findepi commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1688735301
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -20,84 +20,117 @@
import java.io.Closeable;
import java.io.IOException;
+import java.io
findepi commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1688731772
##
core/src/test/java/org/apache/iceberg/util/TestParallelIterable.java:
##
@@ -133,6 +140,47 @@ public CloseableIterator iterator() {
.untilAsserted(() -> a
rdblue commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1687083709
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -20,84 +20,117 @@
import java.io.Closeable;
import java.io.IOException;
+import java.io.
rdblue commented on PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#issuecomment-2243728470
Thanks, @findepi! Good work finding a solution here.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL abov
rdblue merged PR #10691:
URL: https://github.com/apache/iceberg/pull/10691
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: issues-unsubscr...@iceberg.ap
rdblue commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1687077960
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -136,30 +169,33 @@ private boolean checkTasks() {
}
}
- return !close
RussellSpitzer commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1686853441
##
core/src/test/java/org/apache/iceberg/util/TestParallelIterable.java:
##
@@ -133,6 +140,47 @@ public CloseableIterator iterator() {
.untilAsserted(
findepi commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1686847688
##
core/src/test/java/org/apache/iceberg/util/TestParallelIterable.java:
##
@@ -133,6 +140,47 @@ public CloseableIterator iterator() {
.untilAsserted(() -> a
RussellSpitzer commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1686777206
##
core/src/test/java/org/apache/iceberg/util/TestParallelIterable.java:
##
@@ -133,6 +140,47 @@ public CloseableIterator iterator() {
.untilAsserted(
findepi commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1686424317
##
core/src/test/java/org/apache/iceberg/util/TestParallelIterable.java:
##
@@ -133,6 +140,47 @@ public CloseableIterator iterator() {
.untilAsserted(() -> a
findepi commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1686408402
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -20,84 +20,117 @@
import java.io.Closeable;
import java.io.IOException;
+import java.io
findepi commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1686401382
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -136,30 +169,33 @@ private boolean checkTasks() {
}
}
- return !clos
findepi commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1686399659
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -136,30 +169,33 @@ private boolean checkTasks() {
}
}
- return !clos
findepi commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1686399659
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -136,30 +169,33 @@ private boolean checkTasks() {
}
}
- return !clos
stevenzwu commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1685542742
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -20,84 +20,117 @@
import java.io.Closeable;
import java.io.IOException;
+import java.
stevenzwu commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1685542742
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -20,84 +20,117 @@
import java.io.Closeable;
import java.io.IOException;
+import java.
stevenzwu commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1685541998
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -136,30 +169,33 @@ private boolean checkTasks() {
}
}
- return !cl
stevenzwu commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1685540696
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -136,30 +169,33 @@ private boolean checkTasks() {
}
}
- return !cl
stevenzwu commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1685540498
##
core/src/test/java/org/apache/iceberg/util/TestParallelIterable.java:
##
@@ -133,6 +140,47 @@ public CloseableIterator iterator() {
.untilAsserted(() ->
findepi commented on PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#issuecomment-2241150813
@stevenzwu thanks for your review! addressed comments, PTAL
@rdblue i changed the default size limit + some other editorials, PTAL
--
This is an automated message from the Apach
findepi commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1685432875
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -20,65 +20,69 @@
import java.io.Closeable;
import java.io.IOException;
+import java.uti
findepi commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1685431565
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -192,4 +228,78 @@ public synchronized T next() {
return queue.poll();
}
}
+
findepi commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1685431047
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -192,4 +228,78 @@ public synchronized T next() {
return queue.poll();
}
}
+
findepi commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1685430488
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -136,30 +169,33 @@ private boolean checkTasks() {
}
}
- return !clos
findepi commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1685430355
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -192,4 +228,78 @@ public synchronized T next() {
return queue.poll();
}
}
+
findepi commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1685430017
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -20,84 +20,115 @@
import java.io.Closeable;
import java.io.IOException;
+import java.io
findepi commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1685429769
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -107,15 +138,17 @@ public void close() {
*
* @return true if there are pending
findepi commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1685429472
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -136,30 +169,33 @@ private boolean checkTasks() {
}
}
- return !clos
findepi commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1685428633
##
core/src/test/java/org/apache/iceberg/util/TestParallelIterable.java:
##
@@ -133,6 +140,47 @@ public CloseableIterator iterator() {
.untilAsserted(() -> a
findepi commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1685428388
##
core/src/test/java/org/apache/iceberg/util/TestParallelIterable.java:
##
@@ -133,6 +140,47 @@ public CloseableIterator iterator() {
.untilAsserted(() -> a
stevenzwu commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1685247967
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -107,15 +138,17 @@ public void close() {
*
* @return true if there are pendin
RussellSpitzer commented on PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#issuecomment-2239458430
I'm ok with adding a performance testing in another issue, I'll do a full
read throw again this morning.
--
This is an automated message from the Apache Git Service.
To respond
findepi commented on PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#issuecomment-2238785012
thanks @rdblue for your review and approval too!
@RussellSpitzer do you want to take another look?
--
This is an automated message from the Apache Git Service.
To respond to the mes
findepi commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1684130747
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -136,30 +169,33 @@ private boolean checkTasks() {
}
}
- return !clos
rdblue commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1683515066
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -136,30 +169,33 @@ private boolean checkTasks() {
}
}
- return !close
rdblue commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1683506642
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -88,16 +92,26 @@ private ParallelIterator(
@Override
public void close() {
findepi commented on PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#issuecomment-2236165774
> i will address first two now,
addressed. @rdblue PTAL
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use t
findepi commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1682603568
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -20,65 +20,69 @@
import java.io.Closeable;
import java.io.IOException;
+import java.uti
findepi commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1682590569
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -88,16 +92,26 @@ private ParallelIterator(
@Override
public void close() {
findepi commented on PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#issuecomment-2236063813
Thanks @rdblue for your thorough review.
I applied style fixes and outstanding items are
- resuming background tasks earlier
https://github.com/apache/iceberg/pull/10691#discus
findepi commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1682523817
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -88,16 +92,26 @@ private ParallelIterator(
@Override
public void close() {
rdblue commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1681833994
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -192,4 +211,71 @@ public synchronized T next() {
return queue.poll();
}
}
+
+
rdblue commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1681833492
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -192,4 +211,71 @@ public synchronized T next() {
return queue.poll();
}
}
+
+
rdblue commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1681826881
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -88,16 +92,26 @@ private ParallelIterator(
@Override
public void close() {
rdblue commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1681826881
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -88,16 +92,26 @@ private ParallelIterator(
@Override
public void close() {
rdblue commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1681827634
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -20,65 +20,69 @@
import java.io.Closeable;
import java.io.IOException;
+import java.util
rdblue commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1681826881
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -88,16 +92,26 @@ private ParallelIterator(
@Override
public void close() {
findepi closed pull request #10691: Core: Limit memory used by ParallelIterable
URL: https://github.com/apache/iceberg/pull/10691
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
findepi commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1678930650
##
core/src/test/java/org/apache/iceberg/util/TestParallelIterable.java:
##
@@ -133,6 +140,47 @@ public CloseableIterator iterator() {
.untilAsserted(() -> a
findepi commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1678864255
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -136,19 +151,25 @@ private boolean checkTasks() {
}
}
- return !clos
findepi commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1678859403
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -192,4 +209,65 @@ public synchronized T next() {
return queue.poll();
}
}
+
dekimir commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1678458408
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -192,4 +209,65 @@ public synchronized T next() {
return queue.poll();
}
}
+
dekimir commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1678458408
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -192,4 +209,65 @@ public synchronized T next() {
return queue.poll();
}
}
+
RussellSpitzer commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1678296088
##
core/src/test/java/org/apache/iceberg/util/TestParallelIterable.java:
##
@@ -133,6 +140,47 @@ public CloseableIterator iterator() {
.untilAsserted(
RussellSpitzer commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1678282002
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -136,19 +151,25 @@ private boolean checkTasks() {
}
}
- retur
findinpath commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1677791198
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -192,4 +209,65 @@ public synchronized T next() {
return queue.poll();
}
}
findepi commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1677661087
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -192,4 +209,65 @@ public synchronized T next() {
return queue.poll();
}
}
+
findepi commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1677660806
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -192,4 +209,65 @@ public synchronized T next() {
return queue.poll();
}
}
+
findepi commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1677654176
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -20,65 +20,69 @@
import java.io.Closeable;
import java.io.IOException;
+import java.uti
findepi commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1677652693
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -20,65 +20,69 @@
import java.io.Closeable;
import java.io.IOException;
+import java.uti
findepi commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1677649901
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -192,4 +209,65 @@ public synchronized T next() {
return queue.poll();
}
}
+
findepi commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1677648977
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -20,65 +20,69 @@
import java.io.Closeable;
import java.io.IOException;
+import java.uti
findepi commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1677634825
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -88,16 +92,26 @@ private ParallelIterator(
@Override
public void close() {
findinpath commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1677575346
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -20,65 +20,69 @@
import java.io.Closeable;
import java.io.IOException;
+import java.
findinpath commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1677572633
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -192,4 +209,65 @@ public synchronized T next() {
return queue.poll();
}
}
rdblue commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1677182372
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -20,65 +20,69 @@
import java.io.Closeable;
import java.io.IOException;
+import java.util
rdblue commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1677181515
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -126,6 +141,7 @@ private boolean checkTasks() {
} catch (InterruptedException e
rdblue commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1677181134
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -192,4 +209,65 @@ public synchronized T next() {
return queue.poll();
}
}
+
+
rdblue commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1677181095
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -192,4 +209,65 @@ public synchronized T next() {
return queue.poll();
}
}
+
+
rdblue commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1677179876
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -136,19 +152,20 @@ private boolean checkTasks() {
}
}
- return !close
rdblue commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1677179126
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -192,4 +209,65 @@ public synchronized T next() {
return queue.poll();
}
}
+
+
rdblue commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1677178824
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -88,16 +92,26 @@ private ParallelIterator(
@Override
public void close() {
findepi commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1676163120
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -192,4 +209,65 @@ public synchronized T next() {
return queue.poll();
}
}
+
findepi commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1676170877
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -88,7 +91,18 @@ private ParallelIterator(
@Override
public void close() {
findepi commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1676163120
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -192,4 +209,65 @@ public synchronized T next() {
return queue.poll();
}
}
+
findepi commented on PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#issuecomment-2225908686
@stevenzwu thanks for your comments!
> Curious if you have done any performance testing. echo to another comment.
wondering if the default queue size of 10K would affect the thro
findepi commented on PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#issuecomment-2225902783
> > Can't the caller set a lower limit then, by calling the new constructor
overload?
>
> Yes, that's possible but then you already have to inherit quite a few
classes to overloa
stevenzwu commented on PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#issuecomment-2225793651
@findepi thanks for working on this. we also ran into the memory issue
internally when some manifest files are super large (like hundreds of MBs or
GBs).
Curious if you have d
stevenzwu commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1676065032
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -192,4 +209,65 @@ public synchronized T next() {
return queue.poll();
}
}
stevenzwu commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1676065032
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -192,4 +209,65 @@ public synchronized T next() {
return queue.poll();
}
}
stevenzwu commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1676050363
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -88,7 +91,18 @@ private ParallelIterator(
@Override
public void close() {
Fokko commented on PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#issuecomment-2225760543
> Can't the caller set a lower limit then, by calling the new constructor
overload?
Yes, that's possible but then you already have to inherit quite a few
classes to overload certai
dekimir commented on PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#issuecomment-2225671754
> on the other hand, if the limit is too high, it doesn't help either.
Can't the caller set a lower limit then, by calling the new constructor
overload?
--
This is an automate
Fokko commented on PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#issuecomment-2225662200
> If the consumer of `ParallelIterable` is fast enough, then this limit
should have no impact, right?
Yes, I agree, but on the other hand, if the limit is too high, it doesn't
help
dekimir commented on PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#issuecomment-2225641596
> My thinking here is that we bound the queue. The `ParallelIterable` is
often used at places where it is IO intensive. This will limit the parallelism
of calls to the object stores and
Fokko commented on PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#issuecomment-2225632947
My thinking here is that we bound the queue. The `ParallelIterable` is often
used at places where it is IO intensive. This will limit the parallelism of
calls to the object stores and tha
findepi commented on PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#issuecomment-2225622443
@Fokko fair. Also, note this is not a performance improvement or something.
It's "just" bounding memory usage to prevent OOM. As such, I wound't expect
this change to require benchmark
findepi commented on PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#issuecomment-2225620332
@Heltman @losipiuk @alexjo2144 you might want to take a look
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use th
Fokko commented on PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#issuecomment-2225619730
I don't have enough knowledge of this piece of this piece of code to merge
this without any benchmarks or profiling. Maybe @rdblue?
--
This is an automated message from the Apache Git S
findepi commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1675927724
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -88,7 +91,18 @@ private ParallelIterator(
@Override
public void close() {
findepi commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1675922119
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -20,65 +20,68 @@
import java.io.Closeable;
import java.io.IOException;
+import java.uti
findepi commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1675920451
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -20,65 +20,68 @@
import java.io.Closeable;
import java.io.IOException;
+import java.uti
dekimir commented on code in PR #10691:
URL: https://github.com/apache/iceberg/pull/10691#discussion_r1675856578
##
core/src/main/java/org/apache/iceberg/util/ParallelIterable.java:
##
@@ -20,65 +20,68 @@
import java.io.Closeable;
import java.io.IOException;
+import java.uti
93 matches
Mail list logo