def read_pandas( reader, urlpath, blocksize="default", lineterminator=None, compression="infer", sample=256000, enforce=False, assume_missing=False, storage_options=None, include_path_column=False, **kwargs, ): reader_name = reader.__name__ if lineterminator is not None and len(lineterminator) == 1: kwargs["lineterminator"] = lineterminator else: lineterminator = "\n" if include_path_column and isinstance(include_path_column, bool): include_path_column = "path" if "index" in kwargs or "index_col" in kwargs: raise ValueError("Keywords 'index' and 'index_col' not supported. " "Use dd.{0}(...).set_index('my-index') " "instead".format(reader_name)) for kw in ["iterator", "chunksize"]: if kw in kwargs: raise ValueError("{0} not supported for dd.{1}".format( kw, reader_name)) if kwargs.get("nrows", None): raise ValueError("The 'nrows' keyword is not supported by " "`dd.{0}`. To achieve the same behavior, it's " "recommended to use `dd.{0}(...)." "head(n=nrows)`".format(reader_name)) if isinstance(kwargs.get("skiprows"), int): skiprows = lastskiprow = firstrow = kwargs.get("skiprows") elif kwargs.get("skiprows") is None: skiprows = lastskiprow = firstrow = 0 else: # When skiprows is a list, we expect more than max(skiprows) to # be included in the sample. This means that [0,2] will work well, # but [0, 440] might not work. skiprows = set(kwargs.get("skiprows")) lastskiprow = max(skiprows) # find the firstrow that is not skipped, for use as header firstrow = min(set(range(len(skiprows) + 1)) - set(skiprows)) if isinstance(kwargs.get("header"), list): raise TypeError( "List of header rows not supported for dd.{0}".format(reader_name)) if isinstance(kwargs.get("converters"), dict) and include_path_column: path_converter = kwargs.get("converters").get(include_path_column, None) else: path_converter = None # If compression is "infer", inspect the (first) path suffix and # set the proper compression option if the suffix is recongnized. if compression == "infer": # Translate the input urlpath to a simple path list paths = get_fs_token_paths(urlpath, mode="rb", storage_options=storage_options)[2] # Infer compression from first path compression = infer_compression(paths[0]) if blocksize == "default": blocksize = AUTO_BLOCKSIZE if isinstance(blocksize, str): blocksize = parse_bytes(blocksize) if blocksize and compression: # NONE of the compressions should use chunking warn("Warning %s compression does not support breaking apart files\n" "Please ensure that each individual file can fit in memory and\n" "use the keyword ``blocksize=None to remove this message``\n" "Setting ``blocksize=None``" % compression) blocksize = None if compression not in compr: raise NotImplementedError("Compression format %s not installed" % compression) if blocksize and sample and blocksize < sample and lastskiprow != 0: warn("Unexpected behavior can result from passing skiprows when\n" "blocksize is smaller than sample size.\n" "Setting ``sample=blocksize``") sample = blocksize b_lineterminator = lineterminator.encode() b_out = read_bytes( urlpath, delimiter=b_lineterminator, blocksize=blocksize, sample=sample, compression=compression, include_path=include_path_column, **(storage_options or {}), ) if include_path_column: b_sample, values, paths = b_out path = (include_path_column, path_converter) else: b_sample, values = b_out path = None if not isinstance(values[0], (tuple, list)): values = [values] # If we have not sampled, then use the first row of the first values # as a representative sample. if b_sample is False and len(values[0]): b_sample = values[0][0].compute() # Get header row, and check that sample is long enough. If the file # contains a header row, we need at least 2 nonempty rows + the number of # rows to skip. names = kwargs.get("names", None) header = kwargs.get("header", "infer" if names is None else None) need = 1 if header is None else 2 parts = b_sample.split(b_lineterminator, lastskiprow + need) # If the last partition is empty, don't count it nparts = 0 if not parts else len(parts) - int(not parts[-1]) if sample is not False and nparts < lastskiprow + need and len( b_sample) >= sample: raise ValueError("Sample is not large enough to include at least one " "row of data. Please increase the number of bytes " "in `sample` in the call to `read_csv`/`read_table`") header = b"" if header is None else parts[firstrow] + b_lineterminator # Use sample to infer dtypes and check for presence of include_path_column head = reader(BytesIO(b_sample), **kwargs) if include_path_column and (include_path_column in head.columns): raise ValueError("Files already contain the column name: %s, so the " "path column cannot use this name. Please set " "`include_path_column` to a unique name." % include_path_column) specified_dtypes = kwargs.get("dtype", {}) if specified_dtypes is None: specified_dtypes = {} # If specified_dtypes is a single type, then all columns were specified if assume_missing and isinstance(specified_dtypes, dict): # Convert all non-specified integer columns to floats for c in head.columns: if is_integer_dtype(head[c].dtype) and c not in specified_dtypes: head[c] = head[c].astype(float) values = [[list(dsk.dask.values()) for dsk in block] for block in values] return text_blocks_to_pandas( reader, values, header, head, kwargs, enforce=enforce, specified_dtypes=specified_dtypes, path=path, blocksize=blocksize, )
def to_parquet( self, output_path, shuffle=None, preserve_files=False, output_files=None, out_files_per_proc=None, num_threads=0, dtypes=None, cats=None, conts=None, labels=None, suffix=".parquet", partition_on=None, ): """Writes out to a parquet dataset Parameters ---------- output_path : string Path to write processed/shuffled output data shuffle : nvt.io.Shuffle enum How to shuffle the output dataset. For all options, other than `None` (which means no shuffling), the partitions of the underlying dataset/ddf will be randomly ordered. If `PER_PARTITION` is specified, each worker/process will also shuffle the rows within each partition before splitting and appending the data to a number (`out_files_per_proc`) of output files. Output files are distinctly mapped to each worker process. If `PER_WORKER` is specified, each worker will follow the same procedure as `PER_PARTITION`, but will re-shuffle each file after all data is persisted. This results in a full shuffle of the data processed by each worker. To improve performace, this option currently uses host-memory `BytesIO` objects for the intermediate persist stage. The `FULL` option is not yet implemented. partition_on : str or list(str) Columns to use for hive-partitioning. If this option is used, `preserve_files`, `output_files`, and `out_files_per_proc` will all be ignored. Also, the `PER_WORKER` shuffle will not be supported. preserve_files : bool Whether to preserve the original file-to-partition mapping of the base dataset. This option is only available if the base dataset is known, and if it corresponds to csv or parquet format. If True, the `out_files_per_proc` option will be ignored, but the `output_files` option will take precedence. Default is False. output_files : dict, list or int Dictionary mapping of output file names to partition indices. If a list of file names is specified, a contiguous range of output partitions will be mapped to each file. The same procedure is used if an integer is specified, but the file names will be written as "part_*". If anything is specified for `output_files`, the `output_files_per_proc` argument will be ignored. Also, if a dictionary is specified, excluded partition indices will not be written to disk. out_files_per_proc : integer Number of files to create (per process) after shuffling the data. This option will be ignored if `output_files` is specified. num_threads : integer Number of IO threads to use for writing the output dataset. For `0` (default), no dedicated IO threads will be used. dtypes : dict Dictionary containing desired datatypes for output columns. Keys are column names, values are datatypes. suffix : str or False File-name extension to use for all output files. This argument is ignored if a specific list of file names is specified using the ``output_files`` option. If ``preserve_files=True``, this suffix will be appended to the original name of each file, unless the original extension is ".csv", ".parquet", ".avro", or ".orc" (in which case the old extension will be replaced). cats : list of str, optional List of categorical columns conts : list of str, optional List of continuous columns labels : list of str, optional List of label columns """ shuffle = _check_shuffle_arg(shuffle) if isinstance(output_files, dict) or (not output_files and preserve_files): # Do not shuffle partitions if we are preserving files or # if a specific file-partition mapping is already specified ddf = self.to_ddf() else: ddf = self.to_ddf(shuffle=shuffle) # Replace None/False suffix argument with "" suffix = suffix or "" # Convert `output_files` argument to a dict mapping if output_files: # First, repartition ddf if necessary required_npartitions = ddf.npartitions if isinstance(output_files, int): required_npartitions = output_files elif isinstance(output_files, list): required_npartitions = len(output_files) if ddf.npartitions < required_npartitions: ddf = ddf.clear_divisions().repartition(npartitions=required_npartitions) if isinstance(output_files, int): output_files = [f"part_{i}" + suffix for i in range(output_files)] if isinstance(output_files, list): new = {} split = math.ceil(ddf.npartitions / len(output_files)) for i, fn in enumerate(output_files): start = i * split stop = min(start + split, ddf.npartitions) new[fn] = np.arange(start, stop) output_files = new suffix = "" # Don't add a suffix later - Names already include it if not isinstance(output_files, dict): raise TypeError(f"{type(output_files)} not a supported type for `output_files`.") # If we are preserving files, use the stored dictionary, # or use file_partition_map to extract the mapping elif preserve_files: try: _output_files = self.base_dataset.file_partition_map except AttributeError as e: raise AttributeError( f"`to_parquet(..., preserve_files=True)` is not currently supported " f"for datasets with a {type(self.base_dataset.engine)} engine. Check " f"that `dataset.base_dataset` is backed by csv or parquet files." ) from e if suffix == "": output_files = _output_files else: output_files = {} for fn, rgs in _output_files.items(): split_fn = fn.split(".") if split_fn[-1] in ("parquet", "avro", "orc", "csv"): output_files[".".join(split_fn[:-1]) + suffix] = rgs else: output_files[fn + suffix] = rgs suffix = "" # Don't add a suffix later - Names already include it if dtypes: _meta = _set_dtypes(ddf._meta, dtypes) ddf = ddf.map_partitions(_set_dtypes, dtypes, meta=_meta) fs = get_fs_token_paths(output_path)[0] fs.mkdirs(output_path, exist_ok=True) # Output dask_cudf DataFrame to dataset _ddf_to_dataset( ddf, fs, output_path, shuffle, output_files, out_files_per_proc, cats or [], conts or [], labels or [], "parquet", self.client, num_threads, self.cpu, suffix=suffix, partition_on=partition_on, )
def test_parquet(s3, engine, s3so, metadata_file): import s3fs dd = pytest.importorskip("dask.dataframe") pd = pytest.importorskip("pandas") np = pytest.importorskip("numpy") lib = pytest.importorskip(engine) lib_version = parse_version(lib.__version__) if engine == "pyarrow" and lib_version < parse_version("0.13.1"): pytest.skip("pyarrow < 0.13.1 not supported for parquet") if ( engine == "pyarrow" and lib_version.major == 2 and parse_version(s3fs.__version__) > parse_version("0.5.0") ): pytest.skip("#7056 - new s3fs not supported before pyarrow 3.0") url = "s3://%s/test.parquet" % test_bucket_name data = pd.DataFrame( { "i32": np.arange(1000, dtype=np.int32), "i64": np.arange(1000, dtype=np.int64), "f": np.arange(1000, dtype=np.float64), "bhello": np.random.choice(["hello", "you", "people"], size=1000).astype( "O" ), }, index=pd.Index(np.arange(1000), name="foo"), ) df = dd.from_pandas(data, chunksize=500) df.to_parquet( url, engine=engine, storage_options=s3so, write_metadata_file=metadata_file ) files = [f.split("/")[-1] for f in s3.ls(url)] if metadata_file: assert "_common_metadata" in files assert "_metadata" in files assert "part.0.parquet" in files df2 = dd.read_parquet( url, index="foo", gather_statistics=True, engine=engine, storage_options=s3so ) assert len(df2.divisions) > 1 dd.utils.assert_eq(data, df2) # Check that `open_file_options` arguments are # really passed through to fsspec if fsspec_parquet: # Passing `open_file_options` kwargs will fail # if you set an unsupported engine with pytest.raises(ValueError): dd.read_parquet( url, engine=engine, storage_options=s3so, open_file_options={ "precache_options": {"method": "parquet", "engine": "foo"}, }, ).compute() # ...but should work fine if you modify the # maximum block-transfer size (max_block) dd.read_parquet( url, engine=engine, storage_options=s3so, open_file_options={ "precache_options": {"method": "parquet", "max_block": 8_000}, }, ).compute() # Check "open_file_func" fs = get_fs_token_paths(url, storage_options=s3so)[0] def _open(*args, check=True, **kwargs): assert check return fs.open(*args, **kwargs) # Should fail if `check=False` with pytest.raises(AssertionError): dd.read_parquet( url, engine=engine, storage_options=s3so, open_file_options={"open_file_func": _open, "check": False}, ).compute() # Should succeed otherwise df3 = dd.read_parquet( url, engine=engine, storage_options=s3so, open_file_options={"open_file_func": _open}, ) dd.utils.assert_eq(data, df3) # Check that `cache_type="all"` result is same df4 = dd.read_parquet( url, engine=engine, storage_options=s3so, open_file_options={"cache_type": "all"}, ) dd.utils.assert_eq(data, df4)
def __init__( self, out_dir, num_out_files=30, num_threads=0, cats=None, conts=None, labels=None, shuffle=None, fs=None, use_guid=False, bytes_io=False, cpu=False, fns=None, suffix=None, ): # set variables self.out_dir = out_dir self.cats = cats self.conts = conts self.labels = labels self.shuffle = shuffle self.column_names = None if labels and conts: self.column_names = labels + conts self.col_idx = {} self.num_threads = num_threads self.fns = [fns] if isinstance(fns, str) else fns if self.fns: # If specific file names were specified, # ignore `num_out_files` argument self.num_out_files = len(self.fns) else: self.num_out_files = num_out_files self.num_samples = [0] * self.num_out_files self.data_paths = None self.need_cal_col_names = True self.use_guid = use_guid self.bytes_io = bytes_io self.cpu = cpu self.suffix = suffix # Resolve file system self.fs = fs or get_fs_token_paths(str(out_dir))[0] # Only use threading if num_threads > 1 self.queue = None if self.num_threads > 1: # create thread queue and locks self.queue = queue.Queue(num_threads) self.write_locks = [threading.Lock() for _ in range(num_out_files)] # signifies that end-of-data and that the thread should shut down self._eod = object() # create and start threads for _ in range(num_threads): write_thread = threading.Thread(target=self._write_thread, daemon=True) write_thread.start()
def read_bytes( urlpath, delimiter=None, not_zero=False, blocksize="128 MiB", sample="10 kiB", compression=None, include_path=False, **kwargs, ): """Given a path or paths, return delayed objects that read from those paths. The path may be a filename like ``'2015-01-01.csv'`` or a globstring like ``'2015-*-*.csv'``. The path may be preceded by a protocol, like ``s3://`` or ``hdfs://`` if those libraries are installed. This cleanly breaks data by a delimiter if given, so that block boundaries start directly after a delimiter and end on the delimiter. Parameters ---------- urlpath : string or list Absolute or relative filepath(s). Prefix with a protocol like ``s3://`` to read from alternative filesystems. To read from multiple files you can pass a globstring or a list of paths, with the caveat that they must all have the same protocol. delimiter : bytes An optional delimiter, like ``b'\\n'`` on which to split blocks of bytes. not_zero : bool Force seek of start-of-file delimiter, discarding header. blocksize : int, str Chunk size in bytes, defaults to "128 MiB" compression : string or None String like 'gzip' or 'xz'. Must support efficient random access. sample : int, string, or boolean Whether or not to return a header sample. Values can be ``False`` for "no sample requested" Or an integer or string value like ``2**20`` or ``"1 MiB"`` include_path : bool Whether or not to include the path with the bytes representing a particular file. Default is False. **kwargs : dict Extra options that make sense to a particular storage connection, e.g. host, port, username, password, etc. Examples -------- >>> sample, blocks = read_bytes('2015-*-*.csv', delimiter=b'\\n') # doctest: +SKIP >>> sample, blocks = read_bytes('s3://bucket/2015-*-*.csv', delimiter=b'\\n') # doctest: +SKIP >>> sample, paths, blocks = read_bytes('2015-*-*.csv', include_path=True) # doctest: +SKIP Returns ------- sample : bytes The sample header blocks : list of lists of ``dask.Delayed`` Each list corresponds to a file, and each delayed object computes to a block of bytes from that file. paths : list of strings, only included if include_path is True List of same length as blocks, where each item is the path to the file represented in the corresponding block. """ if not isinstance(urlpath, (str, list, tuple, os.PathLike)): raise TypeError("Path should be a string, os.PathLike, list or tuple") fs, fs_token, paths = get_fs_token_paths(urlpath, mode="rb", storage_options=kwargs) if len(paths) == 0: raise OSError("%s resolved to no files" % urlpath) if blocksize is not None: if isinstance(blocksize, str): blocksize = parse_bytes(blocksize) if not is_integer(blocksize): raise TypeError("blocksize must be an integer") blocksize = int(blocksize) if blocksize is None: offsets = [[0]] * len(paths) lengths = [[None]] * len(paths) else: offsets = [] lengths = [] for path in paths: if compression == "infer": comp = infer_compression(path) else: comp = compression if comp is not None: raise ValueError( "Cannot do chunked reads on compressed files. " "To read, set blocksize=None") size = fs.info(path)["size"] if size is None: raise ValueError( "Backing filesystem couldn't determine file size, cannot " "do chunked reads. To read, set blocksize=None.") elif size == 0: # skip empty offsets.append([]) lengths.append([]) else: # shrink blocksize to give same number of parts if size % blocksize and size > blocksize: blocksize1 = size / (size // blocksize) else: blocksize1 = blocksize place = 0 off = [0] length = [] # figure out offsets, spreading around spare bytes while size - place > (blocksize1 * 2) - 1: place += blocksize1 off.append(int(place)) length.append(off[-1] - off[-2]) length.append(size - off[-1]) if not_zero: off[0] = 1 length[0] -= 1 offsets.append(off) lengths.append(length) delayed_read = delayed(read_block_from_file) out = [] for path, offset, length in zip(paths, offsets, lengths): token = tokenize(fs_token, delimiter, path, fs.ukey(path), compression, offset) keys = [f"read-block-{o}-{token}" for o in offset] values = [ delayed_read( OpenFile(fs, path, compression=compression), o, l, delimiter, dask_key_name=key, ) for o, key, l in zip(offset, keys, length) ] out.append(values) if sample: if sample is True: sample = "10 kiB" # backwards compatibility if isinstance(sample, str): sample = parse_bytes(sample) with OpenFile(fs, paths[0], compression=compression) as f: # read block without seek (because we start at zero) if delimiter is None: sample = f.read(sample) else: sample_buff = f.read(sample) while True: new = f.read(sample) if not new: break if delimiter in new: sample_buff = (sample_buff + new.split(delimiter, 1)[0] + delimiter) break sample_buff = sample_buff + new sample = sample_buff if include_path: return sample, out, paths return sample, out
def create_metadata_file( paths, root_dir=None, out_dir=None, engine="pyarrow", storage_options=None, split_every=32, compute=True, compute_kwargs=None, ): """Construct a global _metadata file from a list of parquet files. Dask's read_parquet function is designed to leverage a global _metadata file whenever one is available. The to_parquet function will generate this file automatically by default, but it may not exist if the dataset was generated outside of Dask. This utility provides a mechanism to generate a _metadata file from a list of existing parquet files. NOTE: This utility is not yet supported for the "fastparquet" engine. Parameters ---------- paths : list(string) List of files to collect footer metadata from. root_dir : string, optional Root directory of dataset. The `file_path` fields in the new _metadata file will relative to this directory. If None, a common root directory will be inferred. out_dir : string, optional Directory location to write the final _metadata file. By default, this will be set to `root_dir`. engine : str or Engine, default 'pyarrow' Parquet Engine to use. Only 'pyarrow' is supported if a string is passed. storage_options : dict, optional Key/value pairs to be passed on to the file-system backend, if any. split_every : int, optional The final metadata object that is written to _metadata can be much smaller than the list of footer metadata. In order to avoid the aggregation of all metadata within a single task, a tree reduction is used. This argument specifies the maximum number of metadata inputs to be handled by any one task in the tree. Defaults to 32. 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 """ # Get engine. # Note that "fastparquet" is not yet supported if isinstance(engine, str): if engine not in ("pyarrow", "arrow"): raise ValueError( f"{engine} is not a supported engine for create_metadata_file " "Try engine='pyarrow'.") engine = get_engine(engine) # Process input path list fs, _, paths = get_fs_token_paths(paths, mode="rb", storage_options=storage_options) paths = sorted(paths, key=natural_sort_key) # numeric rather than glob ordering ap_kwargs = {"root": root_dir} if root_dir else {} root_dir, fns = _analyze_paths(paths, fs, **ap_kwargs) out_dir = out_dir or root_dir # Start constructing a raw graph dsk = {} name = "gen-metadata-" + tokenize(paths, fs) collect_name = "collect-" + name agg_name = "agg-" + name # Define a "collect" task for each file in the input list. # Each tasks will: # 1. Extract the footer metadata from a distinct file # 2. Populate the `file_path` field in the metadata # 3. Return the extracted/modified metadata for p, (fn, path) in enumerate(zip(fns, paths)): key = (collect_name, p, 0) dsk[key] = (engine.collect_file_metadata, path, fs, fn) # Build a reduction tree to aggregate all footer metadata # into a single metadata object. Each task in the tree # will take in a list of metadata objects as input, and will # usually output a single (aggregated) metadata object. # The final task in the tree will write the result to disk # instead of returning it (this behavior is triggered by # passing a file path to `engine.aggregate_metadata`). parts = len(paths) widths = [parts] while parts > 1: parts = math.ceil(parts / split_every) widths.append(parts) height = len(widths) 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) dep_task_name = collect_name if depth == 1 else agg_name node_list = [(dep_task_name, p, depth - 1) for p in range(lstart, lstop)] if depth == height - 1: assert group == 0 dsk[name] = (engine.aggregate_metadata, node_list, fs, out_dir) else: dsk[(agg_name, group, depth)] = ( engine.aggregate_metadata, node_list, None, None, ) # Convert the raw graph to a `Delayed` object graph = HighLevelGraph.from_collections(name, dsk, dependencies=[]) out = Delayed(name, graph) # Optionally compute the result if compute: if compute_kwargs is None: compute_kwargs = dict() out = out.compute(**compute_kwargs) return out
def convert_criteo_to_parquet( input_path: str, output_path: str, client, gpu_mem_frac: float = 0.05, ): print("Converting tsv to parquet files") if not output_path: raise RuntimeError( "Intermediate directory must be defined, if the dataset is tsv.") os.makedirs(output_path, exist_ok=True) # split last day into two parts number_of_lines = int( subprocess.check_output(( f'wc -l {os.path.join(input_path, "day_23")}').split()).split()[0]) valid_set_size = number_of_lines // 2 test_set_size = number_of_lines - valid_set_size with open(os.path.join(input_path, "day_23.part1"), "w") as f: subprocess.run([ 'head', '-n', str(test_set_size), str(os.path.join(input_path, "day_23")) ], stdout=f) with open(os.path.join(input_path, "day_23.part2"), "w") as f: subprocess.run([ 'tail', '-n', str(valid_set_size), str(os.path.join(input_path, "day_23")) ], stdout=f) fs = get_fs_token_paths(input_path, mode="rb")[0] file_list = [ x for x in fs.glob(fs.sep.join([input_path, "day_*"])) if not x.endswith("parquet") ] file_list = sorted(file_list, key=natural_sort_key) name_list = _analyze_paths(file_list, fs)[1] cols = CRITEO_CLICK_COLUMNS + CRITEO_CONTINUOUS_COLUMNS + CRITEO_CATEGORICAL_COLUMNS dtypes = {} dtypes[CRITEO_CLICK_COLUMNS[0]] = np.int64 for x in CRITEO_CONTINUOUS_COLUMNS: dtypes[x] = np.int64 for x in CRITEO_CATEGORICAL_COLUMNS: dtypes[x] = "hex" dsk = {} token = tokenize(file_list, name_list, output_path, gpu_mem_frac, fs, cols, dtypes) convert_file_name = "convert_file-" + token for i, (path, name) in enumerate(zip(file_list, name_list)): key = (convert_file_name, i) dsk[key] = (_convert_file, path, name, output_path, gpu_mem_frac, fs, cols, dtypes) write_meta_name = "write-metadata-" + token dsk[write_meta_name] = ( _write_metadata, [(convert_file_name, i) for i in range(len(file_list))], fs, output_path, ) graph = HighLevelGraph.from_collections(write_meta_name, dsk, dependencies=[]) conversion_delayed = Delayed(write_meta_name, graph) if client: conversion_delayed.compute() else: conversion_delayed.compute(scheduler="synchronous") print("Converted")
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_parquet(df, path, engine="auto", compression="default", write_index=True, append=False, ignore_divisions=False, partition_on=None, storage_options=None, write_metadata_file=True, compute=True, **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. 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. **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 """ from dask import delayed 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) # 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", # (or "level_0" if "index" is already a column name) division_info[ "name"] = "index" if "index" not in df.columns else "level_0" # If write_index==True (default), reset the index and record the # name of the original index in `index_cols` (will be `index` if None, # or `level_0` if `index` is already a column name). # `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) df = df.reset_index() index_cols = [c for c in set(df.columns).difference(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, 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, **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) ] # write parts dwrite = delayed(engine.write_partition) parts = [ dwrite(d, path, fs, filename, partition_on, write_metadata_file, fmd=meta, compression=compression, index_cols=index_cols, **kwargs_pass) for d, filename in zip(df.to_delayed(), filenames) ] # single task to complete out = delayed(lambda x: None)(parts) if write_metadata_file: out = delayed(engine.write_metadata)(parts, meta, fs, path, append=append, compression=compression) if compute: out = out.compute() return out
def read_parquet( path, columns=None, filters=None, categories=None, index=None, storage_options=None, engine="auto", gather_statistics=None, ignore_metadata_file=False, metadata_task_size=None, split_row_groups=None, chunksize=None, aggregate_files=None, **kwargs, ): """ Read a Parquet file into a Dask DataFrame This reads a directory of Parquet data into a Dask.dataframe, one file per partition. It selects the index among the sorted columns if any exist. Parameters ---------- path : str or list Source directory for data, or path(s) to individual parquet files. Prefix with a protocol like ``s3://`` to read from alternative filesystems. To read from multiple files you can pass a globstring or a list of paths, with the caveat that they must all have the same protocol. columns : str or list, default None Field name(s) to read in as columns in the output. By default all non-index fields will be read (as determined by the pandas parquet metadata, if present). Provide a single field name instead of a list to read in the data as a Series. filters : Union[List[Tuple[str, str, Any]], List[List[Tuple[str, str, Any]]]], default None List of filters to apply, like ``[[('col1', '==', 0), ...], ...]``. Using this argument will NOT result in row-wise filtering of the final partitions unless ``engine="pyarrow-dataset"`` is also specified. For other engines, filtering is only performed at the partition level, i.e., to prevent the loading of some row-groups and/or files. For the "pyarrow" engines, 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 (preferred for "pyarrow") List[List[Tuple]] notation. Note that the "fastparquet" engine does not currently support DNF for the filtering of partitioned columns (List[Tuple] is required). 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 parquet file metadata (if present). Use False to read all fields as columns. categories : list or dict, default None For any fields listed here, if the parquet encoding is Dictionary, the column will be created with dtype category. Use only if it is guaranteed that the column is encoded as dictionary in all row-groups. If a list, assumes up to 2**16-1 labels; if a dict, specify the number of labels expected; if None, will load categories automatically for data written by dask/fastparquet, not otherwise. storage_options : dict, default None Key/value pairs to be passed on to the file-system backend, if any. engine : str, default 'auto' Parquet reader library to use. Options include: 'auto', 'fastparquet', 'pyarrow', 'pyarrow-dataset', and 'pyarrow-legacy'. Defaults to 'auto', which selects the FastParquetEngine if fastparquet is installed (and ArrowDatasetEngine otherwise). If 'pyarrow' or 'pyarrow-dataset' is specified, the ArrowDatasetEngine (which leverages the pyarrow.dataset API) will be used. If 'pyarrow-legacy' is specified, ArrowLegacyEngine will be used (which leverages the pyarrow.parquet.ParquetDataset API). NOTE: The 'pyarrow-legacy' option (ArrowLegacyEngine) is deprecated for pyarrow>=5. gather_statistics : bool, default None Gather the statistics for each dataset partition. By default, this will only be done if the _metadata file is available. Otherwise, statistics will only be gathered if True, because the footer of every file will be parsed (which is very slow on some systems). ignore_metadata_file : bool, default False Whether to ignore the global ``_metadata`` file (when one is present). If ``True``, or if the global ``_metadata`` file is missing, the parquet metadata may be gathered and processed in parallel. Parallel metadata processing is currently supported for ``ArrowDatasetEngine`` only. metadata_task_size : int, default configurable If parquet metadata is processed in parallel (see ``ignore_metadata_file`` description above), this argument can be used to specify the number of dataset files to be processed by each task in the Dask graph. If this argument is set to ``0``, parallel metadata processing will be disabled. The default values for local and remote filesystems can be specified with the "metadata-task-size-local" and "metadata-task-size-remote" config fields, respectively (see "dataframe.parquet"). split_row_groups : bool or int, default None Default is True if a _metadata file is available or if the dataset is composed of a single file (otherwise defult is False). If True, then each output dataframe partition will correspond to a single parquet-file row-group. If False, each partition will correspond to a complete file. If a positive integer value is given, each dataframe partition will correspond to that number of parquet row-groups (or fewer). Only the "pyarrow" engine supports this argument. chunksize : int or str, default None The desired size of each output ``DataFrame`` partition in terms of total (uncompressed) parquet storage space. If specified, adjacent row-groups and/or files will be aggregated into the same output partition until the cumulative ``total_byte_size`` parquet-metadata statistic reaches this value. Use `aggregate_files` to enable/disable inter-file aggregation. aggregate_files : bool or str, default None Whether distinct file paths may be aggregated into the same output partition. This parameter requires `gather_statistics=True`, and is only used when `chunksize` is specified or when `split_row_groups` is an integer >1. A setting of True means that any two file paths may be aggregated into the same output partition, while False means that inter-file aggregation is prohibited. For "hive-partitioned" datasets, a "partition"-column name can also be specified. In this case, we allow the aggregation of any two files sharing a file path up to, and including, the corresponding directory name. For example, if ``aggregate_files`` is set to ``"section"`` for the directory structure below, ``03.parquet`` and ``04.parquet`` may be aggregated together, but ``01.parquet`` and ``02.parquet`` cannot be. If, however, ``aggregate_files`` is set to ``"region"``, ``01.parquet`` may be aggregated with ``02.parquet``, and ``03.parquet`` may be aggregated with ``04.parquet``:: dataset-path/ ├── region=1/ │ ├── section=a/ │ │ └── 01.parquet │ ├── section=b/ │ └── └── 02.parquet └── region=2/ ├── section=a/ │ ├── 03.parquet └── └── 04.parquet Note that the default behavior of ``aggregate_files`` is False. **kwargs: dict (of dicts) Passthrough key-word arguments for read backend. The top-level keys correspond to the appropriate operation type, and the second level corresponds to the kwargs that will be passed on to the underlying ``pyarrow`` or ``fastparquet`` function. Supported top-level keys: 'dataset' (for opening a ``pyarrow`` dataset), 'file' (for opening a ``fastparquet`` ``ParquetFile``), 'read' (for the backend read function), 'arrow_to_pandas' (for controlling the arguments passed to convert from a ``pyarrow.Table.to_pandas()``) Examples -------- >>> df = dd.read_parquet('s3://bucket/my-parquet-data') # doctest: +SKIP See Also -------- to_parquet pyarrow.parquet.ParquetDataset """ if "read_from_paths" in kwargs: warnings.warn( "`read_from_paths` is no longer supported and will be ignored.", FutureWarning, ) if isinstance(columns, str): df = read_parquet( path, columns=[columns], filters=filters, categories=categories, index=index, storage_options=storage_options, engine=engine, gather_statistics=gather_statistics, ignore_metadata_file=ignore_metadata_file, split_row_groups=split_row_groups, chunksize=chunksize, aggregate_files=aggregate_files, metadata_task_size=metadata_task_size, ) return df[columns] if columns is not None: columns = list(columns) label = "read-parquet-" output_name = label + tokenize( path, columns, filters, categories, index, storage_options, engine, gather_statistics, ignore_metadata_file, metadata_task_size, split_row_groups, chunksize, aggregate_files, ) if isinstance(engine, str): engine = get_engine(engine) if hasattr(path, "name"): path = stringify_path(path) 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 auto_index_allowed = False if index is None: # User is allowing auto-detected index auto_index_allowed = True if index and isinstance(index, str): index = [index] if chunksize or (split_row_groups and int(split_row_groups) > 1 and aggregate_files): # Require `gather_statistics=True` if `chunksize` is used, # or if `split_row_groups>1` and we are aggregating files. if gather_statistics is False: raise ValueError( "read_parquet options require gather_statistics=True") gather_statistics = True read_metadata_result = engine.read_metadata( fs, paths, categories=categories, index=index, gather_statistics=gather_statistics, filters=filters, split_row_groups=split_row_groups, chunksize=chunksize, aggregate_files=aggregate_files, ignore_metadata_file=ignore_metadata_file, metadata_task_size=metadata_task_size, **kwargs, ) # In the future, we may want to give the engine the # option to return a dedicated element for `common_kwargs`. # However, to avoid breaking the API, we just embed this # data in the first element of `parts` for now. # The logic below is inteded to handle backward and forward # compatibility with a user-defined engine. meta, statistics, parts, index = read_metadata_result[:4] common_kwargs = {} aggregation_depth = False if len(parts): # For now, `common_kwargs` and `aggregation_depth` # may be stored in the first element of `parts` common_kwargs = parts[0].pop("common_kwargs", {}) aggregation_depth = parts[0].pop("aggregation_depth", aggregation_depth) # Parse dataset statistics from metadata (if available) parts, divisions, index, index_in_columns = process_statistics( parts, statistics, filters, index, chunksize, split_row_groups, fs, aggregation_depth, ) # Account for index and columns arguments. # Modify `meta` dataframe accordingly meta, index, columns = set_index_columns(meta, index, columns, index_in_columns, auto_index_allowed) if meta.index.name == NONE_LABEL: meta.index.name = None # Set the index that was previously treated as a column if index_in_columns: meta = meta.set_index(index) if meta.index.name == NONE_LABEL: meta.index.name = None if len(divisions) < 2: # empty dataframe - just use meta graph = {(output_name, 0): meta} divisions = (None, None) else: # Create Blockwise layer layer = DataFrameIOLayer( output_name, columns, parts, ParquetFunctionWrapper( engine, fs, meta, columns, index, kwargs, common_kwargs, ), label=label, ) graph = HighLevelGraph({output_name: layer}, {output_name: set()}) return new_dd_object(graph, output_name, meta, divisions)
def read_parquet( path, columns=None, filters=None, categories=None, index=None, storage_options=None, engine="auto", gather_statistics=None, **kwargs ): """ Read a Parquet file into a Dask DataFrame This reads a directory of Parquet data into a Dask.dataframe, one file per partition. It selects the index among the sorted columns if any exist. Parameters ---------- path : string or list Source directory for data, or path(s) to individual parquet files. Prefix with a protocol like ``s3://`` to read from alternative filesystems. To read from multiple files you can pass a globstring or a list of paths, with the caveat that they must all have the same protocol. columns : string, list or None (default) Field name(s) to read in as columns in the output. By default all non-index fields will be read (as determined by the pandas parquet metadata, if present). Provide a single field name instead of a list to read in the data as a Series. filters : list List of filters to apply, like ``[('x', '>', 0), ...]``. This implements row-group (partition) -level filtering only, i.e., to prevent the loading of some chunks of the data, and only if relevant statistics have been included in the metadata. index : string, list, False or None (default) Field name(s) to use as the output frame index. By default will be inferred from the pandas parquet file metadata (if present). Use False to read all fields as columns. categories : list, dict or None For any fields listed here, if the parquet encoding is Dictionary, the column will be created with dtype category. Use only if it is guaranteed that the column is encoded as dictionary in all row-groups. If a list, assumes up to 2**16-1 labels; if a dict, specify the number of labels expected; if None, will load categories automatically for data written by dask/fastparquet, not otherwise. storage_options : dict Key/value pairs to be passed on to the file-system backend, if any. engine : {'auto', 'fastparquet', 'pyarrow'}, default 'auto' Parquet reader library to use. If only one library is installed, it will use that one; if both, it will use 'fastparquet' gather_statistics : bool or None (default). Gather the statistics for each dataset partition. By default, this will only be done if the _metadata file is available. Otherwise, statistics will only be gathered if True, because the footer of every file will be parsed (which is very slow on some systems). **kwargs: dict (of dicts) Passthrough key-word arguments for read backend. The top-level keys correspond to the appropriate operation type, and the second level corresponds to the kwargs that will be passed on to the underlying `pyarrow` or `fastparquet` function. Supported top-level keys: 'dataset' (for opening a `pyarrow` dataset), 'file' (for opening a `fastparquet` `ParquetFile`), and 'read' (for the backend read function) Examples -------- >>> df = dd.read_parquet('s3://bucket/my-parquet-data') # doctest: +SKIP See Also -------- to_parquet """ if isinstance(columns, str): df = read_parquet( path, [columns], filters, categories, index, storage_options, engine, gather_statistics, ) return df[columns] if columns is not None: columns = list(columns) name = "read-parquet-" + tokenize( path, columns, filters, categories, index, storage_options, engine, gather_statistics, ) if isinstance(engine, str): engine = get_engine(engine) if hasattr(path, "name"): path = stringify_path(path) 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 auto_index_allowed = False if index is None: # User is allowing auto-detected index auto_index_allowed = True if index and isinstance(index, str): index = [index] meta, statistics, parts = engine.read_metadata( fs, paths, categories=categories, index=index, gather_statistics=gather_statistics, filters=filters, **kwargs ) if meta.index.name is not None: index = meta.index.name ignore_index_column_intersection = False if columns is None: # User didn't specify columns, so ignore any intersection # of auto-detected values with the index (if necessary) ignore_index_column_intersection = True columns = [c for c in meta.columns] if not set(columns).issubset(set(meta.columns)): raise ValueError( "The following columns were not found in the dataset %s\n" "The following columns were found %s" % (set(columns) - set(meta.columns), meta.columns) ) # Parse dataset statistics from metadata (if available) index_in_columns = False if statistics: result = list( zip( *[ (part, stats) for part, stats in zip(parts, statistics) if stats["num-rows"] > 0 ] ) ) parts, statistics = result or [[], []] if filters: parts, statistics = apply_filters(parts, statistics, filters) out = sorted_columns(statistics) if index and isinstance(index, str): index = [index] if index and out: # Only one valid column out = [o for o in out if o["name"] in index] if index is not False and len(out) == 1: # Use only sorted column with statistics as the index divisions = out[0]["divisions"] if index is None: index_in_columns = True index = [out[0]["name"]] elif index != [out[0]["name"]]: raise ValueError("Specified index is invalid.\nindex: {}".format(index)) elif index is not False and len(out) > 1: if any(o["name"] == "index" for o in out): # Use sorted column named "index" as the index [o] = [o for o in out if o["name"] == "index"] divisions = o["divisions"] if index is None: index = [o["name"]] index_in_columns = True elif index != [o["name"]]: raise ValueError( "Specified index is invalid.\nindex: {}".format(index) ) else: # Multiple sorted columns found, cannot autodetect the index warnings.warn( "Multiple sorted columns found %s, cannot\n " "autodetect index. Will continue without an index.\n" "To pick an index column, use the index= keyword; to \n" "silence this warning use index=False." "" % [o["name"] for o in out], RuntimeWarning, ) index = False divisions = [None] * (len(parts) + 1) else: divisions = [None] * (len(parts) + 1) else: divisions = [None] * (len(parts) + 1) if index: if isinstance(index, str): index = [index] if isinstance(columns, str): columns = [columns] if ignore_index_column_intersection: columns = [col for col in columns if col not in index] if set(index).intersection(columns): if auto_index_allowed: raise ValueError( "Specified index and column arguments must not intersect" " (set index=False or remove the detected index from columns).\n" "index: {} | column: {}".format(index, columns) ) else: raise ValueError( "Specified index and column arguments must not intersect.\n" "index: {} | column: {}".format(index, columns) ) # Leaving index as a column in `meta`, because the index # will be reset below (in case the index was detected after # meta was created) if index_in_columns: meta = meta[columns + index] else: meta = meta[columns] else: meta = meta[list(columns)] def _merge_kwargs(x, y): z = x.copy() z.update(y) return z subgraph = { (name, i): ( read_parquet_part, engine.read_partition, fs, meta, part["piece"], columns, index, _merge_kwargs(part["kwargs"], kwargs or {}), ) for i, part in enumerate(parts) } # Set the index that was previously treated as a column if index_in_columns: meta = meta.set_index(index) if len(divisions) < 2: # empty dataframe - just use meta subgraph = {(name, 0): meta} divisions = (None, None) return new_dd_object(subgraph, name, meta, divisions)
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, "")
def _ensure_filesystem(passed_filesystem, path): if passed_filesystem is None: return get_fs_token_paths( path[0] if isinstance(path, list) else path)[0] return passed_filesystem
def to_hugectr( self, output_path, cats, conts, labels, shuffle=None, file_partition_map=None, out_files_per_proc=None, num_threads=0, dtypes=None, ): """Writes out to a parquet dataset Parameters ---------- output_path : string Path to write processed/shuffled output data cats : list of str List of categorical columns conts : list of str List of continuous columns labels : list of str List of label columns shuffle : nvt.io.Shuffle, optional How to shuffle the output dataset. Shuffling is only performed if the data is written to disk. For all options, other than `None` (which means no shuffling), the partitions of the underlying dataset/ddf will be randomly ordered. If `PER_PARTITION` is specified, each worker/process will also shuffle the rows within each partition before splitting and appending the data to a number (`out_files_per_proc`) of output files. Output files are distinctly mapped to each worker process. If `PER_WORKER` is specified, each worker will follow the same procedure as `PER_PARTITION`, but will re-shuffle each file after all data is persisted. This results in a full shuffle of the data processed by each worker. To improve performace, this option currently uses host-memory `BytesIO` objects for the intermediate persist stage. The `FULL` option is not yet implemented. file_partition_map : dict Dictionary mapping of output file names to partition indices that should be written to that file name. If this argument is passed, only the partitions included in the dictionary will be written to disk, and the `output_files_per_proc` argument will be ignored. out_files_per_proc : integer Number of files to create (per process) after shuffling the data num_threads : integer Number of IO threads to use for writing the output dataset. For `0` (default), no dedicated IO threads will be used. dtypes : dict Dictionary containing desired datatypes for output columns. Keys are column names, values are datatypes. """ # For now, we must move to the GPU to # write an output dataset. # TODO: Support CPU-mode output self.to_gpu() shuffle = _check_shuffle_arg(shuffle) ddf = self.to_ddf(shuffle=shuffle) if dtypes: _meta = _set_dtypes(ddf._meta, dtypes) ddf = ddf.map_partitions(_set_dtypes, dtypes, meta=_meta) fs = get_fs_token_paths(output_path)[0] fs.mkdirs(output_path, exist_ok=True) # Output dask_cudf DataFrame to dataset, _ddf_to_dataset( ddf, fs, output_path, shuffle, file_partition_map, out_files_per_proc, cats, conts, labels, "hugectr", self.client, num_threads, self.cpu, )
def read_parquet( path, columns=None, filters=None, categories=None, index=None, storage_options=None, engine="auto", gather_statistics=None, split_row_groups=None, chunksize=None, **kwargs, ): """ Read a Parquet file into a Dask DataFrame This reads a directory of Parquet data into a Dask.dataframe, one file per partition. It selects the index among the sorted columns if any exist. Parameters ---------- path : string or list Source directory for data, or path(s) to individual parquet files. Prefix with a protocol like ``s3://`` to read from alternative filesystems. To read from multiple files you can pass a globstring or a list of paths, with the caveat that they must all have the same protocol. columns : string, list or None (default) Field name(s) to read in as columns in the output. By default all non-index fields will be read (as determined by the pandas parquet metadata, if present). Provide a single field name instead of a list to read in the data as a Series. filters : Union[List[Tuple[str, str, Any]], List[List[Tuple[str, str, Any]]]] List of filters to apply, like ``[[('x', '=', 0), ...], ...]``. This implements partition-level (hive) filtering only, i.e., to prevent the loading of some row-groups and/or files. 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 (preferred) List[List[Tuple]] notation. index : string, list, False or None (default) Field name(s) to use as the output frame index. By default will be inferred from the pandas parquet file metadata (if present). Use False to read all fields as columns. categories : list, dict or None For any fields listed here, if the parquet encoding is Dictionary, the column will be created with dtype category. Use only if it is guaranteed that the column is encoded as dictionary in all row-groups. If a list, assumes up to 2**16-1 labels; if a dict, specify the number of labels expected; if None, will load categories automatically for data written by dask/fastparquet, not otherwise. storage_options : dict Key/value pairs to be passed on to the file-system backend, if any. engine : {'auto', 'fastparquet', 'pyarrow'}, default 'auto' Parquet reader library to use. If only one library is installed, it will use that one; if both, it will use 'fastparquet' gather_statistics : bool or None (default). Gather the statistics for each dataset partition. By default, this will only be done if the _metadata file is available. Otherwise, statistics will only be gathered if True, because the footer of every file will be parsed (which is very slow on some systems). split_row_groups : bool or int Default is True if a _metadata file is available or if the dataset is composed of a single file (otherwise defult is False). If True, then each output dataframe partition will correspond to a single parquet-file row-group. If False, each partition will correspond to a complete file. If a positive integer value is given, each dataframe partition will correspond to that number of parquet row-groups (or fewer). Only the "pyarrow" engine supports this argument. chunksize : int, str The target task partition size. If set, consecutive row-groups from the same file will be aggregated into the same output partition until the aggregate size reaches this value. **kwargs: dict (of dicts) Passthrough key-word arguments for read backend. The top-level keys correspond to the appropriate operation type, and the second level corresponds to the kwargs that will be passed on to the underlying `pyarrow` or `fastparquet` function. Supported top-level keys: 'dataset' (for opening a `pyarrow` dataset), 'file' (for opening a `fastparquet` `ParquetFile`), 'read' (for the backend read function), 'arrow_to_pandas' (for controlling the arguments passed to convert from a `pyarrow.Table.to_pandas()`) Examples -------- >>> df = dd.read_parquet('s3://bucket/my-parquet-data') # doctest: +SKIP See Also -------- to_parquet """ if isinstance(columns, str): df = read_parquet( path, [columns], filters, categories, index, storage_options, engine, gather_statistics, ) return df[columns] if columns is not None: columns = list(columns) name = "read-parquet-" + tokenize( path, columns, filters, categories, index, storage_options, engine, gather_statistics, ) if isinstance(engine, str): engine = get_engine(engine) if hasattr(path, "name"): path = stringify_path(path) 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 auto_index_allowed = False if index is None: # User is allowing auto-detected index auto_index_allowed = True if index and isinstance(index, str): index = [index] meta, statistics, parts, index = engine.read_metadata( fs, paths, categories=categories, index=index, gather_statistics=gather_statistics, filters=filters, split_row_groups=split_row_groups, **kwargs, ) # Parse dataset statistics from metadata (if available) parts, divisions, index, index_in_columns = process_statistics( parts, statistics, filters, index, chunksize) # Account for index and columns arguments. # Modify `meta` dataframe accordingly meta, index, columns = set_index_columns(meta, index, columns, index_in_columns, auto_index_allowed) if meta.index.name == NONE_LABEL: meta.index.name = None subgraph = BlockwiseParquet(name, engine, fs, meta, columns, index, parts, kwargs) # Set the index that was previously treated as a column if index_in_columns: meta = meta.set_index(index) if meta.index.name == NONE_LABEL: meta.index.name = None if len(divisions) < 2: # empty dataframe - just use meta subgraph = {(name, 0): meta} divisions = (None, None) return new_dd_object(subgraph, name, meta, divisions)
def _groupby_to_disk( ddf, write_func, col_groups, agg_cols, agg_list, out_path, freq_limit, tree_width, on_host, stat_name="categories", concat_groups=False, name_sep="_", max_size=None, nbuckets=None, ): if not col_groups: return {} if concat_groups: if agg_list and agg_list != ["count"]: raise ValueError( "Cannot use concat_groups=True with aggregations other than count" ) if agg_cols: raise ValueError( "Cannot aggregate continuous-column stats with concat_groups=True" ) # Update tree_width tw = {} for col in col_groups: col = [col] if isinstance(col, str) else col if isinstance(col, tuple): col = list(col) col_str = _make_name(*col, sep=name_sep) if tree_width is None: tw[col_str] = 8 elif isinstance(tree_width, int): tw[col_str] = tree_width else: tw[col_str] = tree_width.get(col_str, None) or 8 tree_width = tw # Make dedicated output directory for the categories fs = get_fs_token_paths(out_path)[0] out_path = fs.sep.join([out_path, stat_name]) fs.mkdirs(out_path, exist_ok=True) dsk = {} token = tokenize(ddf, col_groups, out_path, freq_limit, tree_width, on_host) level_1_name = "level_1-" + token split_name = "split-" + token level_2_name = "level_2-" + token level_3_name = "level_3-" + token finalize_labels_name = stat_name + "-" + token for p in range(ddf.npartitions): dsk[(level_1_name, p)] = ( _top_level_groupby, (ddf._name, p), col_groups, tree_width, agg_cols, agg_list, on_host, concat_groups, name_sep, ) k = 0 for c, col in enumerate(col_groups): col = [col] if isinstance(col, str) else col col_str = _make_name(*col, sep=name_sep) for s in range(tree_width[col_str]): dsk[(split_name, p, c, s)] = (getitem, (level_1_name, p), k) k += 1 col_groups_str = [] for c, col in enumerate(col_groups): col = [col] if isinstance(col, str) else col col_str = _make_name(*col, sep=name_sep) col_groups_str.append(col_str) freq_limit_val = None if freq_limit: freq_limit_val = freq_limit[col_str] if isinstance( freq_limit, dict) else freq_limit for s in range(tree_width[col_str]): dsk[(level_2_name, c, s)] = ( _mid_level_groupby, [(split_name, p, c, s) for p in range(ddf.npartitions)], col, agg_cols, agg_list, freq_limit_val, on_host, concat_groups, name_sep, max_size, ) dsk[(level_3_name, c)] = ( write_func, [(level_2_name, c, s) for s in range(tree_width[col_str])], out_path, col, on_host, concat_groups, name_sep, max_size, nbuckets, ) dsk[finalize_labels_name] = ( _finish_labels, [(level_3_name, c) for c, col in enumerate(col_groups)], col_groups_str, ) graph = HighLevelGraph.from_collections(finalize_labels_name, dsk, dependencies=[ddf]) return graph, finalize_labels_name
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).difference(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 read_orc( path, engine="pyarrow", columns=None, index=None, split_stripes=1, aggregate_files=None, storage_options=None, ): """Read dataframe from ORC file(s) Parameters ---------- path: str or list(str) Location of file(s), which can be a full URL with protocol specifier, and may include glob character if a single string. engine: 'pyarrow' or ORCEngine Backend ORC engine to use for IO. Default is "pyarrow". columns: None or list(str) Columns to load. If None, loads all. index: str Column name to set as index. split_stripes: int or False Maximum number of ORC stripes to include in each output-DataFrame partition. Use False to specify a 1-to-1 mapping between files and partitions. Default is 1. aggregate_files : bool, default False Whether distinct file paths may be aggregated into the same output partition. A setting of True means that any two file paths may be aggregated into the same output partition, while False means that inter-file aggregation is prohibited. storage_options: None or dict Further parameters to pass to the bytes backend. Returns ------- Dask.DataFrame (even if there is only one column) Examples -------- >>> df = dd.read_orc('https://github.com/apache/orc/raw/' ... 'master/examples/demo-11-zlib.orc') # doctest: +SKIP """ # Get engine engine = _get_engine(engine) # Process file path(s) storage_options = storage_options or {} fs, fs_token, paths = get_fs_token_paths(path, mode="rb", storage_options=storage_options) # Let backend engine generate a list of parts # from the ORC metadata. The backend should also # return the schema and DataFrame-collection metadata parts, schema, meta = engine.read_metadata( fs, paths, columns, index, split_stripes, aggregate_files, ) # Construct the output collection with from_map return from_map( ORCFunctionWrapper(fs, columns, schema, engine, index), parts, meta=meta, divisions=[None] * (len(parts) + 1), label="read-orc", token=tokenize(fs_token, path, columns), enforce_metadata=False, )
def read_parquet( path, columns=None, filters=None, categories=None, index=None, storage_options=None, engine="auto", gather_statistics=None, split_row_groups=None, read_from_paths=None, chunksize=None, **kwargs, ): """ Read a Parquet file into a Dask DataFrame This reads a directory of Parquet data into a Dask.dataframe, one file per partition. It selects the index among the sorted columns if any exist. Parameters ---------- path : string or list Source directory for data, or path(s) to individual parquet files. Prefix with a protocol like ``s3://`` to read from alternative filesystems. To read from multiple files you can pass a globstring or a list of paths, with the caveat that they must all have the same protocol. columns : string, list or None (default) Field name(s) to read in as columns in the output. By default all non-index fields will be read (as determined by the pandas parquet metadata, if present). Provide a single field name instead of a list to read in the data as a Series. filters : Union[List[Tuple[str, str, Any]], List[List[Tuple[str, str, Any]]]] List of filters to apply, like ``[[('x', '=', 0), ...], ...]``. Using this argument will NOT result in row-wise filtering of the final partitions unless ``engine="pyarrow-dataset"`` is also specified. For other engines, filtering is only performed at the partition level, i.e., to prevent the loading of some row-groups and/or files. For the "pyarrow" engines, 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 (preferred for "pyarrow") List[List[Tuple]] notation. Note that the "fastparquet" engine does not currently support DNF for the filtering of partitioned columns (List[Tuple] is required). index : string, list, False or None (default) Field name(s) to use as the output frame index. By default will be inferred from the pandas parquet file metadata (if present). Use False to read all fields as columns. categories : list, dict or None For any fields listed here, if the parquet encoding is Dictionary, the column will be created with dtype category. Use only if it is guaranteed that the column is encoded as dictionary in all row-groups. If a list, assumes up to 2**16-1 labels; if a dict, specify the number of labels expected; if None, will load categories automatically for data written by dask/fastparquet, not otherwise. storage_options : dict Key/value pairs to be passed on to the file-system backend, if any. engine : str, default 'auto' Parquet reader library to use. Options include: 'auto', 'fastparquet', 'pyarrow', 'pyarrow-dataset', and 'pyarrow-legacy'. Defaults to 'auto', which selects the FastParquetEngine if fastparquet is installed (and ArrowLegacyEngine otherwise). If 'pyarrow-dataset' is specified, the ArrowDatasetEngine (which leverages the pyarrow.dataset API) will be used for newer PyArrow versions (>=1.0.0). If 'pyarrow' or 'pyarrow-legacy' are specified, the ArrowLegacyEngine will be used (which leverages the pyarrow.parquet.ParquetDataset API). NOTE: 'pyarrow-dataset' enables row-wise filtering, but requires pyarrow>=1.0. The behavior of 'pyarrow' will most likely change to ArrowDatasetEngine in a future release, and the 'pyarrow-legacy' option will be deprecated once the ParquetDataset API is deprecated. gather_statistics : bool or None (default). Gather the statistics for each dataset partition. By default, this will only be done if the _metadata file is available. Otherwise, statistics will only be gathered if True, because the footer of every file will be parsed (which is very slow on some systems). split_row_groups : bool or int Default is True if a _metadata file is available or if the dataset is composed of a single file (otherwise defult is False). If True, then each output dataframe partition will correspond to a single parquet-file row-group. If False, each partition will correspond to a complete file. If a positive integer value is given, each dataframe partition will correspond to that number of parquet row-groups (or fewer). Only the "pyarrow" engine supports this argument. read_from_paths : bool or None (default) Only used by ``ArrowDatasetEngine`` when ``filters`` are specified. Determines whether the engine should avoid inserting large pyarrow (``ParquetFileFragment``) objects in the task graph. If this option is True, ``read_partition`` will need to regenerate the appropriate fragment object from the path and row-group IDs. This will reduce the size of the task graph, but will add minor overhead to ``read_partition``. By default (None), ``ArrowDatasetEngine`` will set this option to ``False`` when there are filters. chunksize : int, str The target task partition size. If set, consecutive row-groups from the same file will be aggregated into the same output partition until the aggregate size reaches this value. **kwargs: dict (of dicts) Passthrough key-word arguments for read backend. The top-level keys correspond to the appropriate operation type, and the second level corresponds to the kwargs that will be passed on to the underlying ``pyarrow`` or ``fastparquet`` function. Supported top-level keys: 'dataset' (for opening a ``pyarrow`` dataset), 'file' (for opening a ``fastparquet`` ``ParquetFile``), 'read' (for the backend read function), 'arrow_to_pandas' (for controlling the arguments passed to convert from a ``pyarrow.Table.to_pandas()``) Examples -------- >>> df = dd.read_parquet('s3://bucket/my-parquet-data') # doctest: +SKIP See Also -------- to_parquet """ if isinstance(columns, str): df = read_parquet( path, columns=[columns], filters=filters, categories=categories, index=index, storage_options=storage_options, engine=engine, gather_statistics=gather_statistics, split_row_groups=split_row_groups, read_from_paths=read_from_paths, chunksize=chunksize, ) return df[columns] if columns is not None: columns = list(columns) name = "read-parquet-" + tokenize( path, columns, filters, categories, index, storage_options, engine, gather_statistics, split_row_groups, read_from_paths, chunksize, ) if isinstance(engine, str): engine = get_engine(engine) if hasattr(path, "name"): path = stringify_path(path) 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 auto_index_allowed = False if index is None: # User is allowing auto-detected index auto_index_allowed = True if index and isinstance(index, str): index = [index] read_metadata_result = engine.read_metadata( fs, paths, categories=categories, index=index, gather_statistics=True if chunksize else gather_statistics, filters=filters, split_row_groups=split_row_groups, read_from_paths=read_from_paths, **kwargs, ) # In the future, we may want to give the engine the # option to return a dedicated element for `common_kwargs`. # However, to avoid breaking the API, we just embed this # data in the first element of `parts` for now. # The logic below is inteded to handle backward and forward # compatibility with a user-defined engine. meta, statistics, parts, index = read_metadata_result[:4] common_kwargs = {} if len(read_metadata_result) > 4: # Engine may return common_kwargs as a separate element common_kwargs = read_metadata_result[4] elif len(parts): # If the engine does not return a dedicated # common_kwargs argument, it may be stored in # the first element of `parts` common_kwargs = parts[0].pop("common_kwargs", {}) # Parse dataset statistics from metadata (if available) parts, divisions, index, index_in_columns = process_statistics( parts, statistics, filters, index, chunksize) # Account for index and columns arguments. # Modify `meta` dataframe accordingly meta, index, columns = set_index_columns(meta, index, columns, index_in_columns, auto_index_allowed) if meta.index.name == NONE_LABEL: meta.index.name = None subgraph = ParquetSubgraph( name, engine, fs, meta, columns, index, parts, kwargs, common_kwargs=common_kwargs, ) # Set the index that was previously treated as a column if index_in_columns: meta = meta.set_index(index) if meta.index.name == NONE_LABEL: meta.index.name = None if len(divisions) < 2: # empty dataframe - just use meta subgraph = {(name, 0): meta} divisions = (None, None) return new_dd_object(subgraph, name, meta, divisions)