You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by fs...@apache.org on 2020/04/29 02:47:17 UTC
[arrow] branch master updated: ARROW-3861: [Python]
ParquetDataset.read() respect specified columns and not include partition
columns
This is an automated email from the ASF dual-hosted git repository.
fsaintjacques 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 283e188 ARROW-3861: [Python] ParquetDataset.read() respect specified columns and not include partition columns
283e188 is described below
commit 283e188b151435b1cc18228c48fad9e6e98fcb7e
Author: Joris Van den Bossche <jo...@gmail.com>
AuthorDate: Tue Apr 28 22:46:58 2020 -0400
ARROW-3861: [Python] ParquetDataset.read() respect specified columns and not include partition columns
This is adding a test for it (using both legacy and new dataset), and also small fix for the legacy path.
Closes #7050 from jorisvandenbossche/ARROW-3861-parquet-dataset-read-columns
Authored-by: Joris Van den Bossche <jo...@gmail.com>
Signed-off-by: François Saint-Jacques <fs...@gmail.com>
---
python/pyarrow/parquet.py | 2 ++
python/pyarrow/tests/test_parquet.py | 15 +++++++++++++++
2 files changed, 17 insertions(+)
diff --git a/python/pyarrow/parquet.py b/python/pyarrow/parquet.py
index 376d0a1..b8f135b 100644
--- a/python/pyarrow/parquet.py
+++ b/python/pyarrow/parquet.py
@@ -722,6 +722,8 @@ 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')
diff --git a/python/pyarrow/tests/test_parquet.py b/python/pyarrow/tests/test_parquet.py
index 064f6f7..8a81265 100644
--- a/python/pyarrow/tests/test_parquet.py
+++ b/python/pyarrow/tests/test_parquet.py
@@ -1700,6 +1700,21 @@ def test_create_parquet_dataset_multi_threaded(tempdir):
@pytest.mark.pandas
@parametrize_legacy_dataset
+def test_read_partitioned_columns_selection(tempdir, use_legacy_dataset):
+ # 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)
+ result = dataset.read(columns=["values"])
+ assert result.column_names == ["values"]
+
+
+@pytest.mark.pandas
+@parametrize_legacy_dataset
def test_equivalency(tempdir, use_legacy_dataset):
fs = LocalFileSystem.get_instance()
base_path = tempdir