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

uwe 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 88f6794  ARROW-2763: [Python] Make _metadata file accessible in 
ParquetDataset
88f6794 is described below

commit 88f6794c826259aa425d6232d84c05601ab5e348
Author: Robbie Gruener <robb...@uber.com>
AuthorDate: Sat Jun 30 17:39:17 2018 +0200

    ARROW-2763: [Python] Make _metadata file accessible in ParquetDataset
    
    Author: Robbie Gruener <robb...@uber.com>
    
    Closes #2195 from rgruener/metadata-file and squashes the following commits:
    
    0b9f51d3 <Robbie Gruener> ARROW-2763:  Make _metadata file accessible in 
ParquetDataset
---
 python/pyarrow/parquet.py            | 21 +++++++++++++++------
 python/pyarrow/tests/test_parquet.py | 35 +++++++++++++++++++++++++++++++++++
 2 files changed, 50 insertions(+), 6 deletions(-)

diff --git a/python/pyarrow/parquet.py b/python/pyarrow/parquet.py
index baa5fa4..f97c871 100644
--- a/python/pyarrow/parquet.py
+++ b/python/pyarrow/parquet.py
@@ -729,8 +729,10 @@ class ParquetDataset(object):
 
         self.paths = path_or_paths
 
-        (self.pieces, self.partitions,
-         self.common_metadata_path) = _make_manifest(path_or_paths, self.fs)
+        (self.pieces,
+         self.partitions,
+         self.common_metadata_path,
+         self.metadata_path) = _make_manifest(path_or_paths, self.fs)
 
         if self.common_metadata_path is not None:
             with self.fs.open(self.common_metadata_path) as f:
@@ -738,7 +740,12 @@ class ParquetDataset(object):
         else:
             self.common_metadata = None
 
-        self.metadata = metadata
+        if metadata is None and self.metadata_path is not None:
+            with self.fs.open(self.metadata_path) as f:
+                self.metadata = ParquetFile(f).metadata
+        else:
+            self.metadata = metadata
+
         self.schema = schema
 
         self.split_row_groups = split_row_groups
@@ -756,8 +763,8 @@ class ParquetDataset(object):
         open_file = self._get_open_file_func()
 
         if self.metadata is None and self.schema is None:
-            if self.common_metadata_path is not None:
-                self.schema = open_file(self.common_metadata_path).schema
+            if self.common_metadata is not None:
+                self.schema = self.common_metadata.schema
             else:
                 self.schema = self.pieces[0].get_metadata(open_file).schema
         elif self.schema is None:
@@ -935,6 +942,7 @@ def _ensure_filesystem(fs):
 def _make_manifest(path_or_paths, fs, pathsep='/'):
     partitions = None
     common_metadata_path = None
+    metadata_path = None
 
     if len(path_or_paths) == 1:
         # Dask passes a directory as a list of length 1
@@ -944,6 +952,7 @@ def _make_manifest(path_or_paths, fs, pathsep='/'):
         manifest = ParquetManifest(path_or_paths, filesystem=fs,
                                    pathsep=fs.pathsep)
         common_metadata_path = manifest.common_metadata_path
+        metadata_path = manifest.metadata_path
         pieces = manifest.pieces
         partitions = manifest.partitions
     else:
@@ -962,7 +971,7 @@ def _make_manifest(path_or_paths, fs, pathsep='/'):
             piece = ParquetDatasetPiece(path)
             pieces.append(piece)
 
-    return pieces, partitions, common_metadata_path
+    return pieces, partitions, common_metadata_path, metadata_path
 
 
 _read_table_docstring = """
diff --git a/python/pyarrow/tests/test_parquet.py 
b/python/pyarrow/tests/test_parquet.py
index 5b89e9a..fec65b9 100644
--- a/python/pyarrow/tests/test_parquet.py
+++ b/python/pyarrow/tests/test_parquet.py
@@ -1403,6 +1403,41 @@ def test_read_common_metadata_files(tmpdir):
     _test_read_common_metadata_files(fs, base_path)
 
 
+def _test_read_metadata_files(fs, base_path):
+    import pyarrow.parquet as pq
+
+    N = 100
+    df = pd.DataFrame({
+        'index': np.arange(N),
+        'values': np.random.randn(N)
+    }, columns=['index', 'values'])
+
+    data_path = pjoin(base_path, 'data.parquet')
+
+    table = pa.Table.from_pandas(df)
+
+    with fs.open(data_path, 'wb') as f:
+        _write_table(table, f)
+
+    metadata_path = pjoin(base_path, '_metadata')
+    with fs.open(metadata_path, 'wb') as f:
+        pq.write_metadata(table.schema, f)
+
+    dataset = pq.ParquetDataset(base_path, filesystem=fs)
+    assert dataset.metadata_path == metadata_path
+
+    with fs.open(data_path) as f:
+        metadata_schema = pq.read_metadata(f).schema
+    assert dataset.schema.equals(metadata_schema)
+
+
+@parquet
+def test_read_metadata_files(tmpdir):
+    base_path = str(tmpdir)
+    fs = LocalFileSystem.get_instance()
+    _test_read_metadata_files(fs, base_path)
+
+
 @parquet
 def test_read_schema(tmpdir):
     import pyarrow.parquet as pq

Reply via email to