stevenzwu commented on code in PR #16108:
URL: https://github.com/apache/iceberg/pull/16108#discussion_r3268981410
##########
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:
Two concrete behaviours worth either calling out in the precondition message
or covering with negative tests:
1. **`Executors.newSingleThreadExecutor()` is rejected.** It returns
`Executors$AutoShutdownDelegatedExecutorService` (a wrapper around a 1-thread
`ThreadPoolExecutor`), so `instanceof ThreadPoolExecutor` is `false`. A caller
would get `Unsupported executor type:
java.util.concurrent.Executors$AutoShutdownDelegatedExecutorService` and have
to know to switch to `newFixedThreadPool(1)`. The existing tests use
`newFixedThreadPool(1)`, which papers over this asymmetry.
2. **`Executors.newCachedThreadPool()` is accepted but defeats the layout
invariant.** Its `getMaximumPoolSize()` is `Integer.MAX_VALUE`, so
`manifestWriterCount` collapses to `ceil(fileCount / MIN_FILE_GROUP_SIZE)` —
parallelism is no longer bounded by the supplied pool. The whole reason for the
strict `ThreadPoolExecutor` check is to keep the writer count predictable, but
this is the case where it isn't.
Mentioning the supported shape in the precondition message (e.g. "expected a
fixed-size ThreadPoolExecutor") and adding negative tests for both cases would
make the contract self-documenting for the next maintainer.
##########
api/src/main/java/org/apache/iceberg/SnapshotUpdate.java:
##########
@@ -60,6 +60,18 @@ public interface SnapshotUpdate<ThisT> extends
PendingUpdate<Snapshot> {
*/
ThisT scanManifestsWith(ExecutorService executorService);
+ /**
+ * Use a particular executor to write manifests during commit. The default
worker pool will be
+ * used by default.
+ *
+ * @param executorService the provided executor
+ * @return this for method chaining
+ */
+ default ThisT commitManifestsWith(ExecutorService executorService) {
Review Comment:
Minor contract note: the interface signature here accepts any
`ExecutorService`, but the only shipping implementation
(`SnapshotProducer.commitManifestsWith`) requires a `ThreadPoolExecutor`. A
caller reading just this Javadoc may pass a `ForkJoinPool` or
`Executors.newSingleThreadExecutor()` and only discover the constraint at
runtime.
A single sentence in the Javadoc would close the gap, e.g.:
> Implementations may require a specific executor type (such as
`ThreadPoolExecutor`) to derive parallelism for manifest writes. The default
worker pool will be used otherwise.
--
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]