-
Notifications
You must be signed in to change notification settings - Fork 3k
Use bounded queue to avoid consuming too much memory #4596
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
Conversation
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java
Outdated
Show resolved
Hide resolved
| } catch (IOException e) { | ||
| throw new RuntimeIOException(e, "Failed to close iterable"); | ||
| } catch (IOException | InterruptedException e) { | ||
| throw new RuntimeException("Failed to close iterable", e); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can we add a test case that goes through this path? I'm not sure if throwing when an InterrruptedException occurs is necessarily the correct behavior.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
InterruptedException is thrown by BlockingQueue.put(). Using put() but not add() is because put() will waiting if necessary for space to become available.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
BTW, I think we shouldn't call Thread.sleep in hasNext if we use blocking queue. In our test it's bad for performance when the queue size is small.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
+1 to not using Thread.sleep() anywhere if at all possible. It’s bad for system performance and generally causes instability at any scale. @lirui-apache’s result confirm what I’ve expected / known.
|
Let me be clear. In my scene, my machine has 96 cores, when |
| * Sets the size of the queue, which is used to avoid consuming too much memory. | ||
| */ | ||
| public static final String SCAN_SHARED_QUEUE_SIZE = "iceberg.scan.shared-queue-size"; | ||
| public static final int SCAN_SHARED_QUEUE_SIZE_DEFAULT = 1000; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This seems really small if we're trying to avoid running out of memory. I think it is likely that this will cause a performance bottleneck.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
When use parallel iterator, BaseFileScanTask will be put into queue in parallel, while we consume these BaseFileScanTask serially. So it should not cause a performance bottleneck. We can make this size larger, like 10000, with more memory consuming.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The worker pool is already used for a number of things though and is presently uncapped.
Going from unbounded to bounded is already a potential drain / possible source of bottleneck. I don't know what the right value is for this, but if we're considering this feature, I would say this is the best value to use as a configuration to disable it.
There could be generally other scans taking place and many file scan tasks are unfortunately somewhat variable in size and can be combined etc. So I don't disagree with using the larger value (especially if it's configurable).
That said, if we add this feature, we should add a property to enable or disable it.
We can use 0, or any non-positive value, which is what sets whether or not the caching catalog has a TTL.
Though catalogs general have a boolean property to enable / disable whether or not caching is used entirely. So either iceberg.scan.enable-blocking-queue or iceberg.scan.use-shared-queue, or just setting the value to 0 or -1 to disable it entirely
|
@uncleGen, what is the average file size in the table, the average number of files per manifest, and how many files is your scan producing? Is there anything consuming from the concurrent queue? This seems odd to me since the parallel iterator is self-limiting. When nothing is consuming from the queue, new tasks won't be submitted. |
|
@rdblue |
|
ping @rdblue |
|
Is there a reason that the files have to be 1MB? That’s very very small. You should also consider using Avro for storage at that size. But I think if you compacted files you probably wouldn’t have such issues during scan planning. For ingest, sometimes we can’t avoid that small of files, I know. But scanning that many small files is counterintuitive for most cases. Ideally files that small (possibly coming from a 3rd party or a very very sharded system) would then be ingested into a table that is more ideally tailored to you query needs. Do you run table maintenance actions on your table @uncleGen? |
There is a streaming ingest job, it will create much small files.
I have run |
|
Took a little a look through the code, ParallelIterable.hasNext() calls checkTasks() , which then calls submitNextTask() but only up to taskFuture size (configured thread pool size). submitNextTask is the one that calls tasks.next(), which adds things to the queue. I guess that's why @rdblue is mentioning, it is self-limiting. The problem may be that each tasks.next() does dump the entire nested Iterable into the queue. So in real life example of the planning case, it seems the level parallelism is manifest. New manifests may be blocked to be processed if nobody is consuming, but a single or few manifests having a lot of entries may cause memory pressure as they dump all of the entries to the queue. @lirui-apache @uncleGen is that the case? Probably the fix is to rewriteManifests to have each manifest have more even number of entries? I guess if we use blocking queue, it may be useful and we have another knob to limit the memory, but then it's not truly parallel by manifest, ie some manifests may be blocked a long time before they can submit their entries to the queue. @rdblue let me know if you have any thoughts or if I may have misread the code. |
|
@szehon-ho, that sounds correct to me. I'm reluctant to block the queue because that introduces another bottleneck in cases where you probably don't want one. |
|
Hey @szehon-ho , your understanding about the issue is correct. We did some test of iterating all manifest entries and compute aggregated stats for each partition. We tried various queue sizes ranging from 5 to 10000. In our test the consumer is pretty fast, and even the smallest queue doesn't affect the e2e latency of the job. The result might be different in other use cases where the consumer is not fast enough, but my hunch is that such job latency is bounded by consumer anyway. In production env, we do have a background service to rewrite manifest periodically. But such optimization is asynchronous, which means if users query the table before the rewrite is done, it can still cause OOM. |
|
Is there anything worse than OOM in production env if we do not use a blocking queue ? We may be able to use another solution to limit the flow rather than blocking queue that may cause another bottleneck. |
|
@rdblue @lirui-apache thanks for confirming, so I think using 'rewriteManifest' and specifying the commit.manifest.target-size-byte to a reasonable size (so you have an even number of entries per manifest file), along with a system property: 'iceberg.worker.num-threads' to control how many manifests are read at once. I wonder if that will help that memory problem? So overall, I would double-check how many entries you have per manifest, because it looks like it's parallelizing on the level on manifests. |
|
Good idea setting the number of threads. It could easily be that you're processing too many manifests in parallel because of the number of threads. |
|
Let me clarify our use case. We have an iceberg table partitioned by date, and we run an ETL job every day to sync data from an upstream hive table into this iceberg table. The ETL job basically just runs an I'll check whether |
|
@szehon-ho Do you have an example in mind where blocking queue can hurt performance? IMHO, if a blocking queue blocks the producers, it usually means the consumer is not fast enough, in which case the bottleneck is the consumer, rather than the queue itself. |
|
@lirui-apache, another option here is to add the ability to set parallelism on a per-table basis. Basically to prevent the parallel iterator from submitting so many tasks at once. That may help your situation. That said, I still think the main problem is that your partitions have 50-130k files. Have you tried compacting those? |
|
@rdblue Thanks for the advice. Actually it's not a small-file problem. Each partition has over 200 billion records. We do have optimizations to make sure each query only scans a small portion. But it cannot help if we hit OOM at the planning phase. I also noticed latest code supports planning files with separate pools. So with separate pools + limited manifest entries + limited pool size, I think we can bring the memory usage under control. Although personally I still prefer the blocking queue solution, which seems easier to achieve and more reliable. |
|
@lirui-apache, the files should be added to the queue after being filtered. Won't you need to hold all these files in memory at some point anyway? Or are you avoiding that somehow? |
|
Yea , good idea to adding plan parallelism per table if not there already (it might be in Flink but not Spark) Yea, also curious, in Spark it seems the FileScanTask iterator in the end gets into a concrete array, though I guess its possible in custom code to consume the FileScanTask directly in streaming fashion |
Even for streaming ingest, there are ways to make the small files problem better. For example, if you're ingesting with Flink, you can increase the time between commits. Or you can also play with But if files are only 1MB, even if there's a large amount of data per partition, that is imho by definition going to encounter many of the common problems that encompass the "small files problem". |
|
Also, for compacting small files (as well as avoiding small files in ingest), I just wrote a somewhat in depth summary in this issue that might be relevant to your streaming ingest (and data file rewrite operations): #4875 |
|
@rdblue We implemented file-level index for iceberg and use the index to determine whether a file satisfies the query predicate. Although the index file is much smaller than the data file, we wouldn't want to load them in trino coordinator. So the filtering happens on worker nodes. Trino coordinator schedules the file scan tasks in a streaming fashion, so it doesn't have to hold all of them in memory (it does such thing during query optimization though, which we are trying to avoid). |
|
Actually, I'd prefer to give my +1 to the bound queue solution. Because:
|
|
Agree with @openinx. also +1 to the bound queue solution. I was in the same predicament. |
IIUC, |
|
I do see the points you raise and have admittedly used a BlockingQueue in similar situations in large scale streaming ETL in the past (thinking in terms of manifests in the sort of “envelope” sense that is manifest lists and even just overall snapshot change set). About 10k as the size per worker served me well (though would defer to others as in this sense I was streaming kinesis shards, which aren’t entirely unlike a manifest list given potentially variable size and need to rate limit). Can we introduce a configuration parameter, with a blocking queue size? If we give the parameter a negative value, ideally the user would not use a queue but would keep current behavior (similar to This is also not unlike Would this be achievable? |
+1, this is exactly how we implemented internally. |
Forgive me if I missed this as I’m just coming back to it, but what happens if you lower the core count in iceberg.sc.plan-in-worker-pool via iceberg.worker.num-threads? I’ve heard of other issues with machines with very large number of cores. Maybe, possibly in addition to this, we might consider a configurable max limit and default it to something sane? The other report was from a Spark driver iirc. |
| } | ||
|
|
||
| public static int getInt(String systemProperty, int defaultValue) { | ||
| Preconditions.checkNotNull(systemProperty, "System property name should not be null"); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
General but / FYI - we almost always tend to prefer plain English language with a structure similar to “general type of problem / problem phrase [possible solution only if not clear from stack trace]: offending value”.
we put the bad value at the end after a colon to give us a more continuous search string for logs (instead of mixing them into the sentence).
So for this situation, the suggested sentence would be format(“Invalid value for system property %s: null”, systemProperty);
However for this we can be more specific as null here means missing / not set almost always (and anybody who set it to null will see that). So I’d suggest format(“Invalid value for system property %s: null”, systemProperty); The final : null is debatable but I’d personally put it just to cover any cases where it might be explicitly null and just to match many others preconditions.
Yours is pretty good, but this tends to be our standard. I think it’s most helpful thinking of it in the context of the NPE or IllegalArumentException and full stack trace. What’s invariably helpful is having a common-ish way of writing these with a long enough, specific enough stack trace to be searchable, with less and less specific search query as the phrase is shortened.
Hope that helps for the long term! It really is just a nit but if you’re working on Iceberg offen enough it might help you as a contributor and a user.
| Preconditions.checkNotNull(systemProperty, "System property name should not be null"); | |
| Preconditions.checkNotNull(systemProperty, String.format("Invalid value for system property %s: null", systemProperty)); |
kbendick
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think the better solution is to possibly cap the worker pool size on these 96 core (or very massive number of core) machines. I've heard of other issues, and maybe at a certain point it's best to use 3/4th of the number of threads for this as a ceiling which might resolve general issues with input and output without having to add the overhead of bounded queues in multiple places for a relatively less common configuration - most often I'm using more, smaller machines. But I have worked with very, very large machines before and I understand it's a special kind of situation as performance variances can cause problems.
But I do think we should consider something, though switching the queue out only when requested seems reasonable for now as these very many cored machines (128, 96, etc) are generally not the norm. But they do benefit from restricting work sometimes as differences in performance become more apparent at scale.
|
Also @uncleGen, You might be interested in these two PRs which might help your use case (increase output speed of queue): - #4911 (merged in master and will be released in 0.14.0)
|
|
I also want to mention that if |
|
@lirui-apache, do you want to open a PR for that fix? |
|
@rdblue The issue I mentioned is only critical with blocking queue. Because the producer thread can block forever and if the pool is full of such threads, no new tasks can be run. If we have reached consensus to use blocking queue, I can submit a PR for it. |


In my scene, there are hundreds of thousands of datafiles. If
iceberg.scan.plan-in-worker-poolis enabled, OOM exception happend continually. The root cause is an unbounded queueConcurrentLinkedQueueis used inParallelIterator. The queue will consume much memory.