Skip to content

Commit 57b81ca

Browse files
authored
ARROW-18225: [Python] Fully support filesystem in parquet.write_metadata (apache#14574)
Will fix [ARROW-18225](https://issues.apache.org/jira/browse/ARROW-18225) Authored-by: Miles Granger <miles59923@gmail.com> Signed-off-by: Joris Van den Bossche <jorisvandenbossche@gmail.com>
1 parent cf66f48 commit 57b81ca

2 files changed

Lines changed: 53 additions & 4 deletions

File tree

python/pyarrow/parquet/core.py

Lines changed: 20 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -3379,7 +3379,8 @@ def file_visitor(written_file):
33793379
metadata_collector[-1].set_file_path(outfile)
33803380

33813381

3382-
def write_metadata(schema, where, metadata_collector=None, **kwargs):
3382+
def write_metadata(schema, where, metadata_collector=None, filesystem=None,
3383+
**kwargs):
33833384
"""
33843385
Write metadata-only Parquet file from schema. This can be used with
33853386
`write_to_dataset` to generate `_common_metadata` and `_metadata` sidecar
@@ -3391,6 +3392,9 @@ def write_metadata(schema, where, metadata_collector=None, **kwargs):
33913392
where : string or pyarrow.NativeFile
33923393
metadata_collector : list
33933394
where to collect metadata information.
3395+
filesystem : FileSystem, default None
3396+
If nothing passed, will be inferred from `where` if path-like, else
3397+
`where` is already a file-like object so no filesystem is needed.
33943398
**kwargs : dict,
33953399
Additional kwargs for ParquetWriter class. See docstring for
33963400
`ParquetWriter` for more information.
@@ -3423,16 +3427,28 @@ def write_metadata(schema, where, metadata_collector=None, **kwargs):
34233427
... table.schema, 'dataset_metadata/_metadata',
34243428
... metadata_collector=metadata_collector)
34253429
"""
3426-
writer = ParquetWriter(where, schema, **kwargs)
3430+
filesystem, where = _resolve_filesystem_and_path(where, filesystem)
3431+
3432+
if hasattr(where, "seek"): # file-like
3433+
cursor_position = where.tell()
3434+
3435+
writer = ParquetWriter(where, schema, filesystem, **kwargs)
34273436
writer.close()
34283437

34293438
if metadata_collector is not None:
34303439
# ParquetWriter doesn't expose the metadata until it's written. Write
34313440
# it and read it again.
3432-
metadata = read_metadata(where)
3441+
metadata = read_metadata(where, filesystem=filesystem)
3442+
if hasattr(where, "seek"):
3443+
where.seek(cursor_position) # file-like, set cursor back.
3444+
34333445
for m in metadata_collector:
34343446
metadata.append_row_groups(m)
3435-
metadata.write_metadata_file(where)
3447+
if filesystem is not None:
3448+
with filesystem.open_output_stream(where) as f:
3449+
metadata.write_metadata_file(f)
3450+
else:
3451+
metadata.write_metadata_file(where)
34363452

34373453

34383454
def read_metadata(where, memory_map=False, decryption_properties=None,

python/pyarrow/tests/parquet/test_metadata.py

Lines changed: 33 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -613,3 +613,36 @@ def test_metadata_append_row_groups_diff(t1, t2, expected_error):
613613
meta1.append_row_groups(meta2)
614614
else:
615615
meta1.append_row_groups(meta2)
616+
617+
618+
@pytest.mark.s3
619+
def test_write_metadata_fs_file_combinations(tempdir, s3_example_s3fs):
620+
s3_fs, s3_path = s3_example_s3fs
621+
622+
meta1 = tempdir / "meta1"
623+
meta2 = tempdir / "meta2"
624+
meta3 = tempdir / "meta3"
625+
meta4 = tempdir / "meta4"
626+
meta5 = f"{s3_path}/meta5"
627+
628+
table = pa.table({"col": range(5)})
629+
630+
# plain local path
631+
pq.write_metadata(table.schema, meta1, [])
632+
633+
# Used the localfilesystem to resolve opening an output stream
634+
pq.write_metadata(table.schema, meta2, [], filesystem=LocalFileSystem())
635+
636+
# Can resolve local file URI
637+
pq.write_metadata(table.schema, meta3.as_uri(), [])
638+
639+
# Take a file-like obj all the way thru?
640+
with meta4.open('wb+') as meta4_stream:
641+
pq.write_metadata(table.schema, meta4_stream, [])
642+
643+
# S3FileSystem
644+
pq.write_metadata(table.schema, meta5, [], filesystem=s3_fs)
645+
646+
assert meta1.read_bytes() == meta2.read_bytes() \
647+
== meta3.read_bytes() == meta4.read_bytes() \
648+
== s3_fs.open(meta5).read()

0 commit comments

Comments
 (0)