dramaticlly commented on code in PR #16108:
URL: https://github.com/apache/iceberg/pull/16108#discussion_r3237156267


##########
core/src/main/java/org/apache/iceberg/SnapshotProducer.java:
##########
@@ -164,6 +164,19 @@ public ThisT scanManifestsWith(ExecutorService 
executorService) {
     return self();
   }
 
+  @Override
+  public ThisT commitManifestsWith(ExecutorService executorService) {
+    Preconditions.checkArgument(

Review Comment:
   Thanks Huaxin, the reason on why we require ThreadPoolExecutor for commit is 
that poolSize determines the manifest physical layout 
https://github.com/apache/iceberg/blob/c72c2bdc48e308c8f118f55f9b5c9e053f886894/core/src/main/java/org/apache/iceberg/SnapshotProducer.java#L826-L829.
 For scan, the pool only affects how fast we read; for commit, it changes how 
manifests gets persisted on the disk and how many manifests recorded in the 
manifest list.
   
   A concrete example: say a snapshot is appending 100k entries, 
`ThreadPools.WORKER_THREAD_POOL_SIZE` defaults to the number of available cores 
(8), and the user passes in an ExecutorService with pool size 2.
   - Current PR: manifestWriterCount(2, 100_000) = min(2, 10) = 2 writers → 2 
manifests of ~50k entries each.
   - Fallback to WORKER_THREAD_POOL_SIZE: manifestWriterCount(8, 100_000) = 
min(8, 10) = 8 writers → 8 manifests of ~12.5k entries each.
   
   Require the ThreadPoolExecutor and fail early might be more transparent and 
keeps the signal explicit. WDYT?



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to