def __add__(self, other): if isinstance(other, Tuple): return Tuple(merge(self._dask, other._dask), self._keys + other._keys) return NotImplemented
def from_delayed(dfs, meta=None, divisions=None, prefix="from-delayed", verify_meta=True): """ Create Dask DataFrame from many Dask Delayed objects Parameters ---------- dfs : list of Delayed An iterable of ``dask.delayed.Delayed`` objects, such as come from ``dask.delayed`` These comprise the individual partitions of the resulting dataframe. $META divisions : tuple, str, optional Partition boundaries along the index. For tuple, see https://docs.dask.org/en/latest/dataframe-design.html#partitions For string 'sorted' will compute the delayed values to find index values. Assumes that the indexes are mutually sorted. If None, then won't use index information prefix : str, optional Prefix to prepend to the keys. verify_meta : bool, optional If True check that the partitions have consistent metadata, defaults to True. """ from dask.delayed import Delayed if isinstance(dfs, Delayed): dfs = [dfs] dfs = [ delayed(df) if not isinstance(df, Delayed) and hasattr(df, "key") else df for df in dfs ] for df in dfs: if not isinstance(df, Delayed): raise TypeError("Expected Delayed object, got %s" % type(df).__name__) if meta is None: meta = delayed(make_meta)(dfs[0]).compute() else: meta = make_meta(meta) name = prefix + "-" + tokenize(*dfs) dsk = merge(df.dask for df in dfs) if verify_meta: for (i, df) in enumerate(dfs): dsk[(name, i)] = (check_meta, df.key, meta, "from_delayed") else: for (i, df) in enumerate(dfs): dsk[(name, i)] = df.key if divisions is None or divisions == "sorted": divs = [None] * (len(dfs) + 1) else: divs = tuple(divisions) if len(divs) != len(dfs) + 1: raise ValueError("divisions should be a tuple of len(dfs) + 1") df = new_dd_object(dsk, name, meta, divs) if divisions == "sorted": from ..shuffle import compute_and_set_divisions df = compute_and_set_divisions(df) return df
def start_worker( logdir, scheduler_addr, scheduler_port, worker_addr, nthreads, nprocs, ssh_username, ssh_port, ssh_private_key, nohost, memory_limit, worker_port, nanny_port, remote_python=None, remote_dask_worker="distributed.cli.dask_worker", local_directory=None, ): cmd = ("{python} -m {remote_dask_worker} " "{scheduler_addr}:{scheduler_port} " "--nthreads {nthreads}" + (" --nprocs {nprocs}" if nprocs != 1 else "")) if not nohost: cmd += " --host {worker_addr}" if memory_limit: cmd += " --memory-limit {memory_limit}" if worker_port: cmd += " --worker-port {worker_port}" if nanny_port: cmd += " --nanny-port {nanny_port}" cmd = cmd.format( python=remote_python or sys.executable, remote_dask_worker=remote_dask_worker, scheduler_addr=scheduler_addr, scheduler_port=scheduler_port, worker_addr=worker_addr, nthreads=nthreads, nprocs=nprocs, memory_limit=memory_limit, worker_port=worker_port, nanny_port=nanny_port, ) if local_directory is not None: cmd += " --local-directory {local_directory}".format( local_directory=local_directory) # Optionally redirect stdout and stderr to a logfile if logdir is not None: cmd = "mkdir -p {logdir} && ".format(logdir=logdir) + cmd cmd += "&> {logdir}/dask_scheduler_{addr}.log".format(addr=worker_addr, logdir=logdir) label = "worker {addr}".format(addr=worker_addr) # Create a command dictionary, which contains everything we need to run and # interact with this command. input_queue = Queue() output_queue = Queue() cmd_dict = { "cmd": cmd, "label": label, "address": worker_addr, "input_queue": input_queue, "output_queue": output_queue, "ssh_username": ssh_username, "ssh_port": ssh_port, "ssh_private_key": ssh_private_key, } # Start the thread thread = Thread(target=async_ssh, args=[cmd_dict]) thread.daemon = True thread.start() return merge(cmd_dict, {"thread": thread})
def from_dask_array(x, columns=None, index=None): """ Create a Dask DataFrame from a Dask Array. Converts a 2d array into a DataFrame and a 1d array into a Series. Parameters ---------- x : da.Array columns : list or string list of column names if DataFrame, single string if Series index : dask.dataframe.Index, optional An optional *dask* Index to use for the output Series or DataFrame. The default output index depends on whether `x` has any unknown chunks. If there are any unknown chunks, the output has ``None`` for all the divisions (one per chunk). If all the chunks are known, a default index with known divsions is created. Specifying `index` can be useful if you're conforming a Dask Array to an existing dask Series or DataFrame, and you would like the indices to match. Examples -------- >>> import dask.array as da >>> import dask.dataframe as dd >>> x = da.ones((4, 2), chunks=(2, 2)) >>> df = dd.io.from_dask_array(x, columns=['a', 'b']) >>> df.compute() a b 0 1.0 1.0 1 1.0 1.0 2 1.0 1.0 3 1.0 1.0 See Also -------- dask.bag.to_dataframe: from dask.bag dask.dataframe._Frame.values: Reverse conversion dask.dataframe._Frame.to_records: Reverse conversion """ meta = _meta_from_array(x, columns, index) if x.ndim == 2 and len(x.chunks[1]) > 1: x = x.rechunk({1: x.shape[1]}) name = "from-dask-array" + tokenize(x, columns) to_merge = [] if index is not None: if not isinstance(index, Index): raise ValueError( "'index' must be an instance of dask.dataframe.Index") if index.npartitions != x.numblocks[0]: msg = ("The index and array have different numbers of blocks. " "({} != {})".format(index.npartitions, x.numblocks[0])) raise ValueError(msg) divisions = index.divisions to_merge.append(ensure_dict(index.dask)) index = index.__dask_keys__() elif np.isnan(sum(x.shape)): divisions = [None] * (len(x.chunks[0]) + 1) index = [None] * len(x.chunks[0]) else: divisions = [0] for c in x.chunks[0]: divisions.append(divisions[-1] + c) index = [(np.arange, a, b, 1, "i8") for a, b in zip(divisions[:-1], divisions[1:])] divisions[-1] -= 1 dsk = {} for i, (chunk, ind) in enumerate(zip(x.__dask_keys__(), index)): if x.ndim == 2: chunk = chunk[0] if isinstance(meta, pd.Series): dsk[name, i] = (pd.Series, chunk, ind, x.dtype, meta.name) else: dsk[name, i] = (pd.DataFrame, chunk, ind, meta.columns) to_merge.extend([ensure_dict(x.dask), dsk]) return new_dd_object(merge(*to_merge), name, meta, divisions)
def gen_cluster( nthreads=[("127.0.0.1", 1), ("127.0.0.1", 2)], ncores=None, scheduler="127.0.0.1", timeout=10, security=None, Worker=Worker, client=False, scheduler_kwargs={}, worker_kwargs={}, client_kwargs={}, active_rpc_timeout=1, config={}, clean_kwargs={}, allow_unclosed=False, ): from distributed import Client """ Coroutine test with small cluster @gen_cluster() async def test_foo(scheduler, worker1, worker2): await ... # use tornado coroutines See also: start end """ if ncores is not None: warnings.warn("ncores= has moved to nthreads=", stacklevel=2) nthreads = ncores worker_kwargs = merge( { "memory_limit": system.MEMORY_LIMIT, "death_timeout": 10 }, worker_kwargs) def _(func): if not iscoroutinefunction(func): func = gen.coroutine(func) def test_func(): result = None workers = [] with clean(timeout=active_rpc_timeout, **clean_kwargs) as loop: async def coro(): with dask.config.set(config): s = False for i in range(5): try: s, ws = await start_cluster( nthreads, scheduler, loop, security=security, Worker=Worker, scheduler_kwargs=scheduler_kwargs, worker_kwargs=worker_kwargs, ) except Exception as e: logger.error( "Failed to start gen_cluster, retrying", exc_info=True, ) await asyncio.sleep(1) else: workers[:] = ws args = [s] + workers break if s is False: raise Exception("Could not start cluster") if client: c = await Client( s.address, loop=loop, security=security, asynchronous=True, **client_kwargs, ) args = [c] + args try: future = func(*args) if timeout: future = asyncio.wait_for(future, timeout) result = await future if s.validate: s.validate_state() finally: if client and c.status not in ("closing", "closed"): await c._close(fast=s.status == Status.closed) await end_cluster(s, workers) await asyncio.wait_for(cleanup_global_workers(), 1) try: c = await default_client() except ValueError: pass else: await c._close(fast=True) def get_unclosed(): return [ c for c in Comm._instances if not c.closed() ] + [ c for c in _global_clients.values() if c.status != "closed" ] try: start = time() while time() < start + 5: gc.collect() if not get_unclosed(): break await asyncio.sleep(0.05) else: if allow_unclosed: print(f"Unclosed Comms: {get_unclosed()}") else: raise RuntimeError("Unclosed Comms", get_unclosed()) finally: Comm._instances.clear() _global_clients.clear() return result result = loop.run_sync(coro, timeout=timeout * 2 if timeout else timeout) for w in workers: if getattr(w, "data", None): try: w.data.clear() except EnvironmentError: # zict backends can fail if their storage directory # was already removed pass del w.data return result return test_func return _
def cluster( nworkers=2, nanny=False, worker_kwargs={}, active_rpc_timeout=1, disconnect_timeout=3, scheduler_kwargs={}, ): ws = weakref.WeakSet() enable_proctitle_on_children() with clean(timeout=active_rpc_timeout, threads=False) as loop: if nanny: _run_worker = run_nanny else: _run_worker = run_worker # The scheduler queue will receive the scheduler's address scheduler_q = mp_context.Queue() # Launch scheduler scheduler = mp_context.Process( name="Dask cluster test: Scheduler", target=run_scheduler, args=(scheduler_q, nworkers + 1), kwargs=scheduler_kwargs, ) ws.add(scheduler) scheduler.daemon = True scheduler.start() # Launch workers workers = [] for i in range(nworkers): q = mp_context.Queue() fn = "_test_worker-%s" % uuid.uuid4() kwargs = merge( { "nthreads": 1, "local_directory": fn, "memory_limit": system.MEMORY_LIMIT, }, worker_kwargs, ) proc = mp_context.Process( name="Dask cluster test: Worker", target=_run_worker, args=(q, scheduler_q), kwargs=kwargs, ) ws.add(proc) workers.append({"proc": proc, "queue": q, "dir": fn}) for worker in workers: worker["proc"].start() try: for worker in workers: worker["address"] = worker["queue"].get(timeout=5) except queue.Empty: raise pytest.xfail.Exception("Worker failed to start in test") saddr = scheduler_q.get() start = time() try: try: security = scheduler_kwargs["security"] rpc_kwargs = { "connection_args": security.get_connection_args("client") } except KeyError: rpc_kwargs = {} with rpc(saddr, **rpc_kwargs) as s: while True: nthreads = loop.run_sync(s.ncores) if len(nthreads) == nworkers: break if time() - start > 5: raise Exception("Timeout on cluster creation") # avoid sending processes down to function yield { "address": saddr }, [{ "address": w["address"], "proc": weakref.ref(w["proc"]) } for w in workers] finally: logger.debug("Closing out test cluster") loop.run_sync(lambda: disconnect_all( [w["address"] for w in workers], timeout=disconnect_timeout, rpc_kwargs=rpc_kwargs, )) loop.run_sync(lambda: disconnect( saddr, timeout=disconnect_timeout, rpc_kwargs=rpc_kwargs)) scheduler.terminate() scheduler_q.close() scheduler_q._reader.close() scheduler_q._writer.close() for w in workers: w["proc"].terminate() w["queue"].close() w["queue"]._reader.close() w["queue"]._writer.close() scheduler.join(2) del scheduler for proc in [w["proc"] for w in workers]: proc.join(timeout=2) with suppress(UnboundLocalError): del worker, w, proc del workers[:] for fn in glob("_test_worker-*"): with suppress(OSError): shutil.rmtree(fn) try: client = default_client() except ValueError: pass else: client.close() start = time() while any(proc.is_alive() for proc in ws): text = str(list(ws)) sleep(0.2) assert time() < start + 5, ("Workers still around after five seconds", text)
def to_parquet( df, path, engine="auto", compression="default", write_index=True, append=False, overwrite=False, ignore_divisions=False, partition_on=None, storage_options=None, write_metadata_file=True, compute=True, compute_kwargs=None, schema=None, **kwargs, ): """Store Dask.dataframe to Parquet files Notes ----- Each partition will be written to a separate file. Parameters ---------- df : dask.dataframe.DataFrame path : string or pathlib.Path Destination directory for data. Prepend with protocol like ``s3://`` or ``hdfs://`` for remote data. engine : {'auto', 'fastparquet', 'pyarrow'}, default 'auto' Parquet library to use. If only one library is installed, it will use that one; if both, it will use 'fastparquet'. compression : string or dict, optional Either a string like ``"snappy"`` or a dictionary mapping column names to compressors like ``{"name": "gzip", "values": "snappy"}``. The default is ``"default"``, which uses the default compression for whichever engine is selected. write_index : boolean, optional Whether or not to write the index. Defaults to True. append : bool, optional If False (default), construct data-set from scratch. If True, add new row-group(s) to an existing data-set. In the latter case, the data-set must exist, and the schema must match the input data. overwrite : bool, optional Whether or not to remove the contents of `path` before writing the dataset. The default is False. If True, the specified path must correspond to a directory (but not the current working directory). This option cannot be set to True if `append=True`. NOTE: `overwrite=True` will remove the original data even if the current write operation fails. Use at your own risk. ignore_divisions : bool, optional If False (default) raises error when previous divisions overlap with the new appended divisions. Ignored if append=False. partition_on : list, optional Construct directory-based partitioning by splitting on these fields' values. Each dask partition will result in one or more datafiles, there will be no global groupby. storage_options : dict, optional Key/value pairs to be passed on to the file-system backend, if any. write_metadata_file : bool, optional Whether to write the special "_metadata" file. compute : bool, optional If True (default) then the result is computed immediately. If False then a ``dask.delayed`` object is returned for future computation. compute_kwargs : dict, optional Options to be passed in to the compute method schema : Schema object, dict, or {"infer", None}, optional Global schema to use for the output dataset. Alternatively, a `dict` of pyarrow types can be specified (e.g. `schema={"id": pa.string()}`). For this case, fields excluded from the dictionary will be inferred from `_meta_nonempty`. If "infer", the first non-empty and non-null partition will be used to infer the type for "object" columns. If None (default), we let the backend infer the schema for each distinct output partition. If the partitions produce inconsistent schemas, pyarrow will throw an error when writing the shared _metadata file. Note that this argument is ignored by the "fastparquet" engine. **kwargs : Extra options to be passed on to the specific backend. Examples -------- >>> df = dd.read_csv(...) # doctest: +SKIP >>> dd.to_parquet(df, '/path/to/output/',...) # doctest: +SKIP See Also -------- read_parquet: Read parquet data to dask.dataframe """ if compression == "default": if snappy is not None: compression = "snappy" else: compression = None partition_on = partition_on or [] if isinstance(partition_on, str): partition_on = [partition_on] if set(partition_on) - set(df.columns): raise ValueError("Partitioning on non-existent column. " "partition_on=%s ." "columns=%s" % (str(partition_on), str(list(df.columns)))) if isinstance(engine, str): engine = get_engine(engine) if hasattr(path, "name"): path = stringify_path(path) fs, _, _ = get_fs_token_paths(path, mode="wb", storage_options=storage_options) # Trim any protocol information from the path before forwarding path = fs._strip_protocol(path) if overwrite: if isinstance(fs, LocalFileSystem): working_dir = fs.expand_path(".")[0] if path == working_dir: raise ValueError( "Cannot clear the contents of the current working directory!" ) if append: raise ValueError( "Cannot use both `overwrite=True` and `append=True`!") if fs.isdir(path): # Only remove path contents if # (1) The path exists # (2) The path is a directory # (3) The path is not the current working directory fs.rm(path, recursive=True) # Save divisions and corresponding index name. This is necessary, # because we may be resetting the index to write the file division_info = {"divisions": df.divisions, "name": df.index.name} if division_info["name"] is None: # As of 0.24.2, pandas will rename an index with name=None # when df.reset_index() is called. The default name is "index", # but dask will always change the name to the NONE_LABEL constant if NONE_LABEL not in df.columns: division_info["name"] = NONE_LABEL elif write_index: raise ValueError( "Index must have a name if __null_dask_index__ is a column.") else: warnings.warn( "If read back by Dask, column named __null_dask_index__ " "will be set to the index (and renamed to None).") # There are some "resrved" names that may be used as the default column # name after resetting the index. However, we don't want to treat it as # a "special" name if the string is already used as a "real" column name. reserved_names = [] for name in ["index", "level_0"]: if name not in df.columns: reserved_names.append(name) # If write_index==True (default), reset the index and record the # name of the original index in `index_cols` (we will set the name # to the NONE_LABEL constant if it is originally `None`). # `fastparquet` will use `index_cols` to specify the index column(s) # in the metadata. `pyarrow` will revert the `reset_index` call # below if `index_cols` is populated (because pyarrow will want to handle # index preservation itself). For both engines, the column index # will be written to "pandas metadata" if write_index=True index_cols = [] if write_index: real_cols = set(df.columns) none_index = list(df._meta.index.names) == [None] df = df.reset_index() if none_index: df.columns = [ c if c not in reserved_names else NONE_LABEL for c in df.columns ] index_cols = [c for c in set(df.columns) - real_cols] else: # Not writing index - might as well drop it df = df.reset_index(drop=True) _to_parquet_kwargs = { "engine", "compression", "write_index", "append", "ignore_divisions", "partition_on", "storage_options", "write_metadata_file", "compute", } kwargs_pass = { k: v for k, v in kwargs.items() if k not in _to_parquet_kwargs } # Engine-specific initialization steps to write the dataset. # Possibly create parquet metadata, and load existing stuff if appending meta, schema, i_offset = engine.initialize_write( df, fs, path, append=append, ignore_divisions=ignore_divisions, partition_on=partition_on, division_info=division_info, index_cols=index_cols, schema=schema, **kwargs_pass, ) # Use i_offset and df.npartitions to define file-name list filenames = [ "part.%i.parquet" % (i + i_offset) for i in range(df.npartitions) ] # Construct IO graph dsk = {} name = "to-parquet-" + tokenize( df, fs, path, append, ignore_divisions, partition_on, division_info, index_cols, schema, ) part_tasks = [] kwargs_pass["fmd"] = meta kwargs_pass["compression"] = compression kwargs_pass["index_cols"] = index_cols kwargs_pass["schema"] = schema for d, filename in enumerate(filenames): dsk[(name, d)] = ( apply, engine.write_partition, [ (df._name, d), path, fs, filename, partition_on, write_metadata_file, ], toolz.merge(kwargs_pass, {"head": True}) if d == 0 else kwargs_pass, ) part_tasks.append((name, d)) # Collect metadata and write _metadata if write_metadata_file: dsk[name] = ( apply, engine.write_metadata, [ part_tasks, meta, fs, path, ], { "append": append, "compression": compression }, ) else: dsk[name] = (lambda x: None, part_tasks) graph = HighLevelGraph.from_collections(name, dsk, dependencies=[df]) out = Delayed(name, graph) if compute: if compute_kwargs is None: compute_kwargs = dict() out = out.compute(**compute_kwargs) return out
def __add__(self, other): if not isinstance(other, Tuple): return NotImplemented # pragma: nocover return Tuple(merge(self._dask, other._dask), self._keys + other._keys)