async def test_await(c, s, a, b): x = dask.delayed(inc)(1) x = await x.persist() assert x.key in s.tasks assert a.data or b.data assert all(f.done() for f in futures_of(x))
async def _fit( model: Model, params: Union[List[Params], ParameterSampler, ParameterGrid], X_train: ArrayLike, y_train: ArrayLike, X_test: ArrayLike, y_test: ArrayLike, additional_calls: Callable[[Info], Instructions], fit_params: Dict[str, Any] = None, scorer: Callable[[Model, ArrayLike, ArrayLike], float] = None, random_state=None, verbose: Union[bool, Int, float] = False, prefix: str = "", ) -> Results: if isinstance(verbose, bool): # Always log (other loggers might configured differently) verbose = 1.0 if not 0 <= verbose <= 1: raise ValueError( "verbose={} does not satisfy 0 <= verbose <= 1".format(verbose) ) log_delay = int(1 / float(verbose)) if verbose > 0 else 0 original_model = model fit_params = fit_params or {} client = default_client() rng = check_random_state(random_state) info: Dict[int, History] = {} models: Dict[int, Tuple[Model, Meta]] = {} scores: Dict[int, Meta] = {} 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) info[ident] = [] models[ident] = model for ident in info: m, m2 = await models[ident].result() # assume everything in fit_params is small and make it concrete fit_params = await client.compute(fit_params) # Convert testing data into a single element on the cluster # This assumes that it fits into memory on a single worker if isinstance(X_train, (dd.DataFrame, dd.Series)): X_train = X_train.to_dask_array() if isinstance(X_test, (dd.DataFrame, dd.Series)): X_test = X_test.to_dask_array() if isinstance(y_train, dd.Series): y_train = y_train.to_dask_array() if isinstance(y_test, dd.Series): y_test = y_test.to_dask_array() X_train, y_train, X_test, y_test = dask.persist(X_train, y_train, X_test, y_test) if isinstance(X_test, da.Array): X_test = client.compute(X_test) else: X_test = await client.scatter(X_test) if isinstance(y_test, da.Array): y_test = client.compute(y_test) else: y_test = await client.scatter(y_test) # Convert to batches of delayed objects of numpy arrays X_train = sorted(futures_of(X_train), key=lambda f: f.key) y_train = sorted(futures_of(y_train), key=lambda f: f.key) assert len(X_train) == len(y_train) train_eg = await client.gather(client.map(len, y_train)) msg = "[CV%s] For training there are between %d and %d examples in each chunk" logger.info(msg, prefix, min(train_eg), max(train_eg)) # Order by which we process training data futures order = [] def get_futures(partial_fit_calls): """Policy to get training data futures Currently we compute once, and then keep in memory. Presumably in the future we'll want to let data drop and recompute. This function handles that policy internally, and also controls random access to training data. """ # Shuffle blocks going forward to get uniform-but-random access while partial_fit_calls >= len(order): L = list(range(len(X_train))) rng.shuffle(L) order.extend(L) j = order[partial_fit_calls] return X_train[j], y_train[j] # Submit initial partial_fit and score computations on first batch of data X_future, y_future = get_futures(0) X_future_2, y_future_2 = get_futures(1) _models: Dict[int, Tuple[Model, Meta]] = {} _scores: Dict[int, Meta] = {} _specs: Dict[int, Tuple[Model, Meta]] = {} d_partial_fit = dask.delayed(_partial_fit) d_score = dask.delayed(_score) for ident, model in models.items(): model = d_partial_fit(model, X_future, y_future, fit_params) score = d_score(model, X_test, y_test, scorer) spec = d_partial_fit(model, X_future_2, y_future_2, fit_params) _models[ident] = model _scores[ident] = score _specs[ident] = spec _models, _scores, _specs = dask.persist( _models, _scores, _specs, priority={tuple(_specs.values()): -1} ) _models = {k: list(v.dask.values())[0] for k, v in _models.items()} _scores = {k: list(v.dask.values())[0] for k, v in _scores.items()} _specs = {k: list(v.dask.values())[0] for k, v in _specs.items()} models.update(_models) scores.update(_scores) speculative = _specs new_scores = list(_scores.values()) history = [] start_time = time() # async for future, result in seq: for _i in itertools.count(): metas = await client.gather(new_scores) if log_delay and _i % int(log_delay) == 0: idx = np.argmax([m["score"] for m in metas]) best = metas[idx] msg = "[CV%s] validation score of %0.4f received after %d partial_fit calls" logger.info(msg, prefix, best["score"], best["partial_fit_calls"]) for meta in metas: ident = meta["model_id"] meta["elapsed_wall_time"] = time() - start_time info[ident].append(meta) history.append(meta) instructions = additional_calls(info) fired = set(models) - set(instructions) # Delete the futures of bad/fired models. This cancels speculative tasks for ident in fired: del models[ident] del scores[ident] del info[ident] if not any(instructions.values()): break _models = {} _scores = {} _specs = {} for ident, k in instructions.items(): start = info[ident][-1]["partial_fit_calls"] + 1 if k: k -= 1 model = speculative.pop(ident) for i in range(k): X_future, y_future = get_futures(start + i) model = d_partial_fit(model, X_future, y_future, fit_params) score = d_score(model, X_test, y_test, scorer) X_future, y_future = get_futures(start + k) spec = d_partial_fit(model, X_future, y_future, fit_params) _models[ident] = model _scores[ident] = score _specs[ident] = spec _models2, _scores2, _specs2 = dask.persist( _models, _scores, _specs, priority={tuple(_specs.values()): -1} ) _models2 = { k: v if isinstance(v, Future) else list(v.dask.values())[0] for k, v in _models2.items() } _scores2 = {k: list(v.dask.values())[0] for k, v in _scores2.items()} _specs2 = {k: list(v.dask.values())[0] for k, v in _specs2.items()} models.update(_models2) scores.update(_scores2) speculative = _specs2 new_scores = list(_scores2.values()) models = {k: client.submit(operator.getitem, v, 0) for k, v in models.items()} await wait(models) scores = await client.gather(scores) best = max(scores.items(), key=lambda x: x[1]["score"]) info = defaultdict(list) for h in history: h.pop("_adapt", None) info[h["model_id"]].append(h) info = dict(info) return Results(info, models, history, best)
def _fit( model, params, X_train, y_train, X_test, y_test, additional_calls, fit_params=None, scorer=None, random_state=None, ): original_model = model fit_params = fit_params or {} client = default_client() rng = check_random_state(random_state) info = {} models = {} scores = {} for ident, param in enumerate(params): model = client.submit(_create_model, original_model, ident, **param) info[ident] = [] models[ident] = model # assume everything in fit_params is small and make it concrete fit_params = yield client.compute(fit_params) # Convert testing data into a single element on the cluster # This assumes that it fits into memory on a single worker if isinstance(X_test, da.Array): X_test = client.compute(X_test) else: X_test = yield client.scatter(X_test) if isinstance(y_test, da.Array): y_test = client.compute(y_test) else: y_test = yield client.scatter(y_test) # Convert to batches of delayed objects of numpy arrays X_train, y_train = dask.persist(X_train, y_train) X_train = sorted(futures_of(X_train), key=lambda f: f.key) y_train = sorted(futures_of(y_train), key=lambda f: f.key) assert len(X_train) == len(y_train) # Order by which we process training data futures order = [] def get_futures(partial_fit_calls): """ Policy to get training data futures Currently we compute once, and then keep in memory. Presumably in the future we'll want to let data drop and recompute. This function handles that policy internally, and also controls random access to training data. """ # Shuffle blocks going forward to get uniform-but-random access while partial_fit_calls >= len(order): L = list(range(len(X_train))) rng.shuffle(L) order.extend(L) j = order[partial_fit_calls] return X_train[j], y_train[j] # Submit initial partial_fit and score computations on first batch of data X_future, y_future = get_futures(0) X_future_2, y_future_2 = get_futures(1) _models = {} _scores = {} _specs = {} d_partial_fit = dask.delayed(_partial_fit) d_score = dask.delayed(_score) for ident, model in models.items(): model = d_partial_fit(model, X_future, y_future, fit_params) score = d_score(model, X_test, y_test, scorer) spec = d_partial_fit(model, X_future_2, y_future_2, fit_params) _models[ident] = model _scores[ident] = score _specs[ident] = spec _models, _scores, _specs = dask.persist( _models, _scores, _specs, priority={tuple(_specs.values()): -1}) _models = {k: list(v.dask.values())[0] for k, v in _models.items()} _scores = {k: list(v.dask.values())[0] for k, v in _scores.items()} _specs = {k: list(v.dask.values())[0] for k, v in _specs.items()} models.update(_models) scores.update(_scores) speculative = _specs new_scores = list(_scores.values()) history = [] start_time = time() # async for future, result in seq: while True: metas = yield client.gather(new_scores) for meta in metas: ident = meta["model_id"] meta["elapsed_wall_time"] = time() - start_time info[ident].append(meta) history.append(meta) instructions = additional_calls(info) bad = set(models) - set(instructions) # Delete the futures of bad models. This cancels speculative tasks for ident in bad: del models[ident] del scores[ident] del info[ident] if not any(instructions.values()): break _models = {} _scores = {} _specs = {} for ident, k in instructions.items(): start = info[ident][-1]["partial_fit_calls"] + 1 if k: k -= 1 model = speculative.pop(ident) for i in range(k): X_future, y_future = get_futures(start + i) model = d_partial_fit(model, X_future, y_future, fit_params) score = d_score(model, X_test, y_test, scorer) X_future, y_future = get_futures(start + k) spec = d_partial_fit(model, X_future, y_future, fit_params) _models[ident] = model _scores[ident] = score _specs[ident] = spec _models2, _scores2, _specs2 = dask.persist( _models, _scores, _specs, priority={tuple(_specs.values()): -1}) _models2 = { k: v if isinstance(v, Future) else list(v.dask.values())[0] for k, v in _models2.items() } _scores2 = {k: list(v.dask.values())[0] for k, v in _scores2.items()} _specs2 = {k: list(v.dask.values())[0] for k, v in _specs2.items()} models.update(_models2) scores.update(_scores2) speculative = _specs2 new_scores = list(_scores2.values()) models = { k: client.submit(operator.getitem, v, 0) for k, v in models.items() } yield wait(models) scores = yield client.gather(scores) best = max(scores.items(), key=lambda x: x[1]["score"]) info = defaultdict(list) for h in history: h.pop("_adapt", None) info[h["model_id"]].append(h) info = dict(info) raise gen.Return(Results(info, models, history, best))