[jira] [Commented] (ARROW-3324) [Python] Users reporting memory leaks using pa.pq.ParquetDataset
[ https://issues.apache.org/jira/browse/ARROW-3324?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16729131#comment-16729131 ] Tanya Schlusser commented on ARROW-3324: The file [arrow_3324_leak_on_write.py|https://issues.apache.org/jira/secure/attachment/12953078/arrow_3324_leak_on_write.py] contains a modified version of the stackoverflow post in Wes's comment above, with {{memory_profiler}} to show the memory use. The memory use does increase as the code cycles through multiple calls to {{write_table}}. > [Python] Users reporting memory leaks using pa.pq.ParquetDataset > > > Key: ARROW-3324 > URL: https://issues.apache.org/jira/browse/ARROW-3324 > Project: Apache Arrow > Issue Type: Bug > Components: Python >Reporter: Wes McKinney >Priority: Major > Labels: parquet, pull-request-available > Fix For: 0.12.0 > > Attachments: arrow_3324_leak_on_write.py > > Time Spent: 50m > Remaining Estimate: 0h > > See: > * https://github.com/apache/arrow/issues/2614 > * https://github.com/apache/arrow/issues/2624 -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (ARROW-3324) [Python] Users reporting memory leaks using pa.pq.ParquetDataset
[ https://issues.apache.org/jira/browse/ARROW-3324?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16728819#comment-16728819 ] Tanya Schlusser commented on ARROW-3324: I could not reproduce either of the two GitHub issues above, but could identify a leak using {{memory_profiler}} on the stackoverflow code (copied [this|https://github.com/apache/arrow/blob/master/python/scripts/test_leak.py]) I observed that {{FileSerializer.properties_.use_count()}} increments more than expected whenever {{FileSerializer.AppendRowGroup}} is called. The offending line is {{FileSerializer.metadata_->AppendRowGroup()}}. I believe that the count should only go up once per new row group, instead of once per column plus once per row group. I think the root cause is that in {{RowGroupMetaDataBuilder::RowGroupMetaDataBuilderImpl.Finish}}, the vector of {{column_builders_}} ought to be reset and cleared each time before it is repopulated. I hope to submit a pull request for this even though it may not address all of the issues stated here. Since the GitHub issues were about memory leaks on "read", and the fix is related only to "write", this observation certainly doesn't address everything in this JIRA issue. Even after the fix I'll post, my memory_profiler code still shows an increase in memory use upon additional calls to {{pq.ParquetWriter.write_table}}, which I think is OK because the row group is incrementing with each write too. So I may be wrong or have still missed something. Regardless, I hope these notes are useful to someone. > [Python] Users reporting memory leaks using pa.pq.ParquetDataset > > > Key: ARROW-3324 > URL: https://issues.apache.org/jira/browse/ARROW-3324 > Project: Apache Arrow > Issue Type: Bug > Components: Python >Reporter: Wes McKinney >Priority: Major > Labels: parquet > Fix For: 0.12.0 > > > See: > * https://github.com/apache/arrow/issues/2614 > * https://github.com/apache/arrow/issues/2624 -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (ARROW-3324) [Python] Users reporting memory leaks using pa.pq.ParquetDataset
[ https://issues.apache.org/jira/browse/ARROW-3324?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16665729#comment-16665729 ] Wes McKinney commented on ARROW-3324: - Here's another memory leak report {code} import resource import random import string import pyarrow as pa import pyarrow.parquet as pq import pandas as pd def id_generator(size=6, chars=string.ascii_uppercase + string.digits): return ''.join(random.choice(chars) for _ in range(size)) schema = pa.schema([ pa.field('test', pa.string()), ]) resource.setrlimit(resource.RLIMIT_NOFILE, (100, 100)) number_files = 1 number_rows_increment = 1000 number_iterations = 100 writers = [pq.ParquetWriter('test_'+id_generator()+'.parquet', schema) for i in range(number_files)] for i in range(number_iterations): for writer in writers: table_to_write = pa.Table.from_pandas( pd.DataFrame({'test': [id_generator() for i in range(number_rows_increment)]}), preserve_index=False, schema = schema, nthreads = 1) table_to_write = table_to_write.replace_schema_metadata(None) writer.write_table(table_to_write) print(i) for writer in writers: writer.close() {code} https://stackoverflow.com/questions/53016802/memory-leak-from-pyarrow > [Python] Users reporting memory leaks using pa.pq.ParquetDataset > > > Key: ARROW-3324 > URL: https://issues.apache.org/jira/browse/ARROW-3324 > Project: Apache Arrow > Issue Type: Bug > Components: Python >Reporter: Wes McKinney >Priority: Major > Fix For: 0.12.0 > > > See: > * https://github.com/apache/arrow/issues/2614 > * https://github.com/apache/arrow/issues/2624 -- This message was sent by Atlassian JIRA (v7.6.3#76005)