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

apitrou 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 797c88a9a0 ARROW-12203: [C++][Python] Switch default Parquet version 
to 2.4 (#13280)
797c88a9a0 is described below

commit 797c88a9a0ec73fa4d24554c3af83b841c205681
Author: Raúl Cumplido <[email protected]>
AuthorDate: Wed Jun 1 19:47:45 2022 +0200

    ARROW-12203: [C++][Python] Switch default Parquet version to 2.4 (#13280)
    
    Change the default parquet version to 2.4
    
    Authored-by: Raúl Cumplido <[email protected]>
    Signed-off-by: Antoine Pitrou <[email protected]>
---
 cpp/src/parquet/arrow/arrow_schema_test.cc  | 3 +--
 cpp/src/parquet/properties.h                | 4 ++--
 cpp/src/parquet/properties_test.cc          | 2 +-
 python/pyarrow/_parquet.pyx                 | 6 +++---
 python/pyarrow/parquet/__init__.py          | 8 ++++----
 python/pyarrow/tests/parquet/test_pandas.py | 2 +-
 6 files changed, 12 insertions(+), 13 deletions(-)

diff --git a/cpp/src/parquet/arrow/arrow_schema_test.cc 
b/cpp/src/parquet/arrow/arrow_schema_test.cc
index 6275696098..703b234044 100644
--- a/cpp/src/parquet/arrow/arrow_schema_test.cc
+++ b/cpp/src/parquet/arrow/arrow_schema_test.cc
@@ -836,8 +836,7 @@ TEST_F(TestConvertArrowSchema, ArrowFields) {
       {"int8", ::arrow::int8(), LogicalType::Int(8, true), ParquetType::INT32, 
-1},
       {"uint16", ::arrow::uint16(), LogicalType::Int(16, false), 
ParquetType::INT32, -1},
       {"int16", ::arrow::int16(), LogicalType::Int(16, true), 
ParquetType::INT32, -1},
-      {"uint32", ::arrow::uint32(), LogicalType::None(), ParquetType::INT64,
-       -1},  // Parquet 1.0
+      {"uint32", ::arrow::uint32(), LogicalType::Int(32, false), 
ParquetType::INT32, -1},
       {"int32", ::arrow::int32(), LogicalType::None(), ParquetType::INT32, -1},
       {"uint64", ::arrow::uint64(), LogicalType::Int(64, false), 
ParquetType::INT64, -1},
       {"int64", ::arrow::int64(), LogicalType::None(), ParquetType::INT64, -1},
diff --git a/cpp/src/parquet/properties.h b/cpp/src/parquet/properties.h
index 6117bdb6fd..5c81c75357 100644
--- a/cpp/src/parquet/properties.h
+++ b/cpp/src/parquet/properties.h
@@ -166,7 +166,7 @@ class PARQUET_EXPORT WriterProperties {
           write_batch_size_(DEFAULT_WRITE_BATCH_SIZE),
           max_row_group_length_(DEFAULT_MAX_ROW_GROUP_LENGTH),
           pagesize_(kDefaultDataPageSize),
-          version_(ParquetVersion::PARQUET_1_0),
+          version_(ParquetVersion::PARQUET_2_4),
           data_page_version_(ParquetDataPageVersion::V1),
           created_by_(DEFAULT_CREATED_BY) {}
     virtual ~Builder() {}
@@ -246,7 +246,7 @@ class PARQUET_EXPORT WriterProperties {
     }
 
     /// Specify the Parquet file version.
-    /// Default PARQUET_1_0.
+    /// Default PARQUET_2_4.
     Builder* version(ParquetVersion::type version) {
       version_ = version;
       return this;
diff --git a/cpp/src/parquet/properties_test.cc 
b/cpp/src/parquet/properties_test.cc
index 7ce96e4a7d..3d75569d30 100644
--- a/cpp/src/parquet/properties_test.cc
+++ b/cpp/src/parquet/properties_test.cc
@@ -43,7 +43,7 @@ TEST(TestWriterProperties, Basics) {
 
   ASSERT_EQ(kDefaultDataPageSize, props->data_pagesize());
   ASSERT_EQ(DEFAULT_DICTIONARY_PAGE_SIZE_LIMIT, 
props->dictionary_pagesize_limit());
-  ASSERT_EQ(ParquetVersion::PARQUET_1_0, props->version());
+  ASSERT_EQ(ParquetVersion::PARQUET_2_4, props->version());
   ASSERT_EQ(ParquetDataPageVersion::V1, props->data_page_version());
 }
 
diff --git a/python/pyarrow/_parquet.pyx b/python/pyarrow/_parquet.pyx
index c634ea9669..8812ab1059 100644
--- a/python/pyarrow/_parquet.pyx
+++ b/python/pyarrow/_parquet.pyx
@@ -714,7 +714,7 @@ cdef class FileMetaData(_Weakrefable):
         """
         Parquet format version used in file (str, such as '1.0', '2.4').
 
-        If version is missing or unparsable, will default to assuming '1.0'.
+        If version is missing or unparsable, will default to assuming '2.4'.
         """
         cdef ParquetVersion version = self._metadata.version()
         if version == ParquetVersion_V1:
@@ -726,9 +726,9 @@ cdef class FileMetaData(_Weakrefable):
         elif version == ParquetVersion_V2_6:
             return '2.6'
         else:
-            warnings.warn('Unrecognized file version, assuming 1.0: {}'
+            warnings.warn('Unrecognized file version, assuming 2.4: {}'
                           .format(version))
-            return '1.0'
+            return '2.4'
 
     @property
     def created_by(self):
diff --git a/python/pyarrow/parquet/__init__.py 
b/python/pyarrow/parquet/__init__.py
index b4713a717c..7d1c8a4308 100644
--- a/python/pyarrow/parquet/__init__.py
+++ b/python/pyarrow/parquet/__init__.py
@@ -674,7 +674,7 @@ def _sanitize_table(table, new_schema, flavor):
         return table
 
 
-_parquet_writer_arg_docs = """version : {"1.0", "2.4", "2.6"}, default "1.0"
+_parquet_writer_arg_docs = """version : {"1.0", "2.4", "2.6"}, default "2.4"
     Determine which Parquet logical types are available for use, whether the
     reduced set from the Parquet 1.x.x format or the expanded logical types
     added in later format versions.
@@ -862,7 +862,7 @@ Examples
 
     def __init__(self, where, schema, filesystem=None,
                  flavor=None,
-                 version='1.0',
+                 version='2.4',
                  use_dictionary=True,
                  compression='snappy',
                  write_statistics=True,
@@ -2854,7 +2854,7 @@ read_pandas.__doc__ = _read_table_docstring.format(
     _DNF_filter_doc, "")
 
 
-def write_table(table, where, row_group_size=None, version='1.0',
+def write_table(table, where, row_group_size=None, version='2.4',
                 use_dictionary=True, compression='snappy',
                 write_statistics=True,
                 use_deprecated_int96_timestamps=None,
@@ -3336,7 +3336,7 @@ def read_metadata(where, memory_map=False, 
decryption_properties=None):
       num_columns: 2
       num_rows: 3
       num_row_groups: 1
-      format_version: 1.0
+      format_version: 2.6
       serialized_size: 561
     """
     return ParquetFile(where, memory_map=memory_map,
diff --git a/python/pyarrow/tests/parquet/test_pandas.py 
b/python/pyarrow/tests/parquet/test_pandas.py
index 2c37ac486e..0d0658cc38 100644
--- a/python/pyarrow/tests/parquet/test_pandas.py
+++ b/python/pyarrow/tests/parquet/test_pandas.py
@@ -256,7 +256,7 @@ def test_pandas_parquet_pyfile_roundtrip(tempdir, 
use_legacy_dataset):
     arrow_table = pa.Table.from_pandas(df)
 
     with filename.open('wb') as f:
-        _write_table(arrow_table, f, version="1.0")
+        _write_table(arrow_table, f, version="2.4")
 
     data = io.BytesIO(filename.read_bytes())
 

Reply via email to