Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
3 changes: 3 additions & 0 deletions python/doc/source/api.rst
Original file line number Diff line number Diff line change
Expand Up @@ -239,5 +239,8 @@ Apache Parquet
ParquetDataset
ParquetFile
read_table
read_metadata
read_pandas
read_schema
write_metadata
write_table
30 changes: 30 additions & 0 deletions python/pyarrow/parquet.py
Original file line number Diff line number Diff line change
Expand Up @@ -792,3 +792,33 @@ def write_metadata(schema, where, version='1.0',
)
writer = ParquetWriter(where, schema, **options)
writer.close()


def read_metadata(where):
"""
Read FileMetadata from footer of a single Parquet file

Parameters
----------
where : string (filepath) or file-like object

Returns
-------
metadata : FileMetadata
"""
return ParquetFile(where).metadata


def read_schema(where):
"""
Read effective Arrow schema from Parquet file metadata

Parameters
----------
where : string (filepath) or file-like object

Returns
-------
schema : pyarrow.Schema
"""
return ParquetFile(where).schema.to_arrow_schema()
31 changes: 24 additions & 7 deletions python/pyarrow/tests/test_parquet.py
Original file line number Diff line number Diff line change
Expand Up @@ -124,9 +124,8 @@ def test_pandas_parquet_custom_metadata(tmpdir):
assert b'pandas' in arrow_table.schema.metadata

_write_table(arrow_table, filename.strpath, version="2.0")
pf = pq.ParquetFile(filename.strpath)

md = pf.metadata.metadata
md = pq.read_metadata(filename.strpath).metadata
assert b'pandas' in md

js = json.loads(md[b'pandas'].decode('utf8'))
Expand Down Expand Up @@ -580,7 +579,7 @@ def test_pass_separate_metadata():
_write_table(a_table, buf, compression='snappy', version='2.0')

buf.seek(0)
metadata = pq.ParquetFile(buf).metadata
metadata = pq.read_metadata(buf)

buf.seek(0)

Expand Down Expand Up @@ -776,14 +775,32 @@ def test_read_common_metadata_files(tmpdir):
dataset = pq.ParquetDataset(base_path)
assert dataset.metadata_path == metadata_path

pf = pq.ParquetFile(data_path)
assert dataset.schema.equals(pf.schema)
common_schema = pq.read_metadata(data_path).schema
assert dataset.schema.equals(common_schema)

# handle list of one directory
dataset2 = pq.ParquetDataset([base_path])
assert dataset2.schema.equals(dataset.schema)


@parquet
def test_read_schema(tmpdir):
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(str(tmpdir), 'test.parquet')

table = pa.Table.from_pandas(df)
_write_table(table, data_path)

assert table.schema.equals(pq.read_schema(data_path))


def _filter_partition(df, part_keys):
predicate = np.ones(len(df), dtype=bool)

Expand Down Expand Up @@ -835,7 +852,7 @@ def read_multiple_files(paths, columns=None, nthreads=None, **kwargs):
assert result.equals(expected)

# Read with provided metadata
metadata = pq.ParquetFile(paths[0]).metadata
metadata = pq.read_metadata(paths[0])

result2 = read_multiple_files(paths, metadata=metadata)
assert result2.equals(expected)
Expand All @@ -861,7 +878,7 @@ def read_multiple_files(paths, columns=None, nthreads=None, **kwargs):
t = pa.Table.from_pandas(bad_apple)
_write_table(t, bad_apple_path)

bad_meta = pq.ParquetFile(bad_apple_path).metadata
bad_meta = pq.read_metadata(bad_apple_path)

with pytest.raises(ValueError):
read_multiple_files(paths + [bad_apple_path])
Expand Down