From 8070c7a33780d345446c54720a2418682dc62be3 Mon Sep 17 00:00:00 2001 From: AlenkaF Date: Tue, 5 Dec 2023 10:29:19 +0100 Subject: [PATCH 01/29] Initial commit --- docs/source/python/parquet.rst | 38 +- python/pyarrow/parquet/core.py | 1162 +++-------------- python/pyarrow/tests/parquet/__init__.py | 3 - python/pyarrow/tests/parquet/common.py | 38 +- python/pyarrow/tests/parquet/test_basic.py | 280 ++-- .../parquet/test_compliant_nested_type.py | 21 +- .../pyarrow/tests/parquet/test_data_types.py | 108 +- python/pyarrow/tests/parquet/test_dataset.py | 927 +++---------- python/pyarrow/tests/parquet/test_datetime.py | 16 +- python/pyarrow/tests/parquet/test_pandas.py | 210 +-- .../tests/parquet/test_parquet_file.py | 24 +- .../tests/parquet/test_parquet_writer.py | 27 +- python/pyarrow/tests/test_dataset.py | 54 +- python/pyarrow/tests/test_hdfs.py | 13 +- 14 files changed, 626 insertions(+), 2295 deletions(-) diff --git a/docs/source/python/parquet.rst b/docs/source/python/parquet.rst index 85a9674a689ca..aed76cc42adfd 100644 --- a/docs/source/python/parquet.rst +++ b/docs/source/python/parquet.rst @@ -511,36 +511,20 @@ from a remote filesystem into a pandas dataframe you may need to run ``sort_index`` to maintain row ordering (as long as the ``preserve_index`` option was enabled on write). -.. note:: - - The ParquetDataset is being reimplemented based on the new generic Dataset - API (see the :ref:`dataset` docs for an overview). This is not yet the - default, but can already be enabled by passing the ``use_legacy_dataset=False`` - keyword to :class:`ParquetDataset` or :func:`read_table`:: - - pq.ParquetDataset('dataset_name/', use_legacy_dataset=False) - - Enabling this gives the following new features: - - - Filtering on all columns (using row group statistics) instead of only on - the partition keys. - - More fine-grained partitioning: support for a directory partitioning scheme - in addition to the Hive-like partitioning (e.g. "/2019/11/15/" instead of - "/year=2019/month=11/day=15/"), and the ability to specify a schema for - the partition keys. - - General performance improvement and bug fixes. +Other features: - It also has the following changes in behaviour: +- Filtering on all columns (using row group statistics) instead of only on + the partition keys. +- Fine-grained partitioning: support for a directory partitioning scheme + in addition to the Hive-like partitioning (e.g. "/2019/11/15/" instead of + "/year=2019/month=11/day=15/"), and the ability to specify a schema for + the partition keys. - - The partition keys need to be explicitly included in the ``columns`` - keyword when you want to include them in the result while reading a - subset of the columns +Note: - This new implementation is already enabled in ``read_table``, and in the - future, this will be turned on by default for ``ParquetDataset``. The new - implementation does not yet cover all existing ParquetDataset features (e.g. - specifying the ``metadata``, or the ``pieces`` property API). Feedback is - very welcome. +- The partition keys need to be explicitly included in the ``columns`` + keyword when you want to include them in the result while reading a + subset of the columns Using with Spark diff --git a/python/pyarrow/parquet/core.py b/python/pyarrow/parquet/core.py index db22eb3293c86..e374109b529a5 100644 --- a/python/pyarrow/parquet/core.py +++ b/python/pyarrow/parquet/core.py @@ -1145,182 +1145,6 @@ def _get_pandas_index_columns(keyvalues): # Metadata container providing instructions about reading a single Parquet # file, possibly part of a partitioned dataset - -class ParquetDatasetPiece: - """ - DEPRECATED: A single chunk of a potentially larger Parquet dataset to read. - - The arguments will indicate to read either a single row group or all row - groups, and whether to add partition keys to the resulting pyarrow.Table. - - .. deprecated:: 5.0 - Directly constructing a ``ParquetDatasetPiece`` is deprecated, as well - as accessing the pieces of a ``ParquetDataset`` object. Specify - ``use_legacy_dataset=False`` when constructing the ``ParquetDataset`` - and use the ``ParquetDataset.fragments`` attribute instead. - - Parameters - ---------- - path : str or pathlib.Path - Path to file in the file system where this piece is located. - open_file_func : callable - Function to use for obtaining file handle to dataset piece. - file_options : dict - Options - row_group : int, default None - Row group to load. By default, reads all row groups. - partition_keys : list of tuples - Two-element tuples of ``(column name, ordinal index)``. - """ - - def __init__(self, path, open_file_func=partial(open, mode='rb'), - file_options=None, row_group=None, partition_keys=None): - warnings.warn( - "ParquetDatasetPiece is deprecated as of pyarrow 5.0.0 and will " - "be removed in a future version.", - FutureWarning, stacklevel=2) - self._init( - path, open_file_func, file_options, row_group, partition_keys) - - @staticmethod - def _create(path, open_file_func=partial(open, mode='rb'), - file_options=None, row_group=None, partition_keys=None): - self = ParquetDatasetPiece.__new__(ParquetDatasetPiece) - self._init( - path, open_file_func, file_options, row_group, partition_keys) - return self - - def _init(self, path, open_file_func, file_options, row_group, - partition_keys): - self.path = _stringify_path(path) - self.open_file_func = open_file_func - self.row_group = row_group - self.partition_keys = partition_keys or [] - self.file_options = file_options or {} - - def __eq__(self, other): - if not isinstance(other, ParquetDatasetPiece): - return False - return (self.path == other.path and - self.row_group == other.row_group and - self.partition_keys == other.partition_keys) - - def __repr__(self): - return ('{}({!r}, row_group={!r}, partition_keys={!r})' - .format(type(self).__name__, self.path, - self.row_group, - self.partition_keys)) - - def __str__(self): - result = '' - - if len(self.partition_keys) > 0: - partition_str = ', '.join('{}={}'.format(name, index) - for name, index in self.partition_keys) - result += 'partition[{}] '.format(partition_str) - - result += self.path - - if self.row_group is not None: - result += ' | row_group={}'.format(self.row_group) - - return result - - def get_metadata(self): - """ - Return the file's metadata. - - Returns - ------- - metadata : FileMetaData - The file's metadata - """ - with self.open() as parquet: - return parquet.metadata - - def open(self): - """ - Return instance of ParquetFile. - """ - reader = self.open_file_func(self.path) - if not isinstance(reader, ParquetFile): - reader = ParquetFile(reader, **self.file_options) - - # ensure reader knows it's responsible for closing source - # since we opened the source here internally. - reader._close_source = True - return reader - - def read(self, columns=None, use_threads=True, partitions=None, - file=None, use_pandas_metadata=False): - """ - Read this piece as a pyarrow.Table. - - Parameters - ---------- - columns : list of column names, default None - use_threads : bool, default True - Perform multi-threaded column reads. - partitions : ParquetPartitions, default None - file : file-like object - Passed to ParquetFile. - use_pandas_metadata : bool - If pandas metadata should be used or not. - - Returns - ------- - table : pyarrow.Table - The piece as a pyarrow.Table. - """ - if self.open_file_func is not None: - reader = self.open() - elif file is not None: - reader = ParquetFile(file, **self.file_options) - else: - # try to read the local path - reader = ParquetFile(self.path, **self.file_options) - - options = dict(columns=columns, - use_threads=use_threads, - use_pandas_metadata=use_pandas_metadata) - - if self.row_group is not None: - table = reader.read_row_group(self.row_group, **options) - else: - table = reader.read(**options) - - if len(self.partition_keys) > 0: - if partitions is None: - raise ValueError('Must pass partition sets') - - # Here, the index is the categorical code of the partition where - # this piece is located. Suppose we had - # - # /foo=a/0.parq - # /foo=b/0.parq - # /foo=c/0.parq - # - # Then we assign a=0, b=1, c=2. And the resulting Table pieces will - # have a DictionaryArray column named foo having the constant index - # value as indicated. The distinct categories of the partition have - # been computed in the ParquetManifest - for i, (name, index) in enumerate(self.partition_keys): - # The partition code is the same for all values in this piece - indices = np.full(len(table), index, dtype='i4') - - # This is set of all partition values, computed as part of the - # manifest, so ['a', 'b', 'c'] as in our example above. - dictionary = partitions.levels[i].dictionary - - arr = pa.DictionaryArray.from_arrays(indices, dictionary) - table = table.append_column(name, arr) - - # To ParquetFile the source looked like it was already open, so won't - # actually close it without overriding. - reader.close(force=True) - return table - - class PartitionSet: """ A data structure for cataloguing the observed Parquet partitions at a @@ -1596,13 +1420,6 @@ def _parse_partition(self, dirname): raise NotImplementedError('partition schema: {}' .format(self.partition_scheme)) - def _push_pieces(self, files, part_keys): - self.pieces.extend([ - ParquetDatasetPiece._create(path, partition_keys=part_keys, - open_file_func=self.open_file_func) - for path in files - ]) - def _parse_hive_partition(value): if '=' not in value: @@ -1684,12 +1501,11 @@ def _open_dataset_file(dataset, path, meta=None): >>> import pyarrow.parquet as pq >>> pq.write_to_dataset(table, root_path='dataset_name', -... partition_cols=['year'], -... use_legacy_dataset=False) +... partition_cols=['year']) create a ParquetDataset object from the dataset source: ->>> dataset = pq.ParquetDataset('dataset_name/', use_legacy_dataset=False) +>>> dataset = pq.ParquetDataset('dataset_name/') and read the data: @@ -1704,7 +1520,7 @@ def _open_dataset_file(dataset, path, meta=None): create a ParquetDataset object with filter: ->>> dataset = pq.ParquetDataset('dataset_name/', use_legacy_dataset=False, +>>> dataset = pq.ParquetDataset('dataset_name/', ... filters=[('n_legs','=',4)]) >>> dataset.read().to_pandas() n_legs animal year @@ -1740,10 +1556,7 @@ class ParquetDataset: Rows which do not match the filter predicate will be removed from scanned data. Partition keys embedded in a nested directory structure will be exploited to avoid loading files at all if they contain no matching rows. - If `use_legacy_dataset` is True, filters can only reference partition - keys and only a hive-style directory structure is supported. When - setting `use_legacy_dataset` to False, also within-file level filtering - and different partitioning schemes are supported. + Within-file level filtering and different partitioning schemes are supported. {1} metadata_nthreads : int, default 1 @@ -1751,16 +1564,10 @@ class ParquetDataset: dataset metadata. Increasing this is helpful to read partitioned datasets. {0} -use_legacy_dataset : bool, default False - Set to False to enable the new code path (using the - new Arrow Dataset API). Among other things, this allows to pass - `filters` for all columns and not only the partition keys, enables - different partitioning schemes, etc. pre_buffer : bool, default True Coalesce and issue file reads in parallel to improve performance on high-latency filesystems (e.g. S3, GCS). If True, Arrow will use a - background I/O thread pool. This option is only supported for - use_legacy_dataset=False. If using a filesystem layer that itself + background I/O thread pool. If using a filesystem layer that itself performs readahead (e.g. fsspec's S3FS), disable readahead for best results. Set to False if you want to prioritize minimal memory usage over maximum speed. @@ -1788,206 +1595,31 @@ def __new__(cls, path_or_paths=None, filesystem=None, schema=None, metadata=None, split_row_groups=False, validate_schema=True, filters=None, metadata_nthreads=None, read_dictionary=None, memory_map=False, buffer_size=0, partitioning="hive", - use_legacy_dataset=None, pre_buffer=True, + pre_buffer=True, coerce_int96_timestamp_unit=None, thrift_string_size_limit=None, thrift_container_size_limit=None, page_checksum_verification=False): - extra_msg = "" - if use_legacy_dataset is None: - # if an old filesystem is passed -> still use to old implementation - if isinstance(filesystem, legacyfs.FileSystem): - use_legacy_dataset = True - extra_msg = ( - " The legacy behaviour was still chosen because a " - "deprecated 'pyarrow.filesystem' filesystem was specified " - "(use the filesystems from pyarrow.fs instead)." - ) - # otherwise the default is already False - else: - use_legacy_dataset = False - - if not use_legacy_dataset: - return _ParquetDatasetV2( - path_or_paths, filesystem=filesystem, - filters=filters, - partitioning=partitioning, - read_dictionary=read_dictionary, - memory_map=memory_map, - buffer_size=buffer_size, - pre_buffer=pre_buffer, - coerce_int96_timestamp_unit=coerce_int96_timestamp_unit, - # unsupported keywords - schema=schema, metadata=metadata, - split_row_groups=split_row_groups, - validate_schema=validate_schema, - metadata_nthreads=metadata_nthreads, - thrift_string_size_limit=thrift_string_size_limit, - thrift_container_size_limit=thrift_container_size_limit, - page_checksum_verification=page_checksum_verification, - ) - warnings.warn( - "Passing 'use_legacy_dataset=True' to get the legacy behaviour is " - "deprecated as of pyarrow 11.0.0, and the legacy implementation " - "will be removed in a future version." + extra_msg, - FutureWarning, stacklevel=2) - self = object.__new__(cls) - return self - - def __init__(self, path_or_paths, filesystem=None, schema=None, - metadata=None, split_row_groups=False, validate_schema=True, - filters=None, metadata_nthreads=None, read_dictionary=None, - memory_map=False, buffer_size=0, partitioning="hive", - use_legacy_dataset=None, pre_buffer=True, - coerce_int96_timestamp_unit=None, - thrift_string_size_limit=None, - thrift_container_size_limit=None, - page_checksum_verification=False): - if partitioning != "hive": - raise ValueError( - 'Only "hive" for hive-like partitioning is supported when ' - 'using use_legacy_dataset=True') - if metadata_nthreads is not None: - warnings.warn( - "Specifying the 'metadata_nthreads' argument is deprecated as " - "of pyarrow 8.0.0, and the argument will be removed in a " - "future version", - FutureWarning, stacklevel=2, - ) - else: - metadata_nthreads = 1 - - self._ds_metadata = _ParquetDatasetMetadata() - a_path = path_or_paths - if isinstance(a_path, list): - a_path = a_path[0] - - self._ds_metadata.fs, _ = _get_filesystem_and_path(filesystem, a_path) - if isinstance(path_or_paths, list): - self.paths = [_parse_uri(path) for path in path_or_paths] - else: - self.paths = _parse_uri(path_or_paths) - - self._ds_metadata.read_dictionary = read_dictionary - self._ds_metadata.memory_map = memory_map - self._ds_metadata.buffer_size = buffer_size - - (self._pieces, - self._partitions, - self._common_metadata_path, - self._metadata_path) = _make_manifest( - path_or_paths, self._fs, metadata_nthreads=metadata_nthreads, - open_file_func=partial(_open_dataset_file, self._ds_metadata) + return _ParquetDatasetV2( + path_or_paths, filesystem=filesystem, + filters=filters, + partitioning=partitioning, + read_dictionary=read_dictionary, + memory_map=memory_map, + buffer_size=buffer_size, + pre_buffer=pre_buffer, + coerce_int96_timestamp_unit=coerce_int96_timestamp_unit, + # unsupported keywords + schema=schema, metadata=metadata, + split_row_groups=split_row_groups, + validate_schema=validate_schema, + metadata_nthreads=metadata_nthreads, + thrift_string_size_limit=thrift_string_size_limit, + thrift_container_size_limit=thrift_container_size_limit, + page_checksum_verification=page_checksum_verification, ) - if self._common_metadata_path is not None: - with self._fs.open(self._common_metadata_path) as f: - self._ds_metadata.common_metadata = read_metadata( - f, - memory_map=memory_map - ) - else: - self._ds_metadata.common_metadata = None - - if metadata is not None: - warnings.warn( - "Specifying the 'metadata' argument with 'use_legacy_dataset=" - "True' is deprecated as of pyarrow 8.0.0.", - FutureWarning, stacklevel=2) - - if metadata is None and self._metadata_path is not None: - with self._fs.open(self._metadata_path) as f: - self._metadata = read_metadata(f, memory_map=memory_map) - else: - self._metadata = metadata - - if schema is not None: - warnings.warn( - "Specifying the 'schema' argument with 'use_legacy_dataset=" - "True' is deprecated as of pyarrow 8.0.0. You can still " - "specify it in combination with 'use_legacy_dataset=False', " - "but in that case you need to specify a pyarrow.Schema " - "instead of a ParquetSchema.", - FutureWarning, stacklevel=2) - self._schema = schema - - self.split_row_groups = split_row_groups - - if split_row_groups: - raise NotImplementedError("split_row_groups not yet implemented") - - if filters is not None: - if hasattr(filters, "cast"): - raise TypeError( - "Expressions as filter not supported for legacy dataset") - filters = _check_filters(filters) - self._filter(filters) - - if validate_schema: - self.validate_schemas() - - def __getnewargs_ex__(self): - # when creating a new instance while unpickling, force to use the - # legacy code path to create a ParquetDataset instance - # instead of a _ParquetDatasetV2 instance - return ((), dict(use_legacy_dataset=True)) - - def equals(self, other): - if not isinstance(other, ParquetDataset): - raise TypeError('`other` must be an instance of ParquetDataset') - - if self._fs.__class__ != other._fs.__class__: - return False - for prop in ('paths', '_pieces', '_partitions', - '_common_metadata_path', '_metadata_path', - '_common_metadata', '_metadata', '_schema', - 'split_row_groups'): - if getattr(self, prop) != getattr(other, prop): - return False - for prop in ('memory_map', 'buffer_size'): - if ( - getattr(self._ds_metadata, prop) != - getattr(other._ds_metadata, prop) - ): - return False - - return True - - def __eq__(self, other): - try: - return self.equals(other) - except TypeError: - return NotImplemented - - def validate_schemas(self): - if self._metadata is None and self._schema is None: - if self._common_metadata is not None: - self._schema = self._common_metadata.schema - else: - self._schema = self._pieces[0].get_metadata().schema - elif self._schema is None: - self._schema = self._metadata.schema - - # Verify schemas are all compatible - dataset_schema = self._schema.to_arrow_schema() - # Exclude the partition columns from the schema, they are provided - # by the path, not the DatasetPiece - if self._partitions is not None: - for partition_name in self._partitions.partition_names: - if dataset_schema.get_field_index(partition_name) != -1: - field_idx = dataset_schema.get_field_index(partition_name) - dataset_schema = dataset_schema.remove(field_idx) - - for piece in self._pieces: - file_metadata = piece.get_metadata() - file_schema = file_metadata.schema.to_arrow_schema() - if not dataset_schema.equals(file_schema, check_metadata=False): - raise ValueError('Schema in {!s} was different. \n' - '{!s}\n\nvs\n\n{!s}' - .format(piece, file_schema, - dataset_schema)) - def read(self, columns=None, use_threads=True, use_pandas_metadata=False): """ Read multiple Parquet files as a single pyarrow.Table. @@ -2017,10 +1649,8 @@ def read(self, columns=None, use_threads=True, use_pandas_metadata=False): ... "Brittle stars", "Centipede"]}) >>> import pyarrow.parquet as pq >>> pq.write_to_dataset(table, root_path='dataset_name_read', - ... partition_cols=['year'], - ... use_legacy_dataset=False) - >>> dataset = pq.ParquetDataset('dataset_name_read/', - ... use_legacy_dataset=False) + ... partition_cols=['year']) + >>> dataset = pq.ParquetDataset('dataset_name_read/') Read multiple Parquet files as a single pyarrow.Table: @@ -2082,8 +1712,7 @@ def read_pandas(self, **kwargs): >>> table = pa.Table.from_pandas(df) >>> import pyarrow.parquet as pq >>> pq.write_table(table, 'table.parquet') - >>> dataset = pq.ParquetDataset('table.parquet', - ... use_legacy_dataset=False) + >>> dataset = pq.ParquetDataset('table.parquet') Read dataset including pandas metadata: @@ -2120,263 +1749,6 @@ def all_filters_accept(piece): self._pieces = [p for p in self._pieces if all_filters_accept(p)] - @property - def pieces(self): - """ - DEPRECATED - """ - warnings.warn( - _DEPR_MSG.format( - "ParquetDataset.pieces", - " Specify 'use_legacy_dataset=False' while constructing the " - "ParquetDataset, and then use the '.fragments' attribute " - "instead."), - FutureWarning, stacklevel=2) - return self._pieces - - @property - def partitions(self): - """ - DEPRECATED - """ - warnings.warn( - _DEPR_MSG.format( - "ParquetDataset.partitions", - " Specify 'use_legacy_dataset=False' while constructing the " - "ParquetDataset, and then use the '.partitioning' attribute " - "instead."), - FutureWarning, stacklevel=2) - return self._partitions - - @property - def schema(self): - warnings.warn( - _DEPR_MSG.format( - "ParquetDataset.schema", - " Specify 'use_legacy_dataset=False' while constructing the " - "ParquetDataset, and then use the '.schema' attribute " - "instead (which will return an Arrow schema instead of a " - "Parquet schema)."), - FutureWarning, stacklevel=2) - return self._schema - - @property - def memory_map(self): - """ - DEPRECATED - """ - warnings.warn( - _DEPR_MSG.format("ParquetDataset.memory_map", ""), - FutureWarning, stacklevel=2) - return self._ds_metadata.memory_map - - @property - def read_dictionary(self): - """ - DEPRECATED - """ - warnings.warn( - _DEPR_MSG.format("ParquetDataset.read_dictionary", ""), - FutureWarning, stacklevel=2) - return self._ds_metadata.read_dictionary - - @property - def buffer_size(self): - """ - DEPRECATED - """ - warnings.warn( - _DEPR_MSG.format("ParquetDataset.buffer_size", ""), - FutureWarning, stacklevel=2) - return self._ds_metadata.buffer_size - - _fs = property( - operator.attrgetter('_ds_metadata.fs') - ) - - @property - def fs(self): - """ - DEPRECATED - """ - warnings.warn( - _DEPR_MSG.format( - "ParquetDataset.fs", - " Specify 'use_legacy_dataset=False' while constructing the " - "ParquetDataset, and then use the '.filesystem' attribute " - "instead."), - FutureWarning, stacklevel=2) - return self._ds_metadata.fs - - @property - def metadata(self): - """ - DEPRECATED - """ - warnings.warn( - _DEPR_MSG.format("ParquetDataset.metadata", ""), - FutureWarning, stacklevel=2) - return self._metadata - - @property - def metadata_path(self): - """ - DEPRECATED - """ - warnings.warn( - _DEPR_MSG.format("ParquetDataset.metadata_path", ""), - FutureWarning, stacklevel=2) - return self._metadata_path - - @property - def common_metadata_path(self): - """ - DEPRECATED - """ - warnings.warn( - _DEPR_MSG.format("ParquetDataset.common_metadata_path", ""), - FutureWarning, stacklevel=2) - return self._common_metadata_path - - _common_metadata = property( - operator.attrgetter('_ds_metadata.common_metadata') - ) - - @property - def common_metadata(self): - """ - DEPRECATED - """ - warnings.warn( - _DEPR_MSG.format("ParquetDataset.common_metadata", ""), - FutureWarning, stacklevel=2) - return self._ds_metadata.common_metadata - - @property - def fragments(self): - """ - A list of the Dataset source fragments or pieces with absolute - file paths. To use this property set 'use_legacy_dataset=False' - while constructing ParquetDataset object. - - Examples - -------- - Generate an example dataset: - - >>> import pyarrow as pa - >>> table = pa.table({'year': [2020, 2022, 2021, 2022, 2019, 2021], - ... 'n_legs': [2, 2, 4, 4, 5, 100], - ... 'animal': ["Flamingo", "Parrot", "Dog", "Horse", - ... "Brittle stars", "Centipede"]}) - >>> import pyarrow.parquet as pq - >>> pq.write_to_dataset(table, root_path='dataset_name_fragments', - ... partition_cols=['year'], - ... use_legacy_dataset=False) - >>> dataset = pq.ParquetDataset('dataset_name_fragments/', - ... use_legacy_dataset=False) - - List the fragments: - - >>> dataset.fragments - [>> import pyarrow as pa - >>> table = pa.table({'year': [2020, 2022, 2021, 2022, 2019, 2021], - ... 'n_legs': [2, 2, 4, 4, 5, 100], - ... 'animal': ["Flamingo", "Parrot", "Dog", "Horse", - ... "Brittle stars", "Centipede"]}) - >>> import pyarrow.parquet as pq - >>> pq.write_to_dataset(table, root_path='dataset_name_files', - ... partition_cols=['year'], - ... use_legacy_dataset=False) - >>> dataset = pq.ParquetDataset('dataset_name_files/', - ... use_legacy_dataset=False) - - List the files: - - >>> dataset.files - ['dataset_name_files/year=2019/...-0.parquet', ... - """ - raise NotImplementedError( - "To use this property set 'use_legacy_dataset=False' while " - "constructing the ParquetDataset") - - @property - def filesystem(self): - """ - The filesystem type of the Dataset source. - To use this property set 'use_legacy_dataset=False' - while constructing ParquetDataset object. - """ - raise NotImplementedError( - "To use this property set 'use_legacy_dataset=False' while " - "constructing the ParquetDataset") - - @property - def partitioning(self): - """ - The partitioning of the Dataset source, if discovered. - To use this property set 'use_legacy_dataset=False' - while constructing ParquetDataset object. - """ - raise NotImplementedError( - "To use this property set 'use_legacy_dataset=False' while " - "constructing the ParquetDataset") - - -def _make_manifest(path_or_paths, fs, pathsep='/', metadata_nthreads=1, - open_file_func=None): - partitions = None - common_metadata_path = None - metadata_path = None - - if isinstance(path_or_paths, list) and len(path_or_paths) == 1: - # Dask passes a directory as a list of length 1 - path_or_paths = path_or_paths[0] - - if _is_path_like(path_or_paths) and fs.isdir(path_or_paths): - manifest = ParquetManifest(path_or_paths, filesystem=fs, - open_file_func=open_file_func, - pathsep=getattr(fs, "pathsep", "/"), - metadata_nthreads=metadata_nthreads) - common_metadata_path = manifest.common_metadata_path - metadata_path = manifest.metadata_path - pieces = manifest.pieces - partitions = manifest.partitions - else: - if not isinstance(path_or_paths, list): - path_or_paths = [path_or_paths] - - # List of paths - if len(path_or_paths) == 0: - raise ValueError('Must pass at least one file path') - - pieces = [] - for path in path_or_paths: - if not fs.isfile(path): - raise OSError('Passed non-file path: {}' - .format(path)) - piece = ParquetDatasetPiece._create( - path, open_file_func=open_file_func) - pieces.append(piece) - - return pieces, partitions, common_metadata_path, metadata_path - def _is_local_file_system(fs): return isinstance(fs, LocalFileSystem) or isinstance( @@ -2399,12 +1771,11 @@ class _ParquetDatasetV2: ... "Brittle stars", "Centipede"]}) >>> import pyarrow.parquet as pq >>> pq.write_to_dataset(table, root_path='dataset_v2', - ... partition_cols=['year'], - ... use_legacy_dataset=False) + ... partition_cols=['year']) create a ParquetDataset object from the dataset source: - >>> dataset = pq.ParquetDataset('dataset_v2/', use_legacy_dataset=False) + >>> dataset = pq.ParquetDataset('dataset_v2/') and read the data: @@ -2420,8 +1791,7 @@ class _ParquetDatasetV2: create a ParquetDataset object with filter: >>> dataset = pq.ParquetDataset('dataset_v2/', - ... filters=[('n_legs','=',4)], - ... use_legacy_dataset=False) + ... filters=[('n_legs','=',4)]) >>> dataset.read().to_pandas() n_legs animal year 0 4 Dog 2021 @@ -2533,11 +1903,6 @@ def __init__(self, path_or_paths, filesystem=None, *, filters=None, ignore_prefixes=ignore_prefixes) def equals(self, other): - if isinstance(other, ParquetDataset): - raise TypeError( - "`other` must be an instance of ParquetDataset constructed " - "with `use_legacy_dataset=False`" - ) if not isinstance(other, _ParquetDatasetV2): raise TypeError('`other` must be an instance of ParquetDataset') @@ -2569,10 +1934,8 @@ def schema(self): ... "Brittle stars", "Centipede"]}) >>> import pyarrow.parquet as pq >>> pq.write_to_dataset(table, root_path='dataset_v2_schema', - ... partition_cols=['year'], - ... use_legacy_dataset=False) - >>> dataset = pq.ParquetDataset('dataset_v2_schema/', - ... use_legacy_dataset=False) + ... partition_cols=['year']) + >>> dataset = pq.ParquetDataset('dataset_v2_schema/') Read the schema: @@ -2591,8 +1954,7 @@ def read(self, columns=None, use_threads=True, use_pandas_metadata=False): ---------- columns : List[str] Names of columns to read from the dataset. The partition fields - are not automatically included (in contrast to when setting - ``use_legacy_dataset=True``). + are not automatically included. use_threads : bool, default True Perform multi-threaded column reads. use_pandas_metadata : bool, default False @@ -2615,10 +1977,8 @@ def read(self, columns=None, use_threads=True, use_pandas_metadata=False): ... "Brittle stars", "Centipede"]}) >>> import pyarrow.parquet as pq >>> pq.write_to_dataset(table, root_path='dataset_v2_read', - ... partition_cols=['year'], - ... use_legacy_dataset=False) - >>> dataset = pq.ParquetDataset('dataset_v2_read/', - ... use_legacy_dataset=False) + ... partition_cols=['year']) + >>> dataset = pq.ParquetDataset('dataset_v2_read/') Read the dataset: @@ -2702,8 +2062,7 @@ def read_pandas(self, **kwargs): >>> table = pa.Table.from_pandas(df) >>> import pyarrow.parquet as pq >>> pq.write_table(table, 'table_V2.parquet') - >>> dataset = pq.ParquetDataset('table_V2.parquet', - ... use_legacy_dataset=False) + >>> dataset = pq.ParquetDataset('table_V2.parquet') Read the dataset with pandas metadata: @@ -2743,10 +2102,8 @@ def fragments(self): ... "Brittle stars", "Centipede"]}) >>> import pyarrow.parquet as pq >>> pq.write_to_dataset(table, root_path='dataset_v2_fragments', - ... partition_cols=['year'], - ... use_legacy_dataset=False) - >>> dataset = pq.ParquetDataset('dataset_v2_fragments/', - ... use_legacy_dataset=False) + ... partition_cols=['year']) + >>> dataset = pq.ParquetDataset('dataset_v2_fragments/') List the fragments: @@ -2771,10 +2128,8 @@ def files(self): ... "Brittle stars", "Centipede"]}) >>> import pyarrow.parquet as pq >>> pq.write_to_dataset(table, root_path='dataset_v2_files', - ... partition_cols=['year'], - ... use_legacy_dataset=False) - >>> dataset = pq.ParquetDataset('dataset_v2_files/', - ... use_legacy_dataset=False) + ... partition_cols=['year']) + >>> dataset = pq.ParquetDataset('dataset_v2_files/') List the files: @@ -2829,30 +2184,19 @@ def partitioning(self): Rows which do not match the filter predicate will be removed from scanned data. Partition keys embedded in a nested directory structure will be exploited to avoid loading files at all if they contain no matching rows. - If `use_legacy_dataset` is True, filters can only reference partition - keys and only a hive-style directory structure is supported. When - setting `use_legacy_dataset` to False, also within-file level filtering - and different partitioning schemes are supported. + Within-file level filtering and different partitioning schemes are supported. {3} -use_legacy_dataset : bool, default False - By default, `read_table` uses the new Arrow Datasets API since - pyarrow 1.0.0. Among other things, this allows to pass `filters` - for all columns and not only the partition keys, enables - different partitioning schemes, etc. - Set to True to use the legacy behaviour (this option is deprecated, - and the legacy implementation will be removed in a future version). ignore_prefixes : list, optional Files matching any of these prefixes will be ignored by the - discovery process if use_legacy_dataset=False. + discovery process. This is matched to the basename of a path. By default this is ['.', '_']. Note that discovery happens only if a directory is passed as source. pre_buffer : bool, default True Coalesce and issue file reads in parallel to improve performance on high-latency filesystems (e.g. S3). If True, Arrow will use a - background I/O thread pool. This option is only supported for - use_legacy_dataset=False. If using a filesystem layer that itself + background I/O thread pool. If using a filesystem layer that itself performs readahead (e.g. fsspec's S3FS), disable readahead for best results. coerce_int96_timestamp_unit : str, default None @@ -2961,129 +2305,77 @@ def partitioning(self): """ -def read_table(source, *, columns=None, use_threads=True, metadata=None, +def read_table(source, *, columns=None, use_threads=True, schema=None, use_pandas_metadata=False, read_dictionary=None, memory_map=False, buffer_size=0, partitioning="hive", - filesystem=None, filters=None, use_legacy_dataset=False, + filesystem=None, filters=None, ignore_prefixes=None, pre_buffer=True, coerce_int96_timestamp_unit=None, decryption_properties=None, thrift_string_size_limit=None, thrift_container_size_limit=None, - page_checksum_verification=False): - if not use_legacy_dataset: - if metadata is not None: - raise ValueError( - "The 'metadata' keyword is no longer supported with the new " - "datasets-based implementation. Specify " - "'use_legacy_dataset=True' to temporarily recover the old " - "behaviour." - ) - try: - dataset = _ParquetDatasetV2( - source, - schema=schema, - filesystem=filesystem, - partitioning=partitioning, - memory_map=memory_map, - read_dictionary=read_dictionary, - buffer_size=buffer_size, - filters=filters, - ignore_prefixes=ignore_prefixes, - pre_buffer=pre_buffer, - coerce_int96_timestamp_unit=coerce_int96_timestamp_unit, - thrift_string_size_limit=thrift_string_size_limit, - thrift_container_size_limit=thrift_container_size_limit, - page_checksum_verification=page_checksum_verification, - ) - except ImportError: - # fall back on ParquetFile for simple cases when pyarrow.dataset - # module is not available - if filters is not None: - raise ValueError( - "the 'filters' keyword is not supported when the " - "pyarrow.dataset module is not available" - ) - if partitioning != "hive": - raise ValueError( - "the 'partitioning' keyword is not supported when the " - "pyarrow.dataset module is not available" - ) - if schema is not None: - raise ValueError( - "the 'schema' argument is not supported when the " - "pyarrow.dataset module is not available" - ) - filesystem, path = _resolve_filesystem_and_path(source, filesystem) - if filesystem is not None: - source = filesystem.open_input_file(path) - # TODO test that source is not a directory or a list - dataset = ParquetFile( - source, metadata=metadata, read_dictionary=read_dictionary, - memory_map=memory_map, buffer_size=buffer_size, - pre_buffer=pre_buffer, - coerce_int96_timestamp_unit=coerce_int96_timestamp_unit, - decryption_properties=decryption_properties, - thrift_string_size_limit=thrift_string_size_limit, - thrift_container_size_limit=thrift_container_size_limit, - page_checksum_verification=page_checksum_verification, - ) - - return dataset.read(columns=columns, use_threads=use_threads, - use_pandas_metadata=use_pandas_metadata) + page_checksum_verification=False, metadata=None,): - warnings.warn( - "Passing 'use_legacy_dataset=True' to get the legacy behaviour is " - "deprecated as of pyarrow 8.0.0, and the legacy implementation will " - "be removed in a future version.", - FutureWarning, stacklevel=2) - - if ignore_prefixes is not None: + if metadata is not None: raise ValueError( - "The 'ignore_prefixes' keyword is only supported when " - "use_legacy_dataset=False") - - if page_checksum_verification: - raise ValueError( - "The 'page_checksum_verification' keyword is only supported when " - "use_legacy_dataset=False") - - if schema is not None: - raise ValueError( - "The 'schema' argument is only supported when " - "use_legacy_dataset=False") - - if _is_path_like(source): - with warnings.catch_warnings(): - # Suppress second warning from ParquetDataset constructor - warnings.filterwarnings( - "ignore", "Passing 'use_legacy_dataset", FutureWarning) - pf = ParquetDataset( - source, metadata=metadata, memory_map=memory_map, - read_dictionary=read_dictionary, - buffer_size=buffer_size, - filesystem=filesystem, filters=filters, - partitioning=partitioning, - coerce_int96_timestamp_unit=coerce_int96_timestamp_unit, - use_legacy_dataset=True, - ) - else: - pf = ParquetFile( - source, metadata=metadata, - read_dictionary=read_dictionary, + "The 'metadata' keyword is no longer supported with the new " + "datasets-based implementation." + ) + try: + dataset = _ParquetDatasetV2( + source, + schema=schema, + filesystem=filesystem, + partitioning=partitioning, memory_map=memory_map, + read_dictionary=read_dictionary, buffer_size=buffer_size, + filters=filters, + ignore_prefixes=ignore_prefixes, + pre_buffer=pre_buffer, + coerce_int96_timestamp_unit=coerce_int96_timestamp_unit, + thrift_string_size_limit=thrift_string_size_limit, + thrift_container_size_limit=thrift_container_size_limit, + page_checksum_verification=page_checksum_verification, + ) + except ImportError: + # fall back on ParquetFile for simple cases when pyarrow.dataset + # module is not available + if filters is not None: + raise ValueError( + "the 'filters' keyword is not supported when the " + "pyarrow.dataset module is not available" + ) + if partitioning != "hive": + raise ValueError( + "the 'partitioning' keyword is not supported when the " + "pyarrow.dataset module is not available" + ) + if schema is not None: + raise ValueError( + "the 'schema' argument is not supported when the " + "pyarrow.dataset module is not available" + ) + filesystem, path = _resolve_filesystem_and_path(source, filesystem) + if filesystem is not None: + source = filesystem.open_input_file(path) + # TODO test that source is not a directory or a list + dataset = ParquetFile( + source, metadata=metadata, read_dictionary=read_dictionary, + memory_map=memory_map, buffer_size=buffer_size, + pre_buffer=pre_buffer, coerce_int96_timestamp_unit=coerce_int96_timestamp_unit, - decryption_properties=decryption_properties + decryption_properties=decryption_properties, + thrift_string_size_limit=thrift_string_size_limit, + thrift_container_size_limit=thrift_container_size_limit, + page_checksum_verification=page_checksum_verification, ) - return pf.read(columns=columns, use_threads=use_threads, - use_pandas_metadata=use_pandas_metadata) + return dataset.read(columns=columns, use_threads=use_threads, + use_pandas_metadata=use_pandas_metadata) -read_table.__doc__ = _read_table_docstring.format( - """Read a Table from Parquet format -Note: starting with pyarrow 1.0, the default for `use_legacy_dataset` is -switched to False.""", +read_table.__doc__ = _read_table_docstring.format( + """Read a Table from Parquet format""", "\n".join(("""use_pandas_metadata : bool, default False If True and file has custom pandas schema metadata, ensure that index columns are also loaded.""", _read_docstring_common)), @@ -3234,13 +2526,12 @@ def _mkdir_if_not_exists(fs, path): def write_to_dataset(table, root_path, partition_cols=None, partition_filename_cb=None, filesystem=None, - use_legacy_dataset=None, schema=None, + schema=None, partitioning=None, basename_template=None, use_threads=None, file_visitor=None, existing_data_behavior=None, **kwargs): - """Wrapper around dataset.write_dataset (when use_legacy_dataset=False) or - parquet.write_table (when use_legacy_dataset=True) for writing a Table to + """Wrapper around dataset.write_dataset for writing a Table to Parquet format by partitions. For each combination of partition columns and values, a subdirectories are created in the following @@ -3262,45 +2553,29 @@ def write_to_dataset(table, root_path, partition_cols=None, ---------- table : pyarrow.Table root_path : str, pathlib.Path - The root directory of the dataset + The root directory of the dataset. partition_cols : list, Column names by which to partition the dataset. - Columns are partitioned in the order they are given - partition_filename_cb : callable, - A callback function that takes the partition key(s) as an argument - and allow you to override the partition filename. If nothing is - passed, the filename will consist of a uuid. - This option is only supported for use_legacy_dataset=True. - When use_legacy_dataset=None and this option is specified, - use_legacy_dataset will be set to True. + Columns are partitioned in the order they are given. filesystem : FileSystem, default None If nothing passed, will be inferred based on path. Path will try to be found in the local on-disk filesystem otherwise it will be parsed as an URI to determine the filesystem. - use_legacy_dataset : bool - Default is False. Set to True to use the legacy behaviour - (this option is deprecated, and the legacy implementation will be - removed in a future version). The legacy implementation still - supports the `partition_filename_cb` keyword but is less efficient - when using partition columns. schema : Schema, optional - This option is only supported for use_legacy_dataset=False. + This Schema of the dataset. partitioning : Partitioning or list[str], optional The partitioning scheme specified with the ``pyarrow.dataset.partitioning()`` function or a list of field names. When providing a list of field names, you can use ``partitioning_flavor`` to drive which partitioning type should be used. - This option is only supported for use_legacy_dataset=False. basename_template : str, optional A template string used to generate basenames of written data files. The token '{i}' will be replaced with an automatically incremented integer. If not specified, it defaults to "guid-{i}.parquet". - This option is only supported for use_legacy_dataset=False. use_threads : bool, default True Write files in parallel. If enabled, then maximum parallelism will be used determined by the number of available CPU cores. - This option is only supported for use_legacy_dataset=False. file_visitor : function If set, this function will be called with a WrittenFile instance for each file created during the call. This object will have both @@ -3321,7 +2596,6 @@ def write_to_dataset(table, root_path, partition_cols=None, def file_visitor(written_file): visited_paths.append(written_file.path) - This option is only supported for use_legacy_dataset=False. existing_data_behavior : 'overwrite_or_ignore' | 'error' | \ 'delete_matching' Controls how the dataset will handle data that already exists in @@ -3339,15 +2613,11 @@ def file_visitor(written_file): dataset. The first time each partition directory is encountered the entire directory will be deleted. This allows you to overwrite old partitions completely. - This option is only supported for use_legacy_dataset=False. **kwargs : dict, - When use_legacy_dataset=False, used as additional kwargs for - `dataset.write_dataset` function for matching kwargs, and remainder to - `ParquetFileFormat.make_write_options`. See the docstring - of `write_table` and `dataset.write_dataset` for the available options. - When use_legacy_dataset=True, used as additional kwargs for - `parquet.write_table` function (See docstring for `write_table` - or `ParquetWriter` for more information). + Used as additional kwargs for `dataset.write_dataset` function for + matching kwargs, and remainder to `ParquetFileFormat.make_write_options`. + See the docstring of `write_table` and `dataset.write_dataset` for + the available options. Using `metadata_collector` in kwargs allows one to collect the file metadata instances of dataset pieces. The file paths in the ColumnChunkMetaData will be set relative to `root_path`. @@ -3367,194 +2637,85 @@ def file_visitor(written_file): >>> import pyarrow.parquet as pq >>> pq.write_to_dataset(table, root_path='dataset_name_3', ... partition_cols=['year']) - >>> pq.ParquetDataset('dataset_name_3', use_legacy_dataset=False).files + >>> pq.ParquetDataset('dataset_name_3').files ['dataset_name_3/year=2019/...-0.parquet', ... Write a single Parquet file into the root folder: >>> pq.write_to_dataset(table, root_path='dataset_name_4') - >>> pq.ParquetDataset('dataset_name_4/', use_legacy_dataset=False).files + >>> pq.ParquetDataset('dataset_name_4/').files ['dataset_name_4/...-0.parquet'] """ - # Choose the implementation - if use_legacy_dataset is None: - # if partition_filename_cb is specified -> - # default to the old implementation - if partition_filename_cb: - use_legacy_dataset = True - # otherwise the default is False - else: - use_legacy_dataset = False + + metadata_collector = kwargs.pop('metadata_collector', None) # Check for conflicting keywords - msg_confl_0 = ( - "The '{0}' argument is not supported by use_legacy_dataset={2}. " - "Use only '{1}' instead." - ) - msg_confl_1 = ( - "The '{1}' argument is not supported by use_legacy_dataset={2}. " + msg_confl = ( + "The '{1}' argument is not supported. " "Use only '{0}' instead." ) - msg_confl = msg_confl_0 if use_legacy_dataset else msg_confl_1 if partition_filename_cb is not None and basename_template is not None: raise ValueError(msg_confl.format("basename_template", - "partition_filename_cb", - use_legacy_dataset)) + "partition_filename_cb")) if partition_cols is not None and partitioning is not None: raise ValueError(msg_confl.format("partitioning", - "partition_cols", - use_legacy_dataset)) + "partition_cols")) metadata_collector = kwargs.pop('metadata_collector', None) if metadata_collector is not None and file_visitor is not None: raise ValueError(msg_confl.format("file_visitor", - "metadata_collector", - use_legacy_dataset)) + "metadata_collector")) + + import pyarrow.dataset as ds - # New dataset implementation - if not use_legacy_dataset: - import pyarrow.dataset as ds + # extract write_dataset specific options + # reset assumed to go to make_write_options + write_dataset_kwargs = dict() + for key in inspect.signature(ds.write_dataset).parameters: + if key in kwargs: + write_dataset_kwargs[key] = kwargs.pop(key) + write_dataset_kwargs['max_rows_per_group'] = kwargs.pop( + 'row_group_size', kwargs.pop("chunk_size", None) + ) + # raise for unsupported keywords + msg = ( + "The '{}' argument is not supported with the new dataset " + "implementation." + ) + if metadata_collector is not None: + def file_visitor(written_file): + metadata_collector.append(written_file.metadata) + if partition_filename_cb is not None: + raise ValueError(msg.format("partition_filename_cb")) - # extract write_dataset specific options - # reset assumed to go to make_write_options - write_dataset_kwargs = dict() - for key in inspect.signature(ds.write_dataset).parameters: - if key in kwargs: - write_dataset_kwargs[key] = kwargs.pop(key) - write_dataset_kwargs['max_rows_per_group'] = kwargs.pop( - 'row_group_size', kwargs.pop("chunk_size", None) - ) - # raise for unsupported keywords - msg = ( - "The '{}' argument is not supported with the new dataset " - "implementation." - ) - if metadata_collector is not None: - def file_visitor(written_file): - metadata_collector.append(written_file.metadata) - if partition_filename_cb is not None: - raise ValueError(msg.format("partition_filename_cb")) + # map format arguments + parquet_format = ds.ParquetFileFormat() + write_options = parquet_format.make_write_options(**kwargs) - # map format arguments - parquet_format = ds.ParquetFileFormat() - write_options = parquet_format.make_write_options(**kwargs) + # map old filesystems to new one + if filesystem is not None: + filesystem = _ensure_filesystem(filesystem) - # map old filesystems to new one - if filesystem is not None: - filesystem = _ensure_filesystem(filesystem) - - if partition_cols: - part_schema = table.select(partition_cols).schema - partitioning = ds.partitioning(part_schema, flavor="hive") - - if basename_template is None: - basename_template = guid() + '-{i}.parquet' - - if existing_data_behavior is None: - existing_data_behavior = 'overwrite_or_ignore' - - ds.write_dataset( - table, root_path, filesystem=filesystem, - format=parquet_format, file_options=write_options, schema=schema, - partitioning=partitioning, use_threads=use_threads, - file_visitor=file_visitor, - basename_template=basename_template, - existing_data_behavior=existing_data_behavior, - **write_dataset_kwargs) - return - - # warnings and errors when using legacy implementation - if use_legacy_dataset: - warnings.warn( - "Passing 'use_legacy_dataset=True' to get the legacy behaviour is " - "deprecated as of pyarrow 8.0.0, and the legacy implementation " - "will be removed in a future version.", - FutureWarning, stacklevel=2) - msg2 = ( - "The '{}' argument is not supported with the legacy " - "implementation. To use this argument specify " - "'use_legacy_dataset=False' while constructing the " - "ParquetDataset." - ) - if schema is not None: - raise ValueError(msg2.format("schema")) - if partitioning is not None: - raise ValueError(msg2.format("partitioning")) - if use_threads is not None: - raise ValueError(msg2.format("use_threads")) - if file_visitor is not None: - raise ValueError(msg2.format("file_visitor")) - if existing_data_behavior is not None: - raise ValueError(msg2.format("existing_data_behavior")) - if basename_template is not None: - raise ValueError(msg2.format("basename_template")) - if partition_filename_cb is not None: - warnings.warn( - _DEPR_MSG.format("partition_filename_cb", " Specify " - "'use_legacy_dataset=False' while constructing " - "the ParquetDataset, and then use the " - "'basename_template' parameter instead. For " - "usage see `pyarrow.dataset.write_dataset`"), - FutureWarning, stacklevel=2) + if partition_cols: + part_schema = table.select(partition_cols).schema + partitioning = ds.partitioning(part_schema, flavor="hive") - # Legacy implementation - fs, root_path = legacyfs.resolve_filesystem_and_path(root_path, filesystem) - - _mkdir_if_not_exists(fs, root_path) - - if partition_cols is not None and len(partition_cols) > 0: - df = table.to_pandas() - partition_keys = [df[col] for col in partition_cols] - data_df = df.drop(partition_cols, axis='columns') - data_cols = df.columns.drop(partition_cols) - if len(data_cols) == 0: - raise ValueError('No data left to save outside partition columns') - - subschema = table.schema - - # ARROW-2891: Ensure the output_schema is preserved when writing a - # partitioned dataset - for col in table.schema.names: - if col in partition_cols: - subschema = subschema.remove(subschema.get_field_index(col)) - - # ARROW-17829: avoid deprecation warnings for df.groupby - # https://github.com/pandas-dev/pandas/issues/42795 - if len(partition_keys) == 1: - partition_keys = partition_keys[0] - - for keys, subgroup in data_df.groupby(partition_keys, observed=True): - if not isinstance(keys, tuple): - keys = (keys,) - subdir = '/'.join( - ['{colname}={value}'.format(colname=name, value=val) - for name, val in zip(partition_cols, keys)]) - subtable = pa.Table.from_pandas(subgroup, schema=subschema, - safe=False) - _mkdir_if_not_exists(fs, '/'.join([root_path, subdir])) - if partition_filename_cb: - outfile = partition_filename_cb(keys) - else: - outfile = guid() + '.parquet' - relative_path = '/'.join([subdir, outfile]) - full_path = '/'.join([root_path, relative_path]) - with fs.open(full_path, 'wb') as f: - write_table(subtable, f, metadata_collector=metadata_collector, - **kwargs) - if metadata_collector is not None: - metadata_collector[-1].set_file_path(relative_path) - else: - if partition_filename_cb: - outfile = partition_filename_cb(None) - else: - outfile = guid() + '.parquet' - full_path = '/'.join([root_path, outfile]) - with fs.open(full_path, 'wb') as f: - write_table(table, f, metadata_collector=metadata_collector, - **kwargs) - if metadata_collector is not None: - metadata_collector[-1].set_file_path(outfile) + if basename_template is None: + basename_template = guid() + '-{i}.parquet' + + if existing_data_behavior is None: + existing_data_behavior = 'overwrite_or_ignore' + + ds.write_dataset( + table, root_path, filesystem=filesystem, + format=parquet_format, file_options=write_options, schema=schema, + partitioning=partitioning, use_threads=use_threads, + file_visitor=file_visitor, + basename_template=basename_template, + existing_data_behavior=existing_data_behavior, + **write_dataset_kwargs) + return def write_metadata(schema, where, metadata_collector=None, filesystem=None, @@ -3732,7 +2893,6 @@ def read_schema(where, memory_map=False, decryption_properties=None, "FileEncryptionProperties", "FileMetaData", "ParquetDataset", - "ParquetDatasetPiece", "ParquetFile", "ParquetLogicalType", "ParquetManifest", diff --git a/python/pyarrow/tests/parquet/__init__.py b/python/pyarrow/tests/parquet/__init__.py index 4c4e8240b8736..d08d67d2860f4 100644 --- a/python/pyarrow/tests/parquet/__init__.py +++ b/python/pyarrow/tests/parquet/__init__.py @@ -21,7 +21,4 @@ # Ignore these with pytest ... -m 'not parquet' pytestmark = [ pytest.mark.parquet, - pytest.mark.filterwarnings( - "ignore:Passing 'use_legacy_dataset=True':FutureWarning" - ), ] diff --git a/python/pyarrow/tests/parquet/common.py b/python/pyarrow/tests/parquet/common.py index 4401d3ca6bb75..776b743dded2f 100644 --- a/python/pyarrow/tests/parquet/common.py +++ b/python/pyarrow/tests/parquet/common.py @@ -23,26 +23,6 @@ import pyarrow as pa from pyarrow.tests import util -legacy_filter_mark = pytest.mark.filterwarnings( - "ignore:Passing 'use_legacy:FutureWarning" -) - -parametrize_legacy_dataset = pytest.mark.parametrize( - "use_legacy_dataset", - [pytest.param(True, marks=legacy_filter_mark), - pytest.param(False, marks=pytest.mark.dataset)] -) -parametrize_legacy_dataset_not_supported = pytest.mark.parametrize( - "use_legacy_dataset", - [pytest.param(True, marks=legacy_filter_mark), - pytest.param(False, marks=pytest.mark.skip)] -) -parametrize_legacy_dataset_fixed = pytest.mark.parametrize( - "use_legacy_dataset", - [pytest.param(True, marks=[pytest.mark.xfail, legacy_filter_mark]), - pytest.param(False, marks=pytest.mark.dataset)] -) - def _write_table(table, path, **kwargs): # So we see the ImportError somewhere @@ -65,19 +45,18 @@ def _read_table(*args, **kwargs): def _roundtrip_table(table, read_table_kwargs=None, - write_table_kwargs=None, use_legacy_dataset=False): + write_table_kwargs=None): read_table_kwargs = read_table_kwargs or {} write_table_kwargs = write_table_kwargs or {} writer = pa.BufferOutputStream() _write_table(table, writer, **write_table_kwargs) reader = pa.BufferReader(writer.getvalue()) - return _read_table(reader, use_legacy_dataset=use_legacy_dataset, - **read_table_kwargs) + return _read_table(reader, **read_table_kwargs) def _check_roundtrip(table, expected=None, read_table_kwargs=None, - use_legacy_dataset=False, **write_table_kwargs): + **write_table_kwargs): if expected is None: expected = table @@ -85,20 +64,17 @@ def _check_roundtrip(table, expected=None, read_table_kwargs=None, # intentionally check twice result = _roundtrip_table(table, read_table_kwargs=read_table_kwargs, - write_table_kwargs=write_table_kwargs, - use_legacy_dataset=use_legacy_dataset) + write_table_kwargs=write_table_kwargs) assert result.equals(expected) result = _roundtrip_table(result, read_table_kwargs=read_table_kwargs, - write_table_kwargs=write_table_kwargs, - use_legacy_dataset=use_legacy_dataset) + write_table_kwargs=write_table_kwargs) assert result.equals(expected) -def _roundtrip_pandas_dataframe(df, write_kwargs, use_legacy_dataset=False): +def _roundtrip_pandas_dataframe(df, write_kwargs): table = pa.Table.from_pandas(df) result = _roundtrip_table( - table, write_table_kwargs=write_kwargs, - use_legacy_dataset=use_legacy_dataset) + table, write_table_kwargs=write_kwargs) return result.to_pandas() diff --git a/python/pyarrow/tests/parquet/test_basic.py b/python/pyarrow/tests/parquet/test_basic.py index 83e6ebeb7a1fc..a76c2b900bfa3 100644 --- a/python/pyarrow/tests/parquet/test_basic.py +++ b/python/pyarrow/tests/parquet/test_basic.py @@ -28,7 +28,6 @@ from pyarrow.filesystem import LocalFileSystem, FileSystem from pyarrow.tests import util from pyarrow.tests.parquet.common import (_check_roundtrip, _roundtrip_table, - parametrize_legacy_dataset, _test_dataframe) try: @@ -63,21 +62,20 @@ def test_parquet_invalid_version(tempdir): data_page_version="2.2") -@parametrize_legacy_dataset -def test_set_data_page_size(use_legacy_dataset): +@pytest.mark.dataset +def test_set_data_page_size(): arr = pa.array([1, 2, 3] * 100000) t = pa.Table.from_arrays([arr], names=['f0']) # 128K, 512K page_sizes = [2 << 16, 2 << 18] for target_page_size in page_sizes: - _check_roundtrip(t, data_page_size=target_page_size, - use_legacy_dataset=use_legacy_dataset) + _check_roundtrip(t, data_page_size=target_page_size) +@pytest.mark.dataset @pytest.mark.pandas -@parametrize_legacy_dataset -def test_set_write_batch_size(use_legacy_dataset): +def test_set_write_batch_size(): df = _test_dataframe(100) table = pa.Table.from_pandas(df, preserve_index=False) @@ -86,9 +84,9 @@ def test_set_write_batch_size(use_legacy_dataset): ) +@pytest.mark.dataset @pytest.mark.pandas -@parametrize_legacy_dataset -def test_set_dictionary_pagesize_limit(use_legacy_dataset): +def test_set_dictionary_pagesize_limit(): df = _test_dataframe(100) table = pa.Table.from_pandas(df, preserve_index=False) @@ -100,9 +98,9 @@ def test_set_dictionary_pagesize_limit(use_legacy_dataset): data_page_size=10, version='2.4') +@pytest.mark.dataset @pytest.mark.pandas -@parametrize_legacy_dataset -def test_chunked_table_write(use_legacy_dataset): +def test_chunked_table_write(): # ARROW-232 tables = [] batch = pa.RecordBatch.from_pandas(alltypes_sample(size=10)) @@ -116,66 +114,61 @@ def test_chunked_table_write(use_legacy_dataset): for table in tables: _check_roundtrip( table, version='2.6', - use_legacy_dataset=use_legacy_dataset, data_page_version=data_page_version, use_dictionary=use_dictionary) @pytest.mark.pandas -@parametrize_legacy_dataset -def test_memory_map(tempdir, use_legacy_dataset): +@pytest.mark.dataset +def test_memory_map(tempdir): df = alltypes_sample(size=10) table = pa.Table.from_pandas(df) _check_roundtrip(table, read_table_kwargs={'memory_map': True}, - version='2.6', use_legacy_dataset=use_legacy_dataset) + version='2.6') filename = str(tempdir / 'tmp_file') with open(filename, 'wb') as f: _write_table(table, f, version='2.6') - table_read = pq.read_pandas(filename, memory_map=True, - use_legacy_dataset=use_legacy_dataset) + table_read = pq.read_pandas(filename, memory_map=True) assert table_read.equals(table) @pytest.mark.pandas -@parametrize_legacy_dataset -def test_enable_buffered_stream(tempdir, use_legacy_dataset): +@pytest.mark.dataset +def test_enable_buffered_stream(tempdir): df = alltypes_sample(size=10) table = pa.Table.from_pandas(df) _check_roundtrip(table, read_table_kwargs={'buffer_size': 1025}, - version='2.6', use_legacy_dataset=use_legacy_dataset) + version='2.6') filename = str(tempdir / 'tmp_file') with open(filename, 'wb') as f: _write_table(table, f, version='2.6') - table_read = pq.read_pandas(filename, buffer_size=4096, - use_legacy_dataset=use_legacy_dataset) + table_read = pq.read_pandas(filename, buffer_size=4096) assert table_read.equals(table) -@parametrize_legacy_dataset -def test_special_chars_filename(tempdir, use_legacy_dataset): +@pytest.mark.dataset +def test_special_chars_filename(tempdir): table = pa.Table.from_arrays([pa.array([42])], ["ints"]) filename = "foo # bar" path = tempdir / filename assert not path.exists() _write_table(table, str(path)) assert path.exists() - table_read = _read_table(str(path), use_legacy_dataset=use_legacy_dataset) + table_read = _read_table(str(path)) assert table_read.equals(table) -@parametrize_legacy_dataset -def test_invalid_source(use_legacy_dataset): +@pytest.mark.dataset +def test_invalid_source(): # Test that we provide an helpful error message pointing out # that None wasn't expected when trying to open a Parquet None file. # - # Depending on use_legacy_dataset the message changes slightly - # but in both cases it should point out that None wasn't expected. with pytest.raises(TypeError, match="None"): - pq.read_table(None, use_legacy_dataset=use_legacy_dataset) + pq.read_table(None) with pytest.raises(TypeError, match="None"): pq.ParquetFile(None) @@ -193,8 +186,8 @@ def test_file_with_over_int16_max_row_groups(): @pytest.mark.pandas -@parametrize_legacy_dataset -def test_empty_table_roundtrip(use_legacy_dataset): +@pytest.mark.dataset +def test_empty_table_roundtrip(): df = alltypes_sample(size=10) # Create a non-empty table to infer the types correctly, then slice to 0 @@ -206,19 +199,19 @@ def test_empty_table_roundtrip(use_legacy_dataset): assert table.schema.field('null').type == pa.null() assert table.schema.field('null_list').type == pa.list_(pa.null()) _check_roundtrip( - table, version='2.6', use_legacy_dataset=use_legacy_dataset) + table, version='2.6') @pytest.mark.pandas -@parametrize_legacy_dataset -def test_empty_table_no_columns(use_legacy_dataset): +@pytest.mark.dataset +def test_empty_table_no_columns(): df = pd.DataFrame() empty = pa.Table.from_pandas(df, preserve_index=False) - _check_roundtrip(empty, use_legacy_dataset=use_legacy_dataset) + _check_roundtrip(empty) -@parametrize_legacy_dataset -def test_write_nested_zero_length_array_chunk_failure(use_legacy_dataset): +@pytest.mark.dataset +def test_write_nested_zero_length_array_chunk_failure(): # Bug report in ARROW-3792 cols = OrderedDict( int32=pa.int32(), @@ -243,17 +236,17 @@ def test_write_nested_zero_length_array_chunk_failure(use_legacy_dataset): my_batches = [pa.RecordBatch.from_arrays(batch, schema=pa.schema(cols)) for batch in my_arrays] tbl = pa.Table.from_batches(my_batches, pa.schema(cols)) - _check_roundtrip(tbl, use_legacy_dataset=use_legacy_dataset) + _check_roundtrip(tbl) @pytest.mark.pandas -@parametrize_legacy_dataset -def test_multiple_path_types(tempdir, use_legacy_dataset): +@pytest.mark.dataset +def test_multiple_path_types(tempdir): # Test compatibility with PEP 519 path-like objects path = tempdir / 'zzz.parquet' df = pd.DataFrame({'x': np.arange(10, dtype=np.int64)}) _write_table(df, path) - table_read = _read_table(path, use_legacy_dataset=use_legacy_dataset) + table_read = _read_table(path) df_read = table_read.to_pandas() tm.assert_frame_equal(df, df_read) @@ -261,13 +254,13 @@ def test_multiple_path_types(tempdir, use_legacy_dataset): path = str(tempdir) + 'zzz.parquet' df = pd.DataFrame({'x': np.arange(10, dtype=np.int64)}) _write_table(df, path) - table_read = _read_table(path, use_legacy_dataset=use_legacy_dataset) + table_read = _read_table(path) df_read = table_read.to_pandas() tm.assert_frame_equal(df, df_read) -@parametrize_legacy_dataset -def test_fspath(tempdir, use_legacy_dataset): +@pytest.mark.dataset +def test_fspath(tempdir): # ARROW-12472 support __fspath__ objects without using str() path = tempdir / "test.parquet" table = pa.table({"a": [1, 2, 3]}) @@ -275,9 +268,7 @@ def test_fspath(tempdir, use_legacy_dataset): fs_protocol_obj = util.FSProtocolClass(path) - result = _read_table( - fs_protocol_obj, use_legacy_dataset=use_legacy_dataset - ) + result = _read_table(fs_protocol_obj) assert result.equals(table) # combined with non-local filesystem raises @@ -286,14 +277,11 @@ def test_fspath(tempdir, use_legacy_dataset): @pytest.mark.dataset -@parametrize_legacy_dataset @pytest.mark.parametrize("filesystem", [ None, fs.LocalFileSystem(), LocalFileSystem._get_instance() ]) @pytest.mark.parametrize("name", ("data.parquet", "例.parquet")) -def test_relative_paths(tempdir, use_legacy_dataset, filesystem, name): - if use_legacy_dataset and isinstance(filesystem, fs.FileSystem): - pytest.skip("Passing new filesystem not supported for legacy reader") +def test_relative_paths(tempdir, filesystem, name): # reading and writing from relative paths table = pa.table({"a": [1, 2, 3]}) path = tempdir / name @@ -301,8 +289,7 @@ def test_relative_paths(tempdir, use_legacy_dataset, filesystem, name): # reading pq.write_table(table, str(path)) with util.change_cwd(tempdir): - result = pq.read_table(name, filesystem=filesystem, - use_legacy_dataset=use_legacy_dataset) + result = pq.read_table(name, filesystem=filesystem) assert result.equals(table) path.unlink() @@ -315,12 +302,14 @@ def test_relative_paths(tempdir, use_legacy_dataset, filesystem, name): assert result.equals(table) +@pytest.mark.dataset def test_read_non_existing_file(): # ensure we have a proper error message with pytest.raises(FileNotFoundError): pq.read_table('i-am-not-existing.parquet') +@pytest.mark.dataset def test_file_error_python_exception(): class BogusFile(io.BytesIO): def read(self, *args): @@ -334,24 +323,23 @@ def seek(self, *args): pq.read_table(BogusFile(b"")) -@parametrize_legacy_dataset -def test_parquet_read_from_buffer(tempdir, use_legacy_dataset): +@pytest.mark.dataset +def test_parquet_read_from_buffer(tempdir): # reading from a buffer from python's open() table = pa.table({"a": [1, 2, 3]}) pq.write_table(table, str(tempdir / "data.parquet")) with open(str(tempdir / "data.parquet"), "rb") as f: - result = pq.read_table(f, use_legacy_dataset=use_legacy_dataset) + result = pq.read_table(f) assert result.equals(table) with open(str(tempdir / "data.parquet"), "rb") as f: - result = pq.read_table(pa.PythonFile(f), - use_legacy_dataset=use_legacy_dataset) + result = pq.read_table(pa.PythonFile(f)) assert result.equals(table) -@parametrize_legacy_dataset -def test_byte_stream_split(use_legacy_dataset): +@pytest.mark.dataset +def test_byte_stream_split(): # This is only a smoke test. arr_float = pa.array(list(map(float, range(100)))) arr_int = pa.array(list(map(int, range(100)))) @@ -385,12 +373,11 @@ def test_byte_stream_split(use_legacy_dataset): table = pa.Table.from_arrays([arr_int], names=['tmp']) with pytest.raises(IOError): _check_roundtrip(table, expected=table, use_byte_stream_split=True, - use_dictionary=False, - use_legacy_dataset=use_legacy_dataset) + use_dictionary=False) -@parametrize_legacy_dataset -def test_column_encoding(use_legacy_dataset): +@pytest.mark.dataset +def test_column_encoding(): arr_float = pa.array(list(map(float, range(100)))) arr_int = pa.array(list(map(int, range(100)))) arr_bin = pa.array([str(x) for x in range(100)], type=pa.binary()) @@ -406,30 +393,26 @@ def test_column_encoding(use_legacy_dataset): _check_roundtrip(mixed_table, expected=mixed_table, use_dictionary=False, column_encoding={'a': "BYTE_STREAM_SPLIT", 'b': "PLAIN", - 'c': "PLAIN"}, - use_legacy_dataset=use_legacy_dataset) + 'c': "PLAIN"}) # Check "PLAIN" for all columns. _check_roundtrip(mixed_table, expected=mixed_table, use_dictionary=False, - column_encoding="PLAIN", - use_legacy_dataset=use_legacy_dataset) + column_encoding="PLAIN") # Check "DELTA_BINARY_PACKED" for integer columns. _check_roundtrip(mixed_table, expected=mixed_table, use_dictionary=False, column_encoding={'a': "PLAIN", 'b': "DELTA_BINARY_PACKED", - 'c': "PLAIN"}, - use_legacy_dataset=use_legacy_dataset) + 'c': "PLAIN"}) # Check "DELTA_LENGTH_BYTE_ARRAY" for byte columns. _check_roundtrip(mixed_table, expected=mixed_table, use_dictionary=False, column_encoding={'a': "PLAIN", 'b': "DELTA_BINARY_PACKED", - 'c': "DELTA_LENGTH_BYTE_ARRAY"}, - use_legacy_dataset=use_legacy_dataset) + 'c': "DELTA_LENGTH_BYTE_ARRAY"}) # Check "DELTA_BYTE_ARRAY" for byte columns. _check_roundtrip(mixed_table, expected=mixed_table, @@ -437,14 +420,12 @@ def test_column_encoding(use_legacy_dataset): column_encoding={'a': "PLAIN", 'b': "DELTA_BINARY_PACKED", 'c': "DELTA_BYTE_ARRAY", - 'd': "DELTA_BYTE_ARRAY"}, - use_legacy_dataset=use_legacy_dataset) + 'd': "DELTA_BYTE_ARRAY"}) # Check "RLE" for boolean columns. _check_roundtrip(mixed_table, expected=mixed_table, use_dictionary=False, - column_encoding={'e': "RLE"}, - use_legacy_dataset=use_legacy_dataset) + column_encoding={'e': "RLE"}) # Try to pass "BYTE_STREAM_SPLIT" column encoding for integer column 'b'. # This should throw an error as it is only supports FLOAT and DOUBLE. @@ -455,8 +436,7 @@ def test_column_encoding(use_legacy_dataset): use_dictionary=False, column_encoding={'a': "PLAIN", 'b': "BYTE_STREAM_SPLIT", - 'c': "PLAIN"}, - use_legacy_dataset=use_legacy_dataset) + 'c': "PLAIN"}) # Try to pass use "DELTA_BINARY_PACKED" encoding on float column. # This should throw an error as only integers are supported. @@ -465,8 +445,7 @@ def test_column_encoding(use_legacy_dataset): use_dictionary=False, column_encoding={'a': "DELTA_BINARY_PACKED", 'b': "PLAIN", - 'c': "PLAIN"}, - use_legacy_dataset=use_legacy_dataset) + 'c': "PLAIN"}) # Try to pass "RLE_DICTIONARY". # This should throw an error as dictionary encoding is already used by @@ -474,30 +453,26 @@ def test_column_encoding(use_legacy_dataset): with pytest.raises(ValueError): _check_roundtrip(mixed_table, expected=mixed_table, use_dictionary=False, - column_encoding="RLE_DICTIONARY", - use_legacy_dataset=use_legacy_dataset) + column_encoding="RLE_DICTIONARY") # Try to pass unsupported encoding. with pytest.raises(ValueError): _check_roundtrip(mixed_table, expected=mixed_table, use_dictionary=False, - column_encoding={'a': "MADE_UP_ENCODING"}, - use_legacy_dataset=use_legacy_dataset) + column_encoding={'a': "MADE_UP_ENCODING"}) # Try to pass column_encoding and use_dictionary. # This should throw an error. with pytest.raises(ValueError): _check_roundtrip(mixed_table, expected=mixed_table, use_dictionary=['b'], - column_encoding={'b': "PLAIN"}, - use_legacy_dataset=use_legacy_dataset) + column_encoding={'b': "PLAIN"}) # Try to pass column_encoding and use_dictionary=True (default value). # This should throw an error. with pytest.raises(ValueError): _check_roundtrip(mixed_table, expected=mixed_table, - column_encoding={'b': "PLAIN"}, - use_legacy_dataset=use_legacy_dataset) + column_encoding={'b': "PLAIN"}) # Try to pass column_encoding and use_byte_stream_split on same column. # This should throw an error. @@ -507,8 +482,7 @@ def test_column_encoding(use_legacy_dataset): use_byte_stream_split=['a'], column_encoding={'a': "RLE", 'b': "BYTE_STREAM_SPLIT", - 'c': "PLAIN"}, - use_legacy_dataset=use_legacy_dataset) + 'c': "PLAIN"}) # Try to pass column_encoding and use_byte_stream_split=True. # This should throw an error. @@ -518,54 +492,46 @@ def test_column_encoding(use_legacy_dataset): use_byte_stream_split=True, column_encoding={'a': "RLE", 'b': "BYTE_STREAM_SPLIT", - 'c': "PLAIN"}, - use_legacy_dataset=use_legacy_dataset) + 'c': "PLAIN"}) # Try to pass column_encoding=True. # This should throw an error. with pytest.raises(TypeError): _check_roundtrip(mixed_table, expected=mixed_table, use_dictionary=False, - column_encoding=True, - use_legacy_dataset=use_legacy_dataset) + column_encoding=True) -@parametrize_legacy_dataset -def test_compression_level(use_legacy_dataset): +@pytest.mark.dataset +def test_compression_level(): arr = pa.array(list(map(int, range(1000)))) data = [arr, arr] table = pa.Table.from_arrays(data, names=['a', 'b']) # Check one compression level. _check_roundtrip(table, expected=table, compression="gzip", - compression_level=1, - use_legacy_dataset=use_legacy_dataset) + compression_level=1) # Check another one to make sure that compression_level=1 does not # coincide with the default one in Arrow. _check_roundtrip(table, expected=table, compression="gzip", - compression_level=5, - use_legacy_dataset=use_legacy_dataset) + compression_level=5) # Check that the user can provide a compression per column _check_roundtrip(table, expected=table, - compression={'a': "gzip", 'b': "snappy"}, - use_legacy_dataset=use_legacy_dataset) + compression={'a': "gzip", 'b': "snappy"}) # Check that the user can provide a compression level per column _check_roundtrip(table, expected=table, compression="gzip", - compression_level={'a': 2, 'b': 3}, - use_legacy_dataset=use_legacy_dataset) + compression_level={'a': 2, 'b': 3}) # Check if both LZ4 compressors are working # (level < 3 -> fast, level >= 3 -> HC) _check_roundtrip(table, expected=table, compression="lz4", - compression_level=1, - use_legacy_dataset=use_legacy_dataset) + compression_level=1) _check_roundtrip(table, expected=table, compression="lz4", - compression_level=9, - use_legacy_dataset=use_legacy_dataset) + compression_level=9) # Check that specifying a compression level for a codec which does allow # specifying one, results into an error. @@ -594,8 +560,8 @@ def test_sanitized_spark_field_names(): @pytest.mark.pandas -@parametrize_legacy_dataset -def test_multithreaded_read(use_legacy_dataset): +@pytest.mark.dataset +def test_multithreaded_read(): df = alltypes_sample(size=10000) table = pa.Table.from_pandas(df) @@ -604,19 +570,17 @@ def test_multithreaded_read(use_legacy_dataset): _write_table(table, buf, compression='SNAPPY', version='2.6') buf.seek(0) - table1 = _read_table( - buf, use_threads=True, use_legacy_dataset=use_legacy_dataset) + table1 = _read_table(buf, use_threads=True) buf.seek(0) - table2 = _read_table( - buf, use_threads=False, use_legacy_dataset=use_legacy_dataset) + table2 = _read_table(buf, use_threads=False) assert table1.equals(table2) @pytest.mark.pandas -@parametrize_legacy_dataset -def test_min_chunksize(use_legacy_dataset): +@pytest.mark.dataset +def test_min_chunksize(): data = pd.DataFrame([np.arange(4)], columns=['A', 'B', 'C', 'D']) table = pa.Table.from_pandas(data.reset_index()) @@ -624,7 +588,7 @@ def test_min_chunksize(use_legacy_dataset): _write_table(table, buf, chunk_size=-1) buf.seek(0) - result = _read_table(buf, use_legacy_dataset=use_legacy_dataset) + result = _read_table(buf) assert result.equals(table) @@ -659,57 +623,50 @@ def test_write_error_deletes_incomplete_file(tempdir): assert not filename.exists() -@parametrize_legacy_dataset -def test_read_non_existent_file(tempdir, use_legacy_dataset): +@pytest.mark.dataset +def test_read_non_existent_file(tempdir): path = 'nonexistent-file.parquet' try: - pq.read_table(path, use_legacy_dataset=use_legacy_dataset) + pq.read_table(path) except Exception as e: assert path in e.args[0] -@parametrize_legacy_dataset -def test_read_table_doesnt_warn(datadir, use_legacy_dataset): - if use_legacy_dataset: - msg = "Passing 'use_legacy_dataset=True'" - with pytest.warns(FutureWarning, match=msg): - pq.read_table(datadir / 'v0.7.1.parquet', - use_legacy_dataset=use_legacy_dataset) - else: - with warnings.catch_warnings(): - warnings.simplefilter(action="error") - pq.read_table(datadir / 'v0.7.1.parquet', - use_legacy_dataset=use_legacy_dataset) +@pytest.mark.dataset +def test_read_table_doesnt_warn(datadir): + with warnings.catch_warnings(): + warnings.simplefilter(action="error") + pq.read_table(datadir / 'v0.7.1.parquet') @pytest.mark.pandas -@parametrize_legacy_dataset -def test_zlib_compression_bug(use_legacy_dataset): +@pytest.mark.dataset +def test_zlib_compression_bug(): # ARROW-3514: "zlib deflate failed, output buffer too small" table = pa.Table.from_arrays([pa.array(['abc', 'def'])], ['some_col']) f = io.BytesIO() pq.write_table(table, f, compression='gzip') f.seek(0) - roundtrip = pq.read_table(f, use_legacy_dataset=use_legacy_dataset) + roundtrip = pq.read_table(f) tm.assert_frame_equal(roundtrip.to_pandas(), table.to_pandas()) -@parametrize_legacy_dataset -def test_parquet_file_too_small(tempdir, use_legacy_dataset): +@pytest.mark.dataset +def test_parquet_file_too_small(tempdir): path = str(tempdir / "test.parquet") # TODO(dataset) with datasets API it raises OSError instead with pytest.raises((pa.ArrowInvalid, OSError), match='size is 0 bytes'): with open(path, 'wb') as f: pass - pq.read_table(path, use_legacy_dataset=use_legacy_dataset) + pq.read_table(path) with pytest.raises((pa.ArrowInvalid, OSError), match='size is 4 bytes'): with open(path, 'wb') as f: f.write(b'ffff') - pq.read_table(path, use_legacy_dataset=use_legacy_dataset) + pq.read_table(path) @pytest.mark.pandas @@ -752,7 +709,7 @@ def test_fastparquet_cross_compatibility(tempdir): tm.assert_frame_equal(table_fp.to_pandas(), df) -@parametrize_legacy_dataset +@pytest.mark.dataset @pytest.mark.parametrize('array_factory', [ lambda: pa.array([0, None] * 10), lambda: pa.array([0, None] * 10).dictionary_encode(), @@ -762,7 +719,7 @@ def test_fastparquet_cross_compatibility(tempdir): @pytest.mark.parametrize('use_dictionary', [False, True]) @pytest.mark.parametrize('read_dictionary', [False, True]) def test_buffer_contents( - array_factory, use_dictionary, read_dictionary, use_legacy_dataset + array_factory, use_dictionary, read_dictionary ): # Test that null values are deterministically initialized to zero # after a roundtrip through Parquet. @@ -773,8 +730,7 @@ def test_buffer_contents( bio.seek(0) read_dictionary = ['col'] if read_dictionary else None table = pq.read_table(bio, use_threads=False, - read_dictionary=read_dictionary, - use_legacy_dataset=use_legacy_dataset) + read_dictionary=read_dictionary) for col in table.columns: [chunk] = col.chunks @@ -846,18 +802,6 @@ def test_permutation_of_column_order(tempdir): assert table == table2 -def test_read_table_legacy_deprecated(tempdir): - # ARROW-15870 - table = pa.table({'a': [1, 2, 3]}) - path = tempdir / 'data.parquet' - pq.write_table(table, path) - - with pytest.warns( - FutureWarning, match="Passing 'use_legacy_dataset=True'" - ): - pq.read_table(path, use_legacy_dataset=True) - - def test_thrift_size_limits(tempdir): path = tempdir / 'largethrift.parquet' @@ -942,28 +886,9 @@ def test_page_checksum_verification_write_table(tempdir): with pytest.raises(OSError, match="CRC checksum verification"): _ = corrupted_pq_file.read() - # Case 5: Check that enabling page checksum verification in combination - # with legacy dataset raises an exception - with pytest.raises(ValueError, match="page_checksum_verification"): - _ = pq.read_table(corrupted_path, - page_checksum_verification=True, - use_legacy_dataset=True) - @pytest.mark.dataset -@pytest.mark.parametrize( - "use_legacy_dataset", - [ - False, - pytest.param( - True, - marks=pytest.mark.filterwarnings( - "ignore:Passing 'use_legacy_dataset=True':FutureWarning" - ), - ), - ], -) -def test_checksum_write_to_dataset(tempdir, use_legacy_dataset): +def test_checksum_write_to_dataset(tempdir): """Check that checksum verification works for datasets created with pq.write_to_dataset""" @@ -973,8 +898,7 @@ def test_checksum_write_to_dataset(tempdir, use_legacy_dataset): original_dir_path = tempdir / 'correct_dir' pq.write_to_dataset(table_orig, original_dir_path, - write_page_checksum=True, - use_legacy_dataset=use_legacy_dataset) + write_page_checksum=True) # Read file and verify that the data is correct original_file_path_list = list(original_dir_path.iterdir()) diff --git a/python/pyarrow/tests/parquet/test_compliant_nested_type.py b/python/pyarrow/tests/parquet/test_compliant_nested_type.py index ca1ad7ee32255..fadf092b3f6f7 100644 --- a/python/pyarrow/tests/parquet/test_compliant_nested_type.py +++ b/python/pyarrow/tests/parquet/test_compliant_nested_type.py @@ -18,7 +18,6 @@ import pytest import pyarrow as pa -from pyarrow.tests.parquet.common import parametrize_legacy_dataset try: import pyarrow.parquet as pq @@ -58,16 +57,14 @@ @pytest.mark.pandas -@parametrize_legacy_dataset +@pytest.mark.dataset @parametrize_test_data -def test_write_compliant_nested_type_enable(tempdir, - use_legacy_dataset, test_data): +def test_write_compliant_nested_type_enable(tempdir, test_data): # prepare dataframe for testing df = pd.DataFrame(data=test_data) # verify that we can read/write pandas df with new flag (default behaviour) _roundtrip_pandas_dataframe(df, - write_kwargs={}, - use_legacy_dataset=use_legacy_dataset) + write_kwargs={}) # Write to a parquet file with compliant nested type table = pa.Table.from_pandas(df, preserve_index=False) @@ -83,21 +80,18 @@ def test_write_compliant_nested_type_enable(tempdir, assert new_table.schema.types[0].value_field.name == 'element' # Verify that the new table can be read/written correctly - _check_roundtrip(new_table, - use_legacy_dataset=use_legacy_dataset) + _check_roundtrip(new_table) @pytest.mark.pandas -@parametrize_legacy_dataset +@pytest.mark.dataset @parametrize_test_data -def test_write_compliant_nested_type_disable(tempdir, - use_legacy_dataset, test_data): +def test_write_compliant_nested_type_disable(tempdir,test_data): # prepare dataframe for testing df = pd.DataFrame(data=test_data) # verify that we can read/write with new flag disabled _roundtrip_pandas_dataframe(df, write_kwargs={ - 'use_compliant_nested_type': False}, - use_legacy_dataset=use_legacy_dataset) + 'use_compliant_nested_type': False}) # Write to a parquet file while disabling compliant nested type table = pa.Table.from_pandas(df, preserve_index=False) @@ -114,5 +108,4 @@ def test_write_compliant_nested_type_disable(tempdir, # Verify that the new table can be read/written correctly _check_roundtrip(new_table, - use_legacy_dataset=use_legacy_dataset, use_compliant_nested_type=False) diff --git a/python/pyarrow/tests/parquet/test_data_types.py b/python/pyarrow/tests/parquet/test_data_types.py index 32fe128bbae9b..1ac5f9729b26d 100644 --- a/python/pyarrow/tests/parquet/test_data_types.py +++ b/python/pyarrow/tests/parquet/test_data_types.py @@ -23,8 +23,7 @@ import pyarrow as pa from pyarrow.tests import util -from pyarrow.tests.parquet.common import (_check_roundtrip, - parametrize_legacy_dataset) +from pyarrow.tests.parquet.common import _check_roundtrip try: import pyarrow.parquet as pq @@ -54,9 +53,9 @@ @pytest.mark.pandas -@parametrize_legacy_dataset +@pytest.mark.dataset @pytest.mark.parametrize('chunk_size', [None, 1000]) -def test_parquet_2_0_roundtrip(tempdir, chunk_size, use_legacy_dataset): +def test_parquet_2_0_roundtrip(tempdir, chunk_size): df = alltypes_sample(size=10000, categorical=True) filename = tempdir / 'pandas_roundtrip.parquet' @@ -65,8 +64,7 @@ def test_parquet_2_0_roundtrip(tempdir, chunk_size, use_legacy_dataset): _write_table(arrow_table, filename, version='2.6', chunk_size=chunk_size) - table_read = pq.read_pandas( - filename, use_legacy_dataset=use_legacy_dataset) + table_read = pq.read_pandas(filename) assert table_read.schema.pandas_metadata is not None read_metadata = table_read.schema.metadata @@ -77,8 +75,8 @@ def test_parquet_2_0_roundtrip(tempdir, chunk_size, use_legacy_dataset): @pytest.mark.pandas -@parametrize_legacy_dataset -def test_parquet_1_0_roundtrip(tempdir, use_legacy_dataset): +@pytest.mark.dataset +def test_parquet_1_0_roundtrip(tempdir): size = 10000 np.random.seed(0) df = pd.DataFrame({ @@ -100,7 +98,7 @@ def test_parquet_1_0_roundtrip(tempdir, use_legacy_dataset): filename = tempdir / 'pandas_roundtrip.parquet' arrow_table = pa.Table.from_pandas(df) _write_table(arrow_table, filename, version='1.0') - table_read = _read_table(filename, use_legacy_dataset=use_legacy_dataset) + table_read = _read_table(filename) df_read = table_read.to_pandas() # We pass uint32_t as int64_t if we write Parquet version 1.0 @@ -113,18 +111,18 @@ def test_parquet_1_0_roundtrip(tempdir, use_legacy_dataset): # ----------------------------------------------------------------------------- -def _simple_table_write_read(table, use_legacy_dataset): +def _simple_table_write_read(table): bio = pa.BufferOutputStream() pq.write_table(table, bio) contents = bio.getvalue() return pq.read_table( - pa.BufferReader(contents), use_legacy_dataset=use_legacy_dataset + pa.BufferReader(contents) ) @pytest.mark.pandas -@parametrize_legacy_dataset -def test_direct_read_dictionary(use_legacy_dataset): +@pytest.mark.dataset +def test_direct_read_dictionary(): # ARROW-3325 repeats = 10 nunique = 5 @@ -140,8 +138,7 @@ def test_direct_read_dictionary(use_legacy_dataset): contents = bio.getvalue() result = pq.read_table(pa.BufferReader(contents), - read_dictionary=['f0'], - use_legacy_dataset=use_legacy_dataset) + read_dictionary=['f0']) # Compute dictionary-encoded subfield expected = pa.table([table[0].dictionary_encode()], names=['f0']) @@ -149,8 +146,8 @@ def test_direct_read_dictionary(use_legacy_dataset): @pytest.mark.pandas -@parametrize_legacy_dataset -def test_direct_read_dictionary_subfield(use_legacy_dataset): +@pytest.mark.dataset +def test_direct_read_dictionary_subfield(): repeats = 10 nunique = 5 @@ -163,8 +160,7 @@ def test_direct_read_dictionary_subfield(use_legacy_dataset): pq.write_table(table, bio) contents = bio.getvalue() result = pq.read_table(pa.BufferReader(contents), - read_dictionary=['f0.list.element'], - use_legacy_dataset=use_legacy_dataset) + read_dictionary=['f0.list.element']) arr = pa.array(data[0]) values_as_dict = arr.values.dictionary_encode() @@ -181,8 +177,8 @@ def test_direct_read_dictionary_subfield(use_legacy_dataset): assert result[0].num_chunks == 1 -@parametrize_legacy_dataset -def test_dictionary_array_automatically_read(use_legacy_dataset): +@pytest.mark.dataset +def test_dictionary_array_automatically_read(): # ARROW-3246 # Make a large dictionary, a little over 4MB of data @@ -200,7 +196,7 @@ def test_dictionary_array_automatically_read(use_legacy_dataset): dict_values)) table = pa.table([pa.chunked_array(chunks)], names=['f0']) - result = _simple_table_write_read(table, use_legacy_dataset) + result = _simple_table_write_read(table) assert result.equals(table) @@ -213,8 +209,8 @@ def test_dictionary_array_automatically_read(use_legacy_dataset): @pytest.mark.pandas -@parametrize_legacy_dataset -def test_decimal_roundtrip(tempdir, use_legacy_dataset): +@pytest.mark.dataset +def test_decimal_roundtrip(tempdir): num_values = 10 columns = {} @@ -234,8 +230,7 @@ def test_decimal_roundtrip(tempdir, use_legacy_dataset): string_filename = str(filename) table = pa.Table.from_pandas(expected) _write_table(table, string_filename) - result_table = _read_table( - string_filename, use_legacy_dataset=use_legacy_dataset) + result_table = _read_table(string_filename) result = result_table.to_pandas() tm.assert_frame_equal(result, expected) @@ -259,14 +254,14 @@ def test_decimal_roundtrip_negative_scale(tempdir): # ----------------------------------------------------------------------------- -@parametrize_legacy_dataset +@pytest.mark.dataset @pytest.mark.parametrize('dtype', [int, float]) -def test_single_pylist_column_roundtrip(tempdir, dtype, use_legacy_dataset): +def test_single_pylist_column_roundtrip(tempdir, dtype,): filename = tempdir / 'single_{}_column.parquet'.format(dtype.__name__) data = [pa.array(list(map(dtype, range(5))))] table = pa.Table.from_arrays(data, names=['a']) _write_table(table, filename) - table_read = _read_table(filename, use_legacy_dataset=use_legacy_dataset) + table_read = _read_table(filename) for i in range(table.num_columns): col_written = table[i] col_read = table_read[i] @@ -277,16 +272,16 @@ def test_single_pylist_column_roundtrip(tempdir, dtype, use_legacy_dataset): assert data_written.equals(data_read) -@parametrize_legacy_dataset -def test_empty_lists_table_roundtrip(use_legacy_dataset): +@pytest.mark.dataset +def test_empty_lists_table_roundtrip(): # ARROW-2744: Shouldn't crash when writing an array of empty lists arr = pa.array([[], []], type=pa.list_(pa.int32())) table = pa.Table.from_arrays([arr], ["A"]) - _check_roundtrip(table, use_legacy_dataset=use_legacy_dataset) + _check_roundtrip(table) -@parametrize_legacy_dataset -def test_nested_list_nonnullable_roundtrip_bug(use_legacy_dataset): +@pytest.mark.dataset +def test_nested_list_nonnullable_roundtrip_bug(): # Reproduce failure in ARROW-5630 typ = pa.list_(pa.field("item", pa.float32(), False)) num_rows = 10000 @@ -295,26 +290,23 @@ def test_nested_list_nonnullable_roundtrip_bug(use_legacy_dataset): (num_rows // 10)), type=typ) ], ['a']) _check_roundtrip( - t, data_page_size=4096, use_legacy_dataset=use_legacy_dataset) + t, data_page_size=4096) -@parametrize_legacy_dataset -def test_nested_list_struct_multiple_batches_roundtrip( - tempdir, use_legacy_dataset -): +@pytest.mark.dataset +def test_nested_list_struct_multiple_batches_roundtrip(tempdir): # Reproduce failure in ARROW-11024 data = [[{'x': 'abc', 'y': 'abc'}]]*100 + [[{'x': 'abc', 'y': 'gcb'}]]*100 table = pa.table([pa.array(data)], names=['column']) _check_roundtrip( - table, row_group_size=20, use_legacy_dataset=use_legacy_dataset) + table, row_group_size=20) # Reproduce failure in ARROW-11069 (plain non-nested structs with strings) data = pa.array( [{'a': '1', 'b': '2'}, {'a': '3', 'b': '4'}, {'a': '5', 'b': '6'}]*10 ) table = pa.table({'column': data}) - _check_roundtrip( - table, row_group_size=10, use_legacy_dataset=use_legacy_dataset) + _check_roundtrip(table, row_group_size=10) def test_writing_empty_lists(): @@ -365,9 +357,9 @@ def test_large_list_records(): _check_roundtrip(table) +@pytest.mark.dataset @pytest.mark.pandas -@parametrize_legacy_dataset -def test_parquet_nested_convenience(tempdir, use_legacy_dataset): +def test_parquet_nested_convenience(tempdir): # ARROW-1684 df = pd.DataFrame({ 'a': [[1, 2, 3], None, [4, 5], []], @@ -380,11 +372,11 @@ def test_parquet_nested_convenience(tempdir, use_legacy_dataset): _write_table(table, path) read = pq.read_table( - path, columns=['a'], use_legacy_dataset=use_legacy_dataset) + path, columns=['a']) tm.assert_frame_equal(read.to_pandas(), df[['a']]) read = pq.read_table( - path, columns=['a', 'b'], use_legacy_dataset=use_legacy_dataset) + path, columns=['a', 'b']) tm.assert_frame_equal(read.to_pandas(), df) @@ -420,17 +412,17 @@ def test_large_table_int32_overflow(): _write_table(table, f) -def _simple_table_roundtrip(table, use_legacy_dataset=False, **write_kwargs): +def _simple_table_roundtrip(table, **write_kwargs): stream = pa.BufferOutputStream() _write_table(table, stream, **write_kwargs) buf = stream.getvalue() - return _read_table(buf, use_legacy_dataset=use_legacy_dataset) + return _read_table(buf) @pytest.mark.slow @pytest.mark.large_memory -@parametrize_legacy_dataset -def test_byte_array_exactly_2gb(use_legacy_dataset): +@pytest.mark.dataset +def test_byte_array_exactly_2gb(): # Test edge case reported in ARROW-3762 val = b'x' * (1 << 10) @@ -444,15 +436,15 @@ def test_byte_array_exactly_2gb(use_legacy_dataset): values = pa.chunked_array([base, pa.array(case)]) t = pa.table([values], names=['f0']) result = _simple_table_roundtrip( - t, use_legacy_dataset=use_legacy_dataset, use_dictionary=False) + t, use_dictionary=False) assert t.equals(result) @pytest.mark.slow @pytest.mark.pandas @pytest.mark.large_memory -@parametrize_legacy_dataset -def test_binary_array_overflow_to_chunked(use_legacy_dataset): +@pytest.mark.dataset +def test_binary_array_overflow_to_chunked(): # ARROW-3762 # 2^31 + 1 bytes @@ -462,8 +454,7 @@ def test_binary_array_overflow_to_chunked(use_legacy_dataset): df = pd.DataFrame({'byte_col': values}) tbl = pa.Table.from_pandas(df, preserve_index=False) - read_tbl = _simple_table_roundtrip( - tbl, use_legacy_dataset=use_legacy_dataset) + read_tbl = _simple_table_roundtrip(tbl) col0_data = read_tbl[0] assert isinstance(col0_data, pa.ChunkedArray) @@ -477,8 +468,8 @@ def test_binary_array_overflow_to_chunked(use_legacy_dataset): @pytest.mark.slow @pytest.mark.pandas @pytest.mark.large_memory -@parametrize_legacy_dataset -def test_list_of_binary_large_cell(use_legacy_dataset): +@pytest.mark.dataset +def test_list_of_binary_large_cell(): # ARROW-4688 data = [] @@ -491,8 +482,7 @@ def test_list_of_binary_large_cell(use_legacy_dataset): arr = pa.array(data) table = pa.Table.from_arrays([arr], ['chunky_cells']) - read_table = _simple_table_roundtrip( - table, use_legacy_dataset=use_legacy_dataset) + read_table = _simple_table_roundtrip(table) assert table.equals(read_table) diff --git a/python/pyarrow/tests/parquet/test_dataset.py b/python/pyarrow/tests/parquet/test_dataset.py index a9e99d5d65cf9..25bd8d6dbc7ca 100644 --- a/python/pyarrow/tests/parquet/test_dataset.py +++ b/python/pyarrow/tests/parquet/test_dataset.py @@ -29,9 +29,6 @@ from pyarrow import fs from pyarrow.filesystem import LocalFileSystem from pyarrow.tests import util -from pyarrow.tests.parquet.common import ( - parametrize_legacy_dataset, parametrize_legacy_dataset_fixed, - parametrize_legacy_dataset_not_supported) from pyarrow.util import guid from pyarrow.vendored.version import Version @@ -56,59 +53,6 @@ pytestmark = pytest.mark.parquet -@pytest.mark.pandas -def test_parquet_piece_read(tempdir): - df = _test_dataframe(1000) - table = pa.Table.from_pandas(df) - - path = tempdir / 'parquet_piece_read.parquet' - _write_table(table, path, version='2.6') - - with pytest.warns(FutureWarning): - piece1 = pq.ParquetDatasetPiece(path) - - result = piece1.read() - assert result.equals(table) - - -@pytest.mark.pandas -def test_parquet_piece_open_and_get_metadata(tempdir): - df = _test_dataframe(100) - table = pa.Table.from_pandas(df) - - path = tempdir / 'parquet_piece_read.parquet' - _write_table(table, path, version='2.6') - - with pytest.warns(FutureWarning): - piece = pq.ParquetDatasetPiece(path) - - table1 = piece.read() - assert isinstance(table1, pa.Table) - meta1 = piece.get_metadata() - assert isinstance(meta1, pq.FileMetaData) - - assert table.equals(table1) - - -@pytest.mark.filterwarnings("ignore:ParquetDatasetPiece:FutureWarning") -def test_parquet_piece_basics(): - path = '/baz.parq' - - piece1 = pq.ParquetDatasetPiece(path) - piece2 = pq.ParquetDatasetPiece(path, row_group=1) - piece3 = pq.ParquetDatasetPiece( - path, row_group=1, partition_keys=[('foo', 0), ('bar', 1)]) - - assert str(piece1) == path - assert str(piece2) == '/baz.parq | row_group=1' - assert str(piece3) == 'partition[foo=0, bar=1] /baz.parq | row_group=1' - - assert piece1 == piece1 - assert piece2 == piece2 - assert piece3 == piece3 - assert piece1 != piece3 - - def test_partition_set_dictionary_type(): set1 = pq.PartitionSet('key1', ['foo', 'bar', 'baz']) set2 = pq.PartitionSet('key2', [2007, 2008, 2009]) @@ -121,8 +65,8 @@ def test_partition_set_dictionary_type(): set3.dictionary -@parametrize_legacy_dataset_fixed -def test_filesystem_uri(tempdir, use_legacy_dataset): +@pytest.mark.dataset +def test_filesystem_uri(tempdir): table = pa.table({"a": [1, 2, 3]}) directory = tempdir / "data_dir" @@ -132,72 +76,39 @@ def test_filesystem_uri(tempdir, use_legacy_dataset): # filesystem object result = pq.read_table( - path, filesystem=fs.LocalFileSystem(), - use_legacy_dataset=use_legacy_dataset) + path, filesystem=fs.LocalFileSystem()) assert result.equals(table) # filesystem URI result = pq.read_table( - "data_dir/data.parquet", filesystem=util._filesystem_uri(tempdir), - use_legacy_dataset=use_legacy_dataset) + "data_dir/data.parquet", filesystem=util._filesystem_uri(tempdir)) assert result.equals(table) @pytest.mark.pandas -@parametrize_legacy_dataset -def test_read_partitioned_directory(tempdir, use_legacy_dataset): - fs = LocalFileSystem._get_instance() - _partition_test_for_filesystem(fs, tempdir, use_legacy_dataset) - - -@pytest.mark.filterwarnings("ignore:'ParquetDataset:FutureWarning") -@pytest.mark.pandas -def test_create_parquet_dataset_multi_threaded(tempdir): +@pytest.mark.dataset +def test_read_partitioned_directory(tempdir): fs = LocalFileSystem._get_instance() - base_path = tempdir - - _partition_test_for_filesystem(fs, base_path) - - manifest = pq.ParquetManifest(base_path, filesystem=fs, - metadata_nthreads=1) - with pytest.warns( - FutureWarning, match="Specifying the 'metadata_nthreads'" - ): - dataset = pq.ParquetDataset( - base_path, filesystem=fs, metadata_nthreads=16, - use_legacy_dataset=True - ) - assert len(dataset.pieces) > 0 - partitions = dataset.partitions - assert len(partitions.partition_names) > 0 - assert partitions.partition_names == manifest.partitions.partition_names - assert len(partitions.levels) == len(manifest.partitions.levels) + _partition_test_for_filesystem(fs, tempdir) @pytest.mark.pandas -@parametrize_legacy_dataset -def test_read_partitioned_columns_selection(tempdir, use_legacy_dataset): +@pytest.mark.dataset +def test_read_partitioned_columns_selection(tempdir): # ARROW-3861 - do not include partition columns in resulting table when # `columns` keyword was passed without those columns fs = LocalFileSystem._get_instance() base_path = tempdir _partition_test_for_filesystem(fs, base_path) - dataset = pq.ParquetDataset( - base_path, use_legacy_dataset=use_legacy_dataset) + dataset = pq.ParquetDataset(base_path) result = dataset.read(columns=["values"]) - if use_legacy_dataset: - # ParquetDataset implementation always includes the partition columns - # automatically, and we can't easily "fix" this since dask relies on - # this behaviour (ARROW-8644) - assert result.column_names == ["values", "foo", "bar"] - else: - assert result.column_names == ["values"] + assert result.column_names == ["values"] @pytest.mark.pandas -@parametrize_legacy_dataset -def test_filters_equivalency(tempdir, use_legacy_dataset): +@pytest.mark.dataset +def test_filters_equivalency(tempdir): fs = LocalFileSystem._get_instance() base_path = tempdir @@ -225,7 +136,6 @@ def test_filters_equivalency(tempdir, use_legacy_dataset): base_path, filesystem=fs, filters=[('integer', '=', 1), ('string', '!=', 'b'), ('boolean', '==', 'True')], - use_legacy_dataset=use_legacy_dataset, ) table = dataset.read() result_df = (table.to_pandas().reset_index(drop=True)) @@ -247,8 +157,7 @@ def test_filters_equivalency(tempdir, use_legacy_dataset): [('integer', '=', 0), ('boolean', '==', 'False')] ] dataset = pq.ParquetDataset( - base_path, filesystem=fs, filters=filters, - use_legacy_dataset=use_legacy_dataset) + base_path, filesystem=fs, filters=filters) table = dataset.read() result_df = table.to_pandas().reset_index(drop=True) @@ -262,30 +171,16 @@ def test_filters_equivalency(tempdir, use_legacy_dataset): assert df_filter_2.sum() > 0 assert result_df.shape[0] == (df_filter_1.sum() + df_filter_2.sum()) - if use_legacy_dataset: - # Check for \0 in predicate values. Until they are correctly - # implemented in ARROW-3391, they would otherwise lead to weird - # results with the current code. - with pytest.raises(NotImplementedError): - filters = [[('string', '==', b'1\0a')]] - pq.ParquetDataset(base_path, filesystem=fs, filters=filters, - use_legacy_dataset=True) - with pytest.raises(NotImplementedError): - filters = [[('string', '==', '1\0a')]] - pq.ParquetDataset(base_path, filesystem=fs, filters=filters, - use_legacy_dataset=True) - else: - for filters in [[[('string', '==', b'1\0a')]], - [[('string', '==', '1\0a')]]]: - dataset = pq.ParquetDataset( - base_path, filesystem=fs, filters=filters, - use_legacy_dataset=False) - assert dataset.read().num_rows == 0 + for filters in [[[('string', '==', b'1\0a')]], + [[('string', '==', '1\0a')]]]: + dataset = pq.ParquetDataset( + base_path, filesystem=fs, filters=filters) + assert dataset.read().num_rows == 0 @pytest.mark.pandas -@parametrize_legacy_dataset -def test_filters_cutoff_exclusive_integer(tempdir, use_legacy_dataset): +@pytest.mark.dataset +def test_filters_cutoff_exclusive_integer(tempdir): fs = LocalFileSystem._get_instance() base_path = tempdir @@ -308,7 +203,6 @@ def test_filters_cutoff_exclusive_integer(tempdir, use_legacy_dataset): ('integers', '<', 4), ('integers', '>', 1), ], - use_legacy_dataset=use_legacy_dataset ) table = dataset.read() result_df = (table.to_pandas() @@ -319,15 +213,15 @@ def test_filters_cutoff_exclusive_integer(tempdir, use_legacy_dataset): assert result_list == [2, 3] -@pytest.mark.pandas -@parametrize_legacy_dataset @pytest.mark.xfail( # different error with use_legacy_datasets because result_df is no longer # categorical raises=(TypeError, AssertionError), reason='Loss of type information in creation of categoricals.' ) -def test_filters_cutoff_exclusive_datetime(tempdir, use_legacy_dataset): +@pytest.mark.dataset +@pytest.mark.pandas +def test_filters_cutoff_exclusive_datetime(tempdir): fs = LocalFileSystem._get_instance() base_path = tempdir @@ -356,7 +250,6 @@ def test_filters_cutoff_exclusive_datetime(tempdir, use_legacy_dataset): ('dates', '<', "2018-04-12"), ('dates', '>', "2018-04-10") ], - use_legacy_dataset=use_legacy_dataset ) table = dataset.read() result_df = (table.to_pandas() @@ -389,8 +282,8 @@ def test_filters_inclusive_datetime(tempdir): @pytest.mark.pandas -@parametrize_legacy_dataset -def test_filters_inclusive_integer(tempdir, use_legacy_dataset): +@pytest.mark.dataset +def test_filters_inclusive_integer(tempdir): fs = LocalFileSystem._get_instance() base_path = tempdir @@ -413,7 +306,6 @@ def test_filters_inclusive_integer(tempdir, use_legacy_dataset): ('integers', '<=', 3), ('integers', '>=', 2), ], - use_legacy_dataset=use_legacy_dataset ) table = dataset.read() result_df = (table.to_pandas() @@ -425,8 +317,8 @@ def test_filters_inclusive_integer(tempdir, use_legacy_dataset): @pytest.mark.pandas -@parametrize_legacy_dataset -def test_filters_inclusive_set(tempdir, use_legacy_dataset): +@pytest.mark.dataset +def test_filters_inclusive_set(tempdir): fs = LocalFileSystem._get_instance() base_path = tempdir @@ -451,7 +343,6 @@ def test_filters_inclusive_set(tempdir, use_legacy_dataset): dataset = pq.ParquetDataset( base_path, filesystem=fs, filters=[('string', 'in', 'ab')], - use_legacy_dataset=use_legacy_dataset ) table = dataset.read() result_df = (table.to_pandas().reset_index(drop=True)) @@ -464,7 +355,6 @@ def test_filters_inclusive_set(tempdir, use_legacy_dataset): base_path, filesystem=fs, filters=[('integer', 'in', [1]), ('string', 'in', ('a', 'b')), ('boolean', 'not in', {'False'})], - use_legacy_dataset=use_legacy_dataset ) table = dataset.read() result_df = (table.to_pandas().reset_index(drop=True)) @@ -475,8 +365,8 @@ def test_filters_inclusive_set(tempdir, use_legacy_dataset): @pytest.mark.pandas -@parametrize_legacy_dataset -def test_filters_invalid_pred_op(tempdir, use_legacy_dataset): +@pytest.mark.dataset +def test_filters_invalid_pred_op(tempdir): fs = LocalFileSystem._get_instance() base_path = tempdir @@ -496,49 +386,31 @@ def test_filters_invalid_pred_op(tempdir, use_legacy_dataset): with pytest.raises(TypeError): pq.ParquetDataset(base_path, filesystem=fs, - filters=[('integers', 'in', 3), ], - use_legacy_dataset=use_legacy_dataset) + filters=[('integers', 'in', 3), ]) with pytest.raises(ValueError): pq.ParquetDataset(base_path, filesystem=fs, - filters=[('integers', '=<', 3), ], - use_legacy_dataset=use_legacy_dataset) - - if use_legacy_dataset: - with pytest.raises(ValueError): - pq.ParquetDataset(base_path, - filesystem=fs, - filters=[('integers', 'in', set()), ], - use_legacy_dataset=use_legacy_dataset) - else: - # Dataset API returns empty table instead - dataset = pq.ParquetDataset(base_path, - filesystem=fs, - filters=[('integers', 'in', set()), ], - use_legacy_dataset=use_legacy_dataset) - assert dataset.read().num_rows == 0 + filters=[('integers', '=<', 3), ]) - if use_legacy_dataset: - with pytest.raises(ValueError): - pq.ParquetDataset(base_path, - filesystem=fs, - filters=[('integers', '!=', {3})], - use_legacy_dataset=use_legacy_dataset) - else: - dataset = pq.ParquetDataset(base_path, - filesystem=fs, - filters=[('integers', '!=', {3})], - use_legacy_dataset=use_legacy_dataset) - with pytest.raises(NotImplementedError): - assert dataset.read().num_rows == 0 + # Dataset API returns empty table + dataset = pq.ParquetDataset(base_path, + filesystem=fs, + filters=[('integers', 'in', set()), ]) + assert dataset.read().num_rows == 0 + + dataset = pq.ParquetDataset(base_path, + filesystem=fs, + filters=[('integers', '!=', {3})]) + with pytest.raises(NotImplementedError): + assert dataset.read().num_rows == 0 @pytest.mark.pandas -@parametrize_legacy_dataset_fixed -def test_filters_invalid_column(tempdir, use_legacy_dataset): +@pytest.mark.dataset +def test_filters_invalid_column(tempdir): # ARROW-5572 - raise error on invalid name in filter specification - # works with new dataset / xfail with legacy implementation + # works with new dataset fs = LocalFileSystem._get_instance() base_path = tempdir @@ -556,12 +428,11 @@ def test_filters_invalid_column(tempdir, use_legacy_dataset): msg = r"No match for FieldRef.Name\(non_existent_column\)" with pytest.raises(ValueError, match=msg): pq.ParquetDataset(base_path, filesystem=fs, - filters=[('non_existent_column', '<', 3), ], - use_legacy_dataset=use_legacy_dataset).read() + filters=[('non_existent_column', '<', 3), ]).read() @pytest.mark.pandas -@parametrize_legacy_dataset +@pytest.mark.dataset @pytest.mark.parametrize("filters", ([('integers', '<', 3)], [[('integers', '<', 3)]], @@ -569,7 +440,7 @@ def test_filters_invalid_column(tempdir, use_legacy_dataset): pc.field('nested', 'a') < 3, pc.field('nested', 'b').cast(pa.int64()) < 3)) @pytest.mark.parametrize("read_method", ("read_table", "read_pandas")) -def test_filters_read_table(tempdir, use_legacy_dataset, filters, read_method): +def test_filters_read_table(tempdir, filters, read_method): read = getattr(pq, read_method) # test that filters keyword is passed through in read_table fs = LocalFileSystem._get_instance() @@ -589,24 +460,16 @@ def test_filters_read_table(tempdir, use_legacy_dataset, filters, read_method): _generate_partition_directories(fs, base_path, partition_spec, df) - kwargs = dict(filesystem=fs, filters=filters, - use_legacy_dataset=use_legacy_dataset) + kwargs = dict(filesystem=fs, filters=filters) - # Using Expression in legacy dataset not supported - if use_legacy_dataset and isinstance(filters, pc.Expression): - msg = "Expressions as filter not supported for legacy dataset" - with pytest.raises(TypeError, match=msg): - read(base_path, **kwargs) - else: - table = read(base_path, **kwargs) - assert table.num_rows == 3 + table = read(base_path, **kwargs) + assert table.num_rows == 3 @pytest.mark.pandas -@parametrize_legacy_dataset_fixed -def test_partition_keys_with_underscores(tempdir, use_legacy_dataset): +@pytest.mark.dataset +def test_partition_keys_with_underscores(tempdir): # ARROW-5666 - partition field values with underscores preserve underscores - # xfail with legacy dataset -> they get interpreted as integers fs = LocalFileSystem._get_instance() base_path = tempdir @@ -623,60 +486,51 @@ def test_partition_keys_with_underscores(tempdir, use_legacy_dataset): _generate_partition_directories(fs, base_path, partition_spec, df) - dataset = pq.ParquetDataset( - base_path, use_legacy_dataset=use_legacy_dataset) + dataset = pq.ParquetDataset(base_path) result = dataset.read() assert result.column("year_week").to_pylist() == string_keys @pytest.mark.s3 -@parametrize_legacy_dataset -def test_read_s3fs(s3_example_s3fs, use_legacy_dataset): +@pytest.mark.dataset +def test_read_s3fs(s3_example_s3fs, ): fs, path = s3_example_s3fs path = path + "/test.parquet" table = pa.table({"a": [1, 2, 3]}) _write_table(table, path, filesystem=fs) - result = _read_table( - path, filesystem=fs, use_legacy_dataset=use_legacy_dataset - ) + result = _read_table(path, filesystem=fs) assert result.equals(table) @pytest.mark.s3 -@parametrize_legacy_dataset -def test_read_directory_s3fs(s3_example_s3fs, use_legacy_dataset): +@pytest.mark.dataset +def test_read_directory_s3fs(s3_example_s3fs): fs, directory = s3_example_s3fs path = directory + "/test.parquet" table = pa.table({"a": [1, 2, 3]}) _write_table(table, path, filesystem=fs) - result = _read_table( - directory, filesystem=fs, use_legacy_dataset=use_legacy_dataset - ) + result = _read_table(directory, filesystem=fs) assert result.equals(table) @pytest.mark.pandas -@parametrize_legacy_dataset -def test_read_single_file_list(tempdir, use_legacy_dataset): +@pytest.mark.dataset +def test_read_single_file_list(tempdir): data_path = str(tempdir / 'data.parquet') table = pa.table({"a": [1, 2, 3]}) _write_table(table, data_path) - result = pq.ParquetDataset( - [data_path], use_legacy_dataset=use_legacy_dataset - ).read() + result = pq.ParquetDataset([data_path]).read() assert result.equals(table) @pytest.mark.pandas @pytest.mark.s3 -@parametrize_legacy_dataset -def test_read_partitioned_directory_s3fs_wrapper( - s3_example_s3fs, use_legacy_dataset -): +@pytest.mark.dataset +def test_read_partitioned_directory_s3fs_wrapper(s3_example_s3fs): import s3fs from pyarrow.filesystem import S3FSWrapper @@ -690,23 +544,19 @@ def test_read_partitioned_directory_s3fs_wrapper( _partition_test_for_filesystem(wrapper, path) # Check that we can auto-wrap - dataset = pq.ParquetDataset( - path, filesystem=fs, use_legacy_dataset=use_legacy_dataset - ) + dataset = pq.ParquetDataset(path, filesystem=fs) dataset.read() @pytest.mark.pandas @pytest.mark.s3 -@parametrize_legacy_dataset -def test_read_partitioned_directory_s3fs(s3_example_s3fs, use_legacy_dataset): +@pytest.mark.dataset +def test_read_partitioned_directory_s3fs(s3_example_s3fs): fs, path = s3_example_s3fs - _partition_test_for_filesystem( - fs, path, use_legacy_dataset=use_legacy_dataset - ) + _partition_test_for_filesystem(fs, path) -def _partition_test_for_filesystem(fs, base_path, use_legacy_dataset=True): +def _partition_test_for_filesystem(fs, base_path): foo_keys = [0, 1] bar_keys = ['a', 'b', 'c'] partition_spec = [ @@ -724,8 +574,7 @@ def _partition_test_for_filesystem(fs, base_path, use_legacy_dataset=True): _generate_partition_directories(fs, base_path, partition_spec, df) - dataset = pq.ParquetDataset( - base_path, filesystem=fs, use_legacy_dataset=use_legacy_dataset) + dataset = pq.ParquetDataset(base_path, filesystem=fs) table = dataset.read() result_df = (table.to_pandas() .sort_values(by='index') @@ -735,15 +584,12 @@ def _partition_test_for_filesystem(fs, base_path, use_legacy_dataset=True): .reset_index(drop=True) .reindex(columns=result_df.columns)) - if use_legacy_dataset or Version(pd.__version__) < Version("2.0.0"): - expected_df['foo'] = pd.Categorical(df['foo'], categories=foo_keys) - expected_df['bar'] = pd.Categorical(df['bar'], categories=bar_keys) - else: - # With pandas 2.0.0 Index can store all numeric dtypes (not just - # int64/uint64/float64). Using astype() to create a categorical - # column preserves original dtype (int32) - expected_df['foo'] = expected_df['foo'].astype("category") - expected_df['bar'] = expected_df['bar'].astype("category") + + # With pandas 2.0.0 Index can store all numeric dtypes (not just + # int64/uint64/float64). Using astype() to create a categorical + # column preserves original dtype (int32) + expected_df['foo'] = expected_df['foo'].astype("category") + expected_df['bar'] = expected_df['bar'].astype("category") assert (result_df.columns == ['index', 'values', 'foo', 'bar']).all() @@ -790,83 +636,6 @@ def _visit_level(base_dir, level, part_keys): _visit_level(base_dir, 0, []) -def _test_read_common_metadata_files(fs, base_path): - import pandas as pd - - import pyarrow.parquet as pq - - N = 100 - df = pd.DataFrame({ - 'index': np.arange(N), - 'values': np.random.randn(N) - }, columns=['index', 'values']) - - base_path = str(base_path) - data_path = os.path.join(base_path, 'data.parquet') - - table = pa.Table.from_pandas(df) - - with fs.open(data_path, 'wb') as f: - _write_table(table, f) - - metadata_path = os.path.join(base_path, '_common_metadata') - with fs.open(metadata_path, 'wb') as f: - pq.write_metadata(table.schema, f) - - dataset = pq.ParquetDataset(base_path, filesystem=fs, - use_legacy_dataset=True) - with pytest.warns(FutureWarning): - assert dataset.common_metadata_path == str(metadata_path) - - with fs.open(data_path) as f: - common_schema = pq.read_metadata(f).schema - assert dataset.schema.equals(common_schema) - - # handle list of one directory - dataset2 = pq.ParquetDataset([base_path], filesystem=fs, - use_legacy_dataset=True) - assert dataset2.schema.equals(dataset.schema) - - -@pytest.mark.pandas -@pytest.mark.filterwarnings("ignore:'ParquetDataset.schema:FutureWarning") -def test_read_common_metadata_files(tempdir): - fs = LocalFileSystem._get_instance() - _test_read_common_metadata_files(fs, tempdir) - - -@pytest.mark.pandas -@pytest.mark.filterwarnings("ignore:'ParquetDataset.schema:FutureWarning") -def test_read_metadata_files(tempdir): - fs = LocalFileSystem._get_instance() - - N = 100 - df = pd.DataFrame({ - 'index': np.arange(N), - 'values': np.random.randn(N) - }, columns=['index', 'values']) - - data_path = tempdir / 'data.parquet' - - table = pa.Table.from_pandas(df) - - with fs.open(data_path, 'wb') as f: - _write_table(table, f) - - metadata_path = tempdir / '_metadata' - with fs.open(metadata_path, 'wb') as f: - pq.write_metadata(table.schema, f) - - dataset = pq.ParquetDataset(tempdir, filesystem=fs, - use_legacy_dataset=True) - with pytest.warns(FutureWarning): - assert dataset.metadata_path == str(metadata_path) - - with fs.open(data_path) as f: - metadata_schema = pq.read_metadata(f).schema - assert dataset.schema.equals(metadata_schema) - - def _filter_partition(df, part_keys): predicate = np.ones(len(df), dtype=bool) @@ -883,9 +652,9 @@ def _filter_partition(df, part_keys): return df[predicate].drop(to_drop, axis=1) -@parametrize_legacy_dataset @pytest.mark.pandas -def test_filter_before_validate_schema(tempdir, use_legacy_dataset): +@pytest.mark.dataset +def test_filter_before_validate_schema(tempdir): # ARROW-4076 apply filter before schema validation # to avoid checking unneeded schemas @@ -902,16 +671,13 @@ def test_filter_before_validate_schema(tempdir, use_legacy_dataset): pq.write_table(table2, dir2 / 'data.parquet') # read single file using filter - table = pq.read_table(tempdir, filters=[[('A', '==', 0)]], - use_legacy_dataset=use_legacy_dataset) + table = pq.read_table(tempdir, filters=[[('A', '==', 0)]]) assert table.column('B').equals(pa.chunked_array([[1, 2, 3]])) @pytest.mark.pandas -@pytest.mark.filterwarnings( - "ignore:Specifying the 'metadata':FutureWarning") -@parametrize_legacy_dataset -def test_read_multiple_files(tempdir, use_legacy_dataset): +@pytest.mark.dataset +def test_read_multiple_files(tempdir): nfiles = 10 size = 5 @@ -938,8 +704,7 @@ def test_read_multiple_files(tempdir, use_legacy_dataset): (dirpath / '_SUCCESS.crc').touch() def read_multiple_files(paths, columns=None, use_threads=True, **kwargs): - dataset = pq.ParquetDataset( - paths, use_legacy_dataset=use_legacy_dataset, **kwargs) + dataset = pq.ParquetDataset(paths, **kwargs) return dataset.read(columns=columns, use_threads=use_threads) result = read_multiple_files(paths) @@ -948,36 +713,23 @@ def read_multiple_files(paths, columns=None, use_threads=True, **kwargs): assert result.equals(expected) # Read with provided metadata - # TODO(dataset) specifying metadata not yet supported + # TODO specifying metadata not yet supported metadata = pq.read_metadata(paths[0]) - if use_legacy_dataset: - result2 = read_multiple_files(paths, metadata=metadata) - assert result2.equals(expected) - - with pytest.warns(FutureWarning, match="Specifying the 'schema'"): - result3 = pq.ParquetDataset(dirpath, schema=metadata.schema, - use_legacy_dataset=True).read() - assert result3.equals(expected) - else: - with pytest.raises(ValueError, match="no longer supported"): - pq.read_table(paths, metadata=metadata, use_legacy_dataset=False) + with pytest.raises(ValueError, match="no longer supported"): + pq.read_table(paths, metadata=metadata) # Read column subset to_read = [0, 2, 6, result.num_columns - 1] col_names = [result.field(i).name for i in to_read] - out = pq.read_table( - dirpath, columns=col_names, use_legacy_dataset=use_legacy_dataset - ) + out = pq.read_table(dirpath, columns=col_names) expected = pa.Table.from_arrays([result.column(i) for i in to_read], names=col_names, metadata=result.schema.metadata) assert out.equals(expected) # Read with multiple threads - pq.read_table( - dirpath, use_threads=True, use_legacy_dataset=use_legacy_dataset - ) + pq.read_table(dirpath, use_threads=True) # Test failure modes with non-uniform metadata bad_apple = _test_dataframe(size, seed=i).iloc[:, :4] @@ -986,31 +738,25 @@ def read_multiple_files(paths, columns=None, use_threads=True, **kwargs): t = pa.Table.from_pandas(bad_apple) _write_table(t, bad_apple_path) - if not use_legacy_dataset: - # TODO(dataset) Dataset API skips bad files - return + # TODO(dataset) Dataset API skips bad files - bad_meta = pq.read_metadata(bad_apple_path) + # bad_meta = pq.read_metadata(bad_apple_path) - with pytest.raises(ValueError): - read_multiple_files(paths + [bad_apple_path]) + # with pytest.raises(ValueError): + # read_multiple_files(paths + [bad_apple_path]) - with pytest.raises(ValueError): - read_multiple_files(paths, metadata=bad_meta) - - mixed_paths = [bad_apple_path, paths[0]] + # with pytest.raises(ValueError): + # read_multiple_files(paths, metadata=bad_meta) - with pytest.raises(ValueError): - with pytest.warns(FutureWarning, match="Specifying the 'schema'"): - read_multiple_files(mixed_paths, schema=bad_meta.schema) + # mixed_paths = [bad_apple_path, paths[0]] - with pytest.raises(ValueError): - read_multiple_files(mixed_paths) + # with pytest.raises(ValueError): + # read_multiple_files(mixed_paths) @pytest.mark.pandas -@parametrize_legacy_dataset -def test_dataset_read_pandas(tempdir, use_legacy_dataset): +@pytest.mark.dataset +def test_dataset_read_pandas(tempdir): nfiles = 5 size = 5 @@ -1033,7 +779,7 @@ def test_dataset_read_pandas(tempdir, use_legacy_dataset): frames.append(df) paths.append(path) - dataset = pq.ParquetDataset(dirpath, use_legacy_dataset=use_legacy_dataset) + dataset = pq.ParquetDataset(dirpath) columns = ['uint8', 'strings'] result = dataset.read_pandas(columns=columns).to_pandas() expected = pd.concat([x[columns] for x in frames]) @@ -1047,10 +793,9 @@ def test_dataset_read_pandas(tempdir, use_legacy_dataset): tm.assert_frame_equal(result.reindex(columns=expected.columns), expected) -@pytest.mark.filterwarnings("ignore:'ParquetDataset:FutureWarning") @pytest.mark.pandas -@parametrize_legacy_dataset -def test_dataset_memory_map(tempdir, use_legacy_dataset): +@pytest.mark.dataset +def test_dataset_memory_map(tempdir): # ARROW-2627: Check that we can use ParquetDataset with memory-mapping dirpath = tempdir / guid() dirpath.mkdir() @@ -1061,15 +806,13 @@ def test_dataset_memory_map(tempdir, use_legacy_dataset): _write_table(table, path, version='2.6') dataset = pq.ParquetDataset( - dirpath, memory_map=True, use_legacy_dataset=use_legacy_dataset) + dirpath, memory_map=True) assert dataset.read().equals(table) - if use_legacy_dataset: - assert dataset.pieces[0].read().equals(table) @pytest.mark.pandas -@parametrize_legacy_dataset -def test_dataset_enable_buffered_stream(tempdir, use_legacy_dataset): +@pytest.mark.dataset +def test_dataset_enable_buffered_stream(tempdir): dirpath = tempdir / guid() dirpath.mkdir() @@ -1080,19 +823,17 @@ def test_dataset_enable_buffered_stream(tempdir, use_legacy_dataset): with pytest.raises(ValueError): pq.ParquetDataset( - dirpath, buffer_size=-64, - use_legacy_dataset=use_legacy_dataset) + dirpath, buffer_size=-64) for buffer_size in [128, 1024]: dataset = pq.ParquetDataset( - dirpath, buffer_size=buffer_size, - use_legacy_dataset=use_legacy_dataset) + dirpath, buffer_size=buffer_size) assert dataset.read().equals(table) @pytest.mark.pandas -@parametrize_legacy_dataset -def test_dataset_enable_pre_buffer(tempdir, use_legacy_dataset): +@pytest.mark.dataset +def test_dataset_enable_pre_buffer(tempdir): dirpath = tempdir / guid() dirpath.mkdir() @@ -1103,11 +844,9 @@ def test_dataset_enable_pre_buffer(tempdir, use_legacy_dataset): for pre_buffer in (True, False): dataset = pq.ParquetDataset( - dirpath, pre_buffer=pre_buffer, - use_legacy_dataset=use_legacy_dataset) + dirpath, pre_buffer=pre_buffer) assert dataset.read().equals(table) - actual = pq.read_table(dirpath, pre_buffer=pre_buffer, - use_legacy_dataset=use_legacy_dataset) + actual = pq.read_table(dirpath, pre_buffer=pre_buffer) assert actual.equals(table) @@ -1123,18 +862,15 @@ def _make_example_multifile_dataset(base_path, nfiles=10, file_nrows=5): return paths -def _assert_dataset_paths(dataset, paths, use_legacy_dataset): - if use_legacy_dataset: - assert set(map(str, paths)) == {x.path for x in dataset._pieces} - else: - paths = [str(path.as_posix()) for path in paths] - assert set(paths) == set(dataset._dataset.files) +def _assert_dataset_paths(dataset, paths): + paths = [str(path.as_posix()) for path in paths] + assert set(paths) == set(dataset._dataset.files) @pytest.mark.pandas -@parametrize_legacy_dataset +@pytest.mark.dataset @pytest.mark.parametrize('dir_prefix', ['_', '.']) -def test_ignore_private_directories(tempdir, dir_prefix, use_legacy_dataset): +def test_ignore_private_directories(tempdir, dir_prefix): dirpath = tempdir / guid() dirpath.mkdir() @@ -1144,14 +880,14 @@ def test_ignore_private_directories(tempdir, dir_prefix, use_legacy_dataset): # private directory (dirpath / '{}staging'.format(dir_prefix)).mkdir() - dataset = pq.ParquetDataset(dirpath, use_legacy_dataset=use_legacy_dataset) + dataset = pq.ParquetDataset(dirpath) - _assert_dataset_paths(dataset, paths, use_legacy_dataset) + _assert_dataset_paths(dataset, paths) @pytest.mark.pandas -@parametrize_legacy_dataset -def test_ignore_hidden_files_dot(tempdir, use_legacy_dataset): +@pytest.mark.dataset +def test_ignore_hidden_files_dot(tempdir): dirpath = tempdir / guid() dirpath.mkdir() @@ -1164,14 +900,14 @@ def test_ignore_hidden_files_dot(tempdir, use_legacy_dataset): with (dirpath / '.private').open('wb') as f: f.write(b'gibberish') - dataset = pq.ParquetDataset(dirpath, use_legacy_dataset=use_legacy_dataset) + dataset = pq.ParquetDataset(dirpath) - _assert_dataset_paths(dataset, paths, use_legacy_dataset) + _assert_dataset_paths(dataset, paths) @pytest.mark.pandas -@parametrize_legacy_dataset -def test_ignore_hidden_files_underscore(tempdir, use_legacy_dataset): +@pytest.mark.dataset +def test_ignore_hidden_files_underscore(tempdir): dirpath = tempdir / guid() dirpath.mkdir() @@ -1184,17 +920,15 @@ def test_ignore_hidden_files_underscore(tempdir, use_legacy_dataset): with (dirpath / '_started_321').open('wb') as f: f.write(b'abcd') - dataset = pq.ParquetDataset(dirpath, use_legacy_dataset=use_legacy_dataset) + dataset = pq.ParquetDataset(dirpath) - _assert_dataset_paths(dataset, paths, use_legacy_dataset) + _assert_dataset_paths(dataset, paths) @pytest.mark.pandas -@parametrize_legacy_dataset +@pytest.mark.dataset @pytest.mark.parametrize('dir_prefix', ['_', '.']) -def test_ignore_no_private_directories_in_base_path( - tempdir, dir_prefix, use_legacy_dataset -): +def test_ignore_no_private_directories_in_base_path(tempdir, dir_prefix): # ARROW-8427 - don't ignore explicitly listed files if parent directory # is a private directory dirpath = tempdir / "{0}data".format(dir_prefix) / guid() @@ -1203,17 +937,17 @@ def test_ignore_no_private_directories_in_base_path( paths = _make_example_multifile_dataset(dirpath, nfiles=10, file_nrows=5) - dataset = pq.ParquetDataset(paths, use_legacy_dataset=use_legacy_dataset) - _assert_dataset_paths(dataset, paths, use_legacy_dataset) + dataset = pq.ParquetDataset(paths) + _assert_dataset_paths(dataset, paths) # ARROW-9644 - don't ignore full directory with underscore in base path - dataset = pq.ParquetDataset(dirpath, use_legacy_dataset=use_legacy_dataset) - _assert_dataset_paths(dataset, paths, use_legacy_dataset) + dataset = pq.ParquetDataset(dirpath) + _assert_dataset_paths(dataset, paths) @pytest.mark.pandas -@parametrize_legacy_dataset_fixed -def test_ignore_custom_prefixes(tempdir, use_legacy_dataset): +@pytest.mark.dataset +def test_ignore_custom_prefixes(tempdir): # ARROW-9573 - allow override of default ignore_prefixes part = ["xxx"] * 3 + ["yyy"] * 3 table = pa.table([ @@ -1221,7 +955,6 @@ def test_ignore_custom_prefixes(tempdir, use_legacy_dataset): pa.array(part).dictionary_encode(), ], names=['index', '_part']) - # TODO use_legacy_dataset ARROW-10247 pq.write_to_dataset(table, str(tempdir), partition_cols=['_part']) private_duplicate = tempdir / '_private_duplicate' @@ -1230,32 +963,26 @@ def test_ignore_custom_prefixes(tempdir, use_legacy_dataset): partition_cols=['_part']) read = pq.read_table( - tempdir, use_legacy_dataset=use_legacy_dataset, - ignore_prefixes=['_private']) + tempdir, ignore_prefixes=['_private']) assert read.equals(table) -@parametrize_legacy_dataset_fixed -def test_empty_directory(tempdir, use_legacy_dataset): - # ARROW-5310 - reading empty directory - # fails with legacy implementation +@pytest.mark.dataset +def test_empty_directory(tempdir): empty_dir = tempdir / 'dataset' empty_dir.mkdir() - dataset = pq.ParquetDataset( - empty_dir, use_legacy_dataset=use_legacy_dataset) + dataset = pq.ParquetDataset(empty_dir) result = dataset.read() assert result.num_rows == 0 assert result.num_columns == 0 -@pytest.mark.filterwarnings("ignore:'ParquetDataset.schema:FutureWarning") def _test_write_to_dataset_with_partitions(base_path, - use_legacy_dataset=True, filesystem=None, schema=None, - index_name=None): + index_name=None): import pandas as pd import pandas.testing as tm @@ -1275,8 +1002,7 @@ def _test_write_to_dataset_with_partitions(base_path, output_table = pa.Table.from_pandas(output_df, schema=schema, safe=False, preserve_index=False) pq.write_to_dataset(output_table, base_path, partition_by, - filesystem=filesystem, - use_legacy_dataset=use_legacy_dataset) + filesystem=filesystem) metadata_path = os.path.join(str(base_path), '_common_metadata') @@ -1290,15 +1016,10 @@ def _test_write_to_dataset_with_partitions(base_path, # partitioned dataset dataset = pq.ParquetDataset(base_path, filesystem=filesystem, - validate_schema=True, - use_legacy_dataset=use_legacy_dataset) + validate_schema=True) # ARROW-2209: Ensure the dataset schema also includes the partition columns - if use_legacy_dataset: - with pytest.warns(FutureWarning, match="'ParquetDataset.schema'"): - dataset_cols = set(dataset.schema.to_arrow_schema().names) - else: - # NB schema property is an arrow and not parquet schema - dataset_cols = set(dataset.schema.names) + # NB schema property is an arrow and not parquet schema + dataset_cols = set(dataset.schema.names) assert dataset_cols == set(output_table.schema.names) @@ -1323,7 +1044,6 @@ def _test_write_to_dataset_with_partitions(base_path, def _test_write_to_dataset_no_partitions(base_path, - use_legacy_dataset=True, filesystem=None): import pandas as pd @@ -1347,7 +1067,6 @@ def _test_write_to_dataset_no_partitions(base_path, n = 5 for i in range(n): pq.write_to_dataset(output_table, base_path, - use_legacy_dataset=use_legacy_dataset, filesystem=filesystem) output_files = [file for file in filesystem.ls(str(base_path)) if file.endswith(".parquet")] @@ -1356,8 +1075,7 @@ def _test_write_to_dataset_no_partitions(base_path, # Deduplicated incoming DataFrame should match # original outgoing Dataframe input_table = pq.ParquetDataset( - base_path, filesystem=filesystem, - use_legacy_dataset=use_legacy_dataset + base_path, filesystem=filesystem ).read() input_df = input_table.to_pandas() input_df = input_df.drop_duplicates() @@ -1366,128 +1084,77 @@ def _test_write_to_dataset_no_partitions(base_path, @pytest.mark.pandas -@parametrize_legacy_dataset -def test_write_to_dataset_with_partitions(tempdir, use_legacy_dataset): - _test_write_to_dataset_with_partitions(str(tempdir), use_legacy_dataset) +@pytest.mark.dataset +def test_write_to_dataset_with_partitions(tempdir): + _test_write_to_dataset_with_partitions(str(tempdir)) @pytest.mark.pandas -@parametrize_legacy_dataset -def test_write_to_dataset_with_partitions_and_schema( - tempdir, use_legacy_dataset -): +@pytest.mark.dataset +def test_write_to_dataset_with_partitions_and_schema(tempdir): schema = pa.schema([pa.field('group1', type=pa.string()), pa.field('group2', type=pa.string()), pa.field('num', type=pa.int64()), pa.field('nan', type=pa.int32()), pa.field('date', type=pa.timestamp(unit='us'))]) _test_write_to_dataset_with_partitions( - str(tempdir), use_legacy_dataset, schema=schema) + str(tempdir), schema=schema) @pytest.mark.pandas -@parametrize_legacy_dataset -def test_write_to_dataset_with_partitions_and_index_name( - tempdir, use_legacy_dataset -): +@pytest.mark.dataset +def test_write_to_dataset_with_partitions_and_index_name(tempdir): _test_write_to_dataset_with_partitions( - str(tempdir), use_legacy_dataset, index_name='index_name') + str(tempdir), index_name='index_name') @pytest.mark.pandas -@parametrize_legacy_dataset -def test_write_to_dataset_no_partitions(tempdir, use_legacy_dataset): - _test_write_to_dataset_no_partitions(str(tempdir), use_legacy_dataset) +@pytest.mark.dataset +def test_write_to_dataset_no_partitions(tempdir): + _test_write_to_dataset_no_partitions(str(tempdir)) @pytest.mark.pandas -@parametrize_legacy_dataset -def test_write_to_dataset_pathlib(tempdir, use_legacy_dataset): - _test_write_to_dataset_with_partitions( - tempdir / "test1", use_legacy_dataset) - _test_write_to_dataset_no_partitions( - tempdir / "test2", use_legacy_dataset) +@pytest.mark.dataset +def test_write_to_dataset_pathlib(tempdir): + _test_write_to_dataset_with_partitions(tempdir / "test1") + _test_write_to_dataset_no_partitions(tempdir / "test2") @pytest.mark.pandas @pytest.mark.s3 -@parametrize_legacy_dataset -def test_write_to_dataset_pathlib_nonlocal( - tempdir, s3_example_s3fs, use_legacy_dataset -): +@pytest.mark.dataset +def test_write_to_dataset_pathlib_nonlocal(tempdir, s3_example_s3fs): # pathlib paths are only accepted for local files fs, _ = s3_example_s3fs with pytest.raises(TypeError, match="path-like objects are only allowed"): _test_write_to_dataset_with_partitions( - tempdir / "test1", use_legacy_dataset, filesystem=fs) + tempdir / "test1", filesystem=fs) with pytest.raises(TypeError, match="path-like objects are only allowed"): _test_write_to_dataset_no_partitions( - tempdir / "test2", use_legacy_dataset, filesystem=fs) + tempdir / "test2", filesystem=fs) @pytest.mark.pandas @pytest.mark.s3 -@parametrize_legacy_dataset -def test_write_to_dataset_with_partitions_s3fs( - s3_example_s3fs, use_legacy_dataset -): +@pytest.mark.dataset +def test_write_to_dataset_with_partitions_s3fs(s3_example_s3fs): fs, path = s3_example_s3fs _test_write_to_dataset_with_partitions( - path, use_legacy_dataset, filesystem=fs) + path, filesystem=fs) @pytest.mark.pandas @pytest.mark.s3 -@parametrize_legacy_dataset -def test_write_to_dataset_no_partitions_s3fs( - s3_example_s3fs, use_legacy_dataset -): +@pytest.mark.dataset +def test_write_to_dataset_no_partitions_s3fs(s3_example_s3fs): fs, path = s3_example_s3fs _test_write_to_dataset_no_partitions( - path, use_legacy_dataset, filesystem=fs) - - -@pytest.mark.filterwarnings( - "ignore:'ParquetDataset:FutureWarning", - "ignore:'partition_filename_cb':FutureWarning") -@pytest.mark.pandas -@parametrize_legacy_dataset_not_supported -def test_write_to_dataset_with_partitions_and_custom_filenames( - tempdir, use_legacy_dataset -): - output_df = pd.DataFrame({'group1': list('aaabbbbccc'), - 'group2': list('eefeffgeee'), - 'num': list(range(10)), - 'nan': [np.nan] * 10, - 'date': np.arange('2017-01-01', '2017-01-11', - dtype='datetime64[D]')}) - partition_by = ['group1', 'group2'] - output_table = pa.Table.from_pandas(output_df) - path = str(tempdir) - - def partition_filename_callback(keys): - return "{}-{}.parquet".format(*keys) - - pq.write_to_dataset(output_table, path, - partition_by, partition_filename_callback, - use_legacy_dataset=use_legacy_dataset) - - dataset = pq.ParquetDataset(path, use_legacy_dataset=use_legacy_dataset) - - # ARROW-3538: Ensure partition filenames match the given pattern - # defined in the local function partition_filename_callback - expected_basenames = [ - 'a-e.parquet', 'a-f.parquet', - 'b-e.parquet', 'b-f.parquet', - 'b-g.parquet', 'c-e.parquet' - ] - output_basenames = [os.path.basename(p.path) for p in dataset.pieces] - - assert sorted(expected_basenames) == sorted(output_basenames) + path, filesystem=fs) @pytest.mark.dataset @@ -1502,7 +1169,7 @@ def test_write_to_dataset_filesystem(tempdir): assert result.equals(table) -def _make_dataset_for_pickling(tempdir, use_legacy_dataset=False, N=100): +def _make_dataset_for_pickling(tempdir, N=100): path = tempdir / 'data.parquet' fs = LocalFileSystem._get_instance() @@ -1525,42 +1192,24 @@ def _make_dataset_for_pickling(tempdir, use_legacy_dataset=False, N=100): pq.write_metadata(table.schema, f) dataset = pq.ParquetDataset( - tempdir, filesystem=fs, use_legacy_dataset=use_legacy_dataset) - if use_legacy_dataset: - with pytest.warns(FutureWarning): - assert dataset.metadata_path == str(metadata_path) + tempdir, filesystem=fs) return dataset @pytest.mark.pandas -@parametrize_legacy_dataset -def test_pickle_dataset(tempdir, datadir, use_legacy_dataset, pickle_module): +@pytest.mark.dataset +def test_pickle_dataset(tempdir, pickle_module): def is_pickleable(obj): return obj == pickle_module.loads(pickle_module.dumps(obj)) - dataset = _make_dataset_for_pickling(tempdir, use_legacy_dataset) + dataset = _make_dataset_for_pickling(tempdir) assert is_pickleable(dataset) - if use_legacy_dataset: - with pytest.warns(FutureWarning): - metadata = dataset.metadata - assert is_pickleable(metadata) - assert is_pickleable(metadata.schema) - assert len(metadata.schema) - for column in metadata.schema: - assert is_pickleable(column) - - for piece in dataset._pieces: - assert is_pickleable(piece) - metadata = piece.get_metadata() - assert metadata.num_row_groups - for i in range(metadata.num_row_groups): - assert is_pickleable(metadata.row_group(i)) @pytest.mark.pandas -@parametrize_legacy_dataset -def test_partitioned_dataset(tempdir, use_legacy_dataset): +@pytest.mark.dataset +def test_partitioned_dataset(tempdir): # ARROW-3208: Segmentation fault when reading a Parquet partitioned dataset # to a Parquet file path = tempdir / "ARROW-3208" @@ -1571,27 +1220,22 @@ def test_partitioned_dataset(tempdir, use_legacy_dataset): }) table = pa.Table.from_pandas(df) pq.write_to_dataset(table, root_path=str(path), - partition_cols=['one', 'two'], - use_legacy_dataset=use_legacy_dataset) - table = pq.ParquetDataset( - path, use_legacy_dataset=use_legacy_dataset).read() + partition_cols=['one', 'two']) + table = pq.ParquetDataset(path).read() pq.write_table(table, path / "output.parquet") @pytest.mark.pandas -@parametrize_legacy_dataset -def test_dataset_read_dictionary(tempdir, use_legacy_dataset): +@pytest.mark.dataset +def test_dataset_read_dictionary(tempdir): path = tempdir / "ARROW-3325-dataset" t1 = pa.table([[util.rands(10) for i in range(5)] * 10], names=['f0']) t2 = pa.table([[util.rands(10) for i in range(5)] * 10], names=['f0']) - pq.write_to_dataset(t1, root_path=str(path), - use_legacy_dataset=use_legacy_dataset) - pq.write_to_dataset(t2, root_path=str(path), - use_legacy_dataset=use_legacy_dataset) + pq.write_to_dataset(t1, root_path=str(path)) + pq.write_to_dataset(t2, root_path=str(path)) result = pq.ParquetDataset( - path, read_dictionary=['f0'], - use_legacy_dataset=use_legacy_dataset).read() + path, read_dictionary=['f0']).read() # The order of the chunks is non-deterministic ex_chunks = [t1[0].chunk(0).dictionary_encode(), @@ -1608,7 +1252,6 @@ def test_dataset_read_dictionary(tempdir, use_legacy_dataset): @pytest.mark.dataset @pytest.mark.pandas -@pytest.mark.filterwarnings("ignore:Passing 'use_legacy:FutureWarning") def test_read_table_schema(tempdir): # test that schema keyword is passed through in read_table table = pa.table({'a': pa.array([1, 2, 3], pa.int32())}) @@ -1627,17 +1270,7 @@ def test_read_table_schema(tempdir): expected = pa.table({'a': [1, 2, 3, 1, 2, 3]}, schema=schema) assert result.equals(expected) - # don't allow it with the legacy reader - with pytest.raises( - ValueError, match="The 'schema' argument is only supported" - ): - pq.read_table(tempdir / "data.parquet", schema=schema, - use_legacy_dataset=True) - - # using ParquetDataset directory with non-legacy implementation - result = pq.ParquetDataset( - tempdir, schema=schema, use_legacy_dataset=False - ) + result = pq.ParquetDataset(tempdir, schema=schema) expected = pa.table({'a': [1, 2, 3, 1, 2, 3]}, schema=schema) assert result.read().equals(expected) @@ -1646,23 +1279,22 @@ def test_read_table_schema(tempdir): def test_dataset_unsupported_keywords(): with pytest.raises(ValueError, match="not yet supported with the new"): - pq.ParquetDataset("", use_legacy_dataset=False, metadata=pa.schema([])) + pq.ParquetDataset("", metadata=pa.schema([])) with pytest.raises(ValueError, match="not yet supported with the new"): - pq.ParquetDataset("", use_legacy_dataset=False, validate_schema=False) + pq.ParquetDataset("", validate_schema=False) with pytest.raises(ValueError, match="not yet supported with the new"): - pq.ParquetDataset("", use_legacy_dataset=False, split_row_groups=True) + pq.ParquetDataset("", split_row_groups=True) with pytest.raises(ValueError, match="not yet supported with the new"): - pq.ParquetDataset("", use_legacy_dataset=False, metadata_nthreads=4) + pq.ParquetDataset("", metadata_nthreads=4) with pytest.raises(ValueError, match="no longer supported"): - pq.read_table("", use_legacy_dataset=False, metadata=pa.schema([])) + pq.read_table("", metadata=pa.schema([])) @pytest.mark.dataset -@pytest.mark.filterwarnings("ignore:Passing 'use_legacy:FutureWarning") def test_dataset_partitioning(tempdir): import pyarrow.dataset as ds @@ -1679,42 +1311,27 @@ def test_dataset_partitioning(tempdir): # read_table part = ds.partitioning(field_names=["year", "month", "day"]) result = pq.read_table( - str(root_path), partitioning=part, use_legacy_dataset=False) + str(root_path), partitioning=part) assert result.column_names == ["a", "year", "month", "day"] result = pq.ParquetDataset( - str(root_path), partitioning=part, use_legacy_dataset=False).read() + str(root_path), partitioning=part).read() assert result.column_names == ["a", "year", "month", "day"] - # This raises an error for legacy dataset - with pytest.raises(ValueError): - pq.read_table( - str(root_path), partitioning=part, use_legacy_dataset=True) - - with pytest.raises(ValueError): - pq.ParquetDataset( - str(root_path), partitioning=part, use_legacy_dataset=True) - @pytest.mark.dataset def test_parquet_dataset_new_filesystem(tempdir): # Ensure we can pass new FileSystem object to ParquetDataset - # (use new implementation automatically without specifying - # use_legacy_dataset=False) table = pa.table({'a': [1, 2, 3]}) pq.write_table(table, tempdir / 'data.parquet') - # don't use simple LocalFileSystem (as that gets mapped to legacy one) filesystem = fs.SubTreeFileSystem(str(tempdir), fs.LocalFileSystem()) dataset = pq.ParquetDataset('.', filesystem=filesystem) result = dataset.read() assert result.equals(table) -@pytest.mark.filterwarnings("ignore:'ParquetDataset:FutureWarning") -@parametrize_legacy_dataset -def test_parquet_dataset_partitions_piece_path_with_fsspec( - tempdir, use_legacy_dataset -): +@pytest.mark.dataset +def test_parquet_dataset_partitions_piece_path_with_fsspec(tempdir): # ARROW-10462 ensure that on Windows we properly use posix-style paths # as used by fsspec fsspec = pytest.importorskip("fsspec") @@ -1725,108 +1342,12 @@ def test_parquet_dataset_partitions_piece_path_with_fsspec( # pass a posix-style path (using "/" also on Windows) path = str(tempdir).replace("\\", "/") dataset = pq.ParquetDataset( - path, filesystem=filesystem, use_legacy_dataset=use_legacy_dataset) + path, filesystem=filesystem) # ensure the piece path is also posix-style expected = path + "/data.parquet" assert dataset.pieces[0].path == expected -@pytest.mark.dataset -def test_parquet_dataset_deprecated_properties(tempdir): - table = pa.table({'a': [1, 2, 3]}) - path = tempdir / 'data.parquet' - pq.write_table(table, path) - dataset = pq.ParquetDataset(path, use_legacy_dataset=True) - - with pytest.warns(FutureWarning, match="'ParquetDataset.pieces"): - dataset.pieces - - with pytest.warns(FutureWarning, match="'ParquetDataset.partitions"): - dataset.partitions - - with pytest.warns(FutureWarning, match="'ParquetDataset.memory_map"): - dataset.memory_map - - with pytest.warns(FutureWarning, match="'ParquetDataset.read_dictio"): - dataset.read_dictionary - - with pytest.warns(FutureWarning, match="'ParquetDataset.buffer_size"): - dataset.buffer_size - - with pytest.warns(FutureWarning, match="'ParquetDataset.fs"): - dataset.fs - - with pytest.warns(FutureWarning, match="'ParquetDataset.schema'"): - dataset.schema - - with pytest.warns(FutureWarning, match="'ParquetDataset.common_metadata'"): - dataset.common_metadata - - with pytest.warns(FutureWarning, match="'ParquetDataset.metadata"): - dataset.metadata - - with pytest.warns(FutureWarning, match="'ParquetDataset.metadata_path"): - dataset.metadata_path - - with pytest.warns(FutureWarning, - match="'ParquetDataset.common_metadata_path"): - dataset.common_metadata_path - - dataset2 = pq.ParquetDataset(path, use_legacy_dataset=False) - - with pytest.warns(FutureWarning, match="'ParquetDataset.pieces"): - dataset2.pieces - - -@pytest.mark.dataset -def test_parquet_write_to_dataset_deprecated_properties(tempdir): - table = pa.table({'a': [1, 2, 3]}) - path = tempdir / 'data.parquet' - - with pytest.warns(FutureWarning, - match="Passing 'use_legacy_dataset=True'"): - pq.write_to_dataset(table, path, use_legacy_dataset=True) - - # check also that legacy implementation is set when - # partition_filename_cb is specified - with pytest.warns(FutureWarning, - match="Passing 'use_legacy_dataset=True'"): - pq.write_to_dataset(table, path, - partition_filename_cb=lambda x: 'filename.parquet') - - -@pytest.mark.dataset -def test_parquet_write_to_dataset_unsupported_keywords_in_legacy(tempdir): - table = pa.table({'a': [1, 2, 3]}) - path = tempdir / 'data.parquet' - - with pytest.raises(ValueError, match="schema"): - pq.write_to_dataset(table, path, use_legacy_dataset=True, - schema=pa.schema([ - ('a', pa.int32()) - ])) - - with pytest.raises(ValueError, match="partitioning"): - pq.write_to_dataset(table, path, use_legacy_dataset=True, - partitioning=["a"]) - - with pytest.raises(ValueError, match="use_threads"): - pq.write_to_dataset(table, path, use_legacy_dataset=True, - use_threads=False) - - with pytest.raises(ValueError, match="file_visitor"): - pq.write_to_dataset(table, path, use_legacy_dataset=True, - file_visitor=lambda x: x) - - with pytest.raises(ValueError, match="existing_data_behavior"): - pq.write_to_dataset(table, path, use_legacy_dataset=True, - existing_data_behavior='error') - - with pytest.raises(ValueError, match="basename_template"): - pq.write_to_dataset(table, path, use_legacy_dataset=True, - basename_template='part-{i}.parquet') - - @pytest.mark.dataset def test_parquet_write_to_dataset_exposed_keywords(tempdir): table = pa.table({'a': [1, 2, 3]}) @@ -1841,8 +1362,7 @@ def file_visitor(written_file): pq.write_to_dataset(table, path, partitioning=["a"], file_visitor=file_visitor, - basename_template=basename_template, - use_legacy_dataset=False) + basename_template=basename_template) expected_paths = { path / '1' / 'part-0.parquet', @@ -1853,52 +1373,6 @@ def file_visitor(written_file): assert paths_written_set == expected_paths -@pytest.mark.dataset -def test_write_to_dataset_conflicting_keywords(tempdir): - table = pa.table({'a': [1, 2, 3]}) - path = tempdir / 'data.parquet' - - with pytest.raises(ValueError, match="'basename_template' argument " - "is not supported by use_legacy_dataset=True"): - pq.write_to_dataset(table, path, - use_legacy_dataset=True, - partition_filename_cb=lambda x: 'filename.parquet', - basename_template='file-{i}.parquet') - with pytest.raises(ValueError, match="'partition_filename_cb' argument " - "is not supported by use_legacy_dataset=False"): - pq.write_to_dataset(table, path, - use_legacy_dataset=False, - partition_filename_cb=lambda x: 'filename.parquet', - basename_template='file-{i}.parquet') - - with pytest.raises(ValueError, match="'partitioning' argument " - "is not supported by use_legacy_dataset=True"): - pq.write_to_dataset(table, path, - use_legacy_dataset=True, - partition_cols=["a"], - partitioning=["a"]) - - with pytest.raises(ValueError, match="'partition_cols' argument " - "is not supported by use_legacy_dataset=False"): - pq.write_to_dataset(table, path, - use_legacy_dataset=False, - partition_cols=["a"], - partitioning=["a"]) - - with pytest.raises(ValueError, match="'file_visitor' argument " - "is not supported by use_legacy_dataset=True"): - pq.write_to_dataset(table, path, - use_legacy_dataset=True, - metadata_collector=[], - file_visitor=lambda x: x) - with pytest.raises(ValueError, match="'metadata_collector' argument " - "is not supported by use_legacy_dataset=False"): - pq.write_to_dataset(table, path, - use_legacy_dataset=False, - metadata_collector=[], - file_visitor=lambda x: x) - - @pytest.mark.dataset @pytest.mark.parametrize("write_dataset_kwarg", ( ("create_dir", True), @@ -1926,8 +1400,8 @@ def test_write_to_dataset_kwargs_passed(tempdir, write_dataset_kwarg): @pytest.mark.pandas -@parametrize_legacy_dataset -def test_write_to_dataset_category_observed(tempdir, use_legacy_dataset): +@pytest.mark.dataset +def test_write_to_dataset_category_observed(tempdir): # if we partition on a categorical variable with "unobserved" categories # (values present in the dictionary, but not in the actual data) # ensure those are not creating empty files/directories @@ -1938,8 +1412,7 @@ def test_write_to_dataset_category_observed(tempdir, use_legacy_dataset): table = pa.table(df) path = tempdir / "dataset" pq.write_to_dataset( - table, tempdir / "dataset", partition_cols=["cat"], - use_legacy_dataset=use_legacy_dataset + table, tempdir / "dataset", partition_cols=["cat"] ) subdirs = [f.name for f in path.iterdir() if f.is_dir()] assert len(subdirs) == 2 diff --git a/python/pyarrow/tests/parquet/test_datetime.py b/python/pyarrow/tests/parquet/test_datetime.py index f97c451df7ad7..1e653d45a3940 100644 --- a/python/pyarrow/tests/parquet/test_datetime.py +++ b/python/pyarrow/tests/parquet/test_datetime.py @@ -23,8 +23,7 @@ import pytest import pyarrow as pa -from pyarrow.tests.parquet.common import ( - _check_roundtrip, parametrize_legacy_dataset) +from pyarrow.tests.parquet.common import _check_roundtrip try: import pyarrow.parquet as pq @@ -48,8 +47,8 @@ @pytest.mark.pandas -@parametrize_legacy_dataset -def test_pandas_parquet_datetime_tz(use_legacy_dataset): +@pytest.mark.dataset +def test_pandas_parquet_datetime_tz(): # Pandas v2 defaults to [ns], but Arrow defaults to [us] time units # so we need to cast the pandas dtype. Pandas v1 will always silently # coerce to [ns] due to lack of non-[ns] support. @@ -69,21 +68,20 @@ def test_pandas_parquet_datetime_tz(use_legacy_dataset): _write_table(arrow_table, f) f.seek(0) - table_read = pq.read_pandas(f, use_legacy_dataset=use_legacy_dataset) + table_read = pq.read_pandas(f) df_read = table_read.to_pandas() tm.assert_frame_equal(df, df_read) @pytest.mark.pandas -@parametrize_legacy_dataset -def test_datetime_timezone_tzinfo(use_legacy_dataset): +@pytest.mark.dataset +def test_datetime_timezone_tzinfo(): value = datetime.datetime(2018, 1, 1, 1, 23, 45, tzinfo=datetime.timezone.utc) df = pd.DataFrame({'foo': [value]}) - _roundtrip_pandas_dataframe( - df, write_kwargs={}, use_legacy_dataset=use_legacy_dataset) + _roundtrip_pandas_dataframe(df, write_kwargs={}) @pytest.mark.pandas diff --git a/python/pyarrow/tests/parquet/test_pandas.py b/python/pyarrow/tests/parquet/test_pandas.py index 0ed305bff1945..84f698db73bb9 100644 --- a/python/pyarrow/tests/parquet/test_pandas.py +++ b/python/pyarrow/tests/parquet/test_pandas.py @@ -23,8 +23,6 @@ import pyarrow as pa from pyarrow.fs import LocalFileSystem, SubTreeFileSystem -from pyarrow.tests.parquet.common import ( - parametrize_legacy_dataset, parametrize_legacy_dataset_not_supported) from pyarrow.util import guid from pyarrow.vendored.version import Version @@ -101,8 +99,8 @@ def test_merging_parquet_tables_with_different_pandas_metadata(tempdir): @pytest.mark.pandas -@parametrize_legacy_dataset -def test_pandas_parquet_column_multiindex(tempdir, use_legacy_dataset): +@pytest.mark.dataset +def test_pandas_parquet_column_multiindex(tempdir): df = alltypes_sample(size=10) df.columns = pd.MultiIndex.from_tuples( list(zip(df.columns, df.columns[::-1])), @@ -115,17 +113,14 @@ def test_pandas_parquet_column_multiindex(tempdir, use_legacy_dataset): _write_table(arrow_table, filename) - table_read = pq.read_pandas( - filename, use_legacy_dataset=use_legacy_dataset) + table_read = pq.read_pandas(filename) df_read = table_read.to_pandas() tm.assert_frame_equal(df, df_read) @pytest.mark.pandas -@parametrize_legacy_dataset -def test_pandas_parquet_2_0_roundtrip_read_pandas_no_index_written( - tempdir, use_legacy_dataset -): +@pytest.mark.dataset +def test_pandas_parquet_2_0_roundtrip_read_pandas_no_index_written(tempdir): df = alltypes_sample(size=10000) filename = tempdir / 'pandas_roundtrip.parquet' @@ -137,8 +132,7 @@ def test_pandas_parquet_2_0_roundtrip_read_pandas_no_index_written( assert js['columns'] _write_table(arrow_table, filename) - table_read = pq.read_pandas( - filename, use_legacy_dataset=use_legacy_dataset) + table_read = pq.read_pandas(filename) js = table_read.schema.pandas_metadata assert not js['index_columns'] @@ -150,52 +144,22 @@ def test_pandas_parquet_2_0_roundtrip_read_pandas_no_index_written( tm.assert_frame_equal(df, df_read) -# TODO(dataset) duplicate column selection actually gives duplicate columns now -@pytest.mark.pandas -@parametrize_legacy_dataset_not_supported -def test_pandas_column_selection(tempdir, use_legacy_dataset): - size = 10000 - np.random.seed(0) - df = pd.DataFrame({ - 'uint8': np.arange(size, dtype=np.uint8), - 'uint16': np.arange(size, dtype=np.uint16) - }) - filename = tempdir / 'pandas_roundtrip.parquet' - arrow_table = pa.Table.from_pandas(df) - _write_table(arrow_table, filename) - table_read = _read_table( - filename, columns=['uint8'], use_legacy_dataset=use_legacy_dataset) - df_read = table_read.to_pandas() - - tm.assert_frame_equal(df[['uint8']], df_read) - - # ARROW-4267: Selection of duplicate columns still leads to these columns - # being read uniquely. - table_read = _read_table( - filename, columns=['uint8', 'uint8'], - use_legacy_dataset=use_legacy_dataset) - df_read = table_read.to_pandas() - - tm.assert_frame_equal(df[['uint8']], df_read) - - @pytest.mark.pandas -@parametrize_legacy_dataset -def test_pandas_parquet_native_file_roundtrip(tempdir, use_legacy_dataset): +@pytest.mark.dataset +def test_pandas_parquet_native_file_roundtrip(tempdir): df = _test_dataframe(10000) arrow_table = pa.Table.from_pandas(df) imos = pa.BufferOutputStream() _write_table(arrow_table, imos, version='2.6') buf = imos.getvalue() reader = pa.BufferReader(buf) - df_read = _read_table( - reader, use_legacy_dataset=use_legacy_dataset).to_pandas() + df_read = _read_table(reader).to_pandas() tm.assert_frame_equal(df, df_read) @pytest.mark.pandas -@parametrize_legacy_dataset -def test_read_pandas_column_subset(tempdir, use_legacy_dataset): +@pytest.mark.dataset +def test_read_pandas_column_subset(tempdir): df = _test_dataframe(10000) arrow_table = pa.Table.from_pandas(df) imos = pa.BufferOutputStream() @@ -204,22 +168,20 @@ def test_read_pandas_column_subset(tempdir, use_legacy_dataset): reader = pa.BufferReader(buf) df_read = pq.read_pandas( reader, columns=['strings', 'uint8'], - use_legacy_dataset=use_legacy_dataset ).to_pandas() tm.assert_frame_equal(df[['strings', 'uint8']], df_read) @pytest.mark.pandas -@parametrize_legacy_dataset -def test_pandas_parquet_empty_roundtrip(tempdir, use_legacy_dataset): +@pytest.mark.dataset +def test_pandas_parquet_empty_roundtrip(tempdir): df = _test_dataframe(0) arrow_table = pa.Table.from_pandas(df) imos = pa.BufferOutputStream() _write_table(arrow_table, imos, version='2.6') buf = imos.getvalue() reader = pa.BufferReader(buf) - df_read = _read_table( - reader, use_legacy_dataset=use_legacy_dataset).to_pandas() + df_read = _read_table(reader).to_pandas() tm.assert_frame_equal(df, df_read) @@ -241,8 +203,8 @@ def test_pandas_can_write_nested_data(tempdir): @pytest.mark.pandas -@parametrize_legacy_dataset -def test_pandas_parquet_pyfile_roundtrip(tempdir, use_legacy_dataset): +@pytest.mark.dataset +def test_pandas_parquet_pyfile_roundtrip(tempdir): filename = tempdir / 'pandas_pyfile_roundtrip.parquet' size = 5 df = pd.DataFrame({ @@ -260,14 +222,14 @@ def test_pandas_parquet_pyfile_roundtrip(tempdir, use_legacy_dataset): data = io.BytesIO(filename.read_bytes()) - table_read = _read_table(data, use_legacy_dataset=use_legacy_dataset) + table_read = _read_table(data) df_read = table_read.to_pandas() tm.assert_frame_equal(df, df_read) @pytest.mark.pandas -@parametrize_legacy_dataset -def test_pandas_parquet_configuration_options(tempdir, use_legacy_dataset): +@pytest.mark.dataset +def test_pandas_parquet_configuration_options(tempdir): size = 10000 np.random.seed(0) df = pd.DataFrame({ @@ -289,16 +251,14 @@ def test_pandas_parquet_configuration_options(tempdir, use_legacy_dataset): for use_dictionary in [True, False]: _write_table(arrow_table, filename, version='2.6', use_dictionary=use_dictionary) - table_read = _read_table( - filename, use_legacy_dataset=use_legacy_dataset) + table_read = _read_table(filename) df_read = table_read.to_pandas() tm.assert_frame_equal(df, df_read) for write_statistics in [True, False]: _write_table(arrow_table, filename, version='2.6', write_statistics=write_statistics) - table_read = _read_table(filename, - use_legacy_dataset=use_legacy_dataset) + table_read = _read_table(filename) df_read = table_read.to_pandas() tm.assert_frame_equal(df, df_read) @@ -308,8 +268,7 @@ def test_pandas_parquet_configuration_options(tempdir, use_legacy_dataset): continue _write_table(arrow_table, filename, version='2.6', compression=compression) - table_read = _read_table( - filename, use_legacy_dataset=use_legacy_dataset) + table_read = _read_table(filename) df_read = table_read.to_pandas() tm.assert_frame_equal(df, df_read) @@ -327,8 +286,8 @@ def test_spark_flavor_preserves_pandas_metadata(): @pytest.mark.pandas -@parametrize_legacy_dataset -def test_index_column_name_duplicate(tempdir, use_legacy_dataset): +@pytest.mark.dataset +def test_index_column_name_duplicate(tempdir): data = { 'close': { pd.Timestamp('2017-06-30 01:31:00'): 154.99958999999998, @@ -352,14 +311,14 @@ def test_index_column_name_duplicate(tempdir, use_legacy_dataset): tdfx = pa.Table.from_pandas(dfx) _write_table(tdfx, path) - arrow_table = _read_table(path, use_legacy_dataset=use_legacy_dataset) + arrow_table = _read_table(path) result_df = arrow_table.to_pandas() tm.assert_frame_equal(result_df, dfx) @pytest.mark.pandas -@parametrize_legacy_dataset -def test_multiindex_duplicate_values(tempdir, use_legacy_dataset): +@pytest.mark.dataset +def test_multiindex_duplicate_values(tempdir): num_rows = 3 numbers = list(range(num_rows)) index = pd.MultiIndex.from_arrays( @@ -373,7 +332,7 @@ def test_multiindex_duplicate_values(tempdir, use_legacy_dataset): filename = tempdir / 'dup_multi_index_levels.parquet' _write_table(table, filename) - result_table = _read_table(filename, use_legacy_dataset=use_legacy_dataset) + result_table = _read_table(filename) assert table.equals(result_table) result_df = result_table.to_pandas() @@ -381,8 +340,8 @@ def test_multiindex_duplicate_values(tempdir, use_legacy_dataset): @pytest.mark.pandas -@parametrize_legacy_dataset -def test_backwards_compatible_index_naming(datadir, use_legacy_dataset): +@pytest.mark.dataset +def test_backwards_compatible_index_naming(datadir): expected_string = b"""\ carat cut color clarity depth table price x y z 0.23 Ideal E SI2 61.5 55.0 326 3.95 3.98 2.43 @@ -397,17 +356,14 @@ def test_backwards_compatible_index_naming(datadir, use_legacy_dataset): 0.23 Very Good H VS1 59.4 61.0 338 4.00 4.05 2.39""" expected = pd.read_csv(io.BytesIO(expected_string), sep=r'\s{2,}', index_col=None, header=0, engine='python') - table = _read_table( - datadir / 'v0.7.1.parquet', use_legacy_dataset=use_legacy_dataset) + table = _read_table(datadir / 'v0.7.1.parquet') result = table.to_pandas() tm.assert_frame_equal(result, expected) @pytest.mark.pandas -@parametrize_legacy_dataset -def test_backwards_compatible_index_multi_level_named( - datadir, use_legacy_dataset -): +@pytest.mark.dataset +def test_backwards_compatible_index_multi_level_named(datadir): expected_string = b"""\ carat cut color clarity depth table price x y z 0.23 Ideal E SI2 61.5 55.0 326 3.95 3.98 2.43 @@ -426,17 +382,14 @@ def test_backwards_compatible_index_multi_level_named( header=0, engine='python' ).sort_index() - table = _read_table(datadir / 'v0.7.1.all-named-index.parquet', - use_legacy_dataset=use_legacy_dataset) + table = _read_table(datadir / 'v0.7.1.all-named-index.parquet') result = table.to_pandas() tm.assert_frame_equal(result, expected) @pytest.mark.pandas -@parametrize_legacy_dataset -def test_backwards_compatible_index_multi_level_some_named( - datadir, use_legacy_dataset -): +@pytest.mark.dataset +def test_backwards_compatible_index_multi_level_some_named(datadir): expected_string = b"""\ carat cut color clarity depth table price x y z 0.23 Ideal E SI2 61.5 55.0 326 3.95 3.98 2.43 @@ -456,17 +409,14 @@ def test_backwards_compatible_index_multi_level_some_named( ).sort_index() expected.index = expected.index.set_names(['cut', None, 'clarity']) - table = _read_table(datadir / 'v0.7.1.some-named-index.parquet', - use_legacy_dataset=use_legacy_dataset) + table = _read_table(datadir / 'v0.7.1.some-named-index.parquet') result = table.to_pandas() tm.assert_frame_equal(result, expected) @pytest.mark.pandas -@parametrize_legacy_dataset -def test_backwards_compatible_column_metadata_handling( - datadir, use_legacy_dataset -): +@pytest.mark.dataset +def test_backwards_compatible_column_metadata_handling(datadir): expected = pd.DataFrame( {'a': [1, 2, 3], 'b': [.1, .2, .3], 'c': pd.date_range("2017-01-01", periods=3, tz='Europe/Brussels')}) @@ -476,19 +426,19 @@ def test_backwards_compatible_column_metadata_handling( names=['index', None]) path = datadir / 'v0.7.1.column-metadata-handling.parquet' - table = _read_table(path, use_legacy_dataset=use_legacy_dataset) + table = _read_table(path) result = table.to_pandas() tm.assert_frame_equal(result, expected) table = _read_table( - path, columns=['a'], use_legacy_dataset=use_legacy_dataset) + path, columns=['a']) result = table.to_pandas() tm.assert_frame_equal(result, expected[['a']].reset_index(drop=True)) @pytest.mark.pandas -@parametrize_legacy_dataset -def test_categorical_index_survives_roundtrip(use_legacy_dataset): +@pytest.mark.dataset +def test_categorical_index_survives_roundtrip(): # ARROW-3652, addressed by ARROW-3246 df = pd.DataFrame([['a', 'b'], ['c', 'd']], columns=['c1', 'c2']) df['c1'] = df['c1'].astype('category') @@ -497,15 +447,14 @@ def test_categorical_index_survives_roundtrip(use_legacy_dataset): table = pa.Table.from_pandas(df) bos = pa.BufferOutputStream() pq.write_table(table, bos) - ref_df = pq.read_pandas( - bos.getvalue(), use_legacy_dataset=use_legacy_dataset).to_pandas() + ref_df = pq.read_pandas(bos.getvalue()).to_pandas() assert isinstance(ref_df.index, pd.CategoricalIndex) assert ref_df.index.equals(df.index) @pytest.mark.pandas -@parametrize_legacy_dataset -def test_categorical_order_survives_roundtrip(use_legacy_dataset): +@pytest.mark.dataset +def test_categorical_order_survives_roundtrip(): # ARROW-6302 df = pd.DataFrame({"a": pd.Categorical( ["a", "b", "c", "a"], categories=["b", "c", "d"], ordered=True)}) @@ -515,15 +464,14 @@ def test_categorical_order_survives_roundtrip(use_legacy_dataset): pq.write_table(table, bos) contents = bos.getvalue() - result = pq.read_pandas( - contents, use_legacy_dataset=use_legacy_dataset).to_pandas() + result = pq.read_pandas(contents).to_pandas() tm.assert_frame_equal(result, df) @pytest.mark.pandas -@parametrize_legacy_dataset -def test_pandas_categorical_na_type_row_groups(use_legacy_dataset): +@pytest.mark.dataset +def test_pandas_categorical_na_type_row_groups(): # ARROW-5085 df = pd.DataFrame({"col": [None] * 100, "int": [1.0] * 100}) df_category = df.astype({"col": "category", "int": "category"}) @@ -533,8 +481,7 @@ def test_pandas_categorical_na_type_row_groups(use_legacy_dataset): # it works pq.write_table(table_cat, buf, version='2.6', chunk_size=10) - result = pq.read_table( - buf.getvalue(), use_legacy_dataset=use_legacy_dataset) + result = pq.read_table(buf.getvalue()) # Result is non-categorical assert result[0].equals(table[0]) @@ -542,8 +489,8 @@ def test_pandas_categorical_na_type_row_groups(use_legacy_dataset): @pytest.mark.pandas -@parametrize_legacy_dataset -def test_pandas_categorical_roundtrip(use_legacy_dataset): +@pytest.mark.dataset +def test_pandas_categorical_roundtrip(): # ARROW-5480, this was enabled by ARROW-3246 # Have one of the categories unobserved and include a null (-1) @@ -555,8 +502,7 @@ def test_pandas_categorical_roundtrip(use_legacy_dataset): buf = pa.BufferOutputStream() pq.write_table(pa.table(df), buf) - result = pq.read_table( - buf.getvalue(), use_legacy_dataset=use_legacy_dataset).to_pandas() + result = pq.read_table(buf.getvalue()).to_pandas() assert result.x.dtype == 'category' assert (result.x.cat.categories == categories).all() tm.assert_frame_equal(result, df) @@ -587,41 +533,30 @@ def test_categories_with_string_pyarrow_dtype(tempdir): @pytest.mark.pandas -@parametrize_legacy_dataset -def test_write_to_dataset_pandas_preserve_extensiondtypes( - tempdir, use_legacy_dataset -): +@pytest.mark.dataset +def test_write_to_dataset_pandas_preserve_extensiondtypes(tempdir): df = pd.DataFrame({'part': 'a', "col": [1, 2, 3]}) df['col'] = df['col'].astype("Int64") table = pa.table(df) pq.write_to_dataset( table, str(tempdir / "case1"), partition_cols=['part'], - use_legacy_dataset=use_legacy_dataset ) - result = pq.read_table( - str(tempdir / "case1"), use_legacy_dataset=use_legacy_dataset - ).to_pandas() + result = pq.read_table(str(tempdir / "case1")).to_pandas() tm.assert_frame_equal(result[["col"]], df[["col"]]) - pq.write_to_dataset( - table, str(tempdir / "case2"), use_legacy_dataset=use_legacy_dataset - ) - result = pq.read_table( - str(tempdir / "case2"), use_legacy_dataset=use_legacy_dataset - ).to_pandas() + pq.write_to_dataset(table, str(tempdir / "case2")) + result = pq.read_table(str(tempdir / "case2")).to_pandas() tm.assert_frame_equal(result[["col"]], df[["col"]]) pq.write_table(table, str(tempdir / "data.parquet")) - result = pq.read_table( - str(tempdir / "data.parquet"), use_legacy_dataset=use_legacy_dataset - ).to_pandas() + result = pq.read_table(str(tempdir / "data.parquet")).to_pandas() tm.assert_frame_equal(result[["col"]], df[["col"]]) @pytest.mark.pandas -@parametrize_legacy_dataset -def test_write_to_dataset_pandas_preserve_index(tempdir, use_legacy_dataset): +@pytest.mark.dataset +def test_write_to_dataset_pandas_preserve_index(tempdir): # ARROW-8251 - preserve pandas index in roundtrip df = pd.DataFrame({'part': ['a', 'a', 'b'], "col": [1, 2, 3]}) @@ -632,34 +567,25 @@ def test_write_to_dataset_pandas_preserve_index(tempdir, use_legacy_dataset): pq.write_to_dataset( table, str(tempdir / "case1"), partition_cols=['part'], - use_legacy_dataset=use_legacy_dataset ) - result = pq.read_table( - str(tempdir / "case1"), use_legacy_dataset=use_legacy_dataset - ).to_pandas() + result = pq.read_table(str(tempdir / "case1")).to_pandas() tm.assert_frame_equal(result, df_cat) - pq.write_to_dataset( - table, str(tempdir / "case2"), use_legacy_dataset=use_legacy_dataset - ) - result = pq.read_table( - str(tempdir / "case2"), use_legacy_dataset=use_legacy_dataset - ).to_pandas() + pq.write_to_dataset(table, str(tempdir / "case2")) + result = pq.read_table(str(tempdir / "case2")).to_pandas() tm.assert_frame_equal(result, df) pq.write_table(table, str(tempdir / "data.parquet")) - result = pq.read_table( - str(tempdir / "data.parquet"), use_legacy_dataset=use_legacy_dataset - ).to_pandas() + result = pq.read_table(str(tempdir / "data.parquet")).to_pandas() tm.assert_frame_equal(result, df) @pytest.mark.pandas -@parametrize_legacy_dataset +@pytest.mark.dataset @pytest.mark.parametrize('preserve_index', [True, False, None]) @pytest.mark.parametrize('metadata_fname', ["_metadata", "_common_metadata"]) def test_dataset_read_pandas_common_metadata( - tempdir, use_legacy_dataset, preserve_index, metadata_fname + tempdir, preserve_index, metadata_fname ): # ARROW-1103 nfiles = 5 @@ -696,7 +622,7 @@ def test_dataset_read_pandas_common_metadata( ) pq.write_metadata(table_for_metadata.schema, dirpath / metadata_fname) - dataset = pq.ParquetDataset(dirpath, use_legacy_dataset=use_legacy_dataset) + dataset = pq.ParquetDataset(dirpath) columns = ['uint8', 'strings'] result = dataset.read_pandas(columns=columns).to_pandas() expected = pd.concat([x[columns] for x in frames]) diff --git a/python/pyarrow/tests/parquet/test_parquet_file.py b/python/pyarrow/tests/parquet/test_parquet_file.py index 9f920206a107e..6453308d86b1e 100644 --- a/python/pyarrow/tests/parquet/test_parquet_file.py +++ b/python/pyarrow/tests/parquet/test_parquet_file.py @@ -296,28 +296,6 @@ def test_parquet_file_explicitly_closed(tempdir): table = pa.table({'col1': [0, 1], 'col2': [0, 1]}) pq.write_table(table, fn) - # read_table (legacy) with opened file (will leave open) - with open(fn, 'rb') as f: - pq.read_table(f, use_legacy_dataset=True) - assert not f.closed # Didn't close it internally after read_table - - # read_table (legacy) with unopened file (will close) - with mock.patch.object(pq.ParquetFile, "close") as mock_close: - pq.read_table(fn, use_legacy_dataset=True) - mock_close.assert_called() - - # ParquetDataset test (legacy) with unopened file (will close) - with mock.patch.object(pq.ParquetFile, "close") as mock_close: - pq.ParquetDataset(fn, use_legacy_dataset=True).read() - mock_close.assert_called() - - # ParquetDataset test (legacy) with opened file (will leave open) - with open(fn, 'rb') as f: - # ARROW-8075: support ParquetDataset from file-like, not just path-like - with pytest.raises(TypeError, match='not a path-like object'): - pq.ParquetDataset(f, use_legacy_dataset=True).read() - assert not f.closed - # ParquetFile with opened file (will leave open) with open(fn, 'rb') as f: with pq.ParquetFile(f) as p: @@ -338,7 +316,7 @@ def test_parquet_file_explicitly_closed(tempdir): @pytest.mark.s3 @pytest.mark.parametrize("use_uri", (True, False)) -def test_parquet_file_with_filesystem(tempdir, s3_example_fs, use_uri): +def test_parquet_file_with_filesystem(s3_example_fs, use_uri): s3_fs, s3_uri, s3_path = s3_example_fs args = (s3_uri if use_uri else s3_path,) diff --git a/python/pyarrow/tests/parquet/test_parquet_writer.py b/python/pyarrow/tests/parquet/test_parquet_writer.py index b902541015aa2..16584684f5c7f 100644 --- a/python/pyarrow/tests/parquet/test_parquet_writer.py +++ b/python/pyarrow/tests/parquet/test_parquet_writer.py @@ -20,7 +20,6 @@ import pyarrow as pa from pyarrow import fs from pyarrow.filesystem import FileSystem, LocalFileSystem -from pyarrow.tests.parquet.common import parametrize_legacy_dataset try: import pyarrow.parquet as pq @@ -44,8 +43,7 @@ @pytest.mark.pandas -@parametrize_legacy_dataset -def test_parquet_incremental_file_build(tempdir, use_legacy_dataset): +def test_parquet_incremental_file_build(tempdir): df = _test_dataframe(100) df['unique_id'] = 0 @@ -65,8 +63,7 @@ def test_parquet_incremental_file_build(tempdir, use_legacy_dataset): writer.close() buf = out.getvalue() - result = _read_table( - pa.BufferReader(buf), use_legacy_dataset=use_legacy_dataset) + result = _read_table(pa.BufferReader(buf)) expected = pd.concat(frames, ignore_index=True) tm.assert_frame_equal(result.to_pandas(), expected) @@ -105,8 +102,7 @@ def test_parquet_invalid_writer(tempdir): @pytest.mark.pandas -@parametrize_legacy_dataset -def test_parquet_writer_context_obj(tempdir, use_legacy_dataset): +def test_parquet_writer_context_obj(tempdir): df = _test_dataframe(100) df['unique_id'] = 0 @@ -124,18 +120,14 @@ def test_parquet_writer_context_obj(tempdir, use_legacy_dataset): frames.append(df.copy()) buf = out.getvalue() - result = _read_table( - pa.BufferReader(buf), use_legacy_dataset=use_legacy_dataset) + result = _read_table(pa.BufferReader(buf)) expected = pd.concat(frames, ignore_index=True) tm.assert_frame_equal(result.to_pandas(), expected) @pytest.mark.pandas -@parametrize_legacy_dataset -def test_parquet_writer_context_obj_with_exception( - tempdir, use_legacy_dataset -): +def test_parquet_writer_context_obj_with_exception(tempdir): df = _test_dataframe(100) df['unique_id'] = 0 @@ -160,8 +152,7 @@ def test_parquet_writer_context_obj_with_exception( assert str(e) == error_text buf = out.getvalue() - result = _read_table( - pa.BufferReader(buf), use_legacy_dataset=use_legacy_dataset) + result = _read_table(pa.BufferReader(buf)) expected = pd.concat(frames, ignore_index=True) tm.assert_frame_equal(result.to_pandas(), expected) @@ -340,8 +331,7 @@ def test_parquet_writer_filesystem_buffer_raises(): @pytest.mark.pandas -@parametrize_legacy_dataset -def test_parquet_writer_with_caller_provided_filesystem(use_legacy_dataset): +def test_parquet_writer_with_caller_provided_filesystem(): out = pa.BufferOutputStream() class CustomFS(FileSystem): @@ -368,8 +358,7 @@ def open(self, path, mode='rb'): assert out.closed buf = out.getvalue() - table_read = _read_table( - pa.BufferReader(buf), use_legacy_dataset=use_legacy_dataset) + table_read = _read_table(pa.BufferReader(buf)) df_read = table_read.to_pandas() tm.assert_frame_equal(df_read, df) diff --git a/python/pyarrow/tests/test_dataset.py b/python/pyarrow/tests/test_dataset.py index d5e7015a5d5b9..7c58b85de8091 100644 --- a/python/pyarrow/tests/test_dataset.py +++ b/python/pyarrow/tests/test_dataset.py @@ -1094,8 +1094,6 @@ def _create_dataset_for_fragments(tempdir, chunk_size=None, filesystem=None): @pytest.mark.pandas @pytest.mark.parquet -@pytest.mark.filterwarnings( - "ignore:Passing 'use_legacy_dataset=True':FutureWarning") def test_fragments(tempdir, dataset_reader): table, dataset = _create_dataset_for_fragments(tempdir) @@ -1144,8 +1142,6 @@ def test_fragments_implicit_cast(tempdir): @pytest.mark.pandas @pytest.mark.parquet -@pytest.mark.filterwarnings( - "ignore:Passing 'use_legacy_dataset=True':FutureWarning") def test_fragments_reconstruct(tempdir, dataset_reader, pickle_module): table, dataset = _create_dataset_for_fragments(tempdir) @@ -1208,8 +1204,6 @@ def assert_yields_projected(fragment, row_slice, @pytest.mark.pandas @pytest.mark.parquet -@pytest.mark.filterwarnings( - "ignore:Passing 'use_legacy_dataset=True':FutureWarning") def test_fragments_parquet_row_groups(tempdir, dataset_reader): table, dataset = _create_dataset_for_fragments(tempdir, chunk_size=2) @@ -1276,8 +1270,6 @@ def test_fragments_parquet_row_groups_dictionary(tempdir, dataset_reader): @pytest.mark.pandas @pytest.mark.parquet -@pytest.mark.filterwarnings( - "ignore:Passing 'use_legacy_dataset=True':FutureWarning") def test_fragments_parquet_ensure_metadata(tempdir, open_logging_fs, pickle_module): fs, assert_opens = open_logging_fs _, dataset = _create_dataset_for_fragments( @@ -1388,16 +1380,13 @@ def _create_dataset_all_types(tempdir, chunk_size=None): path = str(tempdir / "test_parquet_dataset_all_types") # write_to_dataset currently requires pandas - pq.write_to_dataset(table, path, use_legacy_dataset=True, - chunk_size=chunk_size) + pq.write_to_dataset(table, path, chunk_size=chunk_size) return table, ds.dataset(path, format="parquet", partitioning="hive") @pytest.mark.pandas @pytest.mark.parquet -@pytest.mark.filterwarnings( - "ignore:Passing 'use_legacy_dataset=True':FutureWarning") def test_parquet_fragment_statistics(tempdir): table, dataset = _create_dataset_all_types(tempdir) @@ -1465,8 +1454,6 @@ def test_parquet_empty_row_group_statistics(tempdir): @pytest.mark.pandas @pytest.mark.parquet -@pytest.mark.filterwarnings( - "ignore:Passing 'use_legacy_dataset=True':FutureWarning") def test_fragments_parquet_row_groups_predicate(tempdir): table, dataset = _create_dataset_for_fragments(tempdir, chunk_size=2) @@ -1491,8 +1478,6 @@ def test_fragments_parquet_row_groups_predicate(tempdir): @pytest.mark.pandas @pytest.mark.parquet -@pytest.mark.filterwarnings( - "ignore:Passing 'use_legacy_dataset=True':FutureWarning") def test_fragments_parquet_row_groups_reconstruct(tempdir, dataset_reader, pickle_module): table, dataset = _create_dataset_for_fragments(tempdir, chunk_size=2) @@ -1536,8 +1521,6 @@ def test_fragments_parquet_row_groups_reconstruct(tempdir, dataset_reader, @pytest.mark.pandas @pytest.mark.parquet -@pytest.mark.filterwarnings( - "ignore:Passing 'use_legacy_dataset=True':FutureWarning") def test_fragments_parquet_subset_ids(tempdir, open_logging_fs, dataset_reader): fs, assert_opens = open_logging_fs @@ -1567,8 +1550,6 @@ def test_fragments_parquet_subset_ids(tempdir, open_logging_fs, @pytest.mark.pandas @pytest.mark.parquet -@pytest.mark.filterwarnings( - "ignore:Passing 'use_legacy_dataset=True':FutureWarning") def test_fragments_parquet_subset_filter(tempdir, open_logging_fs, dataset_reader): fs, assert_opens = open_logging_fs @@ -1602,8 +1583,6 @@ def test_fragments_parquet_subset_filter(tempdir, open_logging_fs, @pytest.mark.pandas @pytest.mark.parquet -@pytest.mark.filterwarnings( - "ignore:Passing 'use_legacy_dataset=True':FutureWarning") def test_fragments_parquet_subset_invalid(tempdir): _, dataset = _create_dataset_for_fragments(tempdir, chunk_size=1) fragment = list(dataset.get_fragments())[0] @@ -3525,10 +3504,7 @@ def test_parquet_dataset_factory_fsspec(tempdir): @pytest.mark.parquet @pytest.mark.pandas # write_to_dataset currently requires pandas -@pytest.mark.parametrize('use_legacy_dataset', [False, True]) -@pytest.mark.filterwarnings( - "ignore:Passing 'use_legacy_dataset=True':FutureWarning") -def test_parquet_dataset_factory_roundtrip(tempdir, use_legacy_dataset): +def test_parquet_dataset_factory_roundtrip(tempdir): # Simple test to ensure we can roundtrip dataset to # _metadata/common_metadata and back. A more complex test # using partitioning will have to wait for ARROW-13269. The @@ -3540,7 +3516,6 @@ def test_parquet_dataset_factory_roundtrip(tempdir, use_legacy_dataset): metadata_collector = [] pq.write_to_dataset( table, str(root_path), metadata_collector=metadata_collector, - use_legacy_dataset=use_legacy_dataset ) metadata_path = str(root_path / '_metadata') # write _metadata file @@ -3864,8 +3839,7 @@ def test_write_to_dataset_given_null_just_works(tempdir): 'col': list(range(4))}, schema=schema) path = str(tempdir / 'test_dataset') - pq.write_to_dataset(table, path, partition_cols=[ - 'part'], use_legacy_dataset=False) + pq.write_to_dataset(table, path, partition_cols=['part']) actual_table = pq.read_table(tempdir / 'test_dataset') # column.equals can handle the difference in chunking but not the fact @@ -3875,28 +3849,6 @@ def test_write_to_dataset_given_null_just_works(tempdir): assert actual_table.column('col').equals(table.column('col')) -@pytest.mark.parquet -@pytest.mark.pandas -@pytest.mark.filterwarnings( - "ignore:Passing 'use_legacy_dataset=True':FutureWarning") -def test_legacy_write_to_dataset_drops_null(tempdir): - schema = pa.schema([ - pa.field('col', pa.int64()), - pa.field('part', pa.dictionary(pa.int32(), pa.string())) - ]) - table = pa.table({'part': ['a', 'a', None, None], - 'col': list(range(4))}, schema=schema) - expected = pa.table( - {'part': ['a', 'a'], 'col': list(range(2))}, schema=schema) - - path = str(tempdir / 'test_dataset') - pq.write_to_dataset(table, path, partition_cols=[ - 'part'], use_legacy_dataset=True) - - actual = pq.read_table(tempdir / 'test_dataset') - assert actual == expected - - def _sort_table(tab, sort_col): import pyarrow.compute as pc sorted_indices = pc.sort_indices( diff --git a/python/pyarrow/tests/test_hdfs.py b/python/pyarrow/tests/test_hdfs.py index 511dbf9a1c4e1..dce81f8890614 100644 --- a/python/pyarrow/tests/test_hdfs.py +++ b/python/pyarrow/tests/test_hdfs.py @@ -27,7 +27,7 @@ from pyarrow.tests import util from pyarrow.tests.parquet.common import _test_dataframe from pyarrow.tests.parquet.test_dataset import ( - _test_read_common_metadata_files, _test_write_to_dataset_with_partitions, + _test_write_to_dataset_with_partitions, _test_write_to_dataset_no_partitions ) from pyarrow.util import guid @@ -360,19 +360,10 @@ def test_read_write_parquet_files_with_uri(self): pq.write_table(table, path, filesystem=self.hdfs) - result = pq.read_table( - path, filesystem=self.hdfs, use_legacy_dataset=True - ).to_pandas() + result = pq.read_table(path, filesystem=self.hdfs).to_pandas() assert_frame_equal(result, df) - @pytest.mark.parquet - @pytest.mark.pandas - def test_read_common_metadata_files(self): - tmpdir = pjoin(self.tmp_path, 'common-metadata-' + guid()) - self.hdfs.mkdir(tmpdir) - _test_read_common_metadata_files(self.hdfs, tmpdir) - @pytest.mark.parquet @pytest.mark.pandas def test_write_to_dataset_with_partitions(self): From e0228fb4ae3eac10bd01d433f5d50859e43bc2a7 Mon Sep 17 00:00:00 2001 From: AlenkaF Date: Wed, 6 Dec 2023 10:51:54 +0100 Subject: [PATCH 02/29] Remove additional code connected to ParquetDatasetPiece --- python/benchmarks/parquet.py | 29 ----- python/pyarrow/parquet/core.py | 109 ------------------- python/pyarrow/tests/parquet/test_dataset.py | 2 +- 3 files changed, 1 insertion(+), 139 deletions(-) diff --git a/python/benchmarks/parquet.py b/python/benchmarks/parquet.py index 3aeca425bc8f0..e459ea2c369b4 100644 --- a/python/benchmarks/parquet.py +++ b/python/benchmarks/parquet.py @@ -29,35 +29,6 @@ pq = None -class ParquetManifestCreation(object): - """Benchmark creating a parquet manifest.""" - - size = 10 ** 6 - tmpdir = None - - param_names = ('num_partitions', 'num_threads') - params = [(10, 100, 1000), (1, 8)] - - def setup(self, num_partitions, num_threads): - if pq is None: - raise NotImplementedError("Parquet support not enabled") - - self.tmpdir = tempfile.mkdtemp('benchmark_parquet') - rnd = np.random.RandomState(42) - num1 = rnd.randint(0, num_partitions, size=self.size) - num2 = rnd.randint(0, 1000, size=self.size) - output_df = pd.DataFrame({'num1': num1, 'num2': num2}) - output_table = pa.Table.from_pandas(output_df) - pq.write_to_dataset(output_table, self.tmpdir, ['num1']) - - def teardown(self, num_partitions, num_threads): - if self.tmpdir is not None: - shutil.rmtree(self.tmpdir) - - def time_manifest_creation(self, num_partitions, num_threads): - pq.ParquetManifest(self.tmpdir, metadata_nthreads=num_threads) - - class ParquetWriteBinary(object): def setup(self): diff --git a/python/pyarrow/parquet/core.py b/python/pyarrow/parquet/core.py index e374109b529a5..e6dbacc87cda8 100644 --- a/python/pyarrow/parquet/core.py +++ b/python/pyarrow/parquet/core.py @@ -1321,106 +1321,6 @@ def filter_accepts_partition(self, part_key, filter, level): filter[1]) -class ParquetManifest: - - def __init__(self, dirpath, open_file_func=None, filesystem=None, - pathsep='/', partition_scheme='hive', metadata_nthreads=1): - filesystem, dirpath = _get_filesystem_and_path(filesystem, dirpath) - self.filesystem = filesystem - self.open_file_func = open_file_func - self.pathsep = pathsep - self.dirpath = _stringify_path(dirpath) - self.partition_scheme = partition_scheme - self.partitions = ParquetPartitions() - self.pieces = [] - self._metadata_nthreads = metadata_nthreads - self._thread_pool = futures.ThreadPoolExecutor( - max_workers=metadata_nthreads) - - self.common_metadata_path = None - self.metadata_path = None - - self._visit_level(0, self.dirpath, []) - - # Due to concurrency, pieces will potentially by out of order if the - # dataset is partitioned so we sort them to yield stable results - self.pieces.sort(key=lambda piece: piece.path) - - if self.common_metadata_path is None: - # _common_metadata is a subset of _metadata - self.common_metadata_path = self.metadata_path - - self._thread_pool.shutdown() - - def _visit_level(self, level, base_path, part_keys): - fs = self.filesystem - - _, directories, files = next(fs.walk(base_path)) - - filtered_files = [] - for path in files: - full_path = self.pathsep.join((base_path, path)) - if path.endswith('_common_metadata'): - self.common_metadata_path = full_path - elif path.endswith('_metadata'): - self.metadata_path = full_path - elif self._should_silently_exclude(path): - continue - else: - filtered_files.append(full_path) - - # ARROW-1079: Filter out "private" directories starting with underscore - filtered_directories = [self.pathsep.join((base_path, x)) - for x in directories - if not _is_private_directory(x)] - - filtered_files.sort() - filtered_directories.sort() - - if len(filtered_files) > 0 and len(filtered_directories) > 0: - raise ValueError('Found files in an intermediate ' - 'directory: {}'.format(base_path)) - elif len(filtered_directories) > 0: - self._visit_directories(level, filtered_directories, part_keys) - else: - self._push_pieces(filtered_files, part_keys) - - def _should_silently_exclude(self, file_name): - return (file_name.endswith('.crc') or # Checksums - file_name.endswith('_$folder$') or # HDFS directories in S3 - file_name.startswith('.') or # Hidden files starting with . - file_name.startswith('_') or # Hidden files starting with _ - file_name in EXCLUDED_PARQUET_PATHS) - - def _visit_directories(self, level, directories, part_keys): - futures_list = [] - for path in directories: - head, tail = _path_split(path, self.pathsep) - name, key = _parse_hive_partition(tail) - - index = self.partitions.get_index(level, name, key) - dir_part_keys = part_keys + [(name, index)] - # If you have less threads than levels, the wait call will block - # indefinitely due to multiple waits within a thread. - if level < self._metadata_nthreads: - future = self._thread_pool.submit(self._visit_level, - level + 1, - path, - dir_part_keys) - futures_list.append(future) - else: - self._visit_level(level + 1, path, dir_part_keys) - if futures_list: - futures.wait(futures_list) - - def _parse_partition(self, dirname): - if self.partition_scheme == 'hive': - return _parse_hive_partition(dirname) - else: - raise NotImplementedError('partition schema: {}' - .format(self.partition_scheme)) - - def _parse_hive_partition(value): if '=' not in value: raise ValueError('Directory name did not appear to be a ' @@ -2077,14 +1977,6 @@ def read_pandas(self, **kwargs): """ return self.read(use_pandas_metadata=True, **kwargs) - @property - def pieces(self): - warnings.warn( - _DEPR_MSG.format("ParquetDataset.pieces", - " Use the '.fragments' attribute instead"), - FutureWarning, stacklevel=2) - return list(self._dataset.get_fragments()) - @property def fragments(self): """ @@ -2895,7 +2787,6 @@ def read_schema(where, memory_map=False, decryption_properties=None, "ParquetDataset", "ParquetFile", "ParquetLogicalType", - "ParquetManifest", "ParquetPartitions", "ParquetReader", "ParquetSchema", diff --git a/python/pyarrow/tests/parquet/test_dataset.py b/python/pyarrow/tests/parquet/test_dataset.py index 25bd8d6dbc7ca..5502bfa49cdd6 100644 --- a/python/pyarrow/tests/parquet/test_dataset.py +++ b/python/pyarrow/tests/parquet/test_dataset.py @@ -1345,7 +1345,7 @@ def test_parquet_dataset_partitions_piece_path_with_fsspec(tempdir): path, filesystem=filesystem) # ensure the piece path is also posix-style expected = path + "/data.parquet" - assert dataset.pieces[0].path == expected + assert dataset.fragments[0].path == expected @pytest.mark.dataset From 94ad4e92e6bce74ab5db4b0eb614b568adc850df Mon Sep 17 00:00:00 2001 From: AlenkaF Date: Wed, 6 Dec 2023 14:38:44 +0100 Subject: [PATCH 03/29] Merge _ParquetDatasetV2 and ParquetDataset --- python/pyarrow/parquet/core.py | 388 ++++++--------------------------- 1 file changed, 70 insertions(+), 318 deletions(-) diff --git a/python/pyarrow/parquet/core.py b/python/pyarrow/parquet/core.py index e6dbacc87cda8..0dbc7476da224 100644 --- a/python/pyarrow/parquet/core.py +++ b/python/pyarrow/parquet/core.py @@ -1321,53 +1321,14 @@ def filter_accepts_partition(self, part_key, filter, level): filter[1]) -def _parse_hive_partition(value): - if '=' not in value: - raise ValueError('Directory name did not appear to be a ' - 'partition: {}'.format(value)) - return value.split('=', 1) - - -def _is_private_directory(x): - _, tail = os.path.split(x) - return (tail.startswith('_') or tail.startswith('.')) and '=' not in tail - - -def _path_split(path, sep): - i = path.rfind(sep) + 1 - head, tail = path[:i], path[i:] - head = head.rstrip(sep) - return head, tail - - EXCLUDED_PARQUET_PATHS = {'_SUCCESS'} -class _ParquetDatasetMetadata: - __slots__ = ('fs', 'memory_map', 'read_dictionary', 'common_metadata', - 'buffer_size') - - -def _open_dataset_file(dataset, path, meta=None): - if (dataset.fs is not None and - not isinstance(dataset.fs, legacyfs.LocalFileSystem)): - path = dataset.fs.open(path, mode='rb') - return ParquetFile( - path, - metadata=meta, - memory_map=dataset.memory_map, - read_dictionary=dataset.read_dictionary, - common_metadata=dataset.common_metadata, - buffer_size=dataset.buffer_size +def _is_local_file_system(fs): + return isinstance(fs, LocalFileSystem) or isinstance( + fs, legacyfs.LocalFileSystem ) - -_DEPR_MSG = ( - "'{}' attribute is deprecated as of pyarrow 5.0.0 and will be removed " - "in a future version.{}" -) - - _read_docstring_common = """\ read_dictionary : list, default None List of names or column paths (for nested types) to read directly @@ -1390,280 +1351,10 @@ def _open_dataset_file(dataset, path, meta=None): you need to specify the field names or a full schema. See the ``pyarrow.dataset.partitioning()`` function for more details.""" + _parquet_dataset_example = """\ Generate an example PyArrow Table and write it to a partitioned dataset: ->>> import pyarrow as pa ->>> table = pa.table({'year': [2020, 2022, 2021, 2022, 2019, 2021], -... 'n_legs': [2, 2, 4, 4, 5, 100], -... 'animal': ["Flamingo", "Parrot", "Dog", "Horse", -... "Brittle stars", "Centipede"]}) - ->>> import pyarrow.parquet as pq ->>> pq.write_to_dataset(table, root_path='dataset_name', -... partition_cols=['year']) - -create a ParquetDataset object from the dataset source: - ->>> dataset = pq.ParquetDataset('dataset_name/') - -and read the data: - ->>> dataset.read().to_pandas() - n_legs animal year -0 5 Brittle stars 2019 -1 2 Flamingo 2020 -2 4 Dog 2021 -3 100 Centipede 2021 -4 2 Parrot 2022 -5 4 Horse 2022 - -create a ParquetDataset object with filter: - ->>> dataset = pq.ParquetDataset('dataset_name/', -... filters=[('n_legs','=',4)]) ->>> dataset.read().to_pandas() - n_legs animal year -0 4 Dog 2021 -1 4 Horse 2022 -""" - - -class ParquetDataset: - - __doc__ = """ -Encapsulates details of reading a complete Parquet dataset possibly -consisting of multiple files and partitions in subdirectories. - -Parameters ----------- -path_or_paths : str or List[str] - A directory name, single file name, or list of file names. -filesystem : FileSystem, default None - If nothing passed, will be inferred based on path. - Path will try to be found in the local on-disk filesystem otherwise - it will be parsed as an URI to determine the filesystem. -schema : pyarrow.parquet.Schema - Use schema obtained elsewhere to validate file schemas. Alternative to - metadata parameter. -metadata : pyarrow.parquet.FileMetaData - Use metadata obtained elsewhere to validate file schemas. -split_row_groups : bool, default False - Divide files into pieces for each row group in the file. -validate_schema : bool, default True - Check that individual file schemas are all the same / compatible. -filters : pyarrow.compute.Expression or List[Tuple] or List[List[Tuple]], default None - Rows which do not match the filter predicate will be removed from scanned - data. Partition keys embedded in a nested directory structure will be - exploited to avoid loading files at all if they contain no matching rows. - Within-file level filtering and different partitioning schemes are supported. - - {1} -metadata_nthreads : int, default 1 - How many threads to allow the thread pool which is used to read the - dataset metadata. Increasing this is helpful to read partitioned - datasets. -{0} -pre_buffer : bool, default True - Coalesce and issue file reads in parallel to improve performance on - high-latency filesystems (e.g. S3, GCS). If True, Arrow will use a - background I/O thread pool. If using a filesystem layer that itself - performs readahead (e.g. fsspec's S3FS), disable readahead for best - results. Set to False if you want to prioritize minimal memory usage - over maximum speed. -coerce_int96_timestamp_unit : str, default None - Cast timestamps that are stored in INT96 format to a particular resolution - (e.g. 'ms'). Setting to None is equivalent to 'ns' and therefore INT96 - timestamps will be inferred as timestamps in nanoseconds. -thrift_string_size_limit : int, default None - If not None, override the maximum total string size allocated - when decoding Thrift structures. The default limit should be - sufficient for most Parquet files. -thrift_container_size_limit : int, default None - If not None, override the maximum total size of containers allocated - when decoding Thrift structures. The default limit should be - sufficient for most Parquet files. -page_checksum_verification : bool, default False - If True, verify the page checksum for each page read from the file. - -Examples --------- -{2} -""".format(_read_docstring_common, _DNF_filter_doc, _parquet_dataset_example) - - def __new__(cls, path_or_paths=None, filesystem=None, schema=None, - metadata=None, split_row_groups=False, validate_schema=True, - filters=None, metadata_nthreads=None, read_dictionary=None, - memory_map=False, buffer_size=0, partitioning="hive", - pre_buffer=True, - coerce_int96_timestamp_unit=None, - thrift_string_size_limit=None, - thrift_container_size_limit=None, - page_checksum_verification=False): - - return _ParquetDatasetV2( - path_or_paths, filesystem=filesystem, - filters=filters, - partitioning=partitioning, - read_dictionary=read_dictionary, - memory_map=memory_map, - buffer_size=buffer_size, - pre_buffer=pre_buffer, - coerce_int96_timestamp_unit=coerce_int96_timestamp_unit, - # unsupported keywords - schema=schema, metadata=metadata, - split_row_groups=split_row_groups, - validate_schema=validate_schema, - metadata_nthreads=metadata_nthreads, - thrift_string_size_limit=thrift_string_size_limit, - thrift_container_size_limit=thrift_container_size_limit, - page_checksum_verification=page_checksum_verification, - ) - - def read(self, columns=None, use_threads=True, use_pandas_metadata=False): - """ - Read multiple Parquet files as a single pyarrow.Table. - - Parameters - ---------- - columns : List[str] - Names of columns to read from the file. - use_threads : bool, default True - Perform multi-threaded column reads - use_pandas_metadata : bool, default False - Passed through to each dataset piece. - - Returns - ------- - pyarrow.Table - Content of the file as a table (of columns). - - Examples - -------- - Generate an example dataset: - - >>> import pyarrow as pa - >>> table = pa.table({'year': [2020, 2022, 2021, 2022, 2019, 2021], - ... 'n_legs': [2, 2, 4, 4, 5, 100], - ... 'animal': ["Flamingo", "Parrot", "Dog", "Horse", - ... "Brittle stars", "Centipede"]}) - >>> import pyarrow.parquet as pq - >>> pq.write_to_dataset(table, root_path='dataset_name_read', - ... partition_cols=['year']) - >>> dataset = pq.ParquetDataset('dataset_name_read/') - - Read multiple Parquet files as a single pyarrow.Table: - - >>> dataset.read(columns=["n_legs"]) - pyarrow.Table - n_legs: int64 - ---- - n_legs: [[5],[2],[4,100],[2,4]] - """ - tables = [] - for piece in self._pieces: - table = piece.read(columns=columns, - use_threads=use_threads, - partitions=self._partitions, - use_pandas_metadata=use_pandas_metadata) - tables.append(table) - - all_data = lib.concat_tables(tables) - - if use_pandas_metadata: - # We need to ensure that this metadata is set in the Table's schema - # so that Table.to_pandas will construct pandas.DataFrame with the - # right index - common_metadata = self._get_common_pandas_metadata() - current_metadata = all_data.schema.metadata or {} - - if common_metadata and b'pandas' not in current_metadata: - all_data = all_data.replace_schema_metadata({ - b'pandas': common_metadata}) - - return all_data - - def read_pandas(self, **kwargs): - """ - Read dataset including pandas metadata, if any. Other arguments passed - through to ParquetDataset.read, see docstring for further details. - - Parameters - ---------- - **kwargs : optional - All additional options to pass to the reader. - - Returns - ------- - pyarrow.Table - Content of the file as a table (of columns). - - Examples - -------- - Generate an example PyArrow Table and write it to a partitioned - dataset: - - >>> import pyarrow as pa - >>> import pandas as pd - >>> df = pd.DataFrame({'year': [2020, 2022, 2021, 2022, 2019, 2021], - ... 'n_legs': [2, 2, 4, 4, 5, 100], - ... 'animal': ["Flamingo", "Parrot", "Dog", "Horse", - ... "Brittle stars", "Centipede"]}) - >>> table = pa.Table.from_pandas(df) - >>> import pyarrow.parquet as pq - >>> pq.write_table(table, 'table.parquet') - >>> dataset = pq.ParquetDataset('table.parquet') - - Read dataset including pandas metadata: - - >>> dataset.read_pandas(columns=["n_legs"]) - pyarrow.Table - n_legs: int64 - ---- - n_legs: [[2,2,4,4,5,100]] - - Select pandas metadata: - - >>> dataset.read_pandas(columns=["n_legs"]).schema.pandas_metadata - {'index_columns': [{'kind': 'range', 'name': None, 'start': 0, ...} - """ - return self.read(use_pandas_metadata=True, **kwargs) - - def _get_common_pandas_metadata(self): - if self._common_metadata is None: - return None - - keyvalues = self._common_metadata.metadata - return keyvalues.get(b'pandas', None) - - def _filter(self, filters): - accepts_filter = self._partitions.filter_accepts_partition - - def one_filter_accepts(piece, filter): - return all(accepts_filter(part_key, filter, level) - for level, part_key in enumerate(piece.partition_keys)) - - def all_filters_accept(piece): - return any(all(one_filter_accepts(piece, f) for f in conjunction) - for conjunction in filters) - - self._pieces = [p for p in self._pieces if all_filters_accept(p)] - - -def _is_local_file_system(fs): - return isinstance(fs, LocalFileSystem) or isinstance( - fs, legacyfs.LocalFileSystem - ) - - -class _ParquetDatasetV2: - """ - ParquetDataset shim using the Dataset API under the hood. - - Examples - -------- - Generate an example PyArrow Table and write it to a partitioned dataset: - >>> import pyarrow as pa >>> table = pa.table({'year': [2020, 2022, 2021, 2022, 2019, 2021], ... 'n_legs': [2, 2, 4, 4, 5, 100], @@ -1696,12 +1387,73 @@ class _ParquetDatasetV2: n_legs animal year 0 4 Dog 2021 1 4 Horse 2022 - """ +""" + + +class ParquetDataset: + __doc__ = """ + Encapsulates details of reading a complete Parquet dataset possibly + consisting of multiple files and partitions in subdirectories. + + Parameters + ---------- + path_or_paths : str or List[str] + A directory name, single file name, or list of file names. + filesystem : FileSystem, default None + If nothing passed, will be inferred based on path. + Path will try to be found in the local on-disk filesystem otherwise + it will be parsed as an URI to determine the filesystem. + schema : pyarrow.parquet.Schema + Optionally provide the Schema for the Dataset, in which case it will + not be inferred from the source. + filters : pyarrow.compute.Expression or List[Tuple] or List[List[Tuple]], default None + Rows which do not match the filter predicate will be removed from scanned + data. Partition keys embedded in a nested directory structure will be + exploited to avoid loading files at all if they contain no matching rows. + Within-file level filtering and different partitioning schemes are supported. + + {1} + {0} + ignore_prefixes : list, optional + Files matching any of these prefixes will be ignored by the + discovery process. + This is matched to the basename of a path. + By default this is ['.', '_']. + Note that discovery happens only if a directory is passed as source. + pre_buffer : bool, default True + Coalesce and issue file reads in parallel to improve performance on + high-latency filesystems (e.g. S3, GCS). If True, Arrow will use a + background I/O thread pool. If using a filesystem layer that itself + performs readahead (e.g. fsspec's S3FS), disable readahead for best + results. Set to False if you want to prioritize minimal memory usage + over maximum speed. + coerce_int96_timestamp_unit : str, default None + Cast timestamps that are stored in INT96 format to a particular resolution + (e.g. 'ms'). Setting to None is equivalent to 'ns' and therefore INT96 + timestamps will be inferred as timestamps in nanoseconds. + decryption_properties : FileDecryptionProperties or None + File-level decryption properties. + The decryption properties can be created using + ``CryptoFactory.file_decryption_properties()``. + thrift_string_size_limit : int, default None + If not None, override the maximum total string size allocated + when decoding Thrift structures. The default limit should be + sufficient for most Parquet files. + thrift_container_size_limit : int, default None + If not None, override the maximum total size of containers allocated + when decoding Thrift structures. The default limit should be + sufficient for most Parquet files. + page_checksum_verification : bool, default False + If True, verify the page checksum for each page read from the file. + + Examples + -------- + """.format(_read_docstring_common, _DNF_filter_doc, _parquet_dataset_example) - def __init__(self, path_or_paths, filesystem=None, *, filters=None, + def __init__(self, path_or_paths, filesystem=None, schema=None, *, filters=None, partitioning="hive", read_dictionary=None, buffer_size=None, memory_map=False, ignore_prefixes=None, pre_buffer=True, - coerce_int96_timestamp_unit=None, schema=None, + coerce_int96_timestamp_unit=None, decryption_properties=None, thrift_string_size_limit=None, thrift_container_size_limit=None, page_checksum_verification=False, @@ -1803,7 +1555,7 @@ def __init__(self, path_or_paths, filesystem=None, *, filters=None, ignore_prefixes=ignore_prefixes) def equals(self, other): - if not isinstance(other, _ParquetDatasetV2): + if not isinstance(other, ParquetDataset): raise TypeError('`other` must be an instance of ParquetDataset') return (self.schema == other.schema and @@ -2213,7 +1965,7 @@ def read_table(source, *, columns=None, use_threads=True, "datasets-based implementation." ) try: - dataset = _ParquetDatasetV2( + dataset = ParquetDataset( source, schema=schema, filesystem=filesystem, From e5ff41d12a09faa8102d3a29b5f307c141234c88 Mon Sep 17 00:00:00 2001 From: AlenkaF Date: Wed, 6 Dec 2023 15:37:53 +0100 Subject: [PATCH 04/29] Remove metadata_collector duplicate in write_to_dataset --- python/pyarrow/parquet/core.py | 1 - 1 file changed, 1 deletion(-) diff --git a/python/pyarrow/parquet/core.py b/python/pyarrow/parquet/core.py index 0dbc7476da224..4171f6b84fa3d 100644 --- a/python/pyarrow/parquet/core.py +++ b/python/pyarrow/parquet/core.py @@ -2306,7 +2306,6 @@ def file_visitor(written_file): raise ValueError(msg_confl.format("partitioning", "partition_cols")) - metadata_collector = kwargs.pop('metadata_collector', None) if metadata_collector is not None and file_visitor is not None: raise ValueError(msg_confl.format("file_visitor", "metadata_collector")) From 31ad7d773d3b5d1165e3311224dae2630d74760b Mon Sep 17 00:00:00 2001 From: AlenkaF Date: Wed, 6 Dec 2023 16:13:54 +0100 Subject: [PATCH 05/29] Linter --- python/pyarrow/parquet/core.py | 12 ++++++------ python/pyarrow/tests/parquet/common.py | 1 - .../tests/parquet/test_compliant_nested_type.py | 2 +- python/pyarrow/tests/parquet/test_dataset.py | 3 +-- python/pyarrow/tests/parquet/test_parquet_file.py | 1 - 5 files changed, 8 insertions(+), 11 deletions(-) diff --git a/python/pyarrow/parquet/core.py b/python/pyarrow/parquet/core.py index 4171f6b84fa3d..1ad05b46ce447 100644 --- a/python/pyarrow/parquet/core.py +++ b/python/pyarrow/parquet/core.py @@ -17,19 +17,16 @@ from collections import defaultdict -from concurrent import futures from contextlib import nullcontext -from functools import partial, reduce +from functools import reduce import inspect import json from collections.abc import Collection -import numpy as np import os import re import operator import urllib.parse -import warnings import pyarrow as pa import pyarrow.lib as lib @@ -1329,6 +1326,7 @@ def _is_local_file_system(fs): fs, legacyfs.LocalFileSystem ) + _read_docstring_common = """\ read_dictionary : list, default None List of names or column paths (for nested types) to read directly @@ -1406,7 +1404,8 @@ class ParquetDataset: schema : pyarrow.parquet.Schema Optionally provide the Schema for the Dataset, in which case it will not be inferred from the source. - filters : pyarrow.compute.Expression or List[Tuple] or List[List[Tuple]], default None + filters : pyarrow.compute.Expression or List[Tuple] or List[List[Tuple]], + default None Rows which do not match the filter predicate will be removed from scanned data. Partition keys embedded in a nested directory structure will be exploited to avoid loading files at all if they contain no matching rows. @@ -1448,6 +1447,7 @@ class ParquetDataset: Examples -------- + {2} """.format(_read_docstring_common, _DNF_filter_doc, _parquet_dataset_example) def __init__(self, path_or_paths, filesystem=None, schema=None, *, filters=None, @@ -2309,7 +2309,7 @@ def file_visitor(written_file): if metadata_collector is not None and file_visitor is not None: raise ValueError(msg_confl.format("file_visitor", "metadata_collector")) - + import pyarrow.dataset as ds # extract write_dataset specific options diff --git a/python/pyarrow/tests/parquet/common.py b/python/pyarrow/tests/parquet/common.py index 776b743dded2f..8365ed5b28543 100644 --- a/python/pyarrow/tests/parquet/common.py +++ b/python/pyarrow/tests/parquet/common.py @@ -18,7 +18,6 @@ import io import numpy as np -import pytest import pyarrow as pa from pyarrow.tests import util diff --git a/python/pyarrow/tests/parquet/test_compliant_nested_type.py b/python/pyarrow/tests/parquet/test_compliant_nested_type.py index fadf092b3f6f7..a4d43ee09b5a5 100644 --- a/python/pyarrow/tests/parquet/test_compliant_nested_type.py +++ b/python/pyarrow/tests/parquet/test_compliant_nested_type.py @@ -86,7 +86,7 @@ def test_write_compliant_nested_type_enable(tempdir, test_data): @pytest.mark.pandas @pytest.mark.dataset @parametrize_test_data -def test_write_compliant_nested_type_disable(tempdir,test_data): +def test_write_compliant_nested_type_disable(tempdir, test_data): # prepare dataframe for testing df = pd.DataFrame(data=test_data) # verify that we can read/write with new flag disabled diff --git a/python/pyarrow/tests/parquet/test_dataset.py b/python/pyarrow/tests/parquet/test_dataset.py index 5502bfa49cdd6..ad430fc76dcf9 100644 --- a/python/pyarrow/tests/parquet/test_dataset.py +++ b/python/pyarrow/tests/parquet/test_dataset.py @@ -584,7 +584,6 @@ def _partition_test_for_filesystem(fs, base_path): .reset_index(drop=True) .reindex(columns=result_df.columns)) - # With pandas 2.0.0 Index can store all numeric dtypes (not just # int64/uint64/float64). Using astype() to create a categorical # column preserves original dtype (int32) @@ -982,7 +981,7 @@ def test_empty_directory(tempdir): def _test_write_to_dataset_with_partitions(base_path, filesystem=None, schema=None, - index_name=None): + index_name=None): import pandas as pd import pandas.testing as tm diff --git a/python/pyarrow/tests/parquet/test_parquet_file.py b/python/pyarrow/tests/parquet/test_parquet_file.py index 6453308d86b1e..93097a1afaac9 100644 --- a/python/pyarrow/tests/parquet/test_parquet_file.py +++ b/python/pyarrow/tests/parquet/test_parquet_file.py @@ -18,7 +18,6 @@ import io import os import sys -from unittest import mock import pytest From d9cc412c9597efcfd59fa0cca813df824285fd70 Mon Sep 17 00:00:00 2001 From: AlenkaF Date: Thu, 7 Dec 2023 13:01:54 +0100 Subject: [PATCH 06/29] Remove PartitionSet, ParquetPartitions and few helper methods --- python/pyarrow/parquet/core.py | 215 ------------------- python/pyarrow/tests/parquet/test_dataset.py | 12 -- 2 files changed, 227 deletions(-) diff --git a/python/pyarrow/parquet/core.py b/python/pyarrow/parquet/core.py index 1ad05b46ce447..fbf94ef93d2d0 100644 --- a/python/pyarrow/parquet/core.py +++ b/python/pyarrow/parquet/core.py @@ -22,14 +22,11 @@ import inspect import json -from collections.abc import Collection import os import re import operator -import urllib.parse import pyarrow as pa -import pyarrow.lib as lib try: import pyarrow._parquet as _parquet @@ -51,28 +48,6 @@ from pyarrow import filesystem as legacyfs from pyarrow.util import guid, _is_path_like, _stringify_path, _deprecate_api -_URI_STRIP_SCHEMES = ('hdfs',) - - -def _parse_uri(path): - path = _stringify_path(path) - parsed_uri = urllib.parse.urlparse(path) - if parsed_uri.scheme in _URI_STRIP_SCHEMES: - return parsed_uri.path - else: - # ARROW-4073: On Windows returning the path with the scheme - # stripped removes the drive letter, if any - return path - - -def _get_filesystem_and_path(passed_filesystem, path): - if passed_filesystem is None: - return legacyfs.resolve_filesystem_and_path(path, passed_filesystem) - else: - passed_filesystem = legacyfs._ensure_filesystem(passed_filesystem) - parsed_path = _parse_uri(path) - return passed_filesystem, parsed_path - def _check_contains_null(val): if isinstance(val, bytes): @@ -1138,186 +1113,6 @@ def _get_pandas_index_columns(keyvalues): ['index_columns']) -# ---------------------------------------------------------------------- -# Metadata container providing instructions about reading a single Parquet -# file, possibly part of a partitioned dataset - -class PartitionSet: - """ - A data structure for cataloguing the observed Parquet partitions at a - particular level. So if we have - - /foo=a/bar=0 - /foo=a/bar=1 - /foo=a/bar=2 - /foo=b/bar=0 - /foo=b/bar=1 - /foo=b/bar=2 - - Then we have two partition sets, one for foo, another for bar. As we visit - levels of the partition hierarchy, a PartitionSet tracks the distinct - values and assigns categorical codes to use when reading the pieces - - Parameters - ---------- - name : str - Name of the partition set. Under which key to collect all values. - keys : list - All possible values that have been collected for that partition set. - """ - - def __init__(self, name, keys=None): - self.name = name - self.keys = keys or [] - self.key_indices = {k: i for i, k in enumerate(self.keys)} - self._dictionary = None - - def get_index(self, key): - """ - Get the index of the partition value if it is known, otherwise assign - one - - Parameters - ---------- - key : str or int - The value for which we want to known the index. - """ - if key in self.key_indices: - return self.key_indices[key] - else: - index = len(self.key_indices) - self.keys.append(key) - self.key_indices[key] = index - return index - - @property - def dictionary(self): - if self._dictionary is not None: - return self._dictionary - - if len(self.keys) == 0: - raise ValueError('No known partition keys') - - # Only integer and string partition types are supported right now - try: - integer_keys = [int(x) for x in self.keys] - dictionary = lib.array(integer_keys) - except ValueError: - dictionary = lib.array(self.keys) - - self._dictionary = dictionary - return dictionary - - @property - def is_sorted(self): - return list(self.keys) == sorted(self.keys) - - -class ParquetPartitions: - - def __init__(self): - self.levels = [] - self.partition_names = set() - - def __len__(self): - return len(self.levels) - - def __getitem__(self, i): - return self.levels[i] - - def equals(self, other): - if not isinstance(other, ParquetPartitions): - raise TypeError('`other` must be an instance of ParquetPartitions') - - return (self.levels == other.levels and - self.partition_names == other.partition_names) - - def __eq__(self, other): - try: - return self.equals(other) - except TypeError: - return NotImplemented - - def get_index(self, level, name, key): - """ - Record a partition value at a particular level, returning the distinct - code for that value at that level. - - Examples - -------- - - partitions.get_index(1, 'foo', 'a') returns 0 - partitions.get_index(1, 'foo', 'b') returns 1 - partitions.get_index(1, 'foo', 'c') returns 2 - partitions.get_index(1, 'foo', 'a') returns 0 - - Parameters - ---------- - level : int - The nesting level of the partition we are observing - name : str - The partition name - key : str or int - The partition value - """ - if level == len(self.levels): - if name in self.partition_names: - raise ValueError('{} was the name of the partition in ' - 'another level'.format(name)) - - part_set = PartitionSet(name) - self.levels.append(part_set) - self.partition_names.add(name) - - return self.levels[level].get_index(key) - - def filter_accepts_partition(self, part_key, filter, level): - p_column, p_value_index = part_key - f_column, op, f_value = filter - if p_column != f_column: - return True - - f_type = type(f_value) - - if op in {'in', 'not in'}: - if not isinstance(f_value, Collection): - raise TypeError( - "'%s' object is not a collection", f_type.__name__) - if not f_value: - raise ValueError("Cannot use empty collection as filter value") - if len({type(item) for item in f_value}) != 1: - raise ValueError("All elements of the collection '%s' must be" - " of same type", f_value) - f_type = type(next(iter(f_value))) - - elif not isinstance(f_value, str) and isinstance(f_value, Collection): - raise ValueError( - "Op '%s' not supported with a collection value", op) - - p_value = f_type(self.levels[level] - .dictionary[p_value_index].as_py()) - - if op == "=" or op == "==": - return p_value == f_value - elif op == "!=": - return p_value != f_value - elif op == '<': - return p_value < f_value - elif op == '>': - return p_value > f_value - elif op == '<=': - return p_value <= f_value - elif op == '>=': - return p_value >= f_value - elif op == 'in': - return p_value in f_value - elif op == 'not in': - return p_value not in f_value - else: - raise ValueError("'%s' is not a valid operator in predicates.", - filter[1]) - - EXCLUDED_PARQUET_PATHS = {'_SUCCESS'} @@ -2160,14 +1955,6 @@ def write_table(table, where, row_group_size=None, version='2.6', """.format(_parquet_writer_arg_docs, _write_table_example) -def _mkdir_if_not_exists(fs, path): - if fs._isfilestore() and not fs.exists(path): - try: - fs.mkdir(path) - except OSError: - assert fs.exists(path) - - def write_to_dataset(table, root_path, partition_cols=None, partition_filename_cb=None, filesystem=None, schema=None, @@ -2538,11 +2325,9 @@ def read_schema(where, memory_map=False, decryption_properties=None, "ParquetDataset", "ParquetFile", "ParquetLogicalType", - "ParquetPartitions", "ParquetReader", "ParquetSchema", "ParquetWriter", - "PartitionSet", "RowGroupMetaData", "Statistics", "read_metadata", diff --git a/python/pyarrow/tests/parquet/test_dataset.py b/python/pyarrow/tests/parquet/test_dataset.py index ad430fc76dcf9..fae8a1d6772de 100644 --- a/python/pyarrow/tests/parquet/test_dataset.py +++ b/python/pyarrow/tests/parquet/test_dataset.py @@ -53,18 +53,6 @@ pytestmark = pytest.mark.parquet -def test_partition_set_dictionary_type(): - set1 = pq.PartitionSet('key1', ['foo', 'bar', 'baz']) - set2 = pq.PartitionSet('key2', [2007, 2008, 2009]) - - assert isinstance(set1.dictionary, pa.StringArray) - assert isinstance(set2.dictionary, pa.IntegerArray) - - set3 = pq.PartitionSet('key2', [datetime.datetime(2007, 1, 1)]) - with pytest.raises(TypeError): - set3.dictionary - - @pytest.mark.dataset def test_filesystem_uri(tempdir): table = pa.table({"a": [1, 2, 3]}) From abf58b9e4e3c31690e309b1a71435b475ac82cd0 Mon Sep 17 00:00:00 2001 From: AlenkaF Date: Thu, 7 Dec 2023 13:03:45 +0100 Subject: [PATCH 07/29] Remove partition_filename_cb --- python/pyarrow/parquet/core.py | 15 ++------------- 1 file changed, 2 insertions(+), 13 deletions(-) diff --git a/python/pyarrow/parquet/core.py b/python/pyarrow/parquet/core.py index fbf94ef93d2d0..474bc7fdfc230 100644 --- a/python/pyarrow/parquet/core.py +++ b/python/pyarrow/parquet/core.py @@ -1956,8 +1956,7 @@ def write_table(table, where, row_group_size=None, version='2.6', def write_to_dataset(table, root_path, partition_cols=None, - partition_filename_cb=None, filesystem=None, - schema=None, + filesystem=None, schema=None, partitioning=None, basename_template=None, use_threads=None, file_visitor=None, existing_data_behavior=None, @@ -2085,10 +2084,6 @@ def file_visitor(written_file): "The '{1}' argument is not supported. " "Use only '{0}' instead." ) - if partition_filename_cb is not None and basename_template is not None: - raise ValueError(msg_confl.format("basename_template", - "partition_filename_cb")) - if partition_cols is not None and partitioning is not None: raise ValueError(msg_confl.format("partitioning", "partition_cols")) @@ -2108,16 +2103,10 @@ def file_visitor(written_file): write_dataset_kwargs['max_rows_per_group'] = kwargs.pop( 'row_group_size', kwargs.pop("chunk_size", None) ) - # raise for unsupported keywords - msg = ( - "The '{}' argument is not supported with the new dataset " - "implementation." - ) + if metadata_collector is not None: def file_visitor(written_file): metadata_collector.append(written_file.metadata) - if partition_filename_cb is not None: - raise ValueError(msg.format("partition_filename_cb")) # map format arguments parquet_format = ds.ParquetFileFormat() From 8e72811c3c61e9b371068bb69840eaaf1de75df5 Mon Sep 17 00:00:00 2001 From: AlenkaF Date: Thu, 7 Dec 2023 16:06:11 +0100 Subject: [PATCH 08/29] Keep use_legacy_dataset but deprecate it --- python/pyarrow/parquet/core.py | 30 ++++++++++++++++++---- python/pyarrow/tests/parquet/test_basic.py | 19 ++++++++++++++ 2 files changed, 44 insertions(+), 5 deletions(-) diff --git a/python/pyarrow/parquet/core.py b/python/pyarrow/parquet/core.py index 474bc7fdfc230..8d24f9a80b76c 100644 --- a/python/pyarrow/parquet/core.py +++ b/python/pyarrow/parquet/core.py @@ -25,6 +25,7 @@ import os import re import operator +import warnings import pyarrow as pa @@ -1253,6 +1254,14 @@ def __init__(self, path_or_paths, filesystem=None, schema=None, *, filters=None, thrift_container_size_limit=None, page_checksum_verification=False, **kwargs): + + use_legacy_dataset = kwargs.pop('use_legacy_dataset', None) + if use_legacy_dataset is not None: + warnings.warn( + "Passing 'use_legacy_dataset' is deprecated as of pyarrow 15.0.0 " + "and will be removed in a future version.", + FutureWarning, stacklevel=2) + import pyarrow.dataset as ds # Raise error for not supported keywords @@ -1747,13 +1756,19 @@ def partitioning(self): def read_table(source, *, columns=None, use_threads=True, schema=None, use_pandas_metadata=False, read_dictionary=None, memory_map=False, buffer_size=0, partitioning="hive", - filesystem=None, filters=None, + filesystem=None, filters=None, use_legacy_dataset=None, ignore_prefixes=None, pre_buffer=True, coerce_int96_timestamp_unit=None, decryption_properties=None, thrift_string_size_limit=None, thrift_container_size_limit=None, page_checksum_verification=False, metadata=None,): + if use_legacy_dataset is not None: + warnings.warn( + "Passing 'use_legacy_dataset' is deprecated as of pyarrow 15.0.0 " + "and will be removed in a future version.", + FutureWarning, stacklevel=2) + if metadata is not None: raise ValueError( "The 'metadata' keyword is no longer supported with the new " @@ -1956,10 +1971,10 @@ def write_table(table, where, row_group_size=None, version='2.6', def write_to_dataset(table, root_path, partition_cols=None, - filesystem=None, schema=None, - partitioning=None, basename_template=None, - use_threads=None, file_visitor=None, - existing_data_behavior=None, + filesystem=None, use_legacy_dataset=None, + schema=None, partitioning=None, + basename_template=None, use_threads=None, + file_visitor=None, existing_data_behavior=None, **kwargs): """Wrapper around dataset.write_dataset for writing a Table to Parquet format by partitions. @@ -2076,6 +2091,11 @@ def file_visitor(written_file): >>> pq.ParquetDataset('dataset_name_4/').files ['dataset_name_4/...-0.parquet'] """ + if use_legacy_dataset is not None: + warnings.warn( + "Passing 'use_legacy_dataset' is deprecated as of pyarrow 15.0.0 " + "and will be removed in a future version.", + FutureWarning, stacklevel=2) metadata_collector = kwargs.pop('metadata_collector', None) diff --git a/python/pyarrow/tests/parquet/test_basic.py b/python/pyarrow/tests/parquet/test_basic.py index a76c2b900bfa3..8a92f7b284102 100644 --- a/python/pyarrow/tests/parquet/test_basic.py +++ b/python/pyarrow/tests/parquet/test_basic.py @@ -938,3 +938,22 @@ def test_checksum_write_to_dataset(tempdir): # checksum verification enabled raises an exception with pytest.raises(OSError, match="CRC checksum verification"): _ = pq.read_table(corrupted_file_path, page_checksum_verification=True) + + +def test_deprecated_use_legacy_dataset(tempdir): + # Test that specifying use_legacy_dataset in ParquetDataset, write_to_dataset + # and read_table doesn't raise an error but gives a warning. + table = pa.table({"a": [1, 2, 3]}) + path = tempdir / "deprecate_legacy" + + msg = "Passing 'use_legacy_dataset'" + with pytest.warns(FutureWarning, match=msg): + pq.write_to_dataset(table, path, use_legacy_dataset=False) + + pq.write_to_dataset(table, path) + + with pytest.warns(FutureWarning, match=msg): + pq.read_table(path, use_legacy_dataset=False) + + with pytest.warns(FutureWarning, match=msg): + pq.ParquetDataset(path, use_legacy_dataset=False) From 863b15e27df8d80f2fb8ec4df5b0ecf8f8699b14 Mon Sep 17 00:00:00 2001 From: AlenkaF Date: Thu, 7 Dec 2023 16:08:32 +0100 Subject: [PATCH 09/29] Lint rst. file --- docs/source/python/parquet.rst | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/source/python/parquet.rst b/docs/source/python/parquet.rst index aed76cc42adfd..d4717897660b6 100644 --- a/docs/source/python/parquet.rst +++ b/docs/source/python/parquet.rst @@ -514,17 +514,17 @@ option was enabled on write). Other features: - Filtering on all columns (using row group statistics) instead of only on - the partition keys. + the partition keys. - Fine-grained partitioning: support for a directory partitioning scheme - in addition to the Hive-like partitioning (e.g. "/2019/11/15/" instead of - "/year=2019/month=11/day=15/"), and the ability to specify a schema for - the partition keys. + in addition to the Hive-like partitioning (e.g. "/2019/11/15/" instead of + "/year=2019/month=11/day=15/"), and the ability to specify a schema for + the partition keys. Note: - The partition keys need to be explicitly included in the ``columns`` - keyword when you want to include them in the result while reading a - subset of the columns + keyword when you want to include them in the result while reading a + subset of the columns Using with Spark From 3e784b4d41872c8c1d18650ae2623034fc827178 Mon Sep 17 00:00:00 2001 From: AlenkaF Date: Thu, 7 Dec 2023 16:41:56 +0100 Subject: [PATCH 10/29] Update test marks --- python/pyarrow/tests/parquet/test_basic.py | 33 +----------- .../parquet/test_compliant_nested_type.py | 1 - .../pyarrow/tests/parquet/test_data_types.py | 14 ----- python/pyarrow/tests/parquet/test_dataset.py | 52 +------------------ 4 files changed, 3 insertions(+), 97 deletions(-) diff --git a/python/pyarrow/tests/parquet/test_basic.py b/python/pyarrow/tests/parquet/test_basic.py index 8a92f7b284102..3c867776ac052 100644 --- a/python/pyarrow/tests/parquet/test_basic.py +++ b/python/pyarrow/tests/parquet/test_basic.py @@ -62,7 +62,6 @@ def test_parquet_invalid_version(tempdir): data_page_version="2.2") -@pytest.mark.dataset def test_set_data_page_size(): arr = pa.array([1, 2, 3] * 100000) t = pa.Table.from_arrays([arr], names=['f0']) @@ -73,7 +72,6 @@ def test_set_data_page_size(): _check_roundtrip(t, data_page_size=target_page_size) -@pytest.mark.dataset @pytest.mark.pandas def test_set_write_batch_size(): df = _test_dataframe(100) @@ -84,7 +82,6 @@ def test_set_write_batch_size(): ) -@pytest.mark.dataset @pytest.mark.pandas def test_set_dictionary_pagesize_limit(): df = _test_dataframe(100) @@ -98,7 +95,6 @@ def test_set_dictionary_pagesize_limit(): data_page_size=10, version='2.4') -@pytest.mark.dataset @pytest.mark.pandas def test_chunked_table_write(): # ARROW-232 @@ -119,7 +115,6 @@ def test_chunked_table_write(): @pytest.mark.pandas -@pytest.mark.dataset def test_memory_map(tempdir): df = alltypes_sample(size=10) @@ -135,7 +130,6 @@ def test_memory_map(tempdir): @pytest.mark.pandas -@pytest.mark.dataset def test_enable_buffered_stream(tempdir): df = alltypes_sample(size=10) @@ -150,7 +144,6 @@ def test_enable_buffered_stream(tempdir): assert table_read.equals(table) -@pytest.mark.dataset def test_special_chars_filename(tempdir): table = pa.Table.from_arrays([pa.array([42])], ["ints"]) filename = "foo # bar" @@ -162,11 +155,9 @@ def test_special_chars_filename(tempdir): assert table_read.equals(table) -@pytest.mark.dataset def test_invalid_source(): # Test that we provide an helpful error message pointing out # that None wasn't expected when trying to open a Parquet None file. - # with pytest.raises(TypeError, match="None"): pq.read_table(None) @@ -186,7 +177,6 @@ def test_file_with_over_int16_max_row_groups(): @pytest.mark.pandas -@pytest.mark.dataset def test_empty_table_roundtrip(): df = alltypes_sample(size=10) @@ -203,14 +193,12 @@ def test_empty_table_roundtrip(): @pytest.mark.pandas -@pytest.mark.dataset def test_empty_table_no_columns(): df = pd.DataFrame() empty = pa.Table.from_pandas(df, preserve_index=False) _check_roundtrip(empty) -@pytest.mark.dataset def test_write_nested_zero_length_array_chunk_failure(): # Bug report in ARROW-3792 cols = OrderedDict( @@ -240,7 +228,6 @@ def test_write_nested_zero_length_array_chunk_failure(): @pytest.mark.pandas -@pytest.mark.dataset def test_multiple_path_types(tempdir): # Test compatibility with PEP 519 path-like objects path = tempdir / 'zzz.parquet' @@ -259,7 +246,6 @@ def test_multiple_path_types(tempdir): tm.assert_frame_equal(df, df_read) -@pytest.mark.dataset def test_fspath(tempdir): # ARROW-12472 support __fspath__ objects without using str() path = tempdir / "test.parquet" @@ -276,7 +262,6 @@ def test_fspath(tempdir): _read_table(fs_protocol_obj, filesystem=FileSystem()) -@pytest.mark.dataset @pytest.mark.parametrize("filesystem", [ None, fs.LocalFileSystem(), LocalFileSystem._get_instance() ]) @@ -302,14 +287,12 @@ def test_relative_paths(tempdir, filesystem, name): assert result.equals(table) -@pytest.mark.dataset def test_read_non_existing_file(): # ensure we have a proper error message with pytest.raises(FileNotFoundError): pq.read_table('i-am-not-existing.parquet') -@pytest.mark.dataset def test_file_error_python_exception(): class BogusFile(io.BytesIO): def read(self, *args): @@ -323,7 +306,6 @@ def seek(self, *args): pq.read_table(BogusFile(b"")) -@pytest.mark.dataset def test_parquet_read_from_buffer(tempdir): # reading from a buffer from python's open() table = pa.table({"a": [1, 2, 3]}) @@ -338,7 +320,6 @@ def test_parquet_read_from_buffer(tempdir): assert result.equals(table) -@pytest.mark.dataset def test_byte_stream_split(): # This is only a smoke test. arr_float = pa.array(list(map(float, range(100)))) @@ -376,7 +357,6 @@ def test_byte_stream_split(): use_dictionary=False) -@pytest.mark.dataset def test_column_encoding(): arr_float = pa.array(list(map(float, range(100)))) arr_int = pa.array(list(map(int, range(100)))) @@ -502,7 +482,6 @@ def test_column_encoding(): column_encoding=True) -@pytest.mark.dataset def test_compression_level(): arr = pa.array(list(map(int, range(1000)))) data = [arr, arr] @@ -560,7 +539,6 @@ def test_sanitized_spark_field_names(): @pytest.mark.pandas -@pytest.mark.dataset def test_multithreaded_read(): df = alltypes_sample(size=10000) @@ -579,7 +557,6 @@ def test_multithreaded_read(): @pytest.mark.pandas -@pytest.mark.dataset def test_min_chunksize(): data = pd.DataFrame([np.arange(4)], columns=['A', 'B', 'C', 'D']) table = pa.Table.from_pandas(data.reset_index()) @@ -623,7 +600,6 @@ def test_write_error_deletes_incomplete_file(tempdir): assert not filename.exists() -@pytest.mark.dataset def test_read_non_existent_file(tempdir): path = 'nonexistent-file.parquet' try: @@ -632,7 +608,6 @@ def test_read_non_existent_file(tempdir): assert path in e.args[0] -@pytest.mark.dataset def test_read_table_doesnt_warn(datadir): with warnings.catch_warnings(): warnings.simplefilter(action="error") @@ -640,7 +615,6 @@ def test_read_table_doesnt_warn(datadir): @pytest.mark.pandas -@pytest.mark.dataset def test_zlib_compression_bug(): # ARROW-3514: "zlib deflate failed, output buffer too small" table = pa.Table.from_arrays([pa.array(['abc', 'def'])], ['some_col']) @@ -652,7 +626,6 @@ def test_zlib_compression_bug(): tm.assert_frame_equal(roundtrip.to_pandas(), table.to_pandas()) -@pytest.mark.dataset def test_parquet_file_too_small(tempdir): path = str(tempdir / "test.parquet") # TODO(dataset) with datasets API it raises OSError instead @@ -709,17 +682,15 @@ def test_fastparquet_cross_compatibility(tempdir): tm.assert_frame_equal(table_fp.to_pandas(), df) -@pytest.mark.dataset @pytest.mark.parametrize('array_factory', [ lambda: pa.array([0, None] * 10), lambda: pa.array([0, None] * 10).dictionary_encode(), lambda: pa.array(["", None] * 10), lambda: pa.array(["", None] * 10).dictionary_encode(), ]) -@pytest.mark.parametrize('use_dictionary', [False, True]) @pytest.mark.parametrize('read_dictionary', [False, True]) def test_buffer_contents( - array_factory, use_dictionary, read_dictionary + array_factory, read_dictionary ): # Test that null values are deterministically initialized to zero # after a roundtrip through Parquet. @@ -782,7 +753,6 @@ def test_reads_over_batch(tempdir): assert table == table2 -@pytest.mark.dataset def test_permutation_of_column_order(tempdir): # ARROW-2366 case = tempdir / "dataset_column_order_permutation" @@ -940,6 +910,7 @@ def test_checksum_write_to_dataset(tempdir): _ = pq.read_table(corrupted_file_path, page_checksum_verification=True) +@pytest.mark.dataset def test_deprecated_use_legacy_dataset(tempdir): # Test that specifying use_legacy_dataset in ParquetDataset, write_to_dataset # and read_table doesn't raise an error but gives a warning. diff --git a/python/pyarrow/tests/parquet/test_compliant_nested_type.py b/python/pyarrow/tests/parquet/test_compliant_nested_type.py index a4d43ee09b5a5..06c23bf1af1e7 100644 --- a/python/pyarrow/tests/parquet/test_compliant_nested_type.py +++ b/python/pyarrow/tests/parquet/test_compliant_nested_type.py @@ -57,7 +57,6 @@ @pytest.mark.pandas -@pytest.mark.dataset @parametrize_test_data def test_write_compliant_nested_type_enable(tempdir, test_data): # prepare dataframe for testing diff --git a/python/pyarrow/tests/parquet/test_data_types.py b/python/pyarrow/tests/parquet/test_data_types.py index 1ac5f9729b26d..e6b66b00428fb 100644 --- a/python/pyarrow/tests/parquet/test_data_types.py +++ b/python/pyarrow/tests/parquet/test_data_types.py @@ -53,7 +53,6 @@ @pytest.mark.pandas -@pytest.mark.dataset @pytest.mark.parametrize('chunk_size', [None, 1000]) def test_parquet_2_0_roundtrip(tempdir, chunk_size): df = alltypes_sample(size=10000, categorical=True) @@ -75,7 +74,6 @@ def test_parquet_2_0_roundtrip(tempdir, chunk_size): @pytest.mark.pandas -@pytest.mark.dataset def test_parquet_1_0_roundtrip(tempdir): size = 10000 np.random.seed(0) @@ -121,7 +119,6 @@ def _simple_table_write_read(table): @pytest.mark.pandas -@pytest.mark.dataset def test_direct_read_dictionary(): # ARROW-3325 repeats = 10 @@ -146,7 +143,6 @@ def test_direct_read_dictionary(): @pytest.mark.pandas -@pytest.mark.dataset def test_direct_read_dictionary_subfield(): repeats = 10 nunique = 5 @@ -177,7 +173,6 @@ def test_direct_read_dictionary_subfield(): assert result[0].num_chunks == 1 -@pytest.mark.dataset def test_dictionary_array_automatically_read(): # ARROW-3246 @@ -209,7 +204,6 @@ def test_dictionary_array_automatically_read(): @pytest.mark.pandas -@pytest.mark.dataset def test_decimal_roundtrip(tempdir): num_values = 10 @@ -254,7 +248,6 @@ def test_decimal_roundtrip_negative_scale(tempdir): # ----------------------------------------------------------------------------- -@pytest.mark.dataset @pytest.mark.parametrize('dtype', [int, float]) def test_single_pylist_column_roundtrip(tempdir, dtype,): filename = tempdir / 'single_{}_column.parquet'.format(dtype.__name__) @@ -272,7 +265,6 @@ def test_single_pylist_column_roundtrip(tempdir, dtype,): assert data_written.equals(data_read) -@pytest.mark.dataset def test_empty_lists_table_roundtrip(): # ARROW-2744: Shouldn't crash when writing an array of empty lists arr = pa.array([[], []], type=pa.list_(pa.int32())) @@ -280,7 +272,6 @@ def test_empty_lists_table_roundtrip(): _check_roundtrip(table) -@pytest.mark.dataset def test_nested_list_nonnullable_roundtrip_bug(): # Reproduce failure in ARROW-5630 typ = pa.list_(pa.field("item", pa.float32(), False)) @@ -293,7 +284,6 @@ def test_nested_list_nonnullable_roundtrip_bug(): t, data_page_size=4096) -@pytest.mark.dataset def test_nested_list_struct_multiple_batches_roundtrip(tempdir): # Reproduce failure in ARROW-11024 data = [[{'x': 'abc', 'y': 'abc'}]]*100 + [[{'x': 'abc', 'y': 'gcb'}]]*100 @@ -357,7 +347,6 @@ def test_large_list_records(): _check_roundtrip(table) -@pytest.mark.dataset @pytest.mark.pandas def test_parquet_nested_convenience(tempdir): # ARROW-1684 @@ -421,7 +410,6 @@ def _simple_table_roundtrip(table, **write_kwargs): @pytest.mark.slow @pytest.mark.large_memory -@pytest.mark.dataset def test_byte_array_exactly_2gb(): # Test edge case reported in ARROW-3762 val = b'x' * (1 << 10) @@ -443,7 +431,6 @@ def test_byte_array_exactly_2gb(): @pytest.mark.slow @pytest.mark.pandas @pytest.mark.large_memory -@pytest.mark.dataset def test_binary_array_overflow_to_chunked(): # ARROW-3762 @@ -468,7 +455,6 @@ def test_binary_array_overflow_to_chunked(): @pytest.mark.slow @pytest.mark.pandas @pytest.mark.large_memory -@pytest.mark.dataset def test_list_of_binary_large_cell(): # ARROW-4688 data = [] diff --git a/python/pyarrow/tests/parquet/test_dataset.py b/python/pyarrow/tests/parquet/test_dataset.py index fae8a1d6772de..bd8d47f60b751 100644 --- a/python/pyarrow/tests/parquet/test_dataset.py +++ b/python/pyarrow/tests/parquet/test_dataset.py @@ -50,10 +50,9 @@ # Marks all of the tests in this module # Ignore these with pytest ... -m 'not parquet' -pytestmark = pytest.mark.parquet +pytestmark = [pytest.mark.parquet, pytest.mark.dataset] -@pytest.mark.dataset def test_filesystem_uri(tempdir): table = pa.table({"a": [1, 2, 3]}) @@ -74,14 +73,12 @@ def test_filesystem_uri(tempdir): @pytest.mark.pandas -@pytest.mark.dataset def test_read_partitioned_directory(tempdir): fs = LocalFileSystem._get_instance() _partition_test_for_filesystem(fs, tempdir) @pytest.mark.pandas -@pytest.mark.dataset def test_read_partitioned_columns_selection(tempdir): # ARROW-3861 - do not include partition columns in resulting table when # `columns` keyword was passed without those columns @@ -95,7 +92,6 @@ def test_read_partitioned_columns_selection(tempdir): @pytest.mark.pandas -@pytest.mark.dataset def test_filters_equivalency(tempdir): fs = LocalFileSystem._get_instance() base_path = tempdir @@ -167,7 +163,6 @@ def test_filters_equivalency(tempdir): @pytest.mark.pandas -@pytest.mark.dataset def test_filters_cutoff_exclusive_integer(tempdir): fs = LocalFileSystem._get_instance() base_path = tempdir @@ -207,7 +202,6 @@ def test_filters_cutoff_exclusive_integer(tempdir): raises=(TypeError, AssertionError), reason='Loss of type information in creation of categoricals.' ) -@pytest.mark.dataset @pytest.mark.pandas def test_filters_cutoff_exclusive_datetime(tempdir): fs = LocalFileSystem._get_instance() @@ -252,7 +246,6 @@ def test_filters_cutoff_exclusive_datetime(tempdir): @pytest.mark.pandas -@pytest.mark.dataset def test_filters_inclusive_datetime(tempdir): # ARROW-11480 path = tempdir / 'timestamps.parquet' @@ -270,7 +263,6 @@ def test_filters_inclusive_datetime(tempdir): @pytest.mark.pandas -@pytest.mark.dataset def test_filters_inclusive_integer(tempdir): fs = LocalFileSystem._get_instance() base_path = tempdir @@ -305,7 +297,6 @@ def test_filters_inclusive_integer(tempdir): @pytest.mark.pandas -@pytest.mark.dataset def test_filters_inclusive_set(tempdir): fs = LocalFileSystem._get_instance() base_path = tempdir @@ -353,7 +344,6 @@ def test_filters_inclusive_set(tempdir): @pytest.mark.pandas -@pytest.mark.dataset def test_filters_invalid_pred_op(tempdir): fs = LocalFileSystem._get_instance() base_path = tempdir @@ -395,7 +385,6 @@ def test_filters_invalid_pred_op(tempdir): @pytest.mark.pandas -@pytest.mark.dataset def test_filters_invalid_column(tempdir): # ARROW-5572 - raise error on invalid name in filter specification # works with new dataset @@ -420,7 +409,6 @@ def test_filters_invalid_column(tempdir): @pytest.mark.pandas -@pytest.mark.dataset @pytest.mark.parametrize("filters", ([('integers', '<', 3)], [[('integers', '<', 3)]], @@ -455,7 +443,6 @@ def test_filters_read_table(tempdir, filters, read_method): @pytest.mark.pandas -@pytest.mark.dataset def test_partition_keys_with_underscores(tempdir): # ARROW-5666 - partition field values with underscores preserve underscores fs = LocalFileSystem._get_instance() @@ -480,7 +467,6 @@ def test_partition_keys_with_underscores(tempdir): @pytest.mark.s3 -@pytest.mark.dataset def test_read_s3fs(s3_example_s3fs, ): fs, path = s3_example_s3fs path = path + "/test.parquet" @@ -492,7 +478,6 @@ def test_read_s3fs(s3_example_s3fs, ): @pytest.mark.s3 -@pytest.mark.dataset def test_read_directory_s3fs(s3_example_s3fs): fs, directory = s3_example_s3fs path = directory + "/test.parquet" @@ -504,7 +489,6 @@ def test_read_directory_s3fs(s3_example_s3fs): @pytest.mark.pandas -@pytest.mark.dataset def test_read_single_file_list(tempdir): data_path = str(tempdir / 'data.parquet') @@ -517,7 +501,6 @@ def test_read_single_file_list(tempdir): @pytest.mark.pandas @pytest.mark.s3 -@pytest.mark.dataset def test_read_partitioned_directory_s3fs_wrapper(s3_example_s3fs): import s3fs @@ -538,7 +521,6 @@ def test_read_partitioned_directory_s3fs_wrapper(s3_example_s3fs): @pytest.mark.pandas @pytest.mark.s3 -@pytest.mark.dataset def test_read_partitioned_directory_s3fs(s3_example_s3fs): fs, path = s3_example_s3fs _partition_test_for_filesystem(fs, path) @@ -640,7 +622,6 @@ def _filter_partition(df, part_keys): @pytest.mark.pandas -@pytest.mark.dataset def test_filter_before_validate_schema(tempdir): # ARROW-4076 apply filter before schema validation # to avoid checking unneeded schemas @@ -663,7 +644,6 @@ def test_filter_before_validate_schema(tempdir): @pytest.mark.pandas -@pytest.mark.dataset def test_read_multiple_files(tempdir): nfiles = 10 size = 5 @@ -742,7 +722,6 @@ def read_multiple_files(paths, columns=None, use_threads=True, **kwargs): @pytest.mark.pandas -@pytest.mark.dataset def test_dataset_read_pandas(tempdir): nfiles = 5 size = 5 @@ -781,7 +760,6 @@ def test_dataset_read_pandas(tempdir): @pytest.mark.pandas -@pytest.mark.dataset def test_dataset_memory_map(tempdir): # ARROW-2627: Check that we can use ParquetDataset with memory-mapping dirpath = tempdir / guid() @@ -798,7 +776,6 @@ def test_dataset_memory_map(tempdir): @pytest.mark.pandas -@pytest.mark.dataset def test_dataset_enable_buffered_stream(tempdir): dirpath = tempdir / guid() dirpath.mkdir() @@ -819,7 +796,6 @@ def test_dataset_enable_buffered_stream(tempdir): @pytest.mark.pandas -@pytest.mark.dataset def test_dataset_enable_pre_buffer(tempdir): dirpath = tempdir / guid() dirpath.mkdir() @@ -855,7 +831,6 @@ def _assert_dataset_paths(dataset, paths): @pytest.mark.pandas -@pytest.mark.dataset @pytest.mark.parametrize('dir_prefix', ['_', '.']) def test_ignore_private_directories(tempdir, dir_prefix): dirpath = tempdir / guid() @@ -873,7 +848,6 @@ def test_ignore_private_directories(tempdir, dir_prefix): @pytest.mark.pandas -@pytest.mark.dataset def test_ignore_hidden_files_dot(tempdir): dirpath = tempdir / guid() dirpath.mkdir() @@ -893,7 +867,6 @@ def test_ignore_hidden_files_dot(tempdir): @pytest.mark.pandas -@pytest.mark.dataset def test_ignore_hidden_files_underscore(tempdir): dirpath = tempdir / guid() dirpath.mkdir() @@ -913,7 +886,6 @@ def test_ignore_hidden_files_underscore(tempdir): @pytest.mark.pandas -@pytest.mark.dataset @pytest.mark.parametrize('dir_prefix', ['_', '.']) def test_ignore_no_private_directories_in_base_path(tempdir, dir_prefix): # ARROW-8427 - don't ignore explicitly listed files if parent directory @@ -933,7 +905,6 @@ def test_ignore_no_private_directories_in_base_path(tempdir, dir_prefix): @pytest.mark.pandas -@pytest.mark.dataset def test_ignore_custom_prefixes(tempdir): # ARROW-9573 - allow override of default ignore_prefixes part = ["xxx"] * 3 + ["yyy"] * 3 @@ -955,7 +926,6 @@ def test_ignore_custom_prefixes(tempdir): assert read.equals(table) -@pytest.mark.dataset def test_empty_directory(tempdir): empty_dir = tempdir / 'dataset' empty_dir.mkdir() @@ -1071,13 +1041,11 @@ def _test_write_to_dataset_no_partitions(base_path, @pytest.mark.pandas -@pytest.mark.dataset def test_write_to_dataset_with_partitions(tempdir): _test_write_to_dataset_with_partitions(str(tempdir)) @pytest.mark.pandas -@pytest.mark.dataset def test_write_to_dataset_with_partitions_and_schema(tempdir): schema = pa.schema([pa.field('group1', type=pa.string()), pa.field('group2', type=pa.string()), @@ -1089,20 +1057,17 @@ def test_write_to_dataset_with_partitions_and_schema(tempdir): @pytest.mark.pandas -@pytest.mark.dataset def test_write_to_dataset_with_partitions_and_index_name(tempdir): _test_write_to_dataset_with_partitions( str(tempdir), index_name='index_name') @pytest.mark.pandas -@pytest.mark.dataset def test_write_to_dataset_no_partitions(tempdir): _test_write_to_dataset_no_partitions(str(tempdir)) @pytest.mark.pandas -@pytest.mark.dataset def test_write_to_dataset_pathlib(tempdir): _test_write_to_dataset_with_partitions(tempdir / "test1") _test_write_to_dataset_no_partitions(tempdir / "test2") @@ -1110,7 +1075,6 @@ def test_write_to_dataset_pathlib(tempdir): @pytest.mark.pandas @pytest.mark.s3 -@pytest.mark.dataset def test_write_to_dataset_pathlib_nonlocal(tempdir, s3_example_s3fs): # pathlib paths are only accepted for local files fs, _ = s3_example_s3fs @@ -1126,7 +1090,6 @@ def test_write_to_dataset_pathlib_nonlocal(tempdir, s3_example_s3fs): @pytest.mark.pandas @pytest.mark.s3 -@pytest.mark.dataset def test_write_to_dataset_with_partitions_s3fs(s3_example_s3fs): fs, path = s3_example_s3fs @@ -1136,7 +1099,6 @@ def test_write_to_dataset_with_partitions_s3fs(s3_example_s3fs): @pytest.mark.pandas @pytest.mark.s3 -@pytest.mark.dataset def test_write_to_dataset_no_partitions_s3fs(s3_example_s3fs): fs, path = s3_example_s3fs @@ -1144,7 +1106,6 @@ def test_write_to_dataset_no_partitions_s3fs(s3_example_s3fs): path, filesystem=fs) -@pytest.mark.dataset @pytest.mark.pandas def test_write_to_dataset_filesystem(tempdir): df = pd.DataFrame({'A': [1, 2, 3]}) @@ -1185,7 +1146,6 @@ def _make_dataset_for_pickling(tempdir, N=100): @pytest.mark.pandas -@pytest.mark.dataset def test_pickle_dataset(tempdir, pickle_module): def is_pickleable(obj): return obj == pickle_module.loads(pickle_module.dumps(obj)) @@ -1195,7 +1155,6 @@ def is_pickleable(obj): @pytest.mark.pandas -@pytest.mark.dataset def test_partitioned_dataset(tempdir): # ARROW-3208: Segmentation fault when reading a Parquet partitioned dataset # to a Parquet file @@ -1213,7 +1172,6 @@ def test_partitioned_dataset(tempdir): @pytest.mark.pandas -@pytest.mark.dataset def test_dataset_read_dictionary(tempdir): path = tempdir / "ARROW-3325-dataset" t1 = pa.table([[util.rands(10) for i in range(5)] * 10], names=['f0']) @@ -1237,7 +1195,6 @@ def test_dataset_read_dictionary(tempdir): assert c1.equals(ex_chunks[0]) -@pytest.mark.dataset @pytest.mark.pandas def test_read_table_schema(tempdir): # test that schema keyword is passed through in read_table @@ -1262,7 +1219,6 @@ def test_read_table_schema(tempdir): assert result.read().equals(expected) -@pytest.mark.dataset def test_dataset_unsupported_keywords(): with pytest.raises(ValueError, match="not yet supported with the new"): @@ -1281,7 +1237,6 @@ def test_dataset_unsupported_keywords(): pq.read_table("", metadata=pa.schema([])) -@pytest.mark.dataset def test_dataset_partitioning(tempdir): import pyarrow.dataset as ds @@ -1306,7 +1261,6 @@ def test_dataset_partitioning(tempdir): assert result.column_names == ["a", "year", "month", "day"] -@pytest.mark.dataset def test_parquet_dataset_new_filesystem(tempdir): # Ensure we can pass new FileSystem object to ParquetDataset table = pa.table({'a': [1, 2, 3]}) @@ -1317,7 +1271,6 @@ def test_parquet_dataset_new_filesystem(tempdir): assert result.equals(table) -@pytest.mark.dataset def test_parquet_dataset_partitions_piece_path_with_fsspec(tempdir): # ARROW-10462 ensure that on Windows we properly use posix-style paths # as used by fsspec @@ -1335,7 +1288,6 @@ def test_parquet_dataset_partitions_piece_path_with_fsspec(tempdir): assert dataset.fragments[0].path == expected -@pytest.mark.dataset def test_parquet_write_to_dataset_exposed_keywords(tempdir): table = pa.table({'a': [1, 2, 3]}) path = tempdir / 'partitioning' @@ -1360,7 +1312,6 @@ def file_visitor(written_file): assert paths_written_set == expected_paths -@pytest.mark.dataset @pytest.mark.parametrize("write_dataset_kwarg", ( ("create_dir", True), ("create_dir", False), @@ -1387,7 +1338,6 @@ def test_write_to_dataset_kwargs_passed(tempdir, write_dataset_kwarg): @pytest.mark.pandas -@pytest.mark.dataset def test_write_to_dataset_category_observed(tempdir): # if we partition on a categorical variable with "unobserved" categories # (values present in the dictionary, but not in the actual data) From e13768e9f5d0a3c4f7c1239f57fd84a123890869 Mon Sep 17 00:00:00 2001 From: AlenkaF Date: Mon, 11 Dec 2023 10:31:30 +0100 Subject: [PATCH 11/29] Clean up the docstrings --- python/pyarrow/parquet/core.py | 200 +++++++++++++++++---------------- 1 file changed, 101 insertions(+), 99 deletions(-) diff --git a/python/pyarrow/parquet/core.py b/python/pyarrow/parquet/core.py index 8d24f9a80b76c..bd7f512c8e917 100644 --- a/python/pyarrow/parquet/core.py +++ b/python/pyarrow/parquet/core.py @@ -1149,102 +1149,104 @@ def _is_local_file_system(fs): _parquet_dataset_example = """\ Generate an example PyArrow Table and write it to a partitioned dataset: - >>> import pyarrow as pa - >>> table = pa.table({'year': [2020, 2022, 2021, 2022, 2019, 2021], - ... 'n_legs': [2, 2, 4, 4, 5, 100], - ... 'animal': ["Flamingo", "Parrot", "Dog", "Horse", - ... "Brittle stars", "Centipede"]}) - >>> import pyarrow.parquet as pq - >>> pq.write_to_dataset(table, root_path='dataset_v2', - ... partition_cols=['year']) +>>> import pyarrow as pa +>>> table = pa.table({'year': [2020, 2022, 2021, 2022, 2019, 2021], +... 'n_legs': [2, 2, 4, 4, 5, 100], +... 'animal': ["Flamingo", "Parrot", "Dog", "Horse", +... "Brittle stars", "Centipede"]}) +>>> import pyarrow.parquet as pq +>>> pq.write_to_dataset(table, root_path='dataset_v2', +... partition_cols=['year']) - create a ParquetDataset object from the dataset source: +create a ParquetDataset object from the dataset source: - >>> dataset = pq.ParquetDataset('dataset_v2/') +>>> dataset = pq.ParquetDataset('dataset_v2/') - and read the data: +and read the data: - >>> dataset.read().to_pandas() - n_legs animal year - 0 5 Brittle stars 2019 - 1 2 Flamingo 2020 - 2 4 Dog 2021 - 3 100 Centipede 2021 - 4 2 Parrot 2022 - 5 4 Horse 2022 +>>> dataset.read().to_pandas() + n_legs animal year +0 5 Brittle stars 2019 +1 2 Flamingo 2020 +2 4 Dog 2021 +3 100 Centipede 2021 +4 2 Parrot 2022 +5 4 Horse 2022 - create a ParquetDataset object with filter: +create a ParquetDataset object with filter: - >>> dataset = pq.ParquetDataset('dataset_v2/', - ... filters=[('n_legs','=',4)]) - >>> dataset.read().to_pandas() - n_legs animal year - 0 4 Dog 2021 - 1 4 Horse 2022 +>>> dataset = pq.ParquetDataset('dataset_v2/', +... filters=[('n_legs','=',4)]) +>>> dataset.read().to_pandas() + n_legs animal year +0 4 Dog 2021 +1 4 Horse 2022 """ class ParquetDataset: __doc__ = """ - Encapsulates details of reading a complete Parquet dataset possibly - consisting of multiple files and partitions in subdirectories. +Encapsulates details of reading a complete Parquet dataset possibly +consisting of multiple files and partitions in subdirectories. - Parameters - ---------- - path_or_paths : str or List[str] - A directory name, single file name, or list of file names. - filesystem : FileSystem, default None - If nothing passed, will be inferred based on path. - Path will try to be found in the local on-disk filesystem otherwise - it will be parsed as an URI to determine the filesystem. - schema : pyarrow.parquet.Schema - Optionally provide the Schema for the Dataset, in which case it will - not be inferred from the source. - filters : pyarrow.compute.Expression or List[Tuple] or List[List[Tuple]], - default None - Rows which do not match the filter predicate will be removed from scanned - data. Partition keys embedded in a nested directory structure will be - exploited to avoid loading files at all if they contain no matching rows. - Within-file level filtering and different partitioning schemes are supported. - - {1} - {0} - ignore_prefixes : list, optional - Files matching any of these prefixes will be ignored by the - discovery process. - This is matched to the basename of a path. - By default this is ['.', '_']. - Note that discovery happens only if a directory is passed as source. - pre_buffer : bool, default True - Coalesce and issue file reads in parallel to improve performance on - high-latency filesystems (e.g. S3, GCS). If True, Arrow will use a - background I/O thread pool. If using a filesystem layer that itself - performs readahead (e.g. fsspec's S3FS), disable readahead for best - results. Set to False if you want to prioritize minimal memory usage - over maximum speed. - coerce_int96_timestamp_unit : str, default None - Cast timestamps that are stored in INT96 format to a particular resolution - (e.g. 'ms'). Setting to None is equivalent to 'ns' and therefore INT96 - timestamps will be inferred as timestamps in nanoseconds. - decryption_properties : FileDecryptionProperties or None - File-level decryption properties. - The decryption properties can be created using - ``CryptoFactory.file_decryption_properties()``. - thrift_string_size_limit : int, default None - If not None, override the maximum total string size allocated - when decoding Thrift structures. The default limit should be - sufficient for most Parquet files. - thrift_container_size_limit : int, default None - If not None, override the maximum total size of containers allocated - when decoding Thrift structures. The default limit should be - sufficient for most Parquet files. - page_checksum_verification : bool, default False - If True, verify the page checksum for each page read from the file. +Parameters +---------- +path_or_paths : str or List[str] + A directory name, single file name, or list of file names. +filesystem : FileSystem, default None + If nothing passed, will be inferred based on path. + Path will try to be found in the local on-disk filesystem otherwise + it will be parsed as an URI to determine the filesystem. +schema : pyarrow.parquet.Schema + Optionally provide the Schema for the Dataset, in which case it will + not be inferred from the source. +filters : pyarrow.compute.Expression or List[Tuple] or List[List[Tuple]], +default None + Rows which do not match the filter predicate will be removed from scanned + data. Partition keys embedded in a nested directory structure will be + exploited to avoid loading files at all if they contain no matching rows. + Within-file level filtering and different partitioning schemes are supported. - Examples - -------- - {2} - """.format(_read_docstring_common, _DNF_filter_doc, _parquet_dataset_example) + {1} +{0} +ignore_prefixes : list, optional + Files matching any of these prefixes will be ignored by the + discovery process. + This is matched to the basename of a path. + By default this is ['.', '_']. + Note that discovery happens only if a directory is passed as source. +pre_buffer : bool, default True + Coalesce and issue file reads in parallel to improve performance on + high-latency filesystems (e.g. S3, GCS). If True, Arrow will use a + background I/O thread pool. If using a filesystem layer that itself + performs readahead (e.g. fsspec's S3FS), disable readahead for best + results. Set to False if you want to prioritize minimal memory usage + over maximum speed. +coerce_int96_timestamp_unit : str, default None + Cast timestamps that are stored in INT96 format to a particular resolution + (e.g. 'ms'). Setting to None is equivalent to 'ns' and therefore INT96 + timestamps will be inferred as timestamps in nanoseconds. +decryption_properties : FileDecryptionProperties or None + File-level decryption properties. + The decryption properties can be created using + ``CryptoFactory.file_decryption_properties()``. +thrift_string_size_limit : int, default None + If not None, override the maximum total string size allocated + when decoding Thrift structures. The default limit should be + sufficient for most Parquet files. +thrift_container_size_limit : int, default None + If not None, override the maximum total size of containers allocated + when decoding Thrift structures. The default limit should be + sufficient for most Parquet files. +page_checksum_verification : bool, default False + If True, verify the page checksum for each page read from the file. +use_legacy_dataset : bool + Deprecated and has no effect from PyArrow version 15.0.0. + +Examples +-------- +{2} +""".format(_read_docstring_common, _DNF_filter_doc, _parquet_dataset_example) def __init__(self, path_or_paths, filesystem=None, schema=None, *, filters=None, partitioning="hive", read_dictionary=None, buffer_size=None, @@ -1253,9 +1255,8 @@ def __init__(self, path_or_paths, filesystem=None, schema=None, *, filters=None, decryption_properties=None, thrift_string_size_limit=None, thrift_container_size_limit=None, page_checksum_verification=False, - **kwargs): + use_legacy_dataset=None): - use_legacy_dataset = kwargs.pop('use_legacy_dataset', None) if use_legacy_dataset is not None: warnings.warn( "Passing 'use_legacy_dataset' is deprecated as of pyarrow 15.0.0 " @@ -1264,15 +1265,6 @@ def __init__(self, path_or_paths, filesystem=None, schema=None, *, filters=None, import pyarrow.dataset as ds - # Raise error for not supported keywords - for keyword, default in [ - ("metadata", None), ("split_row_groups", False), - ("validate_schema", True), ("metadata_nthreads", None)]: - if keyword in kwargs and kwargs[keyword] is not default: - raise ValueError( - "Keyword '{0}' is not yet supported with the new " - "Dataset API".format(keyword)) - # map format arguments read_options = { "pre_buffer": pre_buffer, @@ -1503,7 +1495,12 @@ def _get_common_pandas_metadata(self): def read_pandas(self, **kwargs): """ Read dataset including pandas metadata, if any. Other arguments passed - through to ParquetDataset.read, see docstring for further details. + through to :func:`read`, see docstring for further details. + + Parameters + ---------- + **kwargs : optional + Additional options for :func:`read` Examples -------- @@ -1635,6 +1632,8 @@ def partitioning(self): Within-file level filtering and different partitioning schemes are supported. {3} +use_legacy_dataset : bool + Deprecated and has no effect from PyArrow version 15.0.0. ignore_prefixes : list, optional Files matching any of these prefixes will be ignored by the discovery process. @@ -2006,6 +2005,8 @@ def write_to_dataset(table, root_path, partition_cols=None, If nothing passed, will be inferred based on path. Path will try to be found in the local on-disk filesystem otherwise it will be parsed as an URI to determine the filesystem. + use_legacy_dataset : bool + Deprecated and has no effect from PyArrow version 15.0.0. schema : Schema, optional This Schema of the dataset. partitioning : Partitioning or list[str], optional @@ -2059,10 +2060,11 @@ def file_visitor(written_file): the entire directory will be deleted. This allows you to overwrite old partitions completely. **kwargs : dict, - Used as additional kwargs for `dataset.write_dataset` function for - matching kwargs, and remainder to `ParquetFileFormat.make_write_options`. - See the docstring of `write_table` and `dataset.write_dataset` for - the available options. + Used as additional kwargs for :func:`pyarrow.dataset.write_dataset` + function for matching kwargs, and remainder to + :func:`pyarrow.dataset.ParquetFileFormat.make_write_options`. + See the docstring of :func:`write_table` and + :func:`pyarrow.dataset.write_dataset` for the available options. Using `metadata_collector` in kwargs allows one to collect the file metadata instances of dataset pieces. The file paths in the ColumnChunkMetaData will be set relative to `root_path`. From bf0ce99c7ce4d12f78c10e9892808b8a286f299a Mon Sep 17 00:00:00 2001 From: AlenkaF Date: Mon, 11 Dec 2023 13:23:39 +0100 Subject: [PATCH 12/29] Remove a test for unsupported keywords and update docstrings --- python/pyarrow/parquet/core.py | 6 +++--- python/pyarrow/tests/parquet/test_dataset.py | 18 ------------------ 2 files changed, 3 insertions(+), 21 deletions(-) diff --git a/python/pyarrow/parquet/core.py b/python/pyarrow/parquet/core.py index bd7f512c8e917..f16e619a12684 100644 --- a/python/pyarrow/parquet/core.py +++ b/python/pyarrow/parquet/core.py @@ -1240,7 +1240,7 @@ class ParquetDataset: sufficient for most Parquet files. page_checksum_verification : bool, default False If True, verify the page checksum for each page read from the file. -use_legacy_dataset : bool +use_legacy_dataset : bool, optional Deprecated and has no effect from PyArrow version 15.0.0. Examples @@ -1632,7 +1632,7 @@ def partitioning(self): Within-file level filtering and different partitioning schemes are supported. {3} -use_legacy_dataset : bool +use_legacy_dataset : bool, optional Deprecated and has no effect from PyArrow version 15.0.0. ignore_prefixes : list, optional Files matching any of these prefixes will be ignored by the @@ -2005,7 +2005,7 @@ def write_to_dataset(table, root_path, partition_cols=None, If nothing passed, will be inferred based on path. Path will try to be found in the local on-disk filesystem otherwise it will be parsed as an URI to determine the filesystem. - use_legacy_dataset : bool + use_legacy_dataset : bool, optional Deprecated and has no effect from PyArrow version 15.0.0. schema : Schema, optional This Schema of the dataset. diff --git a/python/pyarrow/tests/parquet/test_dataset.py b/python/pyarrow/tests/parquet/test_dataset.py index bd8d47f60b751..0ab0f78fa39f3 100644 --- a/python/pyarrow/tests/parquet/test_dataset.py +++ b/python/pyarrow/tests/parquet/test_dataset.py @@ -1219,24 +1219,6 @@ def test_read_table_schema(tempdir): assert result.read().equals(expected) -def test_dataset_unsupported_keywords(): - - with pytest.raises(ValueError, match="not yet supported with the new"): - pq.ParquetDataset("", metadata=pa.schema([])) - - with pytest.raises(ValueError, match="not yet supported with the new"): - pq.ParquetDataset("", validate_schema=False) - - with pytest.raises(ValueError, match="not yet supported with the new"): - pq.ParquetDataset("", split_row_groups=True) - - with pytest.raises(ValueError, match="not yet supported with the new"): - pq.ParquetDataset("", metadata_nthreads=4) - - with pytest.raises(ValueError, match="no longer supported"): - pq.read_table("", metadata=pa.schema([])) - - def test_dataset_partitioning(tempdir): import pyarrow.dataset as ds From c9a7924de0e04c78a072db3cfa9b6543b45b006b Mon Sep 17 00:00:00 2001 From: AlenkaF Date: Mon, 11 Dec 2023 14:59:24 +0100 Subject: [PATCH 13/29] Change how we deal with unsupported keywords in ParquetDataset --- python/pyarrow/parquet/core.py | 6 ++++-- python/pyarrow/tests/parquet/test_dataset.py | 5 +---- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/python/pyarrow/parquet/core.py b/python/pyarrow/parquet/core.py index f16e619a12684..d8a06960ee314 100644 --- a/python/pyarrow/parquet/core.py +++ b/python/pyarrow/parquet/core.py @@ -1200,8 +1200,7 @@ class ParquetDataset: schema : pyarrow.parquet.Schema Optionally provide the Schema for the Dataset, in which case it will not be inferred from the source. -filters : pyarrow.compute.Expression or List[Tuple] or List[List[Tuple]], -default None +filters : pyarrow.compute.Expression or List[Tuple] or List[List[Tuple]], default None Rows which do not match the filter predicate will be removed from scanned data. Partition keys embedded in a nested directory structure will be exploited to avoid loading files at all if they contain no matching rows. @@ -1243,6 +1242,9 @@ class ParquetDataset: use_legacy_dataset : bool, optional Deprecated and has no effect from PyArrow version 15.0.0. +Keywords `metadata`, `split_row_groups=True` and `validate_schema=False` +are not yet supported. + Examples -------- {2} diff --git a/python/pyarrow/tests/parquet/test_dataset.py b/python/pyarrow/tests/parquet/test_dataset.py index 0ab0f78fa39f3..a14b227559920 100644 --- a/python/pyarrow/tests/parquet/test_dataset.py +++ b/python/pyarrow/tests/parquet/test_dataset.py @@ -969,11 +969,8 @@ def _test_write_to_dataset_with_partitions(base_path, else: pq.write_metadata(output_table.schema, metadata_path) - # ARROW-2891: Ensure the output_schema is preserved when writing a - # partitioned dataset dataset = pq.ParquetDataset(base_path, - filesystem=filesystem, - validate_schema=True) + filesystem=filesystem) # ARROW-2209: Ensure the dataset schema also includes the partition columns # NB schema property is an arrow and not parquet schema dataset_cols = set(dataset.schema.names) From b6799cfbe68a3289e00378739bc5291ff81d8855 Mon Sep 17 00:00:00 2001 From: AlenkaF Date: Mon, 11 Dec 2023 15:59:15 +0100 Subject: [PATCH 14/29] Some more changes to the docstrings --- python/pyarrow/parquet/core.py | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/python/pyarrow/parquet/core.py b/python/pyarrow/parquet/core.py index d8a06960ee314..c29182cbc2565 100644 --- a/python/pyarrow/parquet/core.py +++ b/python/pyarrow/parquet/core.py @@ -1189,6 +1189,9 @@ class ParquetDataset: Encapsulates details of reading a complete Parquet dataset possibly consisting of multiple files and partitions in subdirectories. +Keywords `metadata`, `split_row_groups=True` and `validate_schema=False` +are not yet supported. + Parameters ---------- path_or_paths : str or List[str] @@ -1242,9 +1245,6 @@ class ParquetDataset: use_legacy_dataset : bool, optional Deprecated and has no effect from PyArrow version 15.0.0. -Keywords `metadata`, `split_row_groups=True` and `validate_schema=False` -are not yet supported. - Examples -------- {2} @@ -1617,8 +1617,6 @@ def partitioning(self): no columns. use_threads : bool, default True Perform multi-threaded column reads. -metadata : FileMetaData - If separately computed schema : Schema, optional Optionally provide the Schema for the parquet dataset, in which case it will not be inferred from the source. @@ -1667,6 +1665,9 @@ def partitioning(self): sufficient for most Parquet files. page_checksum_verification : bool, default False If True, verify the checksum for each page read from the file. +metadata : FileMetaData + This keyword is no longer supported and is being deprecated with + PyArrow version 15.0.0. Returns ------- @@ -1773,7 +1774,8 @@ def read_table(source, *, columns=None, use_threads=True, if metadata is not None: raise ValueError( "The 'metadata' keyword is no longer supported with the new " - "datasets-based implementation." + "datasets-based implementation and will be removed in future " + "versions." ) try: dataset = ParquetDataset( From 45a24092719bcdf5b5d61c5b0d18d84d168ea03d Mon Sep 17 00:00:00 2001 From: AlenkaF Date: Mon, 11 Dec 2023 16:36:53 +0100 Subject: [PATCH 15/29] Update ParquetDataset docstrings --- python/pyarrow/parquet/core.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/pyarrow/parquet/core.py b/python/pyarrow/parquet/core.py index c29182cbc2565..f5e2486f680da 100644 --- a/python/pyarrow/parquet/core.py +++ b/python/pyarrow/parquet/core.py @@ -1251,8 +1251,8 @@ class ParquetDataset: """.format(_read_docstring_common, _DNF_filter_doc, _parquet_dataset_example) def __init__(self, path_or_paths, filesystem=None, schema=None, *, filters=None, - partitioning="hive", read_dictionary=None, buffer_size=None, - memory_map=False, ignore_prefixes=None, pre_buffer=True, + read_dictionary=None, memory_map=False, buffer_size=None, + partitioning="hive", ignore_prefixes=None, pre_buffer=True, coerce_int96_timestamp_unit=None, decryption_properties=None, thrift_string_size_limit=None, thrift_container_size_limit=None, From 4573479e0dddc9f1871a367fa11c8275189e1a1e Mon Sep 17 00:00:00 2001 From: AlenkaF Date: Mon, 11 Dec 2023 17:55:03 +0100 Subject: [PATCH 16/29] Fix docstring examples --- python/pyarrow/parquet/core.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/pyarrow/parquet/core.py b/python/pyarrow/parquet/core.py index f5e2486f680da..94fc844e5351d 100644 --- a/python/pyarrow/parquet/core.py +++ b/python/pyarrow/parquet/core.py @@ -1165,7 +1165,7 @@ def _is_local_file_system(fs): and read the data: >>> dataset.read().to_pandas() - n_legs animal year + n_legs animal year 0 5 Brittle stars 2019 1 2 Flamingo 2020 2 4 Dog 2021 @@ -1178,7 +1178,7 @@ def _is_local_file_system(fs): >>> dataset = pq.ParquetDataset('dataset_v2/', ... filters=[('n_legs','=',4)]) >>> dataset.read().to_pandas() - n_legs animal year + n_legs animal year 0 4 Dog 2021 1 4 Horse 2022 """ From 88c340e44e00d6216d3868c2a8f6a1c25928f4b2 Mon Sep 17 00:00:00 2001 From: AlenkaF Date: Tue, 12 Dec 2023 14:09:28 +0100 Subject: [PATCH 17/29] Remove metadata from read_table --- python/pyarrow/parquet/core.py | 11 +---------- python/pyarrow/tests/parquet/test_dataset.py | 6 ------ 2 files changed, 1 insertion(+), 16 deletions(-) diff --git a/python/pyarrow/parquet/core.py b/python/pyarrow/parquet/core.py index 94fc844e5351d..4ada74e745dcd 100644 --- a/python/pyarrow/parquet/core.py +++ b/python/pyarrow/parquet/core.py @@ -1665,9 +1665,6 @@ def partitioning(self): sufficient for most Parquet files. page_checksum_verification : bool, default False If True, verify the checksum for each page read from the file. -metadata : FileMetaData - This keyword is no longer supported and is being deprecated with - PyArrow version 15.0.0. Returns ------- @@ -1763,7 +1760,7 @@ def read_table(source, *, columns=None, use_threads=True, coerce_int96_timestamp_unit=None, decryption_properties=None, thrift_string_size_limit=None, thrift_container_size_limit=None, - page_checksum_verification=False, metadata=None,): + page_checksum_verification=False): if use_legacy_dataset is not None: warnings.warn( @@ -1771,12 +1768,6 @@ def read_table(source, *, columns=None, use_threads=True, "and will be removed in a future version.", FutureWarning, stacklevel=2) - if metadata is not None: - raise ValueError( - "The 'metadata' keyword is no longer supported with the new " - "datasets-based implementation and will be removed in future " - "versions." - ) try: dataset = ParquetDataset( source, diff --git a/python/pyarrow/tests/parquet/test_dataset.py b/python/pyarrow/tests/parquet/test_dataset.py index a14b227559920..56f592f7c1100 100644 --- a/python/pyarrow/tests/parquet/test_dataset.py +++ b/python/pyarrow/tests/parquet/test_dataset.py @@ -679,12 +679,6 @@ def read_multiple_files(paths, columns=None, use_threads=True, **kwargs): assert result.equals(expected) - # Read with provided metadata - # TODO specifying metadata not yet supported - metadata = pq.read_metadata(paths[0]) - with pytest.raises(ValueError, match="no longer supported"): - pq.read_table(paths, metadata=metadata) - # Read column subset to_read = [0, 2, 6, result.num_columns - 1] From 0cbd03dcbafb76853c0eabb08d74795afd77cc4a Mon Sep 17 00:00:00 2001 From: AlenkaF Date: Tue, 12 Dec 2023 14:28:28 +0100 Subject: [PATCH 18/29] One more metadata case to remove in read_table --- python/pyarrow/parquet/core.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyarrow/parquet/core.py b/python/pyarrow/parquet/core.py index 4ada74e745dcd..08a831c611c50 100644 --- a/python/pyarrow/parquet/core.py +++ b/python/pyarrow/parquet/core.py @@ -1808,7 +1808,7 @@ def read_table(source, *, columns=None, use_threads=True, source = filesystem.open_input_file(path) # TODO test that source is not a directory or a list dataset = ParquetFile( - source, metadata=metadata, read_dictionary=read_dictionary, + source, read_dictionary=read_dictionary, memory_map=memory_map, buffer_size=buffer_size, pre_buffer=pre_buffer, coerce_int96_timestamp_unit=coerce_int96_timestamp_unit, From 5687a1d835354e15aa8c5f575af9c3c96a3de5cd Mon Sep 17 00:00:00 2001 From: AlenkaF Date: Wed, 20 Dec 2023 13:24:46 +0100 Subject: [PATCH 19/29] Remove dataset marks and some unused tempdir parameters --- .../parquet/test_compliant_nested_type.py | 1 - python/pyarrow/tests/parquet/test_datetime.py | 2 -- python/pyarrow/tests/parquet/test_pandas.py | 28 +++---------------- 3 files changed, 4 insertions(+), 27 deletions(-) diff --git a/python/pyarrow/tests/parquet/test_compliant_nested_type.py b/python/pyarrow/tests/parquet/test_compliant_nested_type.py index 06c23bf1af1e7..2345855a3321b 100644 --- a/python/pyarrow/tests/parquet/test_compliant_nested_type.py +++ b/python/pyarrow/tests/parquet/test_compliant_nested_type.py @@ -83,7 +83,6 @@ def test_write_compliant_nested_type_enable(tempdir, test_data): @pytest.mark.pandas -@pytest.mark.dataset @parametrize_test_data def test_write_compliant_nested_type_disable(tempdir, test_data): # prepare dataframe for testing diff --git a/python/pyarrow/tests/parquet/test_datetime.py b/python/pyarrow/tests/parquet/test_datetime.py index 1e653d45a3940..6a9cbd4f73d4f 100644 --- a/python/pyarrow/tests/parquet/test_datetime.py +++ b/python/pyarrow/tests/parquet/test_datetime.py @@ -47,7 +47,6 @@ @pytest.mark.pandas -@pytest.mark.dataset def test_pandas_parquet_datetime_tz(): # Pandas v2 defaults to [ns], but Arrow defaults to [us] time units # so we need to cast the pandas dtype. Pandas v1 will always silently @@ -75,7 +74,6 @@ def test_pandas_parquet_datetime_tz(): @pytest.mark.pandas -@pytest.mark.dataset def test_datetime_timezone_tzinfo(): value = datetime.datetime(2018, 1, 1, 1, 23, 45, tzinfo=datetime.timezone.utc) diff --git a/python/pyarrow/tests/parquet/test_pandas.py b/python/pyarrow/tests/parquet/test_pandas.py index 84f698db73bb9..f194d12876968 100644 --- a/python/pyarrow/tests/parquet/test_pandas.py +++ b/python/pyarrow/tests/parquet/test_pandas.py @@ -99,7 +99,6 @@ def test_merging_parquet_tables_with_different_pandas_metadata(tempdir): @pytest.mark.pandas -@pytest.mark.dataset def test_pandas_parquet_column_multiindex(tempdir): df = alltypes_sample(size=10) df.columns = pd.MultiIndex.from_tuples( @@ -119,7 +118,6 @@ def test_pandas_parquet_column_multiindex(tempdir): @pytest.mark.pandas -@pytest.mark.dataset def test_pandas_parquet_2_0_roundtrip_read_pandas_no_index_written(tempdir): df = alltypes_sample(size=10000) @@ -145,8 +143,7 @@ def test_pandas_parquet_2_0_roundtrip_read_pandas_no_index_written(tempdir): @pytest.mark.pandas -@pytest.mark.dataset -def test_pandas_parquet_native_file_roundtrip(tempdir): +def test_pandas_parquet_native_file_roundtrip(): df = _test_dataframe(10000) arrow_table = pa.Table.from_pandas(df) imos = pa.BufferOutputStream() @@ -158,8 +155,7 @@ def test_pandas_parquet_native_file_roundtrip(tempdir): @pytest.mark.pandas -@pytest.mark.dataset -def test_read_pandas_column_subset(tempdir): +def test_read_pandas_column_subset(): df = _test_dataframe(10000) arrow_table = pa.Table.from_pandas(df) imos = pa.BufferOutputStream() @@ -173,8 +169,7 @@ def test_read_pandas_column_subset(tempdir): @pytest.mark.pandas -@pytest.mark.dataset -def test_pandas_parquet_empty_roundtrip(tempdir): +def test_pandas_parquet_empty_roundtrip(): df = _test_dataframe(0) arrow_table = pa.Table.from_pandas(df) imos = pa.BufferOutputStream() @@ -186,7 +181,7 @@ def test_pandas_parquet_empty_roundtrip(tempdir): @pytest.mark.pandas -def test_pandas_can_write_nested_data(tempdir): +def test_pandas_can_write_nested_data(): data = { "agg_col": [ {"page_type": 1}, @@ -203,7 +198,6 @@ def test_pandas_can_write_nested_data(tempdir): @pytest.mark.pandas -@pytest.mark.dataset def test_pandas_parquet_pyfile_roundtrip(tempdir): filename = tempdir / 'pandas_pyfile_roundtrip.parquet' size = 5 @@ -228,7 +222,6 @@ def test_pandas_parquet_pyfile_roundtrip(tempdir): @pytest.mark.pandas -@pytest.mark.dataset def test_pandas_parquet_configuration_options(tempdir): size = 10000 np.random.seed(0) @@ -286,7 +279,6 @@ def test_spark_flavor_preserves_pandas_metadata(): @pytest.mark.pandas -@pytest.mark.dataset def test_index_column_name_duplicate(tempdir): data = { 'close': { @@ -317,7 +309,6 @@ def test_index_column_name_duplicate(tempdir): @pytest.mark.pandas -@pytest.mark.dataset def test_multiindex_duplicate_values(tempdir): num_rows = 3 numbers = list(range(num_rows)) @@ -340,7 +331,6 @@ def test_multiindex_duplicate_values(tempdir): @pytest.mark.pandas -@pytest.mark.dataset def test_backwards_compatible_index_naming(datadir): expected_string = b"""\ carat cut color clarity depth table price x y z @@ -362,7 +352,6 @@ def test_backwards_compatible_index_naming(datadir): @pytest.mark.pandas -@pytest.mark.dataset def test_backwards_compatible_index_multi_level_named(datadir): expected_string = b"""\ carat cut color clarity depth table price x y z @@ -388,7 +377,6 @@ def test_backwards_compatible_index_multi_level_named(datadir): @pytest.mark.pandas -@pytest.mark.dataset def test_backwards_compatible_index_multi_level_some_named(datadir): expected_string = b"""\ carat cut color clarity depth table price x y z @@ -415,7 +403,6 @@ def test_backwards_compatible_index_multi_level_some_named(datadir): @pytest.mark.pandas -@pytest.mark.dataset def test_backwards_compatible_column_metadata_handling(datadir): expected = pd.DataFrame( {'a': [1, 2, 3], 'b': [.1, .2, .3], @@ -437,7 +424,6 @@ def test_backwards_compatible_column_metadata_handling(datadir): @pytest.mark.pandas -@pytest.mark.dataset def test_categorical_index_survives_roundtrip(): # ARROW-3652, addressed by ARROW-3246 df = pd.DataFrame([['a', 'b'], ['c', 'd']], columns=['c1', 'c2']) @@ -453,7 +439,6 @@ def test_categorical_index_survives_roundtrip(): @pytest.mark.pandas -@pytest.mark.dataset def test_categorical_order_survives_roundtrip(): # ARROW-6302 df = pd.DataFrame({"a": pd.Categorical( @@ -470,7 +455,6 @@ def test_categorical_order_survives_roundtrip(): @pytest.mark.pandas -@pytest.mark.dataset def test_pandas_categorical_na_type_row_groups(): # ARROW-5085 df = pd.DataFrame({"col": [None] * 100, "int": [1.0] * 100}) @@ -489,7 +473,6 @@ def test_pandas_categorical_na_type_row_groups(): @pytest.mark.pandas -@pytest.mark.dataset def test_pandas_categorical_roundtrip(): # ARROW-5480, this was enabled by ARROW-3246 @@ -533,7 +516,6 @@ def test_categories_with_string_pyarrow_dtype(tempdir): @pytest.mark.pandas -@pytest.mark.dataset def test_write_to_dataset_pandas_preserve_extensiondtypes(tempdir): df = pd.DataFrame({'part': 'a', "col": [1, 2, 3]}) df['col'] = df['col'].astype("Int64") @@ -555,7 +537,6 @@ def test_write_to_dataset_pandas_preserve_extensiondtypes(tempdir): @pytest.mark.pandas -@pytest.mark.dataset def test_write_to_dataset_pandas_preserve_index(tempdir): # ARROW-8251 - preserve pandas index in roundtrip @@ -581,7 +562,6 @@ def test_write_to_dataset_pandas_preserve_index(tempdir): @pytest.mark.pandas -@pytest.mark.dataset @pytest.mark.parametrize('preserve_index', [True, False, None]) @pytest.mark.parametrize('metadata_fname', ["_metadata", "_common_metadata"]) def test_dataset_read_pandas_common_metadata( From abe8ab0f21fc06fa8582403f86493f73f749b276 Mon Sep 17 00:00:00 2001 From: AlenkaF Date: Wed, 20 Dec 2023 13:51:03 +0100 Subject: [PATCH 20/29] Removed panads marks --- python/pyarrow/tests/test_dataset.py | 14 -------------- 1 file changed, 14 deletions(-) diff --git a/python/pyarrow/tests/test_dataset.py b/python/pyarrow/tests/test_dataset.py index 7c58b85de8091..23b441881ce58 100644 --- a/python/pyarrow/tests/test_dataset.py +++ b/python/pyarrow/tests/test_dataset.py @@ -1082,7 +1082,6 @@ def _create_dataset_for_fragments(tempdir, chunk_size=None, filesystem=None): path = str(tempdir / "test_parquet_dataset") - # write_to_dataset currently requires pandas pq.write_to_dataset(table, path, partition_cols=["part"], chunk_size=chunk_size) dataset = ds.dataset( @@ -1092,7 +1091,6 @@ def _create_dataset_for_fragments(tempdir, chunk_size=None, filesystem=None): return table, dataset -@pytest.mark.pandas @pytest.mark.parquet def test_fragments(tempdir, dataset_reader): table, dataset = _create_dataset_for_fragments(tempdir) @@ -1140,7 +1138,6 @@ def test_fragments_implicit_cast(tempdir): assert len(list(fragments)) == 1 -@pytest.mark.pandas @pytest.mark.parquet def test_fragments_reconstruct(tempdir, dataset_reader, pickle_module): table, dataset = _create_dataset_for_fragments(tempdir) @@ -1202,7 +1199,6 @@ def assert_yields_projected(fragment, row_slice, dataset_reader.to_table(new_fragment, filter=ds.field('part') == 'a') -@pytest.mark.pandas @pytest.mark.parquet def test_fragments_parquet_row_groups(tempdir, dataset_reader): table, dataset = _create_dataset_for_fragments(tempdir, chunk_size=2) @@ -1254,8 +1250,6 @@ def test_fragments_parquet_num_row_groups(tempdir): @pytest.mark.pandas @pytest.mark.parquet def test_fragments_parquet_row_groups_dictionary(tempdir, dataset_reader): - import pandas as pd - df = pd.DataFrame(dict(col1=['a', 'b'], col2=[1, 2])) df['col1'] = df['col1'].astype("category") @@ -1268,7 +1262,6 @@ def test_fragments_parquet_row_groups_dictionary(tempdir, dataset_reader): assert (df.iloc[0] == result.to_pandas()).all().all() -@pytest.mark.pandas @pytest.mark.parquet def test_fragments_parquet_ensure_metadata(tempdir, open_logging_fs, pickle_module): fs, assert_opens = open_logging_fs @@ -1310,7 +1303,6 @@ def test_fragments_parquet_ensure_metadata(tempdir, open_logging_fs, pickle_modu assert row_group.statistics is not None -@pytest.mark.pandas @pytest.mark.parquet def test_fragments_parquet_pickle_no_metadata(tempdir, open_logging_fs, pickle_module): # https://issues.apache.org/jira/browse/ARROW-15796 @@ -1452,7 +1444,6 @@ def test_parquet_empty_row_group_statistics(tempdir): assert fragments[0].row_groups[0].statistics == {} -@pytest.mark.pandas @pytest.mark.parquet def test_fragments_parquet_row_groups_predicate(tempdir): table, dataset = _create_dataset_for_fragments(tempdir, chunk_size=2) @@ -1476,7 +1467,6 @@ def test_fragments_parquet_row_groups_predicate(tempdir): assert len(row_group_fragments) == 0 -@pytest.mark.pandas @pytest.mark.parquet def test_fragments_parquet_row_groups_reconstruct(tempdir, dataset_reader, pickle_module): @@ -1519,7 +1509,6 @@ def test_fragments_parquet_row_groups_reconstruct(tempdir, dataset_reader, dataset_reader.to_table(new_fragment) -@pytest.mark.pandas @pytest.mark.parquet def test_fragments_parquet_subset_ids(tempdir, open_logging_fs, dataset_reader): @@ -1548,7 +1537,6 @@ def test_fragments_parquet_subset_ids(tempdir, open_logging_fs, assert result.equals(table[:0]) -@pytest.mark.pandas @pytest.mark.parquet def test_fragments_parquet_subset_filter(tempdir, open_logging_fs, dataset_reader): @@ -1581,7 +1569,6 @@ def test_fragments_parquet_subset_filter(tempdir, open_logging_fs, assert subfrag.num_row_groups == 4 -@pytest.mark.pandas @pytest.mark.parquet def test_fragments_parquet_subset_invalid(tempdir): _, dataset = _create_dataset_for_fragments(tempdir, chunk_size=1) @@ -3729,7 +3716,6 @@ def test_dataset_project_only_partition_columns(tempdir, dataset_reader): @pytest.mark.parquet @pytest.mark.pandas def test_dataset_project_null_column(tempdir, dataset_reader): - import pandas as pd df = pd.DataFrame({"col": np.array([None, None, None], dtype='object')}) f = tempdir / "test_dataset_project_null_column.parquet" From 5b1f6eda7b8696989ff4487171770677912c58fb Mon Sep 17 00:00:00 2001 From: AlenkaF Date: Thu, 21 Dec 2023 11:11:40 +0100 Subject: [PATCH 21/29] Add test for duplicate column selection in read_table and remove couple pandas marks --- python/pyarrow/tests/parquet/test_dataset.py | 17 +++++++++++++---- 1 file changed, 13 insertions(+), 4 deletions(-) diff --git a/python/pyarrow/tests/parquet/test_dataset.py b/python/pyarrow/tests/parquet/test_dataset.py index 56f592f7c1100..b48c07b9a9ef7 100644 --- a/python/pyarrow/tests/parquet/test_dataset.py +++ b/python/pyarrow/tests/parquet/test_dataset.py @@ -898,7 +898,6 @@ def test_ignore_no_private_directories_in_base_path(tempdir, dir_prefix): _assert_dataset_paths(dataset, paths) -@pytest.mark.pandas def test_ignore_custom_prefixes(tempdir): # ARROW-9573 - allow override of default ignore_prefixes part = ["xxx"] * 3 + ["yyy"] * 3 @@ -1145,7 +1144,6 @@ def is_pickleable(obj): assert is_pickleable(dataset) -@pytest.mark.pandas def test_partitioned_dataset(tempdir): # ARROW-3208: Segmentation fault when reading a Parquet partitioned dataset # to a Parquet file @@ -1162,7 +1160,6 @@ def test_partitioned_dataset(tempdir): pq.write_table(table, path / "output.parquet") -@pytest.mark.pandas def test_dataset_read_dictionary(tempdir): path = tempdir / "ARROW-3325-dataset" t1 = pa.table([[util.rands(10) for i in range(5)] * 10], names=['f0']) @@ -1186,7 +1183,6 @@ def test_dataset_read_dictionary(tempdir): assert c1.equals(ex_chunks[0]) -@pytest.mark.pandas def test_read_table_schema(tempdir): # test that schema keyword is passed through in read_table table = pa.table({'a': pa.array([1, 2, 3], pa.int32())}) @@ -1210,6 +1206,19 @@ def test_read_table_schema(tempdir): assert result.read().equals(expected) +def test_read_table_duplicate_column_selection(tempdir): + # test that duplicate column selection gives duplicate columns + table = pa.table({'a': pa.array([1, 2, 3], pa.int32()), + 'b': pa.array([1, 2, 3], pa.uint8())}) + pq.write_table(table, tempdir / "data.parquet") + + result = pq.read_table(tempdir / "data.parquet", columns=['a', 'a']) + expected_schema = pa.schema([('a', 'int32'), ('a', 'int32')]) + + assert result.column_names == ['a', 'a'] + assert result.schema == expected_schema + + def test_dataset_partitioning(tempdir): import pyarrow.dataset as ds From 27c9f78f59f46f34aff59f3925269d1d3a7806a0 Mon Sep 17 00:00:00 2001 From: AlenkaF Date: Thu, 21 Dec 2023 12:18:33 +0100 Subject: [PATCH 22/29] Use public atribute --- python/pyarrow/tests/parquet/test_dataset.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyarrow/tests/parquet/test_dataset.py b/python/pyarrow/tests/parquet/test_dataset.py index b48c07b9a9ef7..1825633d2d6fb 100644 --- a/python/pyarrow/tests/parquet/test_dataset.py +++ b/python/pyarrow/tests/parquet/test_dataset.py @@ -821,7 +821,7 @@ def _make_example_multifile_dataset(base_path, nfiles=10, file_nrows=5): def _assert_dataset_paths(dataset, paths): paths = [str(path.as_posix()) for path in paths] - assert set(paths) == set(dataset._dataset.files) + assert set(paths) == set(dataset.files) @pytest.mark.pandas From cbd91cda7c288360b67372bc8c634bcfcb1fd3f0 Mon Sep 17 00:00:00 2001 From: AlenkaF Date: Thu, 21 Dec 2023 12:19:41 +0100 Subject: [PATCH 23/29] Keep issue reference for test_empty_directory --- python/pyarrow/tests/parquet/test_dataset.py | 1 + 1 file changed, 1 insertion(+) diff --git a/python/pyarrow/tests/parquet/test_dataset.py b/python/pyarrow/tests/parquet/test_dataset.py index 1825633d2d6fb..c2879251a5a28 100644 --- a/python/pyarrow/tests/parquet/test_dataset.py +++ b/python/pyarrow/tests/parquet/test_dataset.py @@ -920,6 +920,7 @@ def test_ignore_custom_prefixes(tempdir): def test_empty_directory(tempdir): + # ARROW-5310 empty_dir = tempdir / 'dataset' empty_dir.mkdir() From 360d76257c4702d85b72e9c836f0449299a5c633 Mon Sep 17 00:00:00 2001 From: AlenkaF Date: Thu, 21 Dec 2023 12:20:46 +0100 Subject: [PATCH 24/29] Leave out text in docsting of ParquetDataset --- python/pyarrow/parquet/core.py | 3 --- 1 file changed, 3 deletions(-) diff --git a/python/pyarrow/parquet/core.py b/python/pyarrow/parquet/core.py index 08a831c611c50..df3c62bd44e4e 100644 --- a/python/pyarrow/parquet/core.py +++ b/python/pyarrow/parquet/core.py @@ -1189,9 +1189,6 @@ class ParquetDataset: Encapsulates details of reading a complete Parquet dataset possibly consisting of multiple files and partitions in subdirectories. -Keywords `metadata`, `split_row_groups=True` and `validate_schema=False` -are not yet supported. - Parameters ---------- path_or_paths : str or List[str] From 33bf4920ce21dbc4ed45bc7eefec97b62f4054c3 Mon Sep 17 00:00:00 2001 From: AlenkaF Date: Thu, 21 Dec 2023 12:22:09 +0100 Subject: [PATCH 25/29] Add back one pandas mark that should have been removed --- python/pyarrow/tests/parquet/test_dataset.py | 1 + 1 file changed, 1 insertion(+) diff --git a/python/pyarrow/tests/parquet/test_dataset.py b/python/pyarrow/tests/parquet/test_dataset.py index c2879251a5a28..b6e351bdef9a7 100644 --- a/python/pyarrow/tests/parquet/test_dataset.py +++ b/python/pyarrow/tests/parquet/test_dataset.py @@ -1145,6 +1145,7 @@ def is_pickleable(obj): assert is_pickleable(dataset) +@pytest.mark.pandas def test_partitioned_dataset(tempdir): # ARROW-3208: Segmentation fault when reading a Parquet partitioned dataset # to a Parquet file From 77b4ecbb574704d8b8f6ac2acc46fea7eb5d4e02 Mon Sep 17 00:00:00 2001 From: AlenkaF Date: Thu, 21 Dec 2023 16:11:48 +0100 Subject: [PATCH 26/29] Add ValueError for metadata in FileSystem.read_parquet --- python/pyarrow/filesystem.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/python/pyarrow/filesystem.py b/python/pyarrow/filesystem.py index c1e70a1ee699f..18dfedd74130d 100644 --- a/python/pyarrow/filesystem.py +++ b/python/pyarrow/filesystem.py @@ -223,8 +223,12 @@ def read_parquet(self, path, columns=None, metadata=None, schema=None, table : pyarrow.Table """ from pyarrow.parquet import ParquetDataset - dataset = ParquetDataset(path, schema=schema, metadata=metadata, - filesystem=self) + + if metadata is not None: + raise ValueError( + "Keyword 'metadata' is not supported with the Dataset API") + + dataset = ParquetDataset(path, schema=schema, filesystem=self) return dataset.read(columns=columns, use_threads=use_threads, use_pandas_metadata=use_pandas_metadata) From 4c892768a15e11f4dd01a26b80d1a3afe3a45723 Mon Sep 17 00:00:00 2001 From: AlenkaF Date: Thu, 21 Dec 2023 17:29:29 +0100 Subject: [PATCH 27/29] xfail test_read_multiple_parquet_files --- python/pyarrow/filesystem.py | 8 ++------ python/pyarrow/tests/test_hdfs.py | 2 ++ 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/python/pyarrow/filesystem.py b/python/pyarrow/filesystem.py index 18dfedd74130d..c1e70a1ee699f 100644 --- a/python/pyarrow/filesystem.py +++ b/python/pyarrow/filesystem.py @@ -223,12 +223,8 @@ def read_parquet(self, path, columns=None, metadata=None, schema=None, table : pyarrow.Table """ from pyarrow.parquet import ParquetDataset - - if metadata is not None: - raise ValueError( - "Keyword 'metadata' is not supported with the Dataset API") - - dataset = ParquetDataset(path, schema=schema, filesystem=self) + dataset = ParquetDataset(path, schema=schema, metadata=metadata, + filesystem=self) return dataset.read(columns=columns, use_threads=use_threads, use_pandas_metadata=use_pandas_metadata) diff --git a/python/pyarrow/tests/test_hdfs.py b/python/pyarrow/tests/test_hdfs.py index dce81f8890614..65a5163f5ec65 100644 --- a/python/pyarrow/tests/test_hdfs.py +++ b/python/pyarrow/tests/test_hdfs.py @@ -309,6 +309,8 @@ def _write_multiple_hdfs_pq_files(self, tmpdir): expected = pa.concat_tables(test_data) return expected + @pytest.mark.xfail(reason="legacy.FileSystem.read_parquet used legacy ParquetDataset " + "that has been removed in PyArrow 15.0.0.", raises=TypeError) @pytest.mark.pandas @pytest.mark.parquet def test_read_multiple_parquet_files(self): From 481a85c47ce45243ba972a3287b31675ac3ac76e Mon Sep 17 00:00:00 2001 From: AlenkaF Date: Thu, 21 Dec 2023 17:39:55 +0100 Subject: [PATCH 28/29] Add more xfails for _ensure_filesystem error --- python/pyarrow/tests/test_hdfs.py | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/python/pyarrow/tests/test_hdfs.py b/python/pyarrow/tests/test_hdfs.py index 65a5163f5ec65..58fed9019f8a5 100644 --- a/python/pyarrow/tests/test_hdfs.py +++ b/python/pyarrow/tests/test_hdfs.py @@ -311,6 +311,8 @@ def _write_multiple_hdfs_pq_files(self, tmpdir): @pytest.mark.xfail(reason="legacy.FileSystem.read_parquet used legacy ParquetDataset " "that has been removed in PyArrow 15.0.0.", raises=TypeError) + @pytest.mark.xfail(reason="legacy.FileSystem not supported with ParquetDataset due to " + "legacy path being removed in PyArrow 15.0.0.", raises=TypeError) @pytest.mark.pandas @pytest.mark.parquet def test_read_multiple_parquet_files(self): @@ -345,6 +347,8 @@ def test_read_multiple_parquet_files_with_uri(self): expected.to_pandas() ) + @pytest.mark.xfail(reason="legacy.FileSystem not supported with ParquetDataset due to " + "legacy path being removed in PyArrow 15.0.0.", raises=TypeError) @pytest.mark.pandas @pytest.mark.parquet def test_read_write_parquet_files_with_uri(self): @@ -366,6 +370,8 @@ def test_read_write_parquet_files_with_uri(self): assert_frame_equal(result, df) + @pytest.mark.xfail(reason="legacy.FileSystem not supported with ParquetDataset due to " + "legacy path being removed in PyArrow 15.0.0.", raises=TypeError) @pytest.mark.parquet @pytest.mark.pandas def test_write_to_dataset_with_partitions(self): @@ -374,6 +380,8 @@ def test_write_to_dataset_with_partitions(self): _test_write_to_dataset_with_partitions( tmpdir, filesystem=self.hdfs) + @pytest.mark.xfail(reason="legacy.FileSystem not supported with ParquetDataset due to " + "legacy path being removed in PyArrow 15.0.0.", raises=TypeError) @pytest.mark.parquet @pytest.mark.pandas def test_write_to_dataset_no_partitions(self): From a2e75a4178774ebf41c082677c23b97cc96fb925 Mon Sep 17 00:00:00 2001 From: AlenkaF Date: Thu, 21 Dec 2023 17:52:55 +0100 Subject: [PATCH 29/29] Linter fixed --- python/pyarrow/tests/test_hdfs.py | 22 ++++++++++++---------- 1 file changed, 12 insertions(+), 10 deletions(-) diff --git a/python/pyarrow/tests/test_hdfs.py b/python/pyarrow/tests/test_hdfs.py index 58fed9019f8a5..5b94c200f35de 100644 --- a/python/pyarrow/tests/test_hdfs.py +++ b/python/pyarrow/tests/test_hdfs.py @@ -309,10 +309,9 @@ def _write_multiple_hdfs_pq_files(self, tmpdir): expected = pa.concat_tables(test_data) return expected - @pytest.mark.xfail(reason="legacy.FileSystem.read_parquet used legacy ParquetDataset " - "that has been removed in PyArrow 15.0.0.", raises=TypeError) - @pytest.mark.xfail(reason="legacy.FileSystem not supported with ParquetDataset due to " - "legacy path being removed in PyArrow 15.0.0.", raises=TypeError) + @pytest.mark.xfail(reason="legacy.FileSystem not supported with ParquetDataset " + "due to legacy path being removed in PyArrow 15.0.0.", + raises=TypeError) @pytest.mark.pandas @pytest.mark.parquet def test_read_multiple_parquet_files(self): @@ -347,8 +346,9 @@ def test_read_multiple_parquet_files_with_uri(self): expected.to_pandas() ) - @pytest.mark.xfail(reason="legacy.FileSystem not supported with ParquetDataset due to " - "legacy path being removed in PyArrow 15.0.0.", raises=TypeError) + @pytest.mark.xfail(reason="legacy.FileSystem not supported with ParquetDataset " + "due to legacy path being removed in PyArrow 15.0.0.", + raises=TypeError) @pytest.mark.pandas @pytest.mark.parquet def test_read_write_parquet_files_with_uri(self): @@ -370,8 +370,9 @@ def test_read_write_parquet_files_with_uri(self): assert_frame_equal(result, df) - @pytest.mark.xfail(reason="legacy.FileSystem not supported with ParquetDataset due to " - "legacy path being removed in PyArrow 15.0.0.", raises=TypeError) + @pytest.mark.xfail(reason="legacy.FileSystem not supported with ParquetDataset " + "due to legacy path being removed in PyArrow 15.0.0.", + raises=TypeError) @pytest.mark.parquet @pytest.mark.pandas def test_write_to_dataset_with_partitions(self): @@ -380,8 +381,9 @@ def test_write_to_dataset_with_partitions(self): _test_write_to_dataset_with_partitions( tmpdir, filesystem=self.hdfs) - @pytest.mark.xfail(reason="legacy.FileSystem not supported with ParquetDataset due to " - "legacy path being removed in PyArrow 15.0.0.", raises=TypeError) + @pytest.mark.xfail(reason="legacy.FileSystem not supported with ParquetDataset " + "due to legacy path being removed in PyArrow 15.0.0.", + raises=TypeError) @pytest.mark.parquet @pytest.mark.pandas def test_write_to_dataset_no_partitions(self):