This is an automated email from the ASF dual-hosted git repository.
bkietz 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 8a1c2fa ARROW-9493: [Python] Enable dictionary encoding in read_table
with datasets API
8a1c2fa is described below
commit 8a1c2fa6b97d67b58c3e2913cbc135b1128dc35b
Author: Joris Van den Bossche <[email protected]>
AuthorDate: Wed Jul 15 16:02:25 2020 -0400
ARROW-9493: [Python] Enable dictionary encoding in read_table with datasets
API
Closes #7777 from jorisvandenbossche/ARROW-9139-follow-up
Authored-by: Joris Van den Bossche <[email protected]>
Signed-off-by: Benjamin Kietzman <[email protected]>
---
python/pyarrow/parquet.py | 6 ++++++
python/pyarrow/tests/test_parquet.py | 16 +++++++---------
2 files changed, 13 insertions(+), 9 deletions(-)
diff --git a/python/pyarrow/parquet.py b/python/pyarrow/parquet.py
index 60e6680..59c79ac 100644
--- a/python/pyarrow/parquet.py
+++ b/python/pyarrow/parquet.py
@@ -1413,6 +1413,12 @@ class _ParquetDatasetV2:
)
return
+ # check partitioning to enable dictionary encoding
+ if partitioning == "hive":
+ partitioning = ds.HivePartitioning.discover(
+ max_partition_dictionary_size=-1
+ )
+
# map old filesystems to new one
if filesystem is not None:
filesystem = pyarrow.fs._ensure_filesystem(
diff --git a/python/pyarrow/tests/test_parquet.py
b/python/pyarrow/tests/test_parquet.py
index d213583..a24b1b3 100644
--- a/python/pyarrow/tests/test_parquet.py
+++ b/python/pyarrow/tests/test_parquet.py
@@ -2217,11 +2217,11 @@ def _partition_test_for_filesystem(fs, base_path,
use_legacy_dataset=True):
expected_df = (df.sort_values(by='index')
.reset_index(drop=True)
.reindex(columns=result_df.columns))
+
if use_legacy_dataset:
- # TODO(dataset) Dataset API does not create categorical columns
- # for partition keys
+ # integer partition field not dictionary encoded with new API
expected_df['foo'] = pd.Categorical(df['foo'], categories=foo_keys)
- expected_df['bar'] = pd.Categorical(df['bar'], categories=bar_keys)
+ expected_df['bar'] = pd.Categorical(df['bar'], categories=bar_keys)
assert (result_df.columns == ['index', 'values', 'foo', 'bar']).all()
@@ -2834,10 +2834,9 @@ def _test_write_to_dataset_with_partitions(base_path,
assert partition_by == input_df_cols[-1 * len(partition_by):]
input_df = input_df[cols]
- if use_legacy_dataset:
- # Partitioned columns become 'categorical' dtypes
- for col in partition_by:
- output_df[col] = output_df[col].astype('category')
+ # Partitioned columns become 'categorical' dtypes
+ for col in partition_by:
+ output_df[col] = output_df[col].astype('category')
tm.assert_frame_equal(output_df, input_df)
@@ -2985,8 +2984,7 @@ def test_write_to_dataset_pandas_preserve_index(tempdir,
use_legacy_dataset):
df.index = pd.Index(['a', 'b', 'c'], name="idx")
table = pa.table(df)
df_cat = df[["col", "part"]].copy()
- if use_legacy_dataset:
- df_cat["part"] = df_cat["part"].astype("category")
+ df_cat["part"] = df_cat["part"].astype("category")
pq.write_to_dataset(table, str(tempdir / "case1"), partition_cols=['part'])
result = pq.read_table(