[jira] [Commented] (ARROW-2369) Large (>~20 GB) files written to Parquet via PyArrow are corrupted
[ https://issues.apache.org/jira/browse/ARROW-2369?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16435450#comment-16435450 ] ASF GitHub Bot commented on ARROW-2369: --- pitrou commented on a change in pull request #1866: ARROW-2369: [Python] Fix reading large Parquet files (> 4 GB) URL: https://github.com/apache/arrow/pull/1866#discussion_r181058227 ## File path: cpp/src/arrow/python/io.cc ## @@ -65,14 +65,16 @@ class PythonFile { Status Seek(int64_t position, int whence) { // whence: 0 for relative to start of file, 2 for end of file -PyObject* result = cpp_PyObject_CallMethod(file_, "seek", "(ii)", position, whence); +PyObject* result = cpp_PyObject_CallMethod(file_, "seek", "(ni)", Review comment: Ultimately it is the same format string syntax as documented here: https://docs.python.org/3/c-api/arg.html#building-values This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org > Large (>~20 GB) files written to Parquet via PyArrow are corrupted > -- > > Key: ARROW-2369 > URL: https://issues.apache.org/jira/browse/ARROW-2369 > Project: Apache Arrow > Issue Type: Bug > Components: Python >Affects Versions: 0.9.0 > Environment: Reproduced on Ubuntu + Mac OSX >Reporter: Justin Tan >Assignee: Antoine Pitrou >Priority: Major > Labels: Parquet, bug, pandas, parquetWriter, > pull-request-available, pyarrow > Fix For: 0.10.0 > > Attachments: Screen Shot 2018-03-30 at 11.54.01 pm.png > > > When writing large Parquet files (above 10 GB or so) from Pandas to Parquet > via the command > {{pq.write_table(my_df, 'table.parquet')}} > The write succeeds, but when the parquet file is loaded, the error message > {{ArrowIOError: Invalid parquet file. Corrupt footer.}} > appears. This same error occurs when the parquet file is written chunkwise as > well. When the parquet files are small, say < 5 GB or so (drawn randomly from > the same dataset), everything proceeds as normal. I've also tried this with > Pandas df.to_parquet(), with the same results. > Update: Looks like any DataFrame with size above ~5GB (on disk) returns the > same error. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (ARROW-2369) Large (>~20 GB) files written to Parquet via PyArrow are corrupted
[ https://issues.apache.org/jira/browse/ARROW-2369?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16435447#comment-16435447 ] ASF GitHub Bot commented on ARROW-2369: --- xhochy closed pull request #1866: ARROW-2369: [Python] Fix reading large Parquet files (> 4 GB) URL: https://github.com/apache/arrow/pull/1866 This is a PR merged from a forked repository. As GitHub hides the original diff on merge, it is displayed below for the sake of provenance: As this is a foreign pull request (from a fork), the diff is supplied below (as it won't show otherwise due to GitHub magic): diff --git a/cpp/src/arrow/python/io.cc b/cpp/src/arrow/python/io.cc index 36c193dbfa..155e86f828 100644 --- a/cpp/src/arrow/python/io.cc +++ b/cpp/src/arrow/python/io.cc @@ -65,14 +65,16 @@ class PythonFile { Status Seek(int64_t position, int whence) { // whence: 0 for relative to start of file, 2 for end of file -PyObject* result = cpp_PyObject_CallMethod(file_, "seek", "(ii)", position, whence); +PyObject* result = cpp_PyObject_CallMethod(file_, "seek", "(ni)", + static_cast(position), whence); Py_XDECREF(result); PY_RETURN_IF_ERROR(StatusCode::IOError); return Status::OK(); } Status Read(int64_t nbytes, PyObject** out) { -PyObject* result = cpp_PyObject_CallMethod(file_, "read", "(i)", nbytes); +PyObject* result = +cpp_PyObject_CallMethod(file_, "read", "(n)", static_cast(nbytes)); PY_RETURN_IF_ERROR(StatusCode::IOError); *out = result; return Status::OK(); diff --git a/python/pyarrow/parquet.py b/python/pyarrow/parquet.py index beeedca032..34aa55a9c2 100644 --- a/python/pyarrow/parquet.py +++ b/python/pyarrow/parquet.py @@ -60,7 +60,6 @@ class ParquetFile(object): """ def __init__(self, source, metadata=None, common_metadata=None): self.reader = ParquetReader() -source = _ensure_file(source) self.reader.open(source, metadata=metadata) self.common_metadata = common_metadata self._nested_paths_by_prefix = self._build_nested_paths() diff --git a/python/pyarrow/tests/test_io.py b/python/pyarrow/tests/test_io.py index b29b9f1f6e..1511600777 100644 --- a/python/pyarrow/tests/test_io.py +++ b/python/pyarrow/tests/test_io.py @@ -21,6 +21,7 @@ import os import pytest import sys +import tempfile import weakref import numpy as np @@ -30,6 +31,25 @@ from pyarrow.compat import u, guid import pyarrow as pa + +def check_large_seeks(file_factory): +if sys.platform in ('win32', 'darwin'): +pytest.skip("need sparse file support") +try: +filename = tempfile.mktemp(prefix='test_io') +with open(filename, 'wb') as f: +f.truncate(2 ** 32 + 10) +f.seek(2 ** 32 + 5) +f.write(b'mark\n') +with file_factory(filename) as f: +assert f.seek(2 ** 32 + 5) == 2 ** 32 + 5 +assert f.tell() == 2 ** 32 + 5 +assert f.read(5) == b'mark\n' +assert f.tell() == 2 ** 32 + 10 +finally: +os.unlink(filename) + + # -- # Python file-like objects @@ -83,6 +103,13 @@ def test_python_file_read(): f.close() +def test_python_file_large_seeks(): +def factory(filename): +return pa.PythonFile(open(filename, 'rb')) + +check_large_seeks(factory) + + def test_bytes_reader(): # Like a BytesIO, but zero-copy underneath for C++ consumers data = b'some sample data' @@ -544,6 +571,10 @@ def test_os_file_reader(sample_disk_data): _check_native_file_reader(pa.OSFile, sample_disk_data) +def test_os_file_large_seeks(): +check_large_seeks(pa.OSFile) + + def _try_delete(path): try: os.remove(path) @@ -600,6 +631,10 @@ def test_memory_zero_length(tmpdir): assert memory_map.size() == 0 +def test_memory_map_large_seeks(): +check_large_seeks(pa.memory_map) + + def test_os_file_writer(tmpdir): SIZE = 4096 arr = np.random.randint(0, 256, size=SIZE).astype('u1') This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org > Large (>~20 GB) files written to Parquet via PyArrow are corrupted > -- > > Key: ARROW-2369 > URL: https://issues.apache.org/jira/browse/ARROW-2369 > Project: Apache Arrow > Issue Type: Bug > Components: Python >Affects Versions: 0.9.0 > Environment: Reproduced on Ubuntu + Mac OSX >Reporter: Justin Tan >
[jira] [Commented] (ARROW-2369) Large (>~20 GB) files written to Parquet via PyArrow are corrupted
[ https://issues.apache.org/jira/browse/ARROW-2369?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16435443#comment-16435443 ] ASF GitHub Bot commented on ARROW-2369: --- pitrou commented on a change in pull request #1866: ARROW-2369: [Python] Fix reading large Parquet files (> 4 GB) URL: https://github.com/apache/arrow/pull/1866#discussion_r181057886 ## File path: cpp/src/arrow/python/io.cc ## @@ -65,14 +65,16 @@ class PythonFile { Status Seek(int64_t position, int whence) { // whence: 0 for relative to start of file, 2 for end of file -PyObject* result = cpp_PyObject_CallMethod(file_, "seek", "(ii)", position, whence); +PyObject* result = cpp_PyObject_CallMethod(file_, "seek", "(ni)", Review comment: Yes, `i` takes the given parameter as `int` while `n` takes it as `Py_ssize_t`. This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org > Large (>~20 GB) files written to Parquet via PyArrow are corrupted > -- > > Key: ARROW-2369 > URL: https://issues.apache.org/jira/browse/ARROW-2369 > Project: Apache Arrow > Issue Type: Bug > Components: Python >Affects Versions: 0.9.0 > Environment: Reproduced on Ubuntu + Mac OSX >Reporter: Justin Tan >Assignee: Antoine Pitrou >Priority: Major > Labels: Parquet, bug, pandas, parquetWriter, > pull-request-available, pyarrow > Fix For: 0.10.0 > > Attachments: Screen Shot 2018-03-30 at 11.54.01 pm.png > > > When writing large Parquet files (above 10 GB or so) from Pandas to Parquet > via the command > {{pq.write_table(my_df, 'table.parquet')}} > The write succeeds, but when the parquet file is loaded, the error message > {{ArrowIOError: Invalid parquet file. Corrupt footer.}} > appears. This same error occurs when the parquet file is written chunkwise as > well. When the parquet files are small, say < 5 GB or so (drawn randomly from > the same dataset), everything proceeds as normal. I've also tried this with > Pandas df.to_parquet(), with the same results. > Update: Looks like any DataFrame with size above ~5GB (on disk) returns the > same error. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (ARROW-2369) Large (>~20 GB) files written to Parquet via PyArrow are corrupted
[ https://issues.apache.org/jira/browse/ARROW-2369?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16435440#comment-16435440 ] ASF GitHub Bot commented on ARROW-2369: --- xhochy commented on a change in pull request #1866: ARROW-2369: [Python] Fix reading large Parquet files (> 4 GB) URL: https://github.com/apache/arrow/pull/1866#discussion_r181057233 ## File path: cpp/src/arrow/python/io.cc ## @@ -65,14 +65,16 @@ class PythonFile { Status Seek(int64_t position, int whence) { // whence: 0 for relative to start of file, 2 for end of file -PyObject* result = cpp_PyObject_CallMethod(file_, "seek", "(ii)", position, whence); +PyObject* result = cpp_PyObject_CallMethod(file_, "seek", "(ni)", Review comment: Problem was before that `position` was casted to an int32? This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org > Large (>~20 GB) files written to Parquet via PyArrow are corrupted > -- > > Key: ARROW-2369 > URL: https://issues.apache.org/jira/browse/ARROW-2369 > Project: Apache Arrow > Issue Type: Bug > Components: Python >Affects Versions: 0.9.0 > Environment: Reproduced on Ubuntu + Mac OSX >Reporter: Justin Tan >Assignee: Antoine Pitrou >Priority: Major > Labels: Parquet, bug, pandas, parquetWriter, > pull-request-available, pyarrow > Fix For: 0.10.0 > > Attachments: Screen Shot 2018-03-30 at 11.54.01 pm.png > > > When writing large Parquet files (above 10 GB or so) from Pandas to Parquet > via the command > {{pq.write_table(my_df, 'table.parquet')}} > The write succeeds, but when the parquet file is loaded, the error message > {{ArrowIOError: Invalid parquet file. Corrupt footer.}} > appears. This same error occurs when the parquet file is written chunkwise as > well. When the parquet files are small, say < 5 GB or so (drawn randomly from > the same dataset), everything proceeds as normal. I've also tried this with > Pandas df.to_parquet(), with the same results. > Update: Looks like any DataFrame with size above ~5GB (on disk) returns the > same error. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (ARROW-2369) Large (>~20 GB) files written to Parquet via PyArrow are corrupted
[ https://issues.apache.org/jira/browse/ARROW-2369?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16430650#comment-16430650 ] ASF GitHub Bot commented on ARROW-2369: --- pitrou opened a new pull request #1866: ARROW-2369: [Python] Fix reading large Parquet files (> 4 GB) URL: https://github.com/apache/arrow/pull/1866 - Fix PythonFile.seek() for offsets > 4 GB - Avoid instantiating a PythonFile in ParquetFile, for efficiency This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org > Large (>~20 GB) files written to Parquet via PyArrow are corrupted > -- > > Key: ARROW-2369 > URL: https://issues.apache.org/jira/browse/ARROW-2369 > Project: Apache Arrow > Issue Type: Bug > Components: Python >Affects Versions: 0.9.0 > Environment: Reproduced on Ubuntu + Mac OSX >Reporter: Justin Tan >Assignee: Antoine Pitrou >Priority: Major > Labels: Parquet, bug, pandas, parquetWriter, > pull-request-available, pyarrow > Fix For: 0.10.0 > > Attachments: Screen Shot 2018-03-30 at 11.54.01 pm.png > > > When writing large Parquet files (above 10 GB or so) from Pandas to Parquet > via the command > {{pq.write_table(my_df, 'table.parquet')}} > The write succeeds, but when the parquet file is loaded, the error message > {{ArrowIOError: Invalid parquet file. Corrupt footer.}} > appears. This same error occurs when the parquet file is written chunkwise as > well. When the parquet files are small, say < 5 GB or so (drawn randomly from > the same dataset), everything proceeds as normal. I've also tried this with > Pandas df.to_parquet(), with the same results. > Update: Looks like any DataFrame with size above ~5GB (on disk) returns the > same error. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (ARROW-2369) Large (>~20 GB) files written to Parquet via PyArrow are corrupted
[ https://issues.apache.org/jira/browse/ARROW-2369?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16430572#comment-16430572 ] Antoine Pitrou commented on ARROW-2369: --- Ok, there are two things going on: * when {{write_table()}} is called with a filepath string, it goes through {{PythonFile}}, which is probably inefficient * {{PythonFile.Seek}} doesn't handle seek offsets greater than 2**32 properly: {code:python} >>> f = open('/tmp/empty', 'wb') >>> f.truncate(1<<33 + 10) 8796093022208 >>> f.close() >>> f = open('/tmp/empty', 'rb') >>> paf = pa.PythonFile(f, 'rb') >>> paf.tell() 0 >>> paf.seek(5) 5 >>> paf.tell() 5 >>> paf.seek(1<<33 + 6) 0 >>> paf.tell() 0 >>> f.seek(1<<33 + 6) 549755813888 >>> f.tell() 549755813888 {code} > Large (>~20 GB) files written to Parquet via PyArrow are corrupted > -- > > Key: ARROW-2369 > URL: https://issues.apache.org/jira/browse/ARROW-2369 > Project: Apache Arrow > Issue Type: Bug > Components: Python >Affects Versions: 0.9.0 > Environment: Reproduced on Ubuntu + Mac OSX >Reporter: Justin Tan >Assignee: Antoine Pitrou >Priority: Major > Labels: Parquet, bug, pandas, parquetWriter, pyarrow > Fix For: 0.10.0 > > Attachments: Screen Shot 2018-03-30 at 11.54.01 pm.png > > > When writing large Parquet files (above 10 GB or so) from Pandas to Parquet > via the command > {{pq.write_table(my_df, 'table.parquet')}} > The write succeeds, but when the parquet file is loaded, the error message > {{ArrowIOError: Invalid parquet file. Corrupt footer.}} > appears. This same error occurs when the parquet file is written chunkwise as > well. When the parquet files are small, say < 5 GB or so (drawn randomly from > the same dataset), everything proceeds as normal. I've also tried this with > Pandas df.to_parquet(), with the same results. > Update: Looks like any DataFrame with size above ~5GB (on disk) returns the > same error. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (ARROW-2369) Large (>~20 GB) files written to Parquet via PyArrow are corrupted
[ https://issues.apache.org/jira/browse/ARROW-2369?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16430571#comment-16430571 ] Justin Tan commented on ARROW-2369: --- Looks like the file is readable by early pyarrow versions (0.5.0 - but created by v0.5.0 as well), so maybe something went wrong from 0.5.0 -> 0.9.0 > Large (>~20 GB) files written to Parquet via PyArrow are corrupted > -- > > Key: ARROW-2369 > URL: https://issues.apache.org/jira/browse/ARROW-2369 > Project: Apache Arrow > Issue Type: Bug > Components: Python >Affects Versions: 0.9.0 > Environment: Reproduced on Ubuntu + Mac OSX >Reporter: Justin Tan >Assignee: Antoine Pitrou >Priority: Major > Labels: Parquet, bug, pandas, parquetWriter, pyarrow > Fix For: 0.10.0 > > Attachments: Screen Shot 2018-03-30 at 11.54.01 pm.png > > > When writing large Parquet files (above 10 GB or so) from Pandas to Parquet > via the command > {{pq.write_table(my_df, 'table.parquet')}} > The write succeeds, but when the parquet file is loaded, the error message > {{ArrowIOError: Invalid parquet file. Corrupt footer.}} > appears. This same error occurs when the parquet file is written chunkwise as > well. When the parquet files are small, say < 5 GB or so (drawn randomly from > the same dataset), everything proceeds as normal. I've also tried this with > Pandas df.to_parquet(), with the same results. > Update: Looks like any DataFrame with size above ~5GB (on disk) returns the > same error. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (ARROW-2369) Large (>~20 GB) files written to Parquet via PyArrow are corrupted
[ https://issues.apache.org/jira/browse/ARROW-2369?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16421510#comment-16421510 ] Wes McKinney commented on ARROW-2369: - Sounds like there's a {{uint32_t}} overflow somewhere, which is the sort of thing that often happens on Windows > Large (>~20 GB) files written to Parquet via PyArrow are corrupted > -- > > Key: ARROW-2369 > URL: https://issues.apache.org/jira/browse/ARROW-2369 > Project: Apache Arrow > Issue Type: Bug > Components: Python >Affects Versions: 0.9.0 > Environment: Reproduced on Ubuntu + Mac OSX >Reporter: Justin Tan >Priority: Major > Labels: Parquet, bug, pandas, parquetWriter, pyarrow > Fix For: 0.10.0 > > Attachments: Screen Shot 2018-03-30 at 11.54.01 pm.png > > > When writing large Parquet files (above 10 GB or so) from Pandas to Parquet > via the command > {{pq.write_table(my_df, 'table.parquet')}} > The write succeeds, but when the parquet file is loaded, the error message > {{ArrowIOError: Invalid parquet file. Corrupt footer.}} > appears. This same error occurs when the parquet file is written chunkwise as > well. When the parquet files are small, say < 5 GB or so (drawn randomly from > the same dataset), everything proceeds as normal. I've also tried this with > Pandas df.to_parquet(), with the same results. > Update: Looks like any DataFrame with size above ~5GB (on disk) returns the > same error. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (ARROW-2369) Large (>~20 GB) files written to Parquet via PyArrow are corrupted
[ https://issues.apache.org/jira/browse/ARROW-2369?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16421506#comment-16421506 ] Babak Alipour commented on ARROW-2369: -- I've got the same issue on Win 10, Arrow v0.9.0. The file was created with {{to_parquet(..., engine='pyarrow', compression='brotli')}} and cannot be read with {{read_parquet(..., engine='pyarrow')}} due to {{Invalid parquet file. Corrupt footer. error.}} Surprisingly, the same file created by pyarrow can be read using the fastparquet engine! File size on disk is 5,214,407,947 bytes (brotli compressed). > Large (>~20 GB) files written to Parquet via PyArrow are corrupted > -- > > Key: ARROW-2369 > URL: https://issues.apache.org/jira/browse/ARROW-2369 > Project: Apache Arrow > Issue Type: Bug > Components: Python >Affects Versions: 0.9.0 > Environment: Reproduced on Ubuntu + Mac OSX >Reporter: Justin Tan >Priority: Major > Labels: Parquet, bug, pandas, parquetWriter, pyarrow > Fix For: 0.9.0 > > Attachments: Screen Shot 2018-03-30 at 11.54.01 pm.png > > > When writing large Parquet files (above 10 GB or so) from Pandas to Parquet > via the command > {{pq.write_table(my_df, 'table.parquet')}} > The write succeeds, but when the parquet file is loaded, the error message > {{ArrowIOError: Invalid parquet file. Corrupt footer.}} > appears. This same error occurs when the parquet file is written chunkwise as > well. When the parquet files are small, say < 5 GB or so (drawn randomly from > the same dataset), everything proceeds as normal. I've also tried this with > Pandas df.to_parquet(), with the same results. > Update: Looks like any DataFrame with size above ~5GB (on disk) returns the > same error. -- This message was sent by Atlassian JIRA (v7.6.3#76005)