-
Notifications
You must be signed in to change notification settings - Fork 3.6k
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
[C++] Potential memory leak in Parquet reading with Dataset #37630
Comments
You could upload it in a gist and link that here, perhaps? |
I ended up changing the file extension and that worked :) |
Not too familiar with the async/future code (nor massif outputs), but does this look like the Future objects are not destructed somehow (which has a reference to the parquet metadata object?) (You need to expand the massif output in visualizer to see the Future object at the bottom of the call chain) |
cc @mapleFU @wgtmac @pitrou @felipecrv (Folks that have been opining on the mailing list) |
Future in arrow is an shared_future, which means if reference is hold, the object might not be dtor. I guess maybe not that problem, but I'll try to run the script and reproduce the problem later |
Note that FileFragment in the datasets API caches the parquet metadata (with no option to disable this at the moment). So if you are scanning many files you will see memory grow over the lifetime of the scan as more and more metadatas are cached. I would expect a second scan would not grow the memory. |
Thanks @westonpace, can you give a pointer to where that is happening? |
In
They initially start empty/null and are initialized during a call to |
Hmm, what do you mean by they are cached? Doesn't ParquetFileFragment get dtored after we scanned the fragment? |
Oh I see we keep a vector of Fragment, each Fragment has a shared_ptr to the schema of the file |
Yes, the fragments are part of the dataset. So they stay around until the dataset is destroyed which has to be after the scan. |
In a local build I cleared
after scanning a fragment and things appear to be in better shape. Memory usage goes from 4G -> 700+Mb. There still seem to be some leaking that I couldn't figure out but at least things seem to be in a much better shape after clearing those two fields. |
I was so busy these two weeks, sorry for late reply. Have you find out the reason? Seems that you're suffering from too many |
I believe I might be be experiencing this same problem through the Python API. Having to maintain a local build of Arrow doesn't sound like the right solution, so I wonder if there are ideas of how to achieve the same result as @icexelloss 's last comment, but through Arrow's normal APIs? |
Result<std::vector<std::shared_ptr<FileFragment>>>
ParquetDatasetFactory::CollectParquetFragments(const Partitioning& partitioning) {
std::vector<std::shared_ptr<FileFragment>> fragments(paths_with_row_group_ids_.size());
size_t i = 0;
for (const auto& e : paths_with_row_group_ids_) {
const auto& path = e.first;
auto metadata_subset = metadata_->Subset(e.second);
auto row_groups = Iota(metadata_subset->num_row_groups());
auto partition_expression =
partitioning.Parse(StripPrefix(path, options_.partition_base_dir))
.ValueOr(compute::literal(true));
ARROW_ASSIGN_OR_RAISE(
auto fragment,
format_->MakeFragment({path, filesystem_}, std::move(partition_expression),
physical_schema_, std::move(row_groups)));
RETURN_NOT_OK(fragment->SetMetadata(metadata_subset, manifest_));
fragments[i++] = std::move(fragment);
}
return fragments;
} I noticed that these metadata and manifest is |
Maybe but I think there must be more to it. In my example I have a partitioned parquet dataset on local disk, 8.6GB in total, with 13 columns and 38,747 fragments. Writing this dataset to a new location on disk (i.e. to compact the fragments) consumes all 8GB of RAM on my machine and then swaps to disk. I can't imagine that 13 columns or even 13×38,747 takes upwards of 8GB of memory. |
@mapleFU Sorry I have not found out the reason for the seemly leak of "arrow::Field" in #37630 (comment) post. |
I don't think you can at the moment. Because there is no way to tell Dataset to not to cache these metadata through normal API. I had a local fix but didn't find time the push it out (the fix is actually quite simple - clear the metadata from segment after done reading a segment) |
I do have plan to push the patch upstream proper when I have some spare time FWIW, This is my local patch:
|
You can do [1] https://en.cppreference.com/w/cpp/memory/shared_ptr/reset |
We should definitely make metadata caching an optional feature of the scanner and/or dataset. I think the API could be as simple as...
Any place that is using a dataset "temporarily" should also set this to false (e.g. when we run pyarrow.parquet.read_table it creates a dataset and scans it. That temporary dataset should NOT cache metadata) |
Oh, in your graph the |
I've submitted a draft PR that adds a Here are some memory consumption measurements on a synthetic dataset similar to the one in #45287:
$ /usr/bin/time -f "\n%E real\n%U user\n%S sys\n%M kB peak RSS" python -c 'import pyarrow.dataset as pd; pd.dataset("/home/antoine/arrow/data/bamboo-streaming-parquet-test-data/1000col-dataset/").to_table()'
0:11.01 real
107.66 user
4.37 sys
2069284 kB peak RSS
$ /usr/bin/time -f "\n%E real\n%U user\n%S sys\n%M kB peak RSS" python -c 'import pyarrow.dataset as pd; pd.dataset("/home/antoine/arrow/data/bamboo-streaming-parquet-test-data/1000col-dataset/").to_table(cache_metadata=False)'
0:10.97 real
109.11 user
3.87 sys
1381696 kB peak RSS |
@icexelloss It would be useful it you could test the above PR on a more real-world dataset. |
Thanks @pitrou! We actually already have an internal patch similar to what you have now but still observed higher than expected memory usage when scanning a parquet dataset - my colleague @timothydijamco is working a repro and should have it soon I also noticed that you also cleared |
Describe the bug, including details regarding any error messages, version, and platform.
Version
Arrow 12.0
Platform
Debian 5.4.228
Description
I have been testing "What is the max rss needed to scan through ~100G of data in a parquet stored in gcs using Arrow C++".
The current answer is about ~6G of memory which seems a bit high so I looked into it. What I observed during the process led me to think that there are some potential cache/memory issues in the dataset/parquet cpp code.
Main observation:
(1) As I am scanning through the dataset, I printed out (a) memory allocated by the memory pool from ScanOptions (b) process rss. I found that while (a) stays pretty stable throughout the scan (stays < 1G), (b) keeps increasing during the scan (looks linear to the number of files scanned).
(2) I tested ScanNode in Arrow as well as an in-house library that implements its own "S3Dataset" similar to Arrow dataset, both showing similar rss usage. (Which led me to think the issue is more likely to be in the parquet cpp code instead of dataset code).
(3) Scan the same dataset twice in the same process doesn't increase the max rss.
Following suggestions from the mailing list, I also did a memory profiling of the test program and the results seem to indicate potential memory leak.
Test code
https://gist.github.com/icexelloss/88195de046962e1d043c99d96e1b8b43
Massif output
massif.out.3505660.txt
Component(s)
C++, Parquet
The text was updated successfully, but these errors were encountered: