def read_pieces(serialized_pieces: List[str]): # Implicitly trigger S3 subsystem initialization by importing # pyarrow.fs. import pyarrow.fs # noqa: F401 # Deserialize after loading the filesystem class. pieces: List["pyarrow._dataset.ParquetFileFragment"] = [ cloudpickle.loads(p) for p in serialized_pieces ] import pyarrow as pa from pyarrow.dataset import _get_partition_keys logger.debug(f"Reading {len(pieces)} parquet pieces") use_threads = reader_args.pop("use_threads", False) tables = [] for piece in pieces: table = piece.to_table( use_threads=use_threads, columns=columns, schema=schema, **reader_args) part = _get_partition_keys(piece.partition_expression) if part: for col, value in part.items(): table = table.set_column( table.schema.get_field_index(col), col, pa.array([value] * len(table))) tables.append(table) if len(tables) > 1: table = pa.concat_tables(tables) else: table = tables[0] return table
def read_pieces(serialized_pieces: str) -> Iterator[pa.Table]: # Implicitly trigger S3 subsystem initialization by importing # pyarrow.fs. import pyarrow.fs # noqa: F401 # Deserialize after loading the filesystem class. try: _register_parquet_file_fragment_serialization() pieces: List[ "pyarrow._dataset.ParquetFileFragment" ] = cloudpickle.loads(serialized_pieces) finally: _deregister_parquet_file_fragment_serialization() # Ensure that we're reading at least one dataset fragment. assert len(pieces) > 0 from pyarrow.dataset import _get_partition_keys ctx = DatasetContext.get_current() output_buffer = BlockOutputBuffer( block_udf=_block_udf, target_max_block_size=ctx.target_max_block_size ) logger.debug(f"Reading {len(pieces)} parquet pieces") use_threads = reader_args.pop("use_threads", False) for piece in pieces: part = _get_partition_keys(piece.partition_expression) batches = piece.to_batches( use_threads=use_threads, columns=columns, schema=schema, batch_size=PARQUET_READER_ROW_BATCH_SIZE, **reader_args, ) for batch in batches: table = pyarrow.Table.from_batches([batch], schema=schema) if part: for col, value in part.items(): table = table.set_column( table.schema.get_field_index(col), col, pa.array([value] * len(table)), ) # If the table is empty, drop it. if table.num_rows > 0: output_buffer.add_block(table) if output_buffer.has_next(): yield output_buffer.next() output_buffer.finalize() if output_buffer.has_next(): yield output_buffer.next()
def _read_pieces( block_udf, reader_args, columns, schema, serialized_pieces: List[_SerializedPiece] ) -> Iterator["pyarrow.Table"]: # Deserialize after loading the filesystem class. pieces: List[ "pyarrow._dataset.ParquetFileFragment"] = _deserialize_pieces_with_retry( serialized_pieces) # Ensure that we're reading at least one dataset fragment. assert len(pieces) > 0 import pyarrow as pa from pyarrow.dataset import _get_partition_keys ctx = DatasetContext.get_current() output_buffer = BlockOutputBuffer( block_udf=block_udf, target_max_block_size=ctx.target_max_block_size, ) logger.debug(f"Reading {len(pieces)} parquet pieces") use_threads = reader_args.pop("use_threads", False) for piece in pieces: part = _get_partition_keys(piece.partition_expression) batches = piece.to_batches( use_threads=use_threads, columns=columns, schema=schema, batch_size=PARQUET_READER_ROW_BATCH_SIZE, **reader_args, ) for batch in batches: table = pa.Table.from_batches([batch], schema=schema) if part: for col, value in part.items(): table = table.set_column( table.schema.get_field_index(col), col, pa.array([value] * len(table)), ) # If the table is empty, drop it. if table.num_rows > 0: output_buffer.add_block(table) if output_buffer.has_next(): yield output_buffer.next() output_buffer.finalize() if output_buffer.has_next(): yield output_buffer.next()
def read_pieces(serialized_pieces: List[str]): # Implicitly trigger S3 subsystem initialization by importing # pyarrow.fs. import pyarrow.fs # noqa: F401 # Deserialize after loading the filesystem class. pieces: List["pyarrow._dataset.ParquetFileFragment"] = [ cloudpickle.loads(p) for p in serialized_pieces ] # Ensure that we're reading at least one dataset fragment. assert len(pieces) > 0 from pyarrow.dataset import _get_partition_keys logger.debug(f"Reading {len(pieces)} parquet pieces") use_threads = reader_args.pop("use_threads", False) tables = [] for piece in pieces: table = piece.to_table(use_threads=use_threads, columns=columns, schema=schema, **reader_args) part = _get_partition_keys(piece.partition_expression) if part: for col, value in part.items(): table = table.set_column( table.schema.get_field_index(col), col, pa.array([value] * len(table))) # If the table is empty, drop it. if table.num_rows > 0: tables.append(table) if len(tables) > 1: table = pa.concat_tables(tables, promote=True) elif len(tables) == 1: table = tables[0] if _block_udf is not None: table = _block_udf(table) # If len(tables) == 0, all fragments were empty, and we return the # empty table from the last fragment. return table
def _process_dataset( paths, fs, filters=None, row_groups=None, categorical_partitions=True, ): # Returns: # file_list - Expanded/filtered list of paths # row_groups - Filtered list of row-group selections # partition_keys - list of partition keys for each file # partition_categories - Categories for each partition # The general purpose of this function is to (1) expand # directory input into a list of paths (using the pyarrow # dataset API), (2) to apply row-group filters, and (3) # to discover directory-partitioning information # Deal with case that the user passed in a directory name file_list = paths if len(paths) == 1 and ioutils.is_directory(paths[0]): paths = ioutils.stringify_pathlike(paths[0]) # Convert filters to ds.Expression if filters is not None: filters = pq._filters_to_expression(filters) # Initialize ds.FilesystemDataset dataset = ds.dataset( paths, filesystem=fs, format="parquet", partitioning="hive", ) file_list = dataset.files if len(file_list) == 0: raise FileNotFoundError(f"{paths} could not be resolved to any files") # Deal with directory partitioning # Get all partition keys (without filters) partition_categories = defaultdict(list) file_fragment = None for file_fragment in dataset.get_fragments(): keys = ds._get_partition_keys(file_fragment.partition_expression) if not (keys or partition_categories): # Bail - This is not a directory-partitioned dataset break for k, v in keys.items(): if v not in partition_categories[k]: partition_categories[k].append(v) if not categorical_partitions: # Bail - We don't need to discover all categories. # We only need to save the partition keys from this # first `file_fragment` break if partition_categories and file_fragment is not None: # Check/correct order of `categories` using last file_frag, # because `_get_partition_keys` does NOT preserve the # partition-hierarchy order of the keys. cat_keys = [ part.split("=")[0] for part in file_fragment.path.split(fs.sep) if "=" in part ] if set(partition_categories) == set(cat_keys): partition_categories = { k: partition_categories[k] for k in cat_keys if k in partition_categories } # If we do not have partitioned data and # are not filtering, we can return here if filters is None and not partition_categories: return file_list, row_groups, [], {} # Record initial row_groups input row_groups_map = {} if row_groups is not None: # Make sure paths and row_groups map 1:1 # and save the initial mapping if len(paths) != len(file_list): raise ValueError( "Cannot specify a row_group selection for a directory path.") row_groups_map = {path: rgs for path, rgs in zip(paths, row_groups)} # Apply filters and discover partition columns partition_keys = [] if partition_categories or filters is not None: file_list = [] if filters is not None: row_groups = [] for file_fragment in dataset.get_fragments(filter=filters): path = file_fragment.path # Extract hive-partition keys, and make sure they # are orederd the same as they are in `partition_categories` if partition_categories: raw_keys = ds._get_partition_keys( file_fragment.partition_expression) partition_keys.append([(name, raw_keys[name]) for name in partition_categories.keys() ]) # Apply row-group filtering selection = row_groups_map.get(path, None) if selection is not None or filters is not None: filtered_row_groups = [ rg_info.id for rg_fragment in file_fragment.split_by_row_group( filters, schema=dataset.schema, ) for rg_info in rg_fragment.row_groups ] file_list.append(path) if filters is not None: if selection is None: row_groups.append(filtered_row_groups) else: row_groups.append([ rg_id for rg_id in filtered_row_groups if rg_id in selection ]) return ( file_list, row_groups, partition_keys, partition_categories if categorical_partitions else {}, )
def _read_paths( cls, paths, fs, columns=None, row_groups=None, strings_to_categorical=None, partitions=None, partitioning=None, partition_keys=None, open_file_options=None, **kwargs, ): # Simplify row_groups if all None if row_groups == [None for path in paths]: row_groups = None with ExitStack() as stack: # Non-local filesystem handling paths_or_fobs = paths if not _is_local_filesystem(fs): paths_or_fobs = _open_remote_files( paths_or_fobs, fs, context_stack=stack, **_default_open_file_options(open_file_options, columns, row_groups), ) # Use cudf to read in data df = cudf.read_parquet( paths_or_fobs, engine="cudf", columns=columns, row_groups=row_groups if row_groups else None, strings_to_categorical=strings_to_categorical, **kwargs, ) if partitions and partition_keys is None: # Use `HivePartitioning` by default partitioning = partitioning or {"obj": pa_ds.HivePartitioning} ds = pa_ds.dataset( paths, filesystem=fs, format="parquet", partitioning=partitioning["obj"].discover( *partitioning.get("args", []), **partitioning.get("kwargs", {}), ), ) frag = next(ds.get_fragments()) if frag: # Extract hive-partition keys, and make sure they # are ordered the same as they are in `partitions` raw_keys = pa_ds._get_partition_keys(frag.partition_expression) partition_keys = [(hive_part.name, raw_keys[hive_part.name]) for hive_part in partitions] if partition_keys: if partitions is None: raise ValueError("Must pass partition sets") for i, (name, index2) in enumerate(partition_keys): # Build the column from `codes` directly # (since the category is often a larger dtype) codes = as_column( partitions[i].keys.index(index2), length=len(df), ) df[name] = build_categorical_column( categories=partitions[i].keys, codes=codes, size=codes.size, offset=codes.offset, ordered=False, ) return df