Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

ARROW-16719: [Python] Add path/URI + filesystem handling to parquet.read_metadata #13629

Merged

Conversation

kshitij12345
Copy link
Contributor

Add filesystem support to pq.read_metadata and pq.read_schema.

@github-actions
Copy link

@github-actions
Copy link

⚠️ Ticket has not been started in JIRA, please click 'Start Progress'.

pq.write_table(table, file_path)

# Get expected `metadata` from path.
metadata = pq.read_metadata(tmpdir / '/data.parquet')
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is there a way to get metadata directly from the table?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Try table.schema.metadata.

Copy link
Contributor Author

@kshitij12345 kshitij12345 Jul 17, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I tried that but it leads to segfault. Probably worth an issue? (I don't think the program should crash)

Crash Log
python/pyarrow/tests/parquet/test_metadata.py .........................sFatal Python error: Segmentation fault

Current thread 0x00007f066e2be740 (most recent call first):
  File "/home/kshiteej/Apache/arrow/python/pyarrow/tests/parquet/test_metadata.py", line 549 in test_metadata_schema_filesystem
  File "/home/kshiteej/.conda/envs/pyarrow-dev/lib/python3.9/site-packages/_pytest/python.py", line 192 in pytest_pyfunc_call
  File "/home/kshiteej/.conda/envs/pyarrow-dev/lib/python3.9/site-packages/pluggy/_callers.py", line 39 in _multicall
  File "/home/kshiteej/.conda/envs/pyarrow-dev/lib/python3.9/site-packages/pluggy/_manager.py", line 80 in _hookexec
  File "/home/kshiteej/.conda/envs/pyarrow-dev/lib/python3.9/site-packages/pluggy/_hooks.py", line 265 in __call__
  File "/home/kshiteej/.conda/envs/pyarrow-dev/lib/python3.9/site-packages/_pytest/python.py", line 1761 in runtest
  File "/home/kshiteej/.conda/envs/pyarrow-dev/lib/python3.9/site-packages/_pytest/runner.py", line 166 in pytest_runtest_call
  File "/home/kshiteej/.conda/envs/pyarrow-dev/lib/python3.9/site-packages/pluggy/_callers.py", line 39 in _multicall
  File "/home/kshiteej/.conda/envs/pyarrow-dev/lib/python3.9/site-packages/pluggy/_manager.py", line 80 in _hookexec
  File "/home/kshiteej/.conda/envs/pyarrow-dev/lib/python3.9/site-packages/pluggy/_hooks.py", line 265 in __call__
  File "/home/kshiteej/.conda/envs/pyarrow-dev/lib/python3.9/site-packages/_pytest/runner.py", line 259 in <lambda>
  File "/home/kshiteej/.conda/envs/pyarrow-dev/lib/python3.9/site-packages/_pytest/runner.py", line 338 in from_call
  File "/home/kshiteej/.conda/envs/pyarrow-dev/lib/python3.9/site-packages/_pytest/runner.py", line 258 in call_runtest_hook
  File "/home/kshiteej/.conda/envs/pyarrow-dev/lib/python3.9/site-packages/_pytest/runner.py", line 219 in call_and_report
  File "/home/kshiteej/.conda/envs/pyarrow-dev/lib/python3.9/site-packages/_pytest/runner.py", line 130 in runtestprotocol
  File "/home/kshiteej/.conda/envs/pyarrow-dev/lib/python3.9/site-packages/_pytest/runner.py", line 111 in pytest_runtest_protocol
  File "/home/kshiteej/.conda/envs/pyarrow-dev/lib/python3.9/site-packages/pluggy/_callers.py", line 39 in _multicall
  File "/home/kshiteej/.conda/envs/pyarrow-dev/lib/python3.9/site-packages/pluggy/_manager.py", line 80 in _hookexec
  File "/home/kshiteej/.conda/envs/pyarrow-dev/lib/python3.9/site-packages/pluggy/_hooks.py", line 265 in __call__
  File "/home/kshiteej/.conda/envs/pyarrow-dev/lib/python3.9/site-packages/_pytest/main.py", line 347 in pytest_runtestloop
  File "/home/kshiteej/.conda/envs/pyarrow-dev/lib/python3.9/site-packages/pluggy/_callers.py", line 39 in _multicall
  File "/home/kshiteej/.conda/envs/pyarrow-dev/lib/python3.9/site-packages/pluggy/_manager.py", line 80 in _hookexec
  File "/home/kshiteej/.conda/envs/pyarrow-dev/lib/python3.9/site-packages/pluggy/_hooks.py", line 265 in __call__
  File "/home/kshiteej/.conda/envs/pyarrow-dev/lib/python3.9/site-packages/_pytest/main.py", line 322 in _main
  File "/home/kshiteej/.conda/envs/pyarrow-dev/lib/python3.9/site-packages/_pytest/main.py", line 268 in wrap_session
  File "/home/kshiteej/.conda/envs/pyarrow-dev/lib/python3.9/site-packages/_pytest/main.py", line 315 in pytest_cmdline_main
  File "/home/kshiteej/.conda/envs/pyarrow-dev/lib/python3.9/site-packages/pluggy/_callers.py", line 39 in _multicall
  File "/home/kshiteej/.conda/envs/pyarrow-dev/lib/python3.9/site-packages/pluggy/_manager.py", line 80 in _hookexec
  File "/home/kshiteej/.conda/envs/pyarrow-dev/lib/python3.9/site-packages/pluggy/_hooks.py", line 265 in __call__
  File "/home/kshiteej/.conda/envs/pyarrow-dev/lib/python3.9/site-packages/_pytest/config/__init__.py", line 164 in main
  File "/home/kshiteej/.conda/envs/pyarrow-dev/lib/python3.9/site-packages/_pytest/config/__init__.py", line 187 in console_main
  File "/home/kshiteej/.conda/envs/pyarrow-dev/bin/pytest", line 11 in <module>
Segmentation fault (core dumped)

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Running this locally I can confirm a segfault. I think it happens because the table metadata is (correctly) empty:

>>> import pyarrow as pa
>>> import pyarrow.parquet as pq

>>> table = pa.table({"a": [1, 2, 3]})
>>> file_path = "/tmp/data.parquet"
>>> metadata = table.schema.metadata

>>> pq.read_metadata(file_path).equals(metadata)
zsh: segmentation fault  python

Which would deserve an issue (a warning should be returned without a crash).

Maybe a better option to test ParquetFile metadata would be to inspect individual attributes:

>>> pq.read_metadata(file_path).num_columns == 1
True
>>> pq.read_metadata(file_path).num_rows == 3
True

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

num_columns: 1
num_rows: 3
num_row_groups: 1
format_version: 2.6
serialized_size: 375

From the following metadata attributes, I think num_columns, num_rows, num_row_groups make sense. Should we also check for the other two (I am not sure if they will stay same across versions and platforms)?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ah, they sneaked in 😅. Removed :)

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is there already a JIRA opened for this segfault?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think so. I will open one.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for opening the JIRA

@kshitij12345 kshitij12345 marked this pull request as ready for review July 17, 2022 08:38
@kshitij12345
Copy link
Contributor Author

macOS CI Failure looks unrelated.

@AlenkaF
Copy link
Member

AlenkaF commented Jul 18, 2022

Thank you for working on this issue @kshitij12345! LGTM +1

@jorisvandenbossche can you please have a look before we merge this PR?

Copy link
Member

@jorisvandenbossche jorisvandenbossche left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks good, thanks!
My only comment is about using a context manager to ensure we close the file again

@@ -3411,11 +3416,15 @@ def read_metadata(where, memory_map=False, decryption_properties=None):
format_version: 2.6
serialized_size: 561
"""
filesystem, where = _resolve_filesystem_and_path(where, filesystem)
if filesystem is not None:
where = filesystem.open_input_file(where)
return ParquetFile(where, memory_map=memory_map,
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

When opening the file with open_input_file, we should probably use it in a context manager to ensure we also close the file handle again:

with filesystem.open_input_file(where) as source:
    return ParquetFile(source, ...).metadata

(and the same for read_schema)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am curious as to what is the minimum supported Python version.

I was planning to do something like which requires Python 3.7 or more.

source = filesystem.open_input_file(where) if filesystem is not None else nullcontext()
with source:
    return ParquetFile(
        source, memory_map=memory_map,
        decryption_properties=decryption_properties).schema.to_arrow_schema()

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Python 3.7 is our minimum supported version

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the confirmation. Have addressed the comment.

@kshitij12345
Copy link
Contributor Author

Failures look unrelated. Should I retrigger the CI?

@AlenkaF
Copy link
Member

AlenkaF commented Jul 20, 2022

I think some of the failures are related:

=================================== FAILURES ===================================
_______________________ test_metadata_schema_filesystem ________________________
tmpdir = local('/tmp/pytest-of-root/pytest-0/test_metadata_schema_filesyste0')
    def test_metadata_schema_filesystem(tmpdir):
        table = pa.table({"a": [1, 2, 3]})
        # URI writing to local file.
        fname = "data.parquet"
        file_path = 'file:///' + os.path.join(str(tmpdir), fname)
        pq.write_table(table, file_path)
        # Get expected `metadata` from path.
        metadata = pq.read_metadata(tmpdir / fname)
        schema = table.schema
        assert pq.read_metadata(file_path).equals(metadata)
>       assert pq.read_metadata(
            fname, filesystem=f'file:///{tmpdir}').equals(metadata)
opt/conda/envs/arrow/lib/python3.8/site-packages/pyarrow/tests/parquet/test_metadata.py:553: 
_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ 
opt/conda/envs/arrow/lib/python3.8/site-packages/pyarrow/parquet/__init__.py:3425: in read_metadata
    file = ParquetFile(where, memory_map=memory_map,
opt/conda/envs/arrow/lib/python3.8/site-packages/pyarrow/parquet/__init__.py:287: in __init__
    self.reader.open(
pyarrow/_parquet.pyx:1225: in pyarrow._parquet.ParquetReader.open
    ???
pyarrow/io.pxi:1674: in pyarrow.lib.get_reader
    ???
pyarrow/io.pxi:1665: in pyarrow.lib.get_native_file
    ???
pyarrow/io.pxi:943: in pyarrow.lib.OSFile.__cinit__
    ???
pyarrow/io.pxi:953: in pyarrow.lib.OSFile._open_readable
    ???
pyarrow/error.pxi:144: in pyarrow.lib.pyarrow_internal_check_status
    ???
_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ 
>   ???
E   FileNotFoundError: [Errno 2] Failed to open local file 'data.parquet'. Detail: [errno 2] No such file or directory

@kshitij12345
Copy link
Contributor Author

kshitij12345 commented Jul 20, 2022

Thanks for catching that @AlenkaF!

And Windows strikes 😓 ! I don't access to a Windows system. I think file:/// is not handled by Windows. Do you have any recommendations or is it ok to skip that particular approach on Windows?

Looks to be happening on other platforms as well. My bad, read the CI logs incorrectly.

@kshitij12345
Copy link
Contributor Author

Gentle ping @jorisvandenbossche @AlenkaF

Copy link
Member

@raulcd raulcd left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM minor nit comment

Comment on lines 3475 to 3477
source = filesystem.open_input_file(
where) if filesystem is not None else where
ctx = source if filesystem is not None else nullcontext()
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I find these lines slightly difficult to follow due to using both source and where variables and the two inline ifs for the same case, would something like:

    ctx = nullcontext()
    if filesystem is not None:
        ctx = where = filesystem.open_input_file(where)

be slightly more readable?
In thas case we should use where instead of source on the ParquetFile but this was how it was originally too.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Makes sense. Thanks for the suggestion. Have updated the code. PTAL :)

Copy link
Member

@raulcd raulcd left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @kshitij12345

@kshitij12345
Copy link
Contributor Author

Gentle ping :)

Copy link
Member

@jorisvandenbossche jorisvandenbossche left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks good! Just one comment on the tests that I failed to notice before


import numpy as np
import pytest

import pyarrow as pa
from pyarrow.tests.parquet.common import _check_roundtrip, make_sample_file
from pyarrow.filesystem import LocalFileSystem, FileSystem
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sorry for only noticing this late, but we should import here from pyarrow.fs to use the new filesystems in the tests (pyarrow.filesystem is deprecated)

@jorisvandenbossche
Copy link
Member

@kshitij12345 there are some test failures that actually seem related

assert pq.read_metadata(
fname, filesystem=LocalFileSystem()).equals(metadata)
assert pq.read_metadata(
fname, filesystem=LocalFileSystem.get_instance()).equals(metadata)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The new LocalFileSystem has no get_instance() method, so I think you can just remove this case (since the line above already includes the case of a plain LocalFileSystem())

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks! I was away from keyboard for the week. Will fix it!

@kshitij12345
Copy link
Contributor Author

@jorisvandenbossche CI failure looks irrelevant. PTAL :)

@jorisvandenbossche
Copy link
Member

I pushed a small additional update to the test (mainly changing to use our internal tempdir fixture instead of tmpdir)

Thanks again for the PR @kshitij12345 !

@jorisvandenbossche jorisvandenbossche changed the title ARROW-16719: [Python] Add path/URI /+ filesystem handling to parquet.read_metadata ARROW-16719: [Python] Add path/URI + filesystem handling to parquet.read_metadata Aug 17, 2022
@jorisvandenbossche jorisvandenbossche merged commit 42ed37e into apache:master Aug 17, 2022
@ursabot
Copy link

ursabot commented Aug 17, 2022

Benchmark runs are scheduled for baseline = f6127fc and contender = 42ed37e. 42ed37e is a master commit associated with this PR. Results will be available as each benchmark for each run completes.
Conbench compare runs links:
[Finished ⬇️0.0% ⬆️0.0%] ec2-t3-xlarge-us-east-2
[Finished ⬇️3.44% ⬆️2.89%] test-mac-arm
[Failed ⬇️4.38% ⬆️1.92%] ursa-i9-9960x
[Finished ⬇️5.12% ⬆️2.42%] ursa-thinkcentre-m75q
Buildkite builds:
[Finished] 42ed37e3 ec2-t3-xlarge-us-east-2
[Finished] 42ed37e3 test-mac-arm
[Failed] 42ed37e3 ursa-i9-9960x
[Finished] 42ed37e3 ursa-thinkcentre-m75q
[Finished] f6127fca ec2-t3-xlarge-us-east-2
[Finished] f6127fca test-mac-arm
[Failed] f6127fca ursa-i9-9960x
[Finished] f6127fca ursa-thinkcentre-m75q
Supported benchmarks:
ec2-t3-xlarge-us-east-2: Supported benchmark langs: Python, R. Runs only benchmarks with cloud = True
test-mac-arm: Supported benchmark langs: C++, Python, R
ursa-i9-9960x: Supported benchmark langs: Python, R, JavaScript
ursa-thinkcentre-m75q: Supported benchmark langs: C++, Java

@ursabot
Copy link

ursabot commented Aug 17, 2022

['Python', 'R'] benchmarks have high level of regressions.
test-mac-arm
ursa-i9-9960x

@kshitij12345
Copy link
Contributor Author

@jorisvandenbossche Thank you very much :)

zagto pushed a commit to zagto/arrow that referenced this pull request Oct 7, 2022
…ead_metadata (apache#13629)

Add `filesystem` support to `pq.read_metadata` and `pq.read_schema`.

Lead-authored-by: kshitij12345 <kshitijkalambarkar@gmail.com>
Co-authored-by: Kshiteej K <kshitijkalambarkar@gmail.com>
Co-authored-by: Joris Van den Bossche <jorisvandenbossche@gmail.com>
Signed-off-by: Joris Van den Bossche <jorisvandenbossche@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

5 participants