Hey Wes,

I've opened up a MR for this ARROW-7800.

Tests aren't really done as it was committed from a WIP to demonstrate what
was happening with categorical types and what was needed in order to
sidestep this.

I might need a bit of advice what else to test here.

Rollo

On Fri, Feb 7, 2020 at 7:33 PM Wes McKinney <wesmck...@gmail.com> wrote:

> Cool, feel free to open tickets as needed for the feature and any
> problems you ran into along the way
>
> On the Parquet nested stuff, it's a large project, Micah Kornfield is
> working on it now
> (
> https://lists.apache.org/thread.html/r831c41a3f9b353cfc187dc2092515fe707d4ebf13cbb727524f1c600%40%3Cdev.arrow.apache.org%3E
> )
> so I suggest coordinating with him on that e-mail thread
>
> On Fri, Feb 7, 2020 at 12:27 PM Rollo Konig-Brock <roll...@gmail.com>
> wrote:
> >
> > Hey Wes,
> >
> > I've implemented a get_batches function on the parquet_file. I'm not a
> huge fan of the way I've exposed the API. It's essentially like this:
> >
> > ```
> > file_ = ParquetFile('file/path.parquet', batch_size=100)
> >
> > for batch in file_.get_batches():
> >      pass
> > ```
> >
> > And I encountered some problems with DictionaryArrays having their
> dictionary sliced off in later batches (which I fixed with a bit of a god
> awful hack - but I feel really needs to be fixed in the Cpp level).
> >
> > I'd like to start contributing back, before I write up an MR, I'd like
> to create a ticket with the acceptance criteria and a fix for the
> DictionaryArrays slicing issue this exposed. What I've written is good
> enough for the project I'm working on but it was done on company time and
> has some warts.
> >
> > I'd also like to contribute to getting nested structs working, I'm
> wondering what that involves. I might be free in a few months after my
> current contract ends and I think it would a positive thing to contribute
> to Arrow in any time I have off.
> >
> > Rollo
> >
> >
> >
> > On 30/01/2020, 12:14, "Rollo Konig-Brock" <roll...@gmail.com> wrote:
> >
> >     Hi Wes,
> >
> >     Thank you for letting me know about the GetRecordBatchReader. I
> think that exposing the C++ API for chunked reads though Cython is probably
> the path of least resistance to our project. Depending on personal
> circumstance, legal constraints and maturity of what get's written, I'll
> try my best to contribute back any extensions to the Python component.
> >
> >     The Datasets API project looks really interesting.
> >
> >     All the best,
> >     Rollo
> >
> >
> >
> >     On 29/01/2020, 16:57, "Wes McKinney" <wesmck...@gmail.com> wrote:
> >
> >         [CAUTION] EXTERNAL EMAIL ..
> >
> >
> >         hi Rollo,
> >
> >         Two quick points:
> >
> >         * There is a C++ API already providing chunk-based reads
> >         ("GetRecordBatchReader"). That it is not yet exposed in Python in
> >         pyarrow.parquet is a result of a lack of a volunteer to do so in
> the
> >         past
> >         * Several people are actively working on the "Datasets API"
> project in
> >         C++ (with bindings in Python and R) which will provide a holistic
> >         approach to chunked (and thus more memory constrained) dataset
> reading
> >         across arbitrary file formats (not just Parquet). This is what I
> see
> >         as the long term solution to the problem you're describing. See
> [1]
> >
> >         We'd welcome your contributions to this work. If you have the
> >         financial resources to sponsor developers to increase efforts on
> this
> >         I'd be happy to speak with you about that offline.
> >
> >         - Wes
> >
> >         [1]:
> https://nam04.safelinks.protection.outlook.com/?url=https%3A%2F%2Fdocs.google.com%2Fdocument%2Fd%2F1bVhzifD38qDypnSjtf8exvpP3sSB5x_Kw9m-n66FB2c%2Fedit&amp;data=02%7C01%7CRollo.Konigbrock%40ihsmarkit.com%7Cceebc5748a864692b26308d7a4dc4262%7Cc1156c2fa3bb4fc4ac073eab96da8d10%7C1%7C0%7C637159138220274549&amp;sdata=9NPqsrjHRGNlgD%2BSvVEanlyJOIV%2FR9CBGncvuyZnv1E%3D&amp;reserved=0
> >
> >         On Wed, Jan 29, 2020 at 9:45 AM Rollo Konig-Brock <
> roll...@gmail.com> wrote:
> >         >
> >         > Dear Arrow Developers,
> >         >
> >         >
> >         >
> >         > I’m having memory issues with certain Parquet files. Parquet
> uses run length encoding for certain columns, meaning that an array of
> int64s could take up a couple thousand bytes on disk, but then a few
> hundred megabytes when loaded into a pyarrow.Table. I’m a little surprised
> there’s no option to keep the underlying Arrow array as encoded data.
> >         >
> >         >
> >         >
> >         > An example of this with a file I’ve created (attached) which
> is just 1’000’000 repetitions of a single int64 value:
> >         >
> >         >
> >         >
> >         > import psutil
> >         >
> >         > import os
> >         >
> >         > import gc
> >         >
> >         > import pyarrow.parquet
> >         >
> >         >
> >         >
> >         > suffixes = ['B', 'KB', 'MB', 'GB', 'TB', 'PB']
> >         >
> >         > current_thread = psutil.Process(os.getpid())
> >         >
> >         >
> >         >
> >         > def human_memory_size(nbytes):
> >         >
> >         >     i = 0
> >         >
> >         >     while nbytes >= 1024 and i < len(suffixes)-1:
> >         >
> >         >         nbytes /= 1024.
> >         >
> >         >         i += 1
> >         >
> >         >     f = ('%.2f' % nbytes).rstrip('0').rstrip('.')
> >         >
> >         >     return '%s %s' % (f, suffixes[i])
> >         >
> >         >
> >         >
> >         >
> >         >
> >         > def log_memory_usage(msg):
> >         >
> >         >     print(msg,
> human_memory_size(current_thread.memory_info().rss))
> >         >
> >         >
> >         >
> >         > log_memory_usage('Initial Memory usage')
> >         >
> >         >
> >         >
> >         > print('Size of parquet file',
> human_memory_size(os.stat('rle_bomb.parquet').st_size))
> >         >
> >         >
> >         >
> >         > pf = pyarrow.parquet.ParquetFile('rle_bomb.parquet')
> >         >
> >         >
> >         >
> >         > table = pf.read()
> >         >
> >         >
> >         >
> >         > log_memory_usage('Loaded memory usage')
> >         >
> >         >
> >         >
> >         > This will produce the following output:
> >         >
> >         >
> >         >
> >         > Initial Memory usage 27.11 MB
> >         >
> >         > Size of parquet file 3.62 KB
> >         >
> >         > Loaded schema 27.71 MB
> >         >
> >         > Loaded memory usage 997.9 MB
> >         >
> >         >
> >         >
> >         > This poses a bit of a problem particularly when running this
> code in servers as there doesn’t seem to be a way of preventing a memory
> explosion given the PyArrow API. I’m at a bit of a loss at how to control
> for this, there does not seem to be a method to do something like iterate
> over the Parquet columns in set chunks (where the size could be calculated
> accurately).
> >         >
> >         >
> >         >
> >         > All the best,
> >         >
> >         > Rollo Konig-Brock
> >         >
> >         >
> >         >
> >         >
> >
> >
>

Reply via email to