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_files( read_paths: List[str], fs: Union["pyarrow.fs.FileSystem", _S3FileSystemWrapper], ) -> Iterable[Block]: logger.debug(f"Reading {len(read_paths)} files.") if isinstance(fs, _S3FileSystemWrapper): fs = fs.unwrap() ctx = DatasetContext.get_current() output_buffer = BlockOutputBuffer( block_udf=_block_udf, target_max_block_size=ctx.target_max_block_size ) for read_path in read_paths: compression = open_stream_args.pop("compression", None) if compression is None: import pyarrow as pa try: # If no compression manually given, try to detect # compression codec from path. compression = pa.Codec.detect(read_path).name except (ValueError, TypeError): # Arrow's compression inference on the file path # doesn't work for Snappy, so we double-check ourselves. import pathlib suffix = pathlib.Path(read_path).suffix if suffix and suffix[1:] == "snappy": compression = "snappy" else: compression = None if compression == "snappy": # Pass Snappy compression as a reader arg, so datasource subclasses # can manually handle streaming decompression in # self._read_stream(). reader_args["compression"] = compression reader_args["filesystem"] = fs elif compression is not None: # Non-Snappy compression, pass as open_input_stream() arg so Arrow # can take care of streaming decompression for us. open_stream_args["compression"] = compression with self._open_input_source(fs, read_path, **open_stream_args) as f: for data in read_stream(f, read_path, **reader_args): output_buffer.add_block(data) if output_buffer.has_next(): yield output_buffer.next() output_buffer.finalize() if output_buffer.has_next(): yield output_buffer.next()
def read_files( read_paths: List[str], fs: Union["pyarrow.fs.FileSystem", _S3FileSystemWrapper], ) -> Iterable[Block]: logger.debug(f"Reading {len(read_paths)} files.") if isinstance(fs, _S3FileSystemWrapper): fs = fs.unwrap() ctx = DatasetContext.get_current() output_buffer = BlockOutputBuffer( block_udf=_block_udf, target_max_block_size=ctx.target_max_block_size ) for read_path in read_paths: with fs.open_input_stream(read_path, **open_stream_args) as f: for data in read_stream(f, read_path, **reader_args): output_buffer.add_block(data) if output_buffer.has_next(): yield output_buffer.next() output_buffer.finalize() if output_buffer.has_next(): yield output_buffer.next()