def get_random_ddf(chunk_size, num_chunks, frac_match, chunk_type, args): parts = [chunk_size for i in range(num_chunks)] device_type = True if args.type == "gpu" else False meta = generate_chunk(0, 4, 1, chunk_type, None, device_type) divisions = [None] * (len(parts) + 1) name = "generate-data-" + tokenize(chunk_size, num_chunks, frac_match, chunk_type) graph = {(name, i): ( generate_chunk, i, part, len(parts), chunk_type, frac_match, device_type, ) for i, part in enumerate(parts)} ddf = new_dd_object(graph, name, meta, divisions) if chunk_type == "build": if not args.no_shuffle: divisions = [i for i in range(num_chunks)] + [num_chunks] return ddf.set_index("shuffle", divisions=tuple(divisions)) else: del ddf["shuffle"] return ddf
def to_ddf(self, columns=None, cpu=None): # Check if we are using cpu cpu = self.cpu if cpu is None else cpu if cpu: raise ValueError("cpu=True not supported for AvroDatasetEngine.") # Get list of pieces for each output pieces, meta = self.process_metadata(columns=columns) # TODO: Remove warning and avoid use of uavro in read_partition when # cudf#6529 is fixed (https://github.com/rapidsai/cudf/issues/6529) if len(pieces) > len(self.paths): warnings.warn( "Row-subset selection in cudf avro reader is currently broken. " "Using uavro engine until cudf#6529 is addressed. " "EXPECT POOR PERFORMANCE!! (compared to cuio-based reader)" ) # Construct collection token = tokenize(self.fs, self.paths, self.part_size, columns) read_avro_name = "read-avro-partition-" + token dsk = { (read_avro_name, i): (AvroDatasetEngine.read_partition, self.fs, piece, columns) for i, piece in enumerate(pieces) } return new_dd_object(dsk, read_avro_name, meta.iloc[:0], [None] * (len(pieces) + 1))
def _loc_element(self, iindexer, cindexer): name = "loc-%s" % tokenize(iindexer, self.obj) part = self._get_partitions(iindexer) if iindexer < self.obj.divisions[0] or iindexer > self.obj.divisions[ -1]: raise KeyError("the label [%s] is not in the index" % str(iindexer)) dsk = { (name, 0): ( methods.loc, (self._name, part), slice(iindexer, iindexer), cindexer, ) } meta = self._make_meta(iindexer, cindexer) graph = HighLevelGraph.from_collections(name, dsk, dependencies=[self.obj]) return new_dd_object(graph, name, meta=meta, divisions=[iindexer, iindexer])
def to_ddf(self, columns=None): pieces = self.pieces name = "parquet-to-ddf-" + tokenize(self.fs_token, pieces, columns) dsk = {(name, p): (ParquetDatasetEngine.read_piece, piece, columns) for p, piece in enumerate(pieces)} meta = self.meta_empty(columns=columns) divisions = [None] * (len(pieces) + 1) return new_dd_object(dsk, name, meta, divisions)
def _move_ddf(self, destination): """Move the collection between cpu and gpu memory.""" _ddf = self._ddf if (self.moved_collection and isinstance(_ddf.dask, HighLevelGraph) and hasattr(_ddf.dask, "key_dependencies")): # If our collection has already been moved, and if the # underlying graph is a `HighLevelGraph`, we can just # drop the last "from_pandas-..." layer if the current # destination is "cpu", or we can drop the last # "to_pandas-..." layer if the destination is "gpu". search_name = "from_pandas-" if destination == "cpu" else "to_pandas-" pandas_conversion_layer = None pandas_conversion_dep = None for k, v in _ddf.dask.dependents.items(): if k.startswith(search_name) and v == set(): pandas_conversion_layer = k break if pandas_conversion_layer: deps = [ d for d in _ddf.dask.dependencies[pandas_conversion_layer] ] if len(deps) == 1: pandas_conversion_dep = deps[0] if pandas_conversion_layer and pandas_conversion_dep: # We have met the criteria to remove the last "from/to_pandas-" layer new_layers = { k: v for k, v in _ddf.dask.layers.items() if k != pandas_conversion_layer } new_deps = { k: v for k, v in _ddf.dask.dependencies.items() if k != pandas_conversion_layer } hlg = HighLevelGraph( layers=new_layers, dependencies=new_deps, key_dependencies=_ddf.dask.key_dependencies, ) _meta = (_ddf._meta.to_pandas() if destination == "cpu" else cudf.from_pandas(_ddf._meta)) return new_dd_object(hlg, pandas_conversion_dep, _meta, _ddf.divisions) if destination == "cpu": # Just extend the existing graph to move the collection to cpu return _ddf.to_dask_dataframe() elif destination == "gpu": # Just extend the existing graph to move the collection to gpu return dask_cudf.from_dask_dataframe(_ddf) else: raise ValueError(f"destination {destination} not recognized.")
def from_array(x, chunksize=50000, columns=None, meta=None): """Read any sliceable array into a Dask Dataframe Uses getitem syntax to pull slices out of the array. The array need not be a NumPy array but must support slicing syntax x[50000:100000] and have 2 dimensions: x.ndim == 2 or have a record dtype: x.dtype == [('name', 'O'), ('balance', 'i8')] Parameters ---------- x : array_like chunksize : int, optional The number of rows per partition to use. columns : list or string, optional list of column names if DataFrame, single string if Series meta : object, optional An optional `meta` parameter can be passed for dask to specify the concrete dataframe type to use for partitions of the Dask dataframe. By default, pandas DataFrame is used. Returns ------- dask.DataFrame or dask.Series A dask DataFrame/Series """ if isinstance(x, da.Array): return from_dask_array(x, columns=columns, meta=meta) meta = _meta_from_array(x, columns, meta=meta) divisions = tuple(range(0, len(x), chunksize)) divisions = divisions + (len(x) - 1, ) token = tokenize(x, chunksize, columns) name = "from_array-" + token dsk = {} for i in range(0, int(ceil(len(x) / chunksize))): data = (getitem, x, slice(i * chunksize, (i + 1) * chunksize)) if is_series_like(meta): dsk[name, i] = (type(meta), data, None, meta.dtype, meta.name) else: dsk[name, i] = (type(meta), data, None, meta.columns) return new_dd_object(dsk, name, meta, divisions)
def hash_merge( lhs, left_on, rhs, right_on, how="inner", npartitions=None, suffixes=("_x", "_y"), shuffle=None, indicator=False, ): if npartitions is None: npartitions = max(lhs.npartitions, rhs.npartitions) lhs2 = lhs.repartition(columns=left_on, npartitions=npartitions) rhs2 = rhs.repartition(columns=right_on, npartitions=npartitions) kwargs = dict( how=how, left_on=left_on, right_on=right_on, suffixes=suffixes, indicator=indicator, ) meta = lhs._meta_nonempty.merge(rhs._meta_nonempty, **kwargs) if isinstance(left_on, list): left_on = (list, tuple(left_on)) if isinstance(right_on, list): right_on = (list, tuple(right_on)) token = dask.base.tokenize(lhs2, rhs2, npartitions, shuffle, **kwargs) name = "hash-join-" + token kwargs["empty_index_dtype"] = meta.index.dtype dsk = { (name, i): ( dask.utils.apply, merge_chunk, [(lhs2._name, i), (rhs2._name, i)], kwargs, ) for i in range(npartitions) } divisions = [None] * (npartitions + 1) graph = HighLevelGraph.from_collections(name, dsk, dependencies=[lhs2, rhs2]) return new_dd_object(graph, name, meta, divisions)
def rearrange_by_column_disk(df, column, npartitions=None, compute=False): """Shuffle using local disk See Also -------- rearrange_by_column_tasks: Same function, but using tasks rather than partd Has a more informative docstring """ if npartitions is None: npartitions = df.npartitions token = tokenize(df, column, npartitions) always_new_token = uuid.uuid1().hex p = ("zpartd-" + always_new_token, ) dsk1 = {p: (maybe_buffered_partd(), )} # Partition data on disk name = "shuffle-partition-" + always_new_token dsk2 = {(name, i): (shuffle_group_3, key, column, npartitions, p) for i, key in enumerate(df.__dask_keys__())} dependencies = [] if compute: graph = HighLevelGraph.merge(df.dask, dsk1, dsk2) graph = HighLevelGraph.from_collections(name, graph, dependencies=[df]) keys = [p, sorted(dsk2)] pp, values = compute_as_if_collection(DataFrame, graph, keys) dsk1 = {p: pp} dsk2 = dict(zip(sorted(dsk2), values)) else: dependencies.append(df) # Barrier barrier_token = "barrier-" + always_new_token dsk3 = {barrier_token: (barrier, list(dsk2))} # Collect groups name = "shuffle-collect-" + token dsk4 = {(name, i): (collect, p, i, df._meta, barrier_token) for i in range(npartitions)} divisions = (None, ) * (npartitions + 1) layer = toolz.merge(dsk1, dsk2, dsk3, dsk4) graph = HighLevelGraph.from_collections(name, layer, dependencies=dependencies) return new_dd_object(graph, name, df._meta, divisions)
def get_random_ddf(chunk_size, num_chunks, frac_match, chunk_type): parts = [chunk_size for i in range(num_chunks)] meta = generate_chunk(0, 4, 1, None, None) divisions = [None] * (len(parts) + 1) name = "generate-data-" + tokenize(chunk_size, num_chunks, frac_match, chunk_type) graph = {(name, i): (generate_chunk, i, part, len(parts), chunk_type, frac_match) for i, part in enumerate(parts)} return new_dd_object(graph, name, meta, divisions)
def to_ddf(self, columns=None, shuffle=False, seed=None): """Convert `Dataset` object to `dask_cudf.DataFrame` Parameters ----------- columns : str or list(str); default None Columns to include in output `DataFrame`. If not specified, the output will contain all known columns in the Dataset. shuffle : bool; default False Whether to shuffle the order of partitions in the output `dask_cudf.DataFrame`. Note that this does not shuffle the rows within each partition. This is because the data is not actually loaded into memory for this operation. seed : int; Optional The random seed to use if `shuffle=True`. If nothing is specified, the current system time will be used by the `random` std library. """ # Use DatasetEngine to create ddf ddf = self.engine.to_ddf(columns=columns) # Shuffle the partitions of ddf (optional) if shuffle and ddf.npartitions > 1: # Start with ordered partitions inds = list(range(ddf.npartitions)) # Use random std library to reorder partitions random.seed(seed) random.shuffle(inds) # Construct new high-level graph (HLG) name = ddf._name new_name = "shuffle-partitions-" + tokenize(ddf) dsk = {(new_name, i): (lambda x: x, (name, ind)) for i, ind in enumerate(inds)} new_graph = HighLevelGraph.from_collections(new_name, dsk, dependencies=[ddf]) # Convert the HLG to a Dask collection divisions = [None] * (ddf.npartitions + 1) ddf = new_dd_object(new_graph, new_name, ddf._meta, divisions) # Special dtype conversion (optional) if self.dtypes: _meta = _set_dtypes(ddf._meta, self.dtypes) return ddf.map_partitions(_set_dtypes, self.dtypes, meta=_meta) return ddf
def _simple_shuffle(ddf, plan): # Construct graph for a simple shuffle token = tokenize(ddf, plan) name = "shuffled-" + token final_tasks = collections.defaultdict(list) ignore_index = True for i, p in enumerate(plan): final_tasks[(name, p)].append((ddf._name, i)) dsk = {k: (_concat, v, ignore_index) for k, v in final_tasks.items()} # Conver to a DataFrame collection graph = HighLevelGraph.from_collections(name, dsk, dependencies=[ddf]) divisions = [None] * (len(dsk) + 1) return new_dd_object(graph, name, ddf._meta, divisions)
def __getitem__(self, key): obj = self.obj xs, ys = key # handle numeric values as x and/or y coordinate index if type(xs) is not slice: xs = slice(xs, xs) if type(ys) is not slice: ys = slice(ys, ys) if xs.step is not None or ys.step is not None: raise ValueError("Slice step not supported.") xmin, ymin, xmax, ymax = obj.spatial_partitions.total_bounds bbox = box( xs.start if xs.start is not None else xmin, ys.start if ys.start is not None else ymin, xs.stop if xs.stop is not None else xmax, ys.stop if ys.stop is not None else ymax, ) if self.obj.spatial_partitions is not None: partition_idx = np.nonzero( np.asarray(self.obj.spatial_partitions.intersects(bbox)))[0] else: raise NotImplementedError name = "cx-%s" % tokenize(key, self.obj) if len(partition_idx): # construct graph (based on LocIndexer from dask) dsk = {} for i, part in enumerate(partition_idx): dsk[name, i] = (_cx_part, (self.obj._name, part), bbox) divisions = [self.obj.divisions[i] for i in partition_idx ] + [self.obj.divisions[partition_idx[-1] + 1]] else: # TODO can a dask dataframe have 0 partitions? dsk = {(name, 0): self.obj._meta.head(0)} divisions = [None, None] graph = HighLevelGraph.from_collections(name, dsk, dependencies=[self.obj]) return new_dd_object(graph, name, meta=self.obj._meta, divisions=divisions)
def _handle_frame_argument(arg): dsk = {} prevs_parts_dsk, prevs = _get_previous_partitions(arg, before) dsk.update(prevs_parts_dsk) nexts_parts_dsk, nexts = _get_nexts_partitions(arg, after) dsk.update(nexts_parts_dsk) name_a = "overlap-concat-" + tokenize(arg) for i, (prev, current, next) in enumerate(zip(prevs, arg.__dask_keys__(), nexts)): key = (name_a, i) dsk[key] = (_combined_parts, prev, current, next, before, after) graph = HighLevelGraph.from_collections(name_a, dsk, dependencies=[arg]) return new_dd_object(graph, name_a, meta, divisions)
def _loc_list(self, iindexer, cindexer): name = "loc-%s" % tokenize(iindexer, self.obj) parts = self._get_partitions(iindexer) meta = self._make_meta(iindexer, cindexer) if len(iindexer): dsk = {} divisions = [] items = sorted(parts.items()) for i, (div, indexer) in enumerate(items): dsk[name, i] = (methods.loc, (self._name, div), indexer, cindexer) # append minimum value as division divisions.append(sorted(indexer)[0]) # append maximum value of the last division divisions.append(sorted(items[-1][1])[-1]) graph = HighLevelGraph.from_collections(name, dsk, dependencies=[self.obj]) else: divisions = [None, None] dsk = {(name, 0): meta.head(0)} graph = HighLevelGraph.from_collections(name, dsk) return new_dd_object(graph, name, meta=meta, divisions=divisions)
def fix_overlap(ddf, mins, maxes, lens): """Ensures that the upper bound on each partition of ddf (except the last) is exclusive This is accomplished by first removing empty partitions, then altering existing partitions as needed to include all the values for a particular index value in one partition. """ name = "fix-overlap-" + tokenize(ddf, mins, maxes, lens) non_empties = [i for i, length in enumerate(lens) if length != 0] # drop empty partitions by mapping each partition in a new graph to a particular # partition on the old graph. dsk = {(name, i): (ddf._name, div) for i, div in enumerate(non_empties)} ddf_keys = list(dsk.values()) divisions = tuple(mins) + (maxes[-1], ) overlap = [i for i in range(1, len(mins)) if mins[i] >= maxes[i - 1]] frames = [] for i in overlap: # `frames` is a list of data from previous partitions that we may want to # move to partition i. Here, we add "overlap" from the previous partition # (i-1) to this list. frames.append((get_overlap, ddf_keys[i - 1], divisions[i])) # Make sure that any data added from partition i-1 to `frames` is removed # from partition i-1. dsk[(name, i - 1)] = (drop_overlap, dsk[(name, i - 1)], divisions[i]) # We do not want to move "overlap" from the previous partition (i-1) into # this partition (i) if the data from this partition will need to be moved # to the next partition (i+1) anyway. If we concatenate data too early, # we may lose rows (https://github.com/dask/dask/issues/6972). if i == len(mins) - 2 or divisions[i] != divisions[i + 1]: frames.append(ddf_keys[i]) dsk[(name, i)] = (methods.concat, frames) frames = [] graph = HighLevelGraph.from_collections(name, dsk, dependencies=[ddf]) return new_dd_object(graph, name, ddf._meta, divisions)
def _loc_slice(self, iindexer, cindexer): name = "loc-%s" % tokenize(iindexer, cindexer, self) assert isinstance(iindexer, slice) assert iindexer.step in (None, 1) if iindexer.start is not None: start = self._get_partitions(iindexer.start) else: start = 0 if iindexer.stop is not None: stop = self._get_partitions(iindexer.stop) else: stop = self.obj.npartitions - 1 if iindexer.start is None and self.obj.known_divisions: istart = ( self.obj.divisions[0] if iindexer.stop is None else min(self.obj.divisions[0], iindexer.stop) ) else: istart = self._coerce_loc_index(iindexer.start) if iindexer.stop is None and self.obj.known_divisions: istop = ( self.obj.divisions[-1] if iindexer.start is None else max(self.obj.divisions[-1], iindexer.start) ) else: istop = self._coerce_loc_index(iindexer.stop) if stop == start: dsk = { (name, 0): ( methods.loc, (self._name, start), slice(iindexer.start, iindexer.stop), cindexer, ) } divisions = [istart, istop] else: dsk = { (name, 0): ( methods.loc, (self._name, start), slice(iindexer.start, None), cindexer, ) } for i in range(1, stop - start): if cindexer is None: dsk[name, i] = (self._name, start + i) else: dsk[name, i] = ( methods.loc, (self._name, start + i), slice(None, None), cindexer, ) dsk[name, stop - start] = ( methods.loc, (self._name, stop), slice(None, iindexer.stop), cindexer, ) if iindexer.start is None: div_start = self.obj.divisions[0] else: div_start = max(istart, self.obj.divisions[start]) if iindexer.stop is None: div_stop = self.obj.divisions[-1] else: div_stop = min(istop, self.obj.divisions[stop + 1]) divisions = ( (div_start,) + self.obj.divisions[start + 1 : stop + 1] + (div_stop,) ) assert len(divisions) == len(dsk) + 1 meta = self._make_meta(iindexer, cindexer) graph = HighLevelGraph.from_collections(name, dsk, dependencies=[self.obj]) return new_dd_object(graph, name, meta=meta, divisions=divisions)
def read_file(path, npartitions=None, chunksize=None, layer=None, columns=None, **kwargs): """ Read a GIS file into a Dask GeoDataFrame. This function requires `pyogrio <https://github.com/geopandas/pyogrio/>`__. Parameters ---------- path : str The absolute or relative path to the file or URL to be opened. npartitions : int, optional The number of partitions to create. Either this or `chunksize` should be specified. chunksize : int, optional The number of rows per partition to use. Either this or `npartitions` should be specified. layer : int or str, optional (default: first layer) If an integer is provided, it corresponds to the index of the layer with the data source. If a string is provided, it must match the name of the layer in the data source. Defaults to first layer in data source. columns : list-like, optional (default: all columns) List of column names to import from the data source. Column names must exactly match the names in the data source, and will be returned in the order they occur in the data source. To avoid reading any columns, pass an empty list-like. """ try: import pyogrio except ImportError as err: raise ImportError( "The 'read_file' function requires the 'pyogrio' package, but it is " "not installed or does not import correctly." f"\nImporting pyogrio resulted in: {str(err)}") from dask.layers import DataFrameIOLayer # TODO smart inference for a good default partition size ? if (npartitions is None) == (chunksize is None): raise ValueError( "Exactly one of npartitions and chunksize must be specified.") if "skip_features" in kwargs or "max_features" in kwargs: # TODO we currently use those keywords already for reading in each # partition (we would need to take those into account for determining # the part start/ends) raise ValueError( "The 'skip_features'/'max_feature' keywords are not yet supported") if kwargs: raise ValueError("Additional pyogrio keywords are not yet supported") total_size = pyogrio.read_info(path, layer=layer)["features"] if chunksize is None: chunksize = int(ceil(total_size / npartitions)) # TODO this could be inferred from read_info ? read_geometry = True if columns is not None and "geometry" not in columns: read_geometry = False meta = pyogrio.read_dataframe(path, layer=layer, columns=columns, read_geometry=read_geometry, max_features=5) # Define parts parts = [] row_offset = 0 divs = [row_offset] while row_offset < total_size: batch_size = min(chunksize, total_size - row_offset) parts.append((path, row_offset, batch_size)) row_offset += batch_size divs.append(row_offset) # Set the last division value to be the largest index value in the last partition divs[-1] = divs[-1] - 1 # Create Blockwise layer label = "read-file-" output_name = label + tokenize(path, chunksize, layer, columns) layer = DataFrameIOLayer( output_name, columns, parts, FileFunctionWrapper(layer, columns), label=label, ) graph = HighLevelGraph({output_name: layer}, {output_name: set()}) return new_dd_object(graph, output_name, meta, divs)
def rearrange_by_column_tasks(df, column, max_branch=32, npartitions=None, ignore_index=False): """Order divisions of DataFrame so that all values within column(s) align This enacts a task-based shuffle. It contains most of the tricky logic around the complex network of tasks. Typically before this function is called a new column, ``"_partitions"`` has been added to the dataframe, containing the output partition number of every row. This function produces a new dataframe where every row is in the proper partition. It accomplishes this by splitting each input partition into several pieces, and then concatenating pieces from different input partitions into output partitions. If there are enough partitions then it does this work in stages to avoid scheduling overhead. Lets explain the motivation for this further. Imagine that we have 1000 input partitions and 1000 output partitions. In theory we could split each input into 1000 pieces, and then move the 1 000 000 resulting pieces around, and then concatenate them all into 1000 output groups. This would be fine, but the central scheduling overhead of 1 000 000 tasks would become a bottleneck. Instead we do this in stages so that we split each of the 1000 inputs into 30 pieces (we now have 30 000 pieces) move those around, concatenate back down to 1000, and then do the same process again. This has the same result as the full transfer, but now we've moved data twice (expensive) but done so with only 60 000 tasks (cheap). Note that the `column` input may correspond to a list of columns (rather than just a single column name). In this case, the `shuffle_group` and `shuffle_group_2` functions will use hashing to map each row to an output partition. This approach may require the same rows to be hased multiple times, but avoids the need to assign a new "_partitions" column. Parameters ---------- df: dask.dataframe.DataFrame column: str or list A column name on which we want to split, commonly ``"_partitions"`` which is assigned by functions upstream. This could also be a list of columns (in which case shuffle_group will create a hash array/column). max_branch: int The maximum number of splits per input partition. Defaults to 32. If there are more partitions than this then the shuffling will occur in stages in order to avoid creating npartitions**2 tasks Increasing this number increases scheduling overhead but decreases the number of full-dataset transfers that we have to make. npartitions: Optional[int] The desired number of output partitions Returns ------- df3: dask.dataframe.DataFrame See also -------- rearrange_by_column_disk: same operation, but uses partd rearrange_by_column: parent function that calls this or rearrange_by_column_disk shuffle_group: does the actual splitting per-partition """ max_branch = max_branch or 32 if (npartitions or df.npartitions) <= max_branch: # We are creating a small number of output partitions. # No need for staged shuffling. Staged shuffling will # sometimes require extra work/communication in this case. token = tokenize(df, column, npartitions) shuffle_name = f"simple-shuffle-{token}" npartitions = npartitions or df.npartitions shuffle_layer = SimpleShuffleLayer( shuffle_name, column, npartitions, df.npartitions, ignore_index, df._name, df._meta, ) graph = HighLevelGraph.from_collections(shuffle_name, shuffle_layer, dependencies=[df]) return new_dd_object(graph, shuffle_name, df._meta, [None] * (npartitions + 1)) n = df.npartitions stages = int(math.ceil(math.log(n) / math.log(max_branch))) if stages > 1: k = int(math.ceil(n**(1 / stages))) else: k = n inputs = [ tuple(digit(i, j, k) for j in range(stages)) for i in range(k**stages) ] npartitions_orig = df.npartitions token = tokenize(df, stages, column, n, k) for stage in range(stages): stage_name = f"shuffle-{stage}-{token}" stage_layer = ShuffleLayer( stage_name, column, inputs, stage, npartitions, n, k, ignore_index, df._name, df._meta, ) graph = HighLevelGraph.from_collections(stage_name, stage_layer, dependencies=[df]) df = new_dd_object(graph, stage_name, df._meta, df.divisions) if npartitions is not None and npartitions != npartitions_orig: token = tokenize(df, npartitions) repartition_group_token = "repartition-group-" + token dsk = {(repartition_group_token, i): ( shuffle_group_2, k, column, ignore_index, npartitions, ) for i, k in enumerate(df.__dask_keys__())} repartition_get_name = "repartition-get-" + token for p in range(npartitions): dsk[(repartition_get_name, p)] = ( shuffle_group_get, (repartition_group_token, p % npartitions_orig), p, ) graph2 = HighLevelGraph.from_collections(repartition_get_name, dsk, dependencies=[df]) df2 = new_dd_object(graph2, repartition_get_name, df._meta, [None] * (npartitions + 1)) else: df2 = df df2.divisions = (None, ) * (npartitions_orig + 1) return df2
if none_chunksize: if not isinstance(npartitions, int): raise TypeError( "Please provide npartitions as an int, or possibly as None if you specify chunksize." ) chunksize = int(ceil(nrows / npartitions)) elif not isinstance(chunksize, int): raise TypeError( "Please provide chunksize as an int, or possibly as None if you specify npartitions." ) name = name or ("from_pandas-" + tokenize(data, chunksize)) if not nrows: return new_dd_object({(name, 0): data}, name, data, [None, None]) if data.index.isna().any() and not data.index.is_numeric(): raise NotImplementedError( "Index in passed data is non-numeric and contains nulls, which Dask does not entirely support.\n" "Consider passing `data.loc[~data.isna()]` instead.") if sort: if not data.index.is_monotonic_increasing: data = data.sort_index(ascending=True) divisions, locations = sorted_division_locations(data.index, chunksize=chunksize) else: locations = list(range(0, nrows, chunksize)) + [len(data)] divisions = [None] * len(locations)
def _apply_offset(self, df: dd.DataFrame, offset: int, end: int) -> dd.DataFrame: """ Limit the dataframe to the window [offset, end]. That is unfortunately, not so simple as we do not know how many items we have in each partition. We have therefore no other way than to calculate (!!!) the sizes of each partition (this means we need to compute the dataframe already here). After that, we can create a new dataframe from the old dataframe by calculating for each partition if and how much it should be used. We do this via generating our own dask computation graph as we need to pass the partition number to the selection function, which is not possible with normal "map_partitions". """ # As we need to calculate the partition size, we better persist # the df. I think... # TODO: check if this is the best thing to do df = df.persist() # First, we need to find out which partitions we want to use. # Therefore we count the total number of entries partition_borders = df.map_partitions(lambda x: len(x)).compute() partition_borders = partition_borders.cumsum().to_dict() # Now we let each of the partitions figure out, how much it needs to return # using these partition borders # For this, we generate out own dask computation graph (as it does not really) # fit well with one of the already present methods # (a) we define a method to be calculated on each partition # This method returns the part of the partition, which falls between [offset, fetch] def select_from_to(df, partition_index): this_partition_border_left = (partition_borders[partition_index - 1] if partition_index > 0 else 0) this_partition_border_right = partition_borders[partition_index] if (end and end < this_partition_border_left) or ( offset and offset >= this_partition_border_right): return df.iloc[0:0] from_index = max(offset - this_partition_border_left, 0) if offset else 0 to_index = ( min(end, this_partition_border_right) if end else this_partition_border_right) - this_partition_border_left return df.iloc[from_index:to_index] # Then we (b) define a task graph. It should calculate the function above on each of the partitions of # df (specified by (df._name, i) for each partition i). As an argument, we pass the partition_index. dask_graph_name = df._name + "-limit" dask_graph_dict = {} for partition_index in range(df.npartitions): dask_graph_dict[(dask_graph_name, partition_index)] = ( select_from_to, (df._name, partition_index), partition_index, ) # We replace df with our new graph graph = HighLevelGraph.from_collections(dask_graph_name, dask_graph_dict, dependencies=[df]) return new_dd_object(graph, dask_graph_name, df._meta, df.divisions)
def groupby_agg( ddf, gb_cols, aggs_in, split_every=None, split_out=None, dropna=True, sep="___", sort=False, as_index=True, ): """ Optimized groupby aggregation for Dask-CuDF. This aggregation algorithm only supports the following options: {"count", "mean", "std", "var", "sum", "min", "max"} This "optimized" approach is more performant than the algorithm in `dask.dataframe`, because it allows the cudf backend to perform multiple aggregations at once. """ # Deal with default split_out and split_every params if split_every is False: split_every = ddf.npartitions split_every = split_every or 8 split_out = split_out or 1 # Standardize `gb_cols` and `columns` lists aggs = aggs_in.copy() if isinstance(gb_cols, str): gb_cols = [gb_cols] columns = [c for c in ddf.columns if c not in gb_cols] str_cols_out = False if isinstance(aggs, dict): # Use `str_cols_out` to specify if the output columns # will have str (rather than MultiIndex/tuple) names. # This happens when all values in the `aggs` dict are # strings (no lists) str_cols_out = True for col in aggs: if isinstance(aggs[col], str): aggs[col] = [aggs[col]] else: str_cols_out = False if col in gb_cols: columns.append(col) # Assert that aggregations are supported _supported = {"count", "mean", "std", "var", "sum", "min", "max"} if not _is_supported(aggs, _supported): raise ValueError( f"Supported aggs include {_supported} for groupby_agg API. " f"Aggregations must be specified with dict or list syntax.") # Always convert aggs to dict for consistency if isinstance(aggs, list): aggs = {col: aggs for col in columns} # Begin graph construction dsk = {} token = tokenize(ddf, gb_cols, aggs) partition_agg_name = "groupby_partition_agg-" + token tree_reduce_name = "groupby_tree_reduce-" + token gb_agg_name = "groupby_agg-" + token for p in range(ddf.npartitions): # Perform groupby aggregation on each partition. # Split each result into `split_out` chunks (by hashing `gb_cols`) dsk[(partition_agg_name, p)] = ( _groupby_partition_agg, (ddf._name, p), gb_cols, aggs, columns, split_out, dropna, sort, sep, ) # Pick out each chunk using `getitem` for s in range(split_out): dsk[(tree_reduce_name, p, s, 0)] = ( getitem, (partition_agg_name, p), s, ) # Build reduction tree parts = ddf.npartitions widths = [parts] while parts > 1: parts = math.ceil(parts / split_every) widths.append(parts) height = len(widths) for s in range(split_out): for depth in range(1, height): for group in range(widths[depth]): p_max = widths[depth - 1] lstart = split_every * group lstop = min(lstart + split_every, p_max) node_list = [(tree_reduce_name, p, s, depth - 1) for p in range(lstart, lstop)] dsk[(tree_reduce_name, group, s, depth)] = ( _tree_node_agg, node_list, gb_cols, split_out, dropna, sort, sep, ) # Final output partitions. _aggs = aggs.copy() if str_cols_out: # Metadata should use `str` for dict values if that is # what the user originally specified (column names will # be str, rather than tuples). for col in aggs: _aggs[col] = _aggs[col][0] _meta = ddf._meta.groupby(gb_cols, as_index=as_index).agg(_aggs) for s in range(split_out): dsk[(gb_agg_name, s)] = ( _finalize_gb_agg, (tree_reduce_name, 0, s, height - 1), gb_cols, aggs, columns, _meta.columns, as_index, sort, sep, str_cols_out, ) divisions = [None] * (split_out + 1) graph = HighLevelGraph.from_collections(gb_agg_name, dsk, dependencies=[ddf]) return new_dd_object(graph, gb_agg_name, _meta, divisions)
def regenerate_dataset( cls, dataset, output_path, columns=None, file_size=None, part_size=None, cats=None, conts=None, labels=None, storage_options=None, ): """Regenerate an NVTabular Dataset for efficient processing. Example Usage:: dataset = Dataset("/path/to/data_pq", engine="parquet") dataset.regenerate_dataset( out_path, part_size="1MiB", file_size="10MiB" ) Parameters ----------- dataset : Dataset Input `Dataset` object (to be regenerated). output_path : string Root directory path to use for the new (regenerated) dataset. columns : list[string], optional Subset of columns to include in the regenerated dataset. file_size : int or string, optional Desired size of each output file. part_size : int or string, optional Desired partition size to use within regeneration algorithm. Note that this is effectively the size of each contiguous write operation in cudf. cats : list[string], optional Categorical column list. conts : list[string], optional Continuous column list. labels : list[string], optional Label column list. storage_options : dict, optional Storage-option kwargs to pass through to the `fsspec` file-system interface. Returns ------- result : int or Delayed If `compute=True` (default), the return value will be an integer corresponding to the number of generated data files. If `False`, the returned value will be a `Delayed` object. """ # Specify ideal file size and partition size row_group_size = 128_000_000 file_size = parse_bytes(file_size) or row_group_size * 100 part_size = parse_bytes(part_size) or row_group_size * 10 part_size = min(part_size, file_size) fs, _, _ = get_fs_token_paths(output_path, mode="wb", storage_options=storage_options) # Start by converting the original dataset to a Dask-Dataframe # object in CPU memory. We avoid GPU memory in case the original # dataset is prone to OOM errors. _ddf = dataset.engine.to_ddf(columns=columns, cpu=True) # Prepare general metadata (gmd) gmd = {} cats = cats or [] conts = conts or [] labels = labels or [] if not len(cats + conts + labels): warnings.warn( "General-metadata information not detected! " "Please pass lists for `cats`, `conts`, and `labels` as" "arguments to `regenerate_dataset` to ensure a complete " "and correct _metadata.json file.") col_idx = {str(name): i for i, name in enumerate(_ddf.columns)} gmd["cats"] = [{"col_name": c, "index": col_idx[c]} for c in cats] gmd["conts"] = [{"col_name": c, "index": col_idx[c]} for c in conts] gmd["labels"] = [{"col_name": c, "index": col_idx[c]} for c in labels] # Get list of partition lengths token = tokenize( dataset, output_path, columns, part_size, file_size, cats, conts, labels, storage_options, ) getlen_name = "getlen-" + token name = "all-" + getlen_name dsk = {(getlen_name, i): (len, (_ddf._name, i)) for i in range(_ddf.npartitions)} dsk[name] = [(getlen_name, i) for i in range(_ddf.npartitions)] graph = HighLevelGraph.from_collections(name, dsk, dependencies=[_ddf]) size_list = Delayed(name, graph).compute() # Get memory usage per row using first partition p0_mem_size = _ddf.partitions[0].memory_usage( deep=True, index=True).sum().compute() mem_per_row = int(float(p0_mem_size) / float(size_list[0])) # Determine the number of rows to assign to each output partition # and the number of output partitions to assign to each output file rows_per_part = int(part_size / mem_per_row) parts_per_file = int(file_size / part_size) # Construct re-partition graph dsk2 = {} repartition_name = "repartition-" + token split_name = "split-" + repartition_name getitem_name = "getitem-" + repartition_name gets = defaultdict(list) out_parts = 0 remaining_out_part_rows = rows_per_part for i, in_part_size in enumerate(size_list): # The `split` dictionary will be passed to this input # partition to dictate how that partition will be split # into different output partitions/files. The "key" of # this dict is the output partition, and the value is a # tuple specifying the (start, end) row range. split = {} last = 0 while in_part_size >= remaining_out_part_rows: gets[out_parts].append(i) split[out_parts] = (last, last + remaining_out_part_rows) last += remaining_out_part_rows in_part_size = in_part_size - remaining_out_part_rows remaining_out_part_rows = rows_per_part out_parts += 1 if in_part_size: gets[out_parts].append(i) split[out_parts] = (last, last + in_part_size) remaining_out_part_rows -= in_part_size if remaining_out_part_rows == 0: remaining_out_part_rows = rows_per_part out_parts += 1 dsk2[(split_name, i)] = (_split_part, (_ddf._name, i), split) npartitions = max(gets) + 1 for k, v_list in gets.items(): last = None _concat_list = [] for v in v_list: key = (getitem_name, v, k) _concat_list.append(key) dsk2[key] = (operator.getitem, (split_name, v), k) ignore_index = True dsk2[(repartition_name, k)] = (_concat, _concat_list, ignore_index) graph2 = HighLevelGraph.from_collections(repartition_name, dsk2, dependencies=[_ddf]) divisions = [None] * (npartitions + 1) _ddf2 = new_dd_object(graph2, repartition_name, _ddf._meta, divisions) # Make sure the root directory exists fs.mkdirs(output_path, exist_ok=True) # Construct rewrite graph dsk3 = {} rewrite_name = "rewrite-" + token write_data_name = "write-data-" + rewrite_name write_metadata_name = "write-metadata-" + rewrite_name inputs = [] final_inputs = [] for i in range(_ddf2.npartitions): index = i // parts_per_file nex_index = (i + 1) // parts_per_file package_task = (index != nex_index) or (i == (_ddf2.npartitions - 1)) fn = f"part.{index}.parquet" inputs.append((repartition_name, i)) if package_task: final_inputs.append((write_data_name, i)) dsk3[(write_data_name, i)] = ( _write_data, inputs, output_path, fs, fn, ) inputs = [] # Final task collects and writes all metadata dsk3[write_metadata_name] = ( _write_metadata_file, final_inputs, fs, output_path, gmd, ) graph3 = HighLevelGraph.from_collections(write_metadata_name, dsk3, dependencies=[_ddf2]) return Delayed(write_metadata_name, graph3)
def from_dask_array(x, columns=None, index=None, meta=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 divisions 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. meta : object, optional An optional `meta` parameter can be passed for dask to specify the concrete dataframe type to be returned. By default, pandas DataFrame is used. 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, meta=meta) name = "from-dask-array-" + tokenize(x, columns) graph_dependencies = [x] arrays_and_indices = [x.name, "ij" if x.ndim == 2 else "i"] numblocks = {x.name: x.numblocks} if index is not None: # An index is explicitly given by the caller, so we can pass it through to the # initializer after a few checks. 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 graph_dependencies.append(index) arrays_and_indices.extend([index._name, "i"]) numblocks[index._name] = (index.npartitions, ) elif np.isnan(sum(x.shape)): # The shape of the incoming array is not known in at least one dimension. As # such, we can't create an index for the entire output DataFrame and we set # the divisions to None to represent that. divisions = [None] * (len(x.chunks[0]) + 1) else: # The shape of the incoming array is known and we don't have an explicit index. # Create a mapping of chunk number in the incoming array to # (start row, stop row) tuples. These tuples will be used to create a sequential # RangeIndex later on that is continuous over the whole DataFrame. divisions = [0] stop = 0 index_mapping = {} for i, increment in enumerate(x.chunks[0]): stop += increment index_mapping[(i, )] = (divisions[i], stop) divisions.append(stop) divisions[-1] -= 1 arrays_and_indices.extend( [BlockwiseDepDict(mapping=index_mapping), "i"]) if is_series_like(meta): kwargs = { "dtype": x.dtype, "name": meta.name, "initializer": type(meta) } else: kwargs = {"columns": meta.columns, "initializer": type(meta)} blk = blockwise( _partition_from_array, name, "i", *arrays_and_indices, numblocks=numblocks, concatenate=True, # kwargs passed through to the DataFrame/Series initializer **kwargs, ) graph = HighLevelGraph.from_collections(name, blk, dependencies=graph_dependencies) return new_dd_object(graph, name, meta, divisions)
def map_overlap( func, df, before, after, *args, meta=no_default, enforce_metadata=True, transform_divisions=True, align_dataframes=True, **kwargs, ): """Apply a function to each partition, sharing rows with adjacent partitions. Parameters ---------- func : function The function applied to each partition. If this function accepts the special ``partition_info`` keyword argument, it will recieve information on the partition's relative location within the dataframe. df: dd.DataFrame, dd.Series args, kwargs : Positional and keyword arguments to pass to the function. Positional arguments are computed on a per-partition basis, while keyword arguments are shared across all partitions. The partition itself will be the first positional argument, with all other arguments passed *after*. Arguments can be ``Scalar``, ``Delayed``, or regular Python objects. DataFrame-like args (both dask and pandas) will be repartitioned to align (if necessary) before applying the function; see ``align_dataframes`` to control this behavior. enforce_metadata : bool, default True Whether to enforce at runtime that the structure of the DataFrame produced by ``func`` actually matches the structure of ``meta``. This will rename and reorder columns for each partition, and will raise an error if this doesn't work or types don't match. before : int or timedelta The rows to prepend to partition ``i`` from the end of partition ``i - 1``. after : int or timedelta The rows to append to partition ``i`` from the beginning of partition ``i + 1``. transform_divisions : bool, default True Whether to apply the function onto the divisions and apply those transformed divisions to the output. align_dataframes : bool, default True Whether to repartition DataFrame- or Series-like args (both dask and pandas) so their divisions align before applying the function. This requires all inputs to have known divisions. Single-partition inputs will be split into multiple partitions. If False, all inputs must have either the same number of partitions or a single partition. Single-partition inputs will be broadcast to every partition of multi-partition inputs. $META See Also -------- dd.DataFrame.map_overlap """ args = (df, ) + args dfs = [df for df in args if isinstance(df, _Frame)] if isinstance(before, datetime.timedelta) or isinstance( after, datetime.timedelta): if not is_datetime64_any_dtype( dfs[0].index._meta_nonempty.inferred_type): raise TypeError( "Must have a `DatetimeIndex` when using string offset " "for `before` and `after`") else: if not (isinstance(before, Integral) and before >= 0 and isinstance(after, Integral) and after >= 0): raise ValueError("before and after must be positive integers") name = kwargs.pop("token", None) parent_meta = kwargs.pop("parent_meta", None) assert callable(func) if name is not None: token = tokenize(meta, before, after, *args, **kwargs) else: name = "overlap-" + funcname(func) token = tokenize(func, meta, before, after, *args, **kwargs) name = f"{name}-{token}" if align_dataframes: args = _maybe_from_pandas(args) try: args = _maybe_align_partitions(args) except ValueError as e: raise ValueError( f"{e}. If you don't want the partitions to be aligned, and are " "calling `map_overlap` directly, pass `align_dataframes=False`." ) from e meta = _get_meta_map_partitions(args, dfs, func, kwargs, meta, parent_meta) if all(isinstance(arg, Scalar) for arg in args): layer = { (name, 0): ( apply, func, (tuple, [(arg._name, 0) for arg in args]), kwargs, ) } graph = HighLevelGraph.from_collections(name, layer, dependencies=args) return Scalar(graph, name, meta) args2 = [] dependencies = [] divisions = _get_divisions_map_partitions(align_dataframes, transform_divisions, dfs, func, args, kwargs) def _handle_frame_argument(arg): dsk = {} prevs_parts_dsk, prevs = _get_previous_partitions(arg, before) dsk.update(prevs_parts_dsk) nexts_parts_dsk, nexts = _get_nexts_partitions(arg, after) dsk.update(nexts_parts_dsk) name_a = "overlap-concat-" + tokenize(arg) for i, (prev, current, next) in enumerate(zip(prevs, arg.__dask_keys__(), nexts)): key = (name_a, i) dsk[key] = (_combined_parts, prev, current, next, before, after) graph = HighLevelGraph.from_collections(name_a, dsk, dependencies=[arg]) return new_dd_object(graph, name_a, meta, divisions) for arg in args: if isinstance(arg, _Frame): arg = _handle_frame_argument(arg) args2.append(arg) dependencies.append(arg) continue arg = normalize_arg(arg) arg2, collections = unpack_collections(arg) if collections: args2.append(arg2) dependencies.extend(collections) else: args2.append(arg) kwargs3 = {} simple = True for k, v in kwargs.items(): v = normalize_arg(v) v, collections = unpack_collections(v) dependencies.extend(collections) kwargs3[k] = v if collections: simple = False if has_keyword(func, "partition_info"): partition_info = {(i, ): { "number": i, "division": division } for i, division in enumerate(divisions[:-1])} args2.insert(0, BlockwiseDepDict(partition_info)) orig_func = func def func(partition_info, *args, **kwargs): return orig_func(*args, **kwargs, partition_info=partition_info) if enforce_metadata: dsk = partitionwise_graph( apply_and_enforce, name, func, before, after, *args2, dependencies=dependencies, _func=overlap_chunk, _meta=meta, **kwargs3, ) else: kwargs4 = kwargs if simple else kwargs3 dsk = partitionwise_graph( overlap_chunk, name, func, before, after, *args2, **kwargs4, dependencies=dependencies, ) graph = HighLevelGraph.from_collections(name, dsk, dependencies=dependencies) return new_dd_object(graph, name, meta, divisions)
def shuffle( df: DataFrame, column_names: List[str], npartitions: Optional[int] = None, ignore_index: bool = False, ) -> DataFrame: """Order divisions of DataFrame so that all values within column(s) align This enacts a task-based shuffle using explicit-comms. It requires a full dataset read, serialization and shuffle. This is expensive. If possible you should avoid shuffles. This does not preserve a meaningful index/partitioning scheme. This is not deterministic if done in parallel. Requires an activate client. Parameters ---------- df: dask.dataframe.DataFrame Dataframe to shuffle column_names: list of strings List of column names on which we want to split. npartitions: int or None The desired number of output partitions. If None, the number of output partitions equals `df.npartitions` ignore_index: bool Ignore index during shuffle. If True, performance may improve, but index values will not be preserved. Returns ------- df: dask.dataframe.DataFrame Shuffled dataframe Developer Notes --------------- The implementation consist of three steps: (a) Extend the dask graph of `df` with a call to `shuffle_group()` for each dataframe partition and submit the graph. (b) Submit a task on each worker that shuffle (all-to-all communicate) the groups from (a) and return a list of dataframe-partitions. (c) Submit a dask graph that extract (using `getitem()`) individual dataframe-partitions from (b). """ c = comms.default_comms() # As default we preserve number of partitions if npartitions is None: npartitions = df.npartitions # Step (a): partition/group each dataframe-partition name = ("explicit-comms-shuffle-group-" f"{tokenize(df, column_names, npartitions, ignore_index)}") df = df.persist( ) # Making sure optimizations are apply on the existing graph dsk = dict(df.__dask_graph__()) output_keys = [] for input_key in df.__dask_keys__(): output_key = (name, input_key[1]) dsk[output_key] = ( shuffle_group, input_key, column_names, 0, npartitions, npartitions, ignore_index, npartitions, ) output_keys.append(output_key) # Compute `df_groups`, which is a list of futures, one future per partition in `df`. # Each future points to a dict of length `df.npartitions` that maps each # partition-id to a DataFrame. df_groups = compute_as_if_collection(type(df), dsk, output_keys, sync=False) wait(df_groups) for f in df_groups: # Check for errors if f.status == "error": f.result() # raise exception # Step (b): find out which workers has what part of `df_groups`, # find the number of output each worker should have, # and submit `local_shuffle()` on each worker. key_to_part = {str(part.key): part for part in df_groups} in_parts = defaultdict(list) # Map worker -> [list of futures] for key, workers in c.client.who_has(df_groups).items(): # Note, if multiple workers have the part, we pick the first worker in_parts[first(workers)].append(key_to_part[key]) # Let's create a dict that specifices the number of partitions each worker has in_nparts = {} workers = set() # All ranks that have a partition of `df` for rank, worker in enumerate(c.worker_addresses): nparts = len(in_parts.get(worker, ())) if nparts > 0: in_nparts[rank] = nparts workers.add(rank) workers_sorted = sorted(workers) # Find the output partitions for each worker div = npartitions // len(workers) rank_to_out_part_ids = {} # rank -> [list of partition id] for i, rank in enumerate(workers_sorted): rank_to_out_part_ids[rank] = list(range(div * i, div * (i + 1))) for rank, i in zip(workers_sorted, range(div * len(workers), npartitions)): rank_to_out_part_ids[rank].append(i) # Run `local_shuffle()` on each worker result_futures = {} for rank, worker in enumerate(c.worker_addresses): if rank in workers: result_futures[rank] = c.submit( worker, local_shuffle, in_nparts, in_parts[worker], rank_to_out_part_ids, ignore_index, ) distributed.wait(list(result_futures.values())) del df_groups # Step (c): extract individual dataframe-partitions name = f"explicit-comms-shuffle-getitem-{tokenize(name)}" dsk = {} meta = None for rank, parts in rank_to_out_part_ids.items(): for i, part_id in enumerate(parts): dsk[(name, part_id)] = (getitem, result_futures[rank], i) if meta is None: # Get the meta from the first output partition meta = delayed(make_meta)(delayed(getitem)( result_futures[rank], i)).compute() assert meta is not None divs = [None] * (len(dsk) + 1) return new_dd_object(dsk, name, meta, divs).persist()
def read_feather( path, columns=None, filters=None, index=None, storage_options=None, ): """Read a Feather dataset into a Dask-GeoPandas DataFrame. Parameters ---------- path: str or list(str) Source directory for data, or path(s) to individual Feather files. Paths can be a full URL with protocol specifier, and may include glob character if a single string. columns: None or list(str) Columns to load. If None, loads all. filters : list (of list) of tuples or pyarrow.dataset.Expression, default None Row-wise filter to apply while reading the dataset. Can be specified as a ``pyarrow.dataset.Expression`` object or using a list of tuples notation, like ``[[('col1', '==', 0), ...], ...]``. The filter will be applied both at the partition level, this is to prevent the loading of some files, as at the file level to filter the actual rows. For the list of tuples format, predicates can be expressed in disjunctive normal form (DNF). This means that the innermost tuple describes a single column predicate. These inner predicates are combined with an AND conjunction into a larger predicate. The outer-most list then combines all of the combined filters with an OR disjunction. Predicates can also be expressed as a List[Tuple]. These are evaluated as an AND conjunction. To express OR in predictates, one must use the List[List[Tuple]] notation. index : str, list or False, default None Field name(s) to use as the output frame index. By default will be inferred from the pandas metadata (if present in the files). Use False to read all fields as columns. storage_options : dict, default None Key/value pairs to be passed on to the file-system backend, if any (inferred from the path, such as "s3://..."). Please see ``fsspec`` for more details. Returns ------- dask_geopandas.GeoDataFrame (even if there is only one column) """ if index is False: raise NotImplementedError( "Specifying index=False is not yet implemented") # Get engine engine = FeatherDatasetEngine # Process file path(s) storage_options = storage_options or {} fs, _, paths = get_fs_token_paths(path, mode="rb", storage_options=storage_options) paths = sorted(paths, key=natural_sort_key) # numeric rather than glob ordering # Let backend engine generate a list of parts from the dataset metadata parts, meta, schema, filter = engine.read_metadata( fs, paths, columns, filters, index, ) # Update meta to be a GeoDataFrame meta = _update_meta_to_geodataframe(meta, schema.metadata) # Construct spatial partitioning information, if available spatial_partitions = geopandas.GeoSeries( [ _get_partition_bounds(frag.physical_schema.metadata) for frag in parts ], crs=meta.crs, ) if spatial_partitions.isna().any(): spatial_partitions = None # Construct and return a Blockwise layer label = "read-feather-" output_name = label + tokenize(path, columns, filters, index) layer = DataFrameIOLayer( output_name, columns, parts, FeatherFunctionWrapper(engine, fs, columns, filter, schema, index), label=label, ) graph = HighLevelGraph({output_name: layer}, {output_name: set()}) result = new_dd_object(graph, output_name, meta, [None] * (len(parts) + 1)) result.spatial_partitions = spatial_partitions return result
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 or Future An iterable of ``dask.delayed.Delayed`` objects, such as come from ``dask.delayed`` or an iterable of ``distributed.Future`` objects, such as come from ``client.submit`` interface. 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) if not dfs: dfs = [delayed(make_meta)(meta)] 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") name = prefix + "-" + tokenize(*dfs) layer = DataFrameIOLayer( name=name, columns=None, inputs=BlockwiseDepDict( {(i, ): inp.key for i, inp in enumerate(dfs)}, produces_keys=True, ), io_func=partial(check_meta, meta=meta, funcname="from_delayed") if verify_meta else lambda x: x, ) df = new_dd_object(HighLevelGraph.from_collections(name, layer, dfs), name, meta, divs) if divisions == "sorted": from dask.dataframe.shuffle import compute_and_set_divisions df = compute_and_set_divisions(df) return df
def from_map( func, *iterables, args=None, meta=None, divisions=None, label=None, token=None, enforce_metadata=True, **kwargs, ): """Create a DataFrame collection from a custom function map WARNING: The ``from_map`` API is experimental, and stability is not yet guaranteed. Use at your own risk! Parameters ---------- func : callable Function used to create each partition. If ``func`` satisfies the ``DataFrameIOFunction`` protocol, column projection will be enabled. *iterables : Iterable objects Iterable objects to map to each output partition. All iterables must be the same length. This length determines the number of partitions in the output collection (only one element of each iterable will be passed to ``func`` for each partition). args : list or tuple, optional Positional arguments to broadcast to each output partition. Note that these arguments will always be passed to ``func`` after the ``iterables`` positional arguments. $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 label : str, optional String to use as the function-name label in the output collection-key names. token : str, optional String to use as the "token" in the output collection-key names. enforce_metadata : bool, default True Whether to enforce at runtime that the structure of the DataFrame produced by ``func`` actually matches the structure of ``meta``. This will rename and reorder columns for each partition, and will raise an error if this doesn't work or types don't match. **kwargs: Key-word arguments to broadcast to each output partition. These same arguments will be passed to ``func`` for every output partition. Examples -------- >>> import pandas as pd >>> import dask.dataframe as dd >>> func = lambda x, size=0: pd.Series([x] * size) >>> inputs = ["A", "B"] >>> dd.from_map(func, inputs, size=2).compute() 0 A 1 A 0 B 1 B dtype: object This API can also be used as an alternative to other file-based IO functions, like ``read_parquet`` (which are already just ``from_map`` wrapper functions): >>> import pandas as pd >>> import dask.dataframe as dd >>> paths = ["0.parquet", "1.parquet", "2.parquet"] >>> dd.from_map(pd.read_parquet, paths).head() # doctest: +SKIP name timestamp 2000-01-01 00:00:00 Laura 2000-01-01 00:00:01 Oliver 2000-01-01 00:00:02 Alice 2000-01-01 00:00:03 Victor 2000-01-01 00:00:04 Bob Since ``from_map`` allows you to map an arbitrary function to any number of iterable objects, it can be a very convenient means of implementing functionality that may be missing from from other DataFrame-creation methods. For example, if you happen to have apriori knowledge about the number of rows in each of the files in a dataset, you can generate a DataFrame collection with a global RangeIndex: >>> import pandas as pd >>> import numpy as np >>> import dask.dataframe as dd >>> paths = ["0.parquet", "1.parquet", "2.parquet"] >>> file_sizes = [86400, 86400, 86400] >>> def func(path, row_offset): ... # Read parquet file and set RangeIndex offset ... df = pd.read_parquet(path) ... return df.set_index( ... pd.RangeIndex(row_offset, row_offset+len(df)) ... ) >>> def get_ddf(paths, file_sizes): ... offsets = [0] + list(np.cumsum(file_sizes)) ... return dd.from_map( ... func, paths, offsets[:-1], divisions=offsets ... ) >>> ddf = get_ddf(paths, file_sizes) # doctest: +SKIP >>> ddf.index # doctest: +SKIP Dask Index Structure: npartitions=3 0 int64 86400 ... 172800 ... 259200 ... dtype: int64 Dask Name: myfunc, 6 tasks See Also -------- dask.dataframe.from_delayed dask.layers.DataFrameIOLayer """ # Input validation if not callable(func): raise ValueError("`func` argument must be `callable`") lengths = set() iterables = list(iterables) for i, iterable in enumerate(iterables): if not isinstance(iterable, Iterable): raise ValueError( f"All elements of `iterables` must be Iterable, got {type(iterable)}" ) try: lengths.add(len(iterable)) except (AttributeError, TypeError): iterables[i] = list(iterable) lengths.add(len(iterables[i])) if len(lengths) == 0: raise ValueError("`from_map` requires at least one Iterable input") elif len(lengths) > 1: raise ValueError("All `iterables` must have the same length") if lengths == {0}: raise ValueError("All `iterables` must have a non-zero length") # Check for `produces_tasks` and `creation_info`. # These options are included in the function signature, # because they are not intended for "public" use. produces_tasks = kwargs.pop("produces_tasks", False) creation_info = kwargs.pop("creation_info", None) if produces_tasks or len(iterables) == 1: if len(iterables) > 1: # Tasks are not detected correctly when they are "packed" # within an outer list/tuple raise ValueError( "Multiple iterables not supported when produces_tasks=True") inputs = iterables[0] packed = False else: inputs = list(zip(*iterables)) packed = True # Define collection name label = label or funcname(func) token = token or tokenize(func, meta, inputs, args, divisions, enforce_metadata, **kwargs) name = f"{label}-{token}" # Get "projectable" column selection. # Note that this relies on the IO function # ducktyping with DataFrameIOFunction column_projection = func.columns if isinstance( func, DataFrameIOFunction) else None # NOTE: Most of the metadata-handling logic used here # is copied directly from `map_partitions` if meta is None: meta = _emulate( func, *(inputs[0] if packed else inputs[:1]), *(args or []), udf=True, **kwargs, ) meta_is_emulated = True else: meta = make_meta(meta) meta_is_emulated = False if not (has_parallel_type(meta) or is_arraylike(meta) and meta.shape): if not meta_is_emulated: raise TypeError( "Meta is not valid, `from_map` expects output to be a pandas object. " "Try passing a pandas object as meta or a dict or tuple representing the " "(name, dtype) of the columns.") # If `meta` is not a pandas object, the concatenated results will be a # different type meta = make_meta(_concat([meta])) # Ensure meta is empty DataFrame meta = make_meta(meta) # Define io_func if packed or args or kwargs or enforce_metadata: io_func = _PackedArgCallable( func, args=args, kwargs=kwargs, meta=meta if enforce_metadata else None, enforce_metadata=enforce_metadata, packed=packed, ) else: io_func = func # Construct DataFrameIOLayer layer = DataFrameIOLayer( name, column_projection, inputs, io_func, label=label, produces_tasks=produces_tasks, creation_info=creation_info, ) # Return new DataFrame-collection object divisions = divisions or [None] * (len(inputs) + 1) graph = HighLevelGraph.from_collections(name, layer, dependencies=[]) return new_dd_object(graph, name, meta, divisions)