[ https://issues.apache.org/jira/browse/HIVE-29052?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17987205#comment-17987205 ]
Stamatis Zampetakis commented on HIVE-29052: -------------------------------------------- I didn't understand the connection with the "allModules" and "dist" profile. Can you please elaborate a bit? Even though the core jar contains just module classes, the addition of the new dependency creates build cycles. I mean that in order to build hive-exec module you need standalone-metastore, and if you want to build standalone-metastore you need hive-exec. In addition, from the moment that the dependency of hive-exec is added in the metastore, developers will start using and extending classes from the ql module (and that's expected if we have a dependency to this module) so coupling will only become worse. Maybe I am missing something but even if we don't release a separate tarball for HMS, the users/customers will be able to use HMS IcerbergCatalog support through the main Hive release tarball. I also support the idea of isolating metastore and releasing it apart but it seems that we are not fully ready to do it yet. I am not sure if the HMS folks are following this thread; if they do it would be nice to weight in. The discussion is slightly turning into a debate that is not very productive. I expressed my concerns with the current proposal so at this point I think its better for me to take a step back and let the final call to Denys and Butao who started this work. Just to be clear, I am not offended neither annoyed :) and I am happy to provide further feedback if needed/requested but I will refrain from bringing further counter-arguments about the coupling of metastore with hive-exec. > Error starting standalone HMS: "IcebergHouseKeeperService class not found" > -------------------------------------------------------------------------- > > Key: HIVE-29052 > URL: https://issues.apache.org/jira/browse/HIVE-29052 > Project: Hive > Issue Type: Task > Components: Standalone Metastore > Affects Versions: 4.1.0, 4.2.0 > Reporter: Butao Zhang > Priority: Major > Labels: pull-request-available > > During local standalone HMS deployment testing, the following exception log > was observed: > > {code:java} > 2025-06-28T17:19:28,840 ERROR [Metastore threads starter thread] > leader.LeaseLeaderElection: Error notifying the listener: > org.apache.hadoop.hive.metastore.leader.CompactorTasks@61e0c33d, leader: true > java.lang.ClassNotFoundException: > org.apache.hadoop.hive.ql.txn.compactor.Initiator > at > java.base/jdk.internal.loader.BuiltinClassLoader.loadClass(BuiltinClassLoader.java:641) > ~[?:?] > at > java.base/jdk.internal.loader.ClassLoaders$AppClassLoader.loadClass(ClassLoaders.java:188) > ~[?:?] > at java.base/java.lang.ClassLoader.loadClass(ClassLoader.java:520) > ~[?:?] > at java.base/java.lang.Class.forName0(Native Method) ~[?:?] > at java.base/java.lang.Class.forName(Class.java:375) ~[?:?] > at > org.apache.hadoop.hive.metastore.leader.CompactorTasks.instantiateThread(CompactorTasks.java:53) > ~[hive-standalone-metastore-server-4.1.0.jar:4.1.0] > at > org.apache.hadoop.hive.metastore.leader.CompactorTasks.getCompactorThreads(CompactorTasks.java:67) > ~[hive-standalone-metastore-server-4.1.0.jar:4.1.0] > at > org.apache.hadoop.hive.metastore.leader.CompactorTasks.takeLeadership(CompactorTasks.java:136) > ~[hive-standalone-metastore-server-4.1.0.jar:4.1.0] > at > org.apache.hadoop.hive.metastore.leader.LeaseLeaderElection.lambda$notifyListener$0(LeaseLeaderElection.java:141) > ~[hive-standalone-metastore-server-4.1.0.jar:4.1.0] > at java.base/java.util.ArrayList.forEach(ArrayList.java:1511) ~[?:?] > at > org.apache.hadoop.hive.metastore.leader.LeaseLeaderElection.notifyListener(LeaseLeaderElection.java:138) > ~[hive-standalone-metastore-server-4.1.0.jar:4.1.0] > at > org.apache.hadoop.hive.metastore.leader.LeaseLeaderElection.doWork(LeaseLeaderElection.java:120) > ~[hive-standalone-metastore-server-4.1.0.jar:4.1.0] > at > org.apache.hadoop.hive.metastore.leader.LeaseLeaderElection.tryBeLeader(LeaseLeaderElection.java:181) > ~[hive-standalone-metastore-server-4.1.0.jar:4.1.0] > at > org.apache.hadoop.hive.metastore.leader.LeaseLeaderElection.tryBeLeader(LeaseLeaderElection.java:63) > ~[hive-standalone-metastore-server-4.1.0.jar:4.1.0] > at > org.apache.hadoop.hive.metastore.leader.LeaderElectionContext.lambda$start$2(LeaderElectionContext.java:125) > ~[hive-standalone-metastore-server-4.1.0.jar:4.1.0] > at java.base/java.lang.Thread.run(Thread.java:833) [?:?] > at > org.apache.hadoop.hive.metastore.leader.LeaderElectionContext.start(LeaderElectionContext.java:136) > [hive-standalone-metastore-server-4.1.0.jar:4.1.0] > at > org.apache.hadoop.hive.metastore.HiveMetaStore$8.run(HiveMetaStore.java:855) > [hive-standalone-metastore-server-4.1.0.jar:4.1.0] > 2025-06-28T17:19:28,847 ERROR [Metastore threads starter thread] > leader.LeaseLeaderElection: Error notifying the listener: > org.apache.hadoop.hive.metastore.leader.HouseKeepingTasks@33d3660, leader: > true > org.apache.hadoop.hive.metastore.api.MetaException: > org.apache.iceberg.mr.hive.metastore.task.IcebergHouseKeeperService class not > found > at > org.apache.hadoop.hive.metastore.utils.JavaUtils.getClass(JavaUtils.java:56) > ~[hive-standalone-metastore-common-4.1.0.jar:4.1.0] > at > org.apache.hadoop.hive.metastore.leader.HouseKeepingTasks.getRemoteOnlyTasks(HouseKeepingTasks.java:72) > ~[hive-standalone-metastore-server-4.1.0.jar:4.1.0] > at > org.apache.hadoop.hive.metastore.leader.HouseKeepingTasks.takeLeadership(HouseKeepingTasks.java:112) > ~[hive-standalone-metastore-server-4.1.0.jar:4.1.0] > at > org.apache.hadoop.hive.metastore.leader.LeaseLeaderElection.lambda$notifyListener$0(LeaseLeaderElection.java:141) > ~[hive-standalone-metastore-server-4.1.0.jar:4.1.0] > at java.base/java.util.ArrayList.forEach(ArrayList.java:1511) ~[?:?] > at > org.apache.hadoop.hive.metastore.leader.LeaseLeaderElection.notifyListener(LeaseLeaderElection.java:138) > ~[hive-standalone-metastore-server-4.1.0.jar:4.1.0] > at > org.apache.hadoop.hive.metastore.leader.LeaseLeaderElection.doWork(LeaseLeaderElection.java:120) > ~[hive-standalone-metastore-server-4.1.0.jar:4.1.0] > at > org.apache.hadoop.hive.metastore.leader.LeaseLeaderElection.tryBeLeader(LeaseLeaderElection.java:181) > ~[hive-standalone-metastore-server-4.1.0.jar:4.1.0] > at > org.apache.hadoop.hive.metastore.leader.LeaseLeaderElection.tryBeLeader(LeaseLeaderElection.java:63) > ~[hive-standalone-metastore-server-4.1.0.jar:4.1.0] > at > org.apache.hadoop.hive.metastore.leader.LeaderElectionContext.lambda$start$2(LeaderElectionContext.java:125) > ~[hive-standalone-metastore-server-4.1.0.jar:4.1.0] > at java.base/java.lang.Thread.run(Thread.java:833) [?:?] > at > org.apache.hadoop.hive.metastore.leader.LeaderElectionContext.start(LeaderElectionContext.java:136) > [hive-standalone-metastore-server-4.1.0.jar:4.1.0] > at > org.apache.hadoop.hive.metastore.HiveMetaStore$8.run(HiveMetaStore.java:855) > [hive-standalone-metastore-server-4.1.0.jar:4.1.0] > {code} -- This message was sent by Atlassian Jira (v8.20.10#820010)