You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by we...@apache.org on 2020/05/05 21:43:33 UTC

[arrow] branch master updated: ARROW-8644: [Python] Restore ParquetDataset behaviour to always include partition column for dask compatibility

This is an automated email from the ASF dual-hosted git repository.

wesm pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/arrow.git


The following commit(s) were added to refs/heads/master by this push:
     new fb4d57a  ARROW-8644: [Python] Restore ParquetDataset behaviour to always include partition column for dask compatibility
fb4d57a is described below

commit fb4d57a799932e08ec3742c2beb204be50014b79
Author: Joris Van den Bossche <jo...@gmail.com>
AuthorDate: Tue May 5 16:43:15 2020 -0500

    ARROW-8644: [Python] Restore ParquetDataset behaviour to always include partition column for dask compatibility
    
    Given that the original change (https://issues.apache.org/jira/browse/ARROW-3861 / https://github.com/apache/arrow/pull/7050) breaks dask's reading of partitioned datasets (it doesn't add the partition column to the list of columns to read, but expects it will still be read automatically), it doesn't seem worth it to me to fix this in the "old" ParquetDataset implementation.
    
    But we can keep the "correct" behaviour in the Datasets API - based implementation going forward.
    
    Closes #7096 from jorisvandenbossche/ARROW-8644-dask-partitioned
    
    Authored-by: Joris Van den Bossche <jo...@gmail.com>
    Signed-off-by: Wes McKinney <we...@apache.org>
---
 docs/source/python/parquet.rst       | 6 ++++++
 python/pyarrow/parquet.py            | 6 +++---
 python/pyarrow/tests/test_parquet.py | 8 +++++++-
 3 files changed, 16 insertions(+), 4 deletions(-)

diff --git a/docs/source/python/parquet.rst b/docs/source/python/parquet.rst
index 039ae96..fb1a10b 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 b8f135b..62123d5 100644
--- a/python/pyarrow/parquet.py
+++ b/python/pyarrow/parquet.py
@@ -722,8 +722,6 @@ class ParquetDatasetPiece:
             # 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')
 
@@ -1418,7 +1416,9 @@ class _ParquetDatasetV2:
         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
diff --git a/python/pyarrow/tests/test_parquet.py b/python/pyarrow/tests/test_parquet.py
index 31de9fe..1697526 100644
--- a/python/pyarrow/tests/test_parquet.py
+++ b/python/pyarrow/tests/test_parquet.py
@@ -1705,7 +1705,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