This is an automated email from the ASF dual-hosted git repository.

difin pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
     new 4b1b2095a36 HIVE-29389: HiveProtoLoggingHook populates duplicate 
entries in TablesRead metrics for tables with multiple partitions (#6253)
4b1b2095a36 is described below

commit 4b1b2095a36392db0d563bdb5263c0ad8cae78db
Author: Tanishq Chugh <[email protected]>
AuthorDate: Thu Jan 1 00:13:22 2026 +0530

    HIVE-29389: HiveProtoLoggingHook populates duplicate entries in TablesRead 
metrics for tables with multiple partitions (#6253)
---
 .../java/org/apache/hadoop/hive/ql/hooks/HiveProtoLoggingHook.java   | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)

diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/hooks/HiveProtoLoggingHook.java 
b/ql/src/java/org/apache/hadoop/hive/ql/hooks/HiveProtoLoggingHook.java
index b426d3011b0..a24f93bf4b8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/hooks/HiveProtoLoggingHook.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/HiveProtoLoggingHook.java
@@ -92,6 +92,7 @@
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -489,13 +490,13 @@ private String getQueueName(ExecutionMode mode, HiveConf 
conf) {
     }
 
     private List<String> getTablesFromEntitySet(Set<? extends Entity> 
entities) {
-      List<String> tableNames = new ArrayList<>();
+      Set<String> tableNames = new HashSet<>();
       for (Entity entity : entities) {
         if (entity.getType() == TABLE || entity.getType() == PARTITION) {
           tableNames.add(entity.getTable().getDbName() + "." + 
entity.getTable().getTableName());
         }
       }
-      return tableNames;
+      return new ArrayList<>(tableNames);
     }
 
     private ExecutionMode getExecutionMode(QueryPlan plan, List<ExecDriver> 
mrTasks,

Reply via email to