Skip to content
Open
Show file tree
Hide file tree
Changes from 2 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
8 changes: 8 additions & 0 deletions dask_ml/model_selection/_incremental.py
Original file line number Diff line number Diff line change
Expand Up @@ -166,6 +166,14 @@ async def _fit(
models: Dict[int, Tuple[Model, Meta]] = {}
scores: Dict[int, Meta] = {}

if "partial_fit" not in dir(model):
raise ValueError(
f"model={model} does not implement `partial_fit`, a "
"requirement for doing incremental hyperparameter "
"optimization. For more detail, see\n\n"
" https://ml.dask.org/hyper-parameter-search.html#hyperparameter-scaling"
)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hooray for informative error messages!


logger.info("[CV%s] creating %d models", prefix, len(params))
for ident, param in enumerate(params):
model = client.submit(_create_model, original_model, ident, **param)
Expand Down
23 changes: 22 additions & 1 deletion tests/model_selection/test_incremental.py
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@
)
from scipy.stats import uniform
from sklearn.base import clone
from sklearn.cluster import MiniBatchKMeans
from sklearn.cluster import KMeans, MiniBatchKMeans
from sklearn.linear_model import SGDClassifier
from sklearn.model_selection import ParameterGrid, ParameterSampler
from sklearn.utils import check_random_state
Expand All @@ -33,6 +33,7 @@
HyperbandSearchCV,
IncrementalSearchCV,
InverseDecaySearchCV,
RandomizedSearchCV,
)
from dask_ml.model_selection._incremental import _partial_fit, _score, fit
from dask_ml.model_selection.utils_test import LinearFunction, _MaybeLinearFunction
Expand Down Expand Up @@ -855,6 +856,26 @@ def test_warns_scores_per_fit(c, s, a, b):
yield search.fit(X, y)


@gen_cluster(client=True)
async def test_raises_if_no_partial_fit(c, s, a, b):
X, y = make_classification(n_samples=20, n_features=3, chunks=(10, -1))
X, y = await c.gather(c.compute([X, y]))
assert isinstance(X, np.ndarray)
assert isinstance(y, np.ndarray)

params = {"n_init": list(range(1, 10))}
model = KMeans(max_iter=5, verbose=1, algorithm="elkan")

search = IncrementalSearchCV(model, params)
with pytest.raises(ValueError, match="does not implement `partial_fit`"):
await search.fit(X, y)

# no partial_fit, but works with a passive search
search2 = RandomizedSearchCV(model, params, n_iter=2)
await search2.fit(X, y)
assert search2.best_score_


@gen_cluster(client=True)
async def test_model_future(c, s, a, b):
X, y = make_classification(n_samples=100, n_features=5, chunks=10)
Expand Down