abstractdog commented on PR #12427: URL: https://github.com/apache/iceberg/pull/12427#issuecomment-2691177743
> Hi Laci, Long time, no see... What is the actual issue where you have find this problematic? Hey, long time no see Peter! the actual use-case was [HIVE-28759](https://issues.apache.org/jira/browse/HIVE-28759), where I'm about to supply an executorService to SnapshotProducer to be able to submit some jobs during HS2 shutdown even though the API supports scanManifestsWith, I wasn't able to achive my goal, because the static final variable was initialized in <clinit> when the class is first loaded and used...so this is an edge-case, when: 1. HS2 starts 2. there are some records in the query history queue that need to be flushed (anything that applies 3) ) 3. no iceberg operations using ThreadPools were running 4. HS2 shutdown so I got an exception like: ``` Caused by: java.lang.IllegalStateException: Shutdown in progress at java.lang.ApplicationShutdownHooks.add(ApplicationShutdownHooks.java:66) ~[?:1.8.0_292] at java.lang.Runtime.addShutdownHook(Runtime.java:211) ~[?:1.8.0_292] at org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors$Application.addShutdownHook(MoreExecutors.java:289) ~[hive-iceberg-handler-3.1.3000.2025.0.20.0-SNAPSHOT.jar:3.1.3000.2025.0.20.0-SNAPSHOT] at org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors$Application.addDelayedShutdownHook(MoreExecutors.java:266) ~[hive-iceberg-handler-3.1.3000.2025.0.20.0-SNAPSHOT.jar:3.1.3000.2025.0.20.0-SNAPSHOT] at org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors$Application.getExitingExecutorService(MoreExecutors.java:241) ~[hive-iceberg-handler-3.1.3000.2025.0.20.0-SNAPSHOT.jar:3.1.3000.2025.0.20.0-SNAPSHOT] at org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors$Application.getExitingExecutorService(MoreExecutors.java:246) ~[hive-iceberg-handler-3.1.3000.2025.0.20.0-SNAPSHOT.jar:3.1.3000.2025.0.20.0-SNAPSHOT] at org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors.getExitingExecutorService(MoreExecutors.java:129) ~[hive-iceberg-handler-3.1.3000.2025.0.20.0-SNAPSHOT.jar:3.1.3000.2025.0.20.0-SNAPSHOT] at org.apache.iceberg.util.ThreadPools.newWorkerPool(ThreadPools.java:89) ~[hive-iceberg-handler-3.1.3000.2025.0.20.0-SNAPSHOT.jar:3.1.3000.2025.0.20.0-SNAPSHOT] at org.apache.iceberg.util.ThreadPools.newWorkerPool(ThreadPools.java:85) ~[hive-iceberg-handler-3.1.3000.2025.0.20.0-SNAPSHOT.jar:3.1.3000.2025.0.20.0-SNAPSHOT] at org.apache.iceberg.util.ThreadPools.<clinit>(ThreadPools.java:45) ~[hive-iceberg-handler-3.1.3000.2025.0.20.0-SNAPSHOT.jar:3.1.3000.2025.0.20.0-SNAPSHOT] at org.apache.iceberg.SnapshotProducer.<init>(SnapshotProducer.java:102) ~[hive-iceberg-handler-3.1.3000.2025.0.20.0-SNAPSHOT.jar:3.1.3000.2025.0.20.0-SNAPSHOT] at org.apache.iceberg.MergingSnapshotProducer.<init>(MergingSnapshotProducer.java:103) ~[hive-iceberg-handler-3.1.3000.2025.0.20.0-SNAPSHOT.jar:3.1.3000.2025.0.20.0-SNAPSHOT] at org.apache.iceberg.MergeAppend.<init>(MergeAppend.java:32) ~[hive-iceberg-handler-3.1.3000.2025.0.20.0-SNAPSHOT.jar:3.1.3000.2025.0.20.0-SNAPSHOT] at org.apache.iceberg.BaseTable.newAppend(BaseTable.java:180) ~[hive-iceberg-handler-3.1.3000.2025.0.20.0-SNAPSHOT.jar:3.1.3000.2025.0.20.0-SNAPSHOT] at org.apache.iceberg.mr.hive.HiveIcebergOutputCommitter.commitWrite(HiveIcebergOutputCommitter.java:555) ~[hive-iceberg-handler-3.1.3000.2025.0.20.0-SNAPSHOT.jar:3.1.3000.2025.0.20.0-SNAPSHOT] at org.apache.iceberg.mr.hive.HiveIcebergOutputCommitter.commitTable(HiveIcebergOutputCommitter.java:494) ~[hive-iceberg-handler-3.1.3000.2025.0.20.0-SNAPSHOT.jar:3.1.3000.2025.0.20.0-SNAPSHOT] at org.apache.iceberg.mr.hive.HiveIcebergOutputCommitter.lambda$commitJobs$4(HiveIcebergOutputCommitter.java:292) ~[hive-iceberg-handler-3.1.3000.2025.0.20.0-SNAPSHOT.jar:3.1.3000.2025.0.20.0-SNAPSHOT] at org.apache.iceberg.util.Tasks$Builder.runTaskWithRetry(Tasks.java:413) ~[hive-iceberg-handler-3.1.3000.2025.0.20.0-SNAPSHOT.jar:3.1.3000.2025.0.20.0-SNAPSHOT] at org.apache.iceberg.util.Tasks$Builder.runSingleThreaded(Tasks.java:219) ~[hive-iceberg-handler-3.1.3000.2025.0.20.0-SNAPSHOT.jar:3.1.3000.2025.0.20.0-SNAPSHOT] at org.apache.iceberg.util.Tasks$Builder.run(Tasks.java:203) ~[hive-iceberg-handler-3.1.3000.2025.0.20.0-SNAPSHOT.jar:3.1.3000.2025.0.20.0-SNAPSHOT] at org.apache.iceberg.util.Tasks$Builder.run(Tasks.java:196) ~[hive-iceberg-handler-3.1.3000.2025.0.20.0-SNAPSHOT.jar:3.1.3000.2025.0.20.0-SNAPSHOT] at org.apache.iceberg.mr.hive.HiveIcebergOutputCommitter.commitJobs(HiveIcebergOutputCommitter.java:286) ~[hive-iceberg-handler-3.1.3000.2025.0.20.0-SNAPSHOT.jar:3.1.3000.2025.0.20.0-SNAPSHOT] at org.apache.iceberg.mr.hive.HiveIcebergStorageHandler.storageHandlerCommit(HiveIcebergStorageHandler.java:828) ~[hive-iceberg-handler-3.1.3000.2025.0.20.0-SNAPSHOT.jar:3.1.3000.2025.0.20.0-SNAPSHOT] at org.apache.hadoop.hive.ql.queryhistory.repository.IcebergRepository.flush(IcebergRepository.java:134) ~[hive-exec-3.1.3000.2025.0.20.0-SNAPSHOT.jar:3.1.3000.2025.0.20.0-SNAPSHOT] at org.apache.hadoop.hive.ql.queryhistory.QueryHistoryService.lambda$doFlush$1(QueryHistoryService.java:205) ~[hive-exec-3.1.3000.2025.0.20.0-SNAPSHOT.jar:3.1.3000.2025.0.20.0-SNAPSHOT] at java.util.concurrent.CompletableFuture$AsyncRun.run(CompletableFuture.java:1640) ~[?:1.8.0_292] ... 4 more ``` even this is just an edge-case, the lazy init might look better in general -- 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.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org