def map_partitions(func, *args, **kwargs): """ Apply Python function on each DataFrame partition. Parameters ---------- func : function Function applied to each partition. args, kwargs : Arguments and keywords to pass to the function. At least one of the args should be a dask_gdf object. """ meta = kwargs.pop('meta', None) if meta is not None: meta = make_meta(meta) if 'token' in kwargs: name = kwargs.pop('token') token = tokenize(meta, *args, **kwargs) else: name = funcname(func) token = tokenize(func, meta, *args, **kwargs) name = '{0}-{1}'.format(name, token) args = align_partitions(args) if meta is None: meta = _emulate(func, *args, **kwargs) meta = make_meta(meta) if all(isinstance(arg, Scalar) for arg in args): dask = { (name, 0): (apply, func, (tuple, [(x._name, 0) for x in args]), kwargs) } return Scalar(merge(dask, *[x.dask for x in args]), name, meta) dfs = [df for df in args if isinstance(df, _Frame)] dsk = {} for i in range(dfs[0].npartitions): values = [(x._name, i if isinstance(x, _Frame) else 0) if isinstance( x, (_Frame, Scalar)) else x for x in args] dsk[(name, i)] = (apply, func, values, kwargs) dasks = [arg.dask for arg in args if isinstance(arg, (_Frame, Scalar))] return new_dd_object(merge(dsk, *dasks), name, meta, args[0].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 to_orc( df, path, engine="pyarrow", write_index=True, storage_options=None, compute=True, compute_kwargs=None, ): """Store Dask.dataframe to ORC 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 : 'pyarrow' or ORCEngine Parquet library to use. If only one library is installed, it will use that one; if both, it will use 'fastparquet'. write_index : boolean, default True Whether or not to write the index. Defaults to True. storage_options : dict, default None Key/value pairs to be passed on to the file-system backend, if any. compute : bool, default True If True (default) then the result is computed immediately. If False then a ``dask.delayed`` object is returned for future computation. compute_kwargs : dict, default True Options to be passed in to the compute method Examples -------- >>> df = dd.read_csv(...) # doctest: +SKIP >>> df.to_orc('/path/to/output/', ...) # doctest: +SKIP See Also -------- read_orc: Read ORC data to dask.dataframe """ # Get engine engine = _get_engine(engine, write=True) 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 not write_index: # Not writing index - might as well drop it df = df.reset_index(drop=True) # Use df.npartitions to define file-name list fs.mkdirs(path, exist_ok=True) filenames = [f"part.{i}.orc" for i in range(df.npartitions)] # Construct IO graph dsk = {} name = "to-orc-" + tokenize( df, fs, path, engine, write_index, storage_options, ) final_name = name + "-final" for d, filename in enumerate(filenames): dsk[(name, d)] = ( apply, engine.write_partition, [ (df._name, d), path, fs, filename, ], ) part_tasks = list(dsk.keys()) dsk[(final_name, 0)] = (lambda x: None, part_tasks) graph = HighLevelGraph.from_collections((final_name, 0), dsk, dependencies=[df]) # Compute or return future if compute: if compute_kwargs is None: compute_kwargs = dict() return compute_as_if_collection(DataFrame, graph, part_tasks, **compute_kwargs) return Scalar(graph, final_name, "")
def to_feather( df, path, write_index=True, storage_options=None, compute=True, compute_kwargs=None, ): """Store Dask.dataframe to Feather files Notes ----- Each partition will be written to a separate file. Parameters ---------- df : dask_geopandas.GeoDataFrame path : string or pathlib.Path Destination directory for data. Prepend with protocol like ``s3://`` or ``hdfs://`` for remote data. write_index : boolean, default True Whether or not to write the index. Defaults to True. 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. compute : bool, default True If True (default) then the result is computed immediately. If False then a ``dask.delayed`` object is returned for future computation. compute_kwargs : dict, default True Options to be passed in to the compute method See Also -------- dask_geopandas.read_feather: Read Feather data to dask.dataframe """ # based on the to_orc function from dask # Get engine engine = FeatherDatasetEngine # Process file path storage_options = storage_options or {} 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 not write_index: # Not writing index - might as well drop it df = df.reset_index(drop=True) # Use df.npartitions to define file-name list fs.mkdirs(path, exist_ok=True) filenames = [f"part.{i}.feather" for i in range(df.npartitions)] # Construct IO graph dsk = {} name = "to-feather-" + tokenize(df, fs, path, write_index, storage_options) part_tasks = [] for d, filename in enumerate(filenames): dsk[(name, d)] = ( apply, engine.write_partition, [ (df._name, d), path, fs, filename, ], ) part_tasks.append((name, d)) dsk[name] = (lambda x: None, part_tasks) graph = HighLevelGraph.from_collections(name, dsk, dependencies=[df]) # Compute or return future if compute: if compute_kwargs is None: compute_kwargs = dict() from dask_geopandas import GeoDataFrame return compute_as_if_collection(GeoDataFrame, graph, part_tasks, **compute_kwargs) return Scalar(graph, name, "")
def to_parquet_binned( df, path, nbins, engine="auto", compression="default", write_index=True, append=False, overwrite=False, ignore_divisions=False, partition_on=None, storage_options=None, custom_metadata=None, write_metadata_file=True, compute=True, compute_kwargs=None, schema=None, **kwargs, ): compute_kwargs = compute_kwargs or {} 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.rstrip("/") == working_dir.rstrip("/"): 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.exists(path) and 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-binned" + 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 if custom_metadata: if b"pandas" in custom_metadata.keys(): raise ValueError( "User-defined key/value metadata (custom_metadata) can not " "contain a b'pandas' key. This key is reserved by Pandas, " "and overwriting the corresponding value can render the " "entire dataset unreadable.") kwargs_pass["custom_metadata"] = custom_metadata # Override write_partition to write binned parquet files engine.write_partition = write_partition_binned for d, filename in enumerate(filenames): dsk[(name, d)] = ( apply, engine.write_partition, [ engine, (df._name, d), path, fs, filename, partition_on, write_metadata_file, nbins, ], toolz.merge(kwargs_pass, {"head": True}) if d == 0 else kwargs_pass, ) part_tasks.append((name, d)) final_name = "metadata-" + name # Collect metadata and write _metadata if write_metadata_file: dsk[(final_name, 0)] = ( apply, engine.write_metadata, [ part_tasks, meta, fs, path, ], { "append": append, "compression": compression }, ) else: dsk[(final_name, 0)] = (lambda x: None, part_tasks) graph = HighLevelGraph.from_collections(final_name, dsk, dependencies=[df]) out = Delayed(name, graph) if compute: return compute_as_if_collection(Scalar, graph, [(final_name, 0)], **compute_kwargs) else: return Scalar(graph, final_name, "")