abstractdog commented on code in PR #5642:
URL: https://github.com/apache/hive/pull/5642#discussion_r1959999285
##########
ql/src/java/org/apache/hadoop/hive/ql/queryhistory/repository/IcebergRepository.java:
##########
@@ -65,6 +71,43 @@ public class IcebergRepository extends AbstractRepository
implements QueryHistor
@VisibleForTesting
TableDesc tableDesc;
+ public void init(HiveConf conf, Schema schema) {
+ super.init(conf, schema);
+ try {
+ overrideIcebergWorkerPool();
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ /**
+ * This is a workaround for Iceberg's exiting executor services, as
described in HIVE-28759.
+ * This method replaces the original executor service with a new one that is
not immediately shut down by the JVM's
+ * shutdown hook, allowing the query history service to flush records from
its queue successfully.
+ * The new executor service retains the thread settings of the original one.
+ */
+ private void overrideIcebergWorkerPool() throws Exception {
+ Field field =
Class.forName("org.apache.iceberg.util.ThreadPools").getDeclaredField("WORKER_POOL");
+ field.setAccessible(true);
+
+ Field modifiersField = Field.class.getDeclaredField("modifiers");
+ modifiersField.setAccessible(true);
+ modifiersField.setInt(field, field.getModifiers() & ~Modifier.FINAL);
+
+ int poolSize = getIcebergWorkerPoolSize();
+ LOG.info("Overriding iceberg worker pool, size: {}", poolSize);
+ field.set(null, Executors.newFixedThreadPool(poolSize,
Review Comment:
totally understand your concern
do you think this is something to work on the iceberg side? I mean, iceberg
worker pool allows tasks to finish 120s after the shutdown, but doesn't let any
tasks be submitted, which is very unfortunate
so the problem is theoretically with the hack itself or the fact I'm trying
to submit a task in a shutdown hook in hs2?
--
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]