From 3e480a91833c7cd401fa120c520e5a51dad2d58a Mon Sep 17 00:00:00 2001 From: Joris Van den Bossche Date: Mon, 4 May 2020 14:11:26 +0200 Subject: [PATCH 1/2] ARROW-8644: [Python] Restore ParquetDataset behaviour to always include partition column for dask compatibility --- python/pyarrow/parquet.py | 2 -- python/pyarrow/tests/test_parquet.py | 8 +++++++- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/python/pyarrow/parquet.py b/python/pyarrow/parquet.py index b8f135b8e0e..376d0a1080e 100644 --- a/python/pyarrow/parquet.py +++ b/python/pyarrow/parquet.py @@ -722,8 +722,6 @@ def read(self, columns=None, use_threads=True, partitions=None, # value as indicated. The distinct categories of the partition have # been computed in the ParquetManifest for i, (name, index) in enumerate(self.partition_keys): - if columns is not None and name not in columns: - continue # The partition code is the same for all values in this piece indices = np.full(len(table), index, dtype='i4') diff --git a/python/pyarrow/tests/test_parquet.py b/python/pyarrow/tests/test_parquet.py index 8a81265598d..56fb81880fa 100644 --- a/python/pyarrow/tests/test_parquet.py +++ b/python/pyarrow/tests/test_parquet.py @@ -1710,7 +1710,13 @@ def test_read_partitioned_columns_selection(tempdir, use_legacy_dataset): dataset = pq.ParquetDataset( base_path, use_legacy_dataset=use_legacy_dataset) result = dataset.read(columns=["values"]) - assert result.column_names == ["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"] @pytest.mark.pandas From 6ceaf674c42585f8c241999a93b3d770609ccb25 Mon Sep 17 00:00:00 2001 From: Joris Van den Bossche Date: Mon, 4 May 2020 20:47:42 +0200 Subject: [PATCH 2/2] add note about different behaviour --- docs/source/python/parquet.rst | 6 ++++++ python/pyarrow/parquet.py | 4 +++- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/docs/source/python/parquet.rst b/docs/source/python/parquet.rst index 039ae965c7c..fb1a10b5b96 100644 --- a/docs/source/python/parquet.rst +++ b/docs/source/python/parquet.rst @@ -396,6 +396,12 @@ option was enabled on write). the partition keys. - General performance improvement and bug fixes. + It also has the following changes in behaviour: + + - 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 + In the future, this will be turned on by default. 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. diff --git a/python/pyarrow/parquet.py b/python/pyarrow/parquet.py index 376d0a1080e..62123d55327 100644 --- a/python/pyarrow/parquet.py +++ b/python/pyarrow/parquet.py @@ -1416,7 +1416,9 @@ def read(self, columns=None, use_threads=True, use_pandas_metadata=False): Parameters ---------- columns : List[str] - Names of columns to read from the dataset. + Names of columns to read from the dataset. The partition fields + are not automatically included (in contrast to when setting + ``use_legacy_dataset=True``). use_threads : bool, default True Perform multi-threaded column reads. use_pandas_metadata : bool, default False