You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by kg...@apache.org on 2020/04/23 15:25:45 UTC

[hive] branch master updated (9299512 -> 014dafc)

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

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


    from 9299512  HIVE-23103: Oracle statement batching (Peter Vary reviewed by Marton Bod, Denys Kuzmenko)
     new 65dc6ca  HIVE-23220: PostExecOrcFileDump listing order may depend on the underlying filesystem (Zoltan Haindrich reviewed by Miklos Gergely)
     new 3cadd2a  HIVE-23164: Server is not properly terminated because of non-daemon threads (Eugene Chung via Zoltan Haindrich)
     new 014dafc  HIVE-23088: Using Strings from log4j breaks non-log4j users (David Lavati via Panagiotis Garefalakis, Zoltan Haindrich)

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../apache/hadoop/hive/ql/QTestMiniClusters.java   |   4 +-
 .../apache/hadoop/hive/ql/QTestRunnerUtils.java    |   6 +-
 .../hadoop/hive/llap/log/LlapWrappedAppender.java  |   1 -
 .../exec/tez/PerPoolTriggerValidatorRunnable.java  |   6 +-
 .../org/apache/hadoop/hive/ql/hooks/HookUtils.java |   4 +-
 .../hadoop/hive/ql/hooks/PostExecOrcFileDump.java  |   3 +
 .../llap/acid_bloom_filter_orc_file_dump.q.out     | 156 ++++++++++-----------
 .../hadoop/hive/metastore/HiveMetaStore.java       |   1 +
 .../apache/hadoop/hive/metastore/ThreadPool.java   |   8 +-
 9 files changed, 100 insertions(+), 89 deletions(-)


[hive] 02/03: HIVE-23164: Server is not properly terminated because of non-daemon threads (Eugene Chung via Zoltan Haindrich)

Posted by kg...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 3cadd2ac61b08b81390907f4b4380396e3a99ad5
Author: Eugene Chung <eu...@navercorp.com>
AuthorDate: Thu Apr 23 14:35:47 2020 +0000

    HIVE-23164: Server is not properly terminated because of non-daemon threads (Eugene Chung via Zoltan Haindrich)
    
    Signed-off-by: Zoltan Haindrich <zh...@cloudera.com>
---
 .../hadoop/hive/ql/exec/tez/PerPoolTriggerValidatorRunnable.java  | 6 +++++-
 .../main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java | 1 +
 .../main/java/org/apache/hadoop/hive/metastore/ThreadPool.java    | 8 ++++++--
 3 files changed, 12 insertions(+), 3 deletions(-)

diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/PerPoolTriggerValidatorRunnable.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/PerPoolTriggerValidatorRunnable.java
index 8f29197..14a688e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/PerPoolTriggerValidatorRunnable.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/PerPoolTriggerValidatorRunnable.java
@@ -20,8 +20,10 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.TimeUnit;
 
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.hadoop.hive.ql.wm.SessionTriggerProvider;
 import org.apache.hadoop.hive.ql.wm.TriggerActionHandler;
 import org.slf4j.Logger;
@@ -46,8 +48,10 @@ public class PerPoolTriggerValidatorRunnable implements Runnable {
   @Override
   public void run() {
     try {
+      ThreadFactory threadFactory = new ThreadFactoryBuilder().setDaemon(true)
+          .setNameFormat("PoolValidator %d").build();
       ScheduledExecutorService validatorExecutorService = Executors
-        .newScheduledThreadPool(sessionTriggerProviders.size());
+          .newScheduledThreadPool(sessionTriggerProviders.size(), threadFactory);
       for (Map.Entry<String, SessionTriggerProvider> entry : sessionTriggerProviders.entrySet()) {
         String poolName = entry.getKey();
         if (!poolValidators.containsKey(poolName)) {
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index 77d3404..32494ae 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -1115,6 +1115,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     public void shutdown() {
       cleanupRawStore();
       PerfLogger.getPerfLogger(false).cleanupPerfLogMetrics();
+      ThreadPool.shutdown();
     }
 
     @Override
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ThreadPool.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ThreadPool.java
index d0fcd25..5dca2b3 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ThreadPool.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ThreadPool.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hive.metastore;
 
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.slf4j.Logger;
@@ -24,6 +25,7 @@ import org.slf4j.LoggerFactory;
 
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadFactory;
 
 /**
  * Utility singleton class to manage all the threads.
@@ -31,7 +33,7 @@ import java.util.concurrent.ScheduledExecutorService;
 public class ThreadPool {
 
   static final private Logger LOG = LoggerFactory.getLogger(ThreadPool.class);
-  private static ThreadPool self = null;
+  private static ThreadPool self;
   private static ScheduledExecutorService pool;
 
   public static synchronized ThreadPool initialize(Configuration conf) {
@@ -43,8 +45,10 @@ public class ThreadPool {
   }
 
   private ThreadPool(Configuration conf) {
+    ThreadFactory threadFactory = new ThreadFactoryBuilder().setDaemon(true)
+        .setNameFormat("Metastore Scheduled Worker %d").build();
     pool = Executors.newScheduledThreadPool(MetastoreConf.getIntVar(conf,
-        MetastoreConf.ConfVars.THREAD_POOL_SIZE));
+        MetastoreConf.ConfVars.THREAD_POOL_SIZE), threadFactory);
   }
 
   public static ScheduledExecutorService getPool() {


[hive] 03/03: HIVE-23088: Using Strings from log4j breaks non-log4j users (David Lavati via Panagiotis Garefalakis, Zoltan Haindrich)

Posted by kg...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 014dafcd7ac4260f7038f969d7c8218682029b86
Author: David Lavati <da...@gmail.com>
AuthorDate: Thu Apr 23 14:37:42 2020 +0000

    HIVE-23088: Using Strings from log4j breaks non-log4j users (David Lavati via Panagiotis Garefalakis, Zoltan Haindrich)
    
    Signed-off-by: Zoltan Haindrich <zh...@cloudera.com>
---
 .../src/main/java/org/apache/hadoop/hive/ql/QTestMiniClusters.java  | 4 ++--
 .../src/main/java/org/apache/hadoop/hive/ql/QTestRunnerUtils.java   | 6 +++---
 .../java/org/apache/hadoop/hive/llap/log/LlapWrappedAppender.java   | 1 -
 ql/src/java/org/apache/hadoop/hive/ql/hooks/HookUtils.java          | 4 ++--
 4 files changed, 7 insertions(+), 8 deletions(-)

diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestMiniClusters.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestMiniClusters.java
index 997b35e..46e2f64 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestMiniClusters.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestMiniClusters.java
@@ -37,6 +37,7 @@ import org.apache.avro.io.BinaryEncoder;
 import org.apache.avro.io.DatumWriter;
 import org.apache.avro.io.EncoderFactory;
 import org.apache.avro.specific.SpecificDatumWriter;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystem;
@@ -62,7 +63,6 @@ import org.apache.hive.druid.MiniDruidCluster;
 import org.apache.hive.kafka.SingleNodeKafkaCluster;
 import org.apache.hive.kafka.Wikipedia;
 import org.apache.hive.testutils.MiniZooKeeperCluster;
-import org.apache.logging.log4j.util.Strings;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.ZooKeeper;
@@ -583,7 +583,7 @@ public class QTestMiniClusters {
     Path userInstallPath;
     if (isLocalFs) {
       String buildDir = QTestSystemProperties.getBuildDir();
-      Preconditions.checkState(Strings.isNotBlank(buildDir));
+      Preconditions.checkState(StringUtils.isNotBlank(buildDir));
       Path path = new Path(fsUriString, buildDir);
 
       // Create a fake fs root for local fs
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestRunnerUtils.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestRunnerUtils.java
index 1026195..5fb138d 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestRunnerUtils.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestRunnerUtils.java
@@ -20,8 +20,8 @@ package org.apache.hadoop.hive.ql;
 
 import java.io.File;
 
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.hive.ql.QTestMiniClusters.MiniClusterType;
-import org.apache.logging.log4j.util.Strings;
 
 public class QTestRunnerUtils {
   public static final String DEFAULT_INIT_SCRIPT = "q_test_init.sql";
@@ -104,7 +104,7 @@ public class QTestRunnerUtils {
         StringBuilder builder = new StringBuilder();
         builder.append("Test ").append(qfiles[i].getName())
             .append(" results check failed with error code ").append(result.getReturnCode());
-        if (Strings.isNotEmpty(result.getCapturedOutput())) {
+        if (StringUtils.isNotEmpty(result.getCapturedOutput())) {
           builder.append(" and diff value ").append(result.getCapturedOutput());
         }
         System.err.println(builder.toString());
@@ -155,7 +155,7 @@ public class QTestRunnerUtils {
         StringBuilder builder = new StringBuilder();
         builder.append("Test ").append(qfiles[i].getName())
             .append(" results check failed with error code ").append(result.getReturnCode());
-        if (Strings.isNotEmpty(result.getCapturedOutput())) {
+        if (StringUtils.isNotEmpty(result.getCapturedOutput())) {
           builder.append(" and diff value ").append(result.getCapturedOutput());
         }
         System.err.println(builder.toString());
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/log/LlapWrappedAppender.java b/llap-server/src/java/org/apache/hadoop/hive/llap/log/LlapWrappedAppender.java
index 5cd6005..f35d244 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/log/LlapWrappedAppender.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/log/LlapWrappedAppender.java
@@ -23,7 +23,6 @@ import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.util.concurrent.atomic.AtomicReference;
 
-import com.google.common.base.Preconditions;
 import org.apache.logging.log4j.core.Appender;
 import org.apache.logging.log4j.core.LogEvent;
 import org.apache.logging.log4j.core.appender.AbstractAppender;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/HookUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/HookUtils.java
index 0841d67..58e95e1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/hooks/HookUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/HookUtils.java
@@ -21,10 +21,10 @@ package org.apache.hadoop.hive.ql.hooks;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.ql.exec.Utilities;
-import org.apache.logging.log4j.util.Strings;
 
 public class HookUtils {
 
@@ -47,7 +47,7 @@ public class HookUtils {
       throws InstantiationException, IllegalAccessException, ClassNotFoundException {
     String csHooks = conf.getVar(hookConfVar);
     List<T> hooks = new ArrayList<>();
-    if (Strings.isBlank(csHooks)) {
+    if (StringUtils.isBlank(csHooks)) {
       return hooks;
     }
     String[] hookClasses = csHooks.split(",");


[hive] 01/03: HIVE-23220: PostExecOrcFileDump listing order may depend on the underlying filesystem (Zoltan Haindrich reviewed by Miklos Gergely)

Posted by kg...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 65dc6cab9544badfb9a117d2a4ce9b8f5e0864f5
Author: Zoltan Haindrich <ki...@rxd.hu>
AuthorDate: Thu Apr 23 14:35:43 2020 +0000

    HIVE-23220: PostExecOrcFileDump listing order may depend on the underlying filesystem (Zoltan Haindrich reviewed by Miklos Gergely)
    
    Signed-off-by: Zoltan Haindrich <zh...@cloudera.com>
---
 .../hadoop/hive/ql/hooks/PostExecOrcFileDump.java  |   3 +
 .../llap/acid_bloom_filter_orc_file_dump.q.out     | 156 ++++++++++-----------
 2 files changed, 81 insertions(+), 78 deletions(-)

diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/PostExecOrcFileDump.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/PostExecOrcFileDump.java
index 87c3db2..ecda606 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/hooks/PostExecOrcFileDump.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/PostExecOrcFileDump.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hive.ql.hooks;
 
 import java.io.IOException;
 import java.io.PrintStream;
+import java.util.Collections;
 import java.util.List;
 
 import org.slf4j.Logger;
@@ -101,6 +102,8 @@ public class PostExecOrcFileDump implements ExecuteWithHookContext {
     List<FileStatus> fileList = HdfsUtils.listLocatedStatus(fs, dir,
         hiddenFileFilter);
 
+    Collections.sort(fileList);
+
     for (FileStatus fileStatus : fileList) {
       if (fileStatus.isDirectory()) {
 
diff --git a/ql/src/test/results/clientpositive/llap/acid_bloom_filter_orc_file_dump.q.out b/ql/src/test/results/clientpositive/llap/acid_bloom_filter_orc_file_dump.q.out
index da805b0..28fccd6 100644
--- a/ql/src/test/results/clientpositive/llap/acid_bloom_filter_orc_file_dump.q.out
+++ b/ql/src/test/results/clientpositive/llap/acid_bloom_filter_orc_file_dump.q.out
@@ -87,31 +87,31 @@ Stripe Statistics:
   Stripe 1:
     Column 0: count: 1 hasNull: false
     Column 1: count: 1 hasNull: false bytesOnDisk: 6 min: 0 max: 0 sum: 0
-    Column 2: count: 1 hasNull: false bytesOnDisk: 6 min: 2 max: 2 sum: 2
+    Column 2: count: 1 hasNull: false bytesOnDisk: 6 min: 1 max: 1 sum: 1
     Column 3: count: 1 hasNull: false bytesOnDisk: 9 min: 536870912 max: 536870912 sum: 536870912
     Column 4: count: 1 hasNull: false bytesOnDisk: 6 min: 0 max: 0 sum: 0
-    Column 5: count: 1 hasNull: false bytesOnDisk: 6 min: 2 max: 2 sum: 2
+    Column 5: count: 1 hasNull: false bytesOnDisk: 6 min: 1 max: 1 sum: 1
     Column 6: count: 1 hasNull: false
-    Column 7: count: 1 hasNull: false bytesOnDisk: 13 min: 2345 max: 2345 sum: 4
-    Column 8: count: 1 hasNull: false bytesOnDisk: 13 min: 2345 max: 2345 sum: 4
-    Column 9: count: 1 hasNull: false bytesOnDisk: 7 min: 2345 max: 2345 sum: 2345
-    Column 10: count: 1 hasNull: false bytesOnDisk: 7 min: 2345 max: 2345 sum: 2345
+    Column 7: count: 1 hasNull: false bytesOnDisk: 14 min: 12345 max: 12345 sum: 5
+    Column 8: count: 1 hasNull: false bytesOnDisk: 14 min: 12345 max: 12345 sum: 5
+    Column 9: count: 1 hasNull: false bytesOnDisk: 7 min: 12345 max: 12345 sum: 12345
+    Column 10: count: 1 hasNull: false bytesOnDisk: 7 min: 12345 max: 12345 sum: 12345
 
 File Statistics:
   Column 0: count: 1 hasNull: false
   Column 1: count: 1 hasNull: false bytesOnDisk: 6 min: 0 max: 0 sum: 0
-  Column 2: count: 1 hasNull: false bytesOnDisk: 6 min: 2 max: 2 sum: 2
+  Column 2: count: 1 hasNull: false bytesOnDisk: 6 min: 1 max: 1 sum: 1
   Column 3: count: 1 hasNull: false bytesOnDisk: 9 min: 536870912 max: 536870912 sum: 536870912
   Column 4: count: 1 hasNull: false bytesOnDisk: 6 min: 0 max: 0 sum: 0
-  Column 5: count: 1 hasNull: false bytesOnDisk: 6 min: 2 max: 2 sum: 2
+  Column 5: count: 1 hasNull: false bytesOnDisk: 6 min: 1 max: 1 sum: 1
   Column 6: count: 1 hasNull: false
-  Column 7: count: 1 hasNull: false bytesOnDisk: 13 min: 2345 max: 2345 sum: 4
-  Column 8: count: 1 hasNull: false bytesOnDisk: 13 min: 2345 max: 2345 sum: 4
-  Column 9: count: 1 hasNull: false bytesOnDisk: 7 min: 2345 max: 2345 sum: 2345
-  Column 10: count: 1 hasNull: false bytesOnDisk: 7 min: 2345 max: 2345 sum: 2345
+  Column 7: count: 1 hasNull: false bytesOnDisk: 14 min: 12345 max: 12345 sum: 5
+  Column 8: count: 1 hasNull: false bytesOnDisk: 14 min: 12345 max: 12345 sum: 5
+  Column 9: count: 1 hasNull: false bytesOnDisk: 7 min: 12345 max: 12345 sum: 12345
+  Column 10: count: 1 hasNull: false bytesOnDisk: 7 min: 12345 max: 12345 sum: 12345
 
 Stripes:
-  Stripe: offset: 3 data: 73 rows: 1 tail: 103 index: 595
+  Stripe: offset: 3 data: 75 rows: 1 tail: 100 index: 597
     Stream: column 0 section ROW_INDEX start: 3 length 11
     Stream: column 1 section ROW_INDEX start: 14 length 24
     Stream: column 2 section ROW_INDEX start: 38 length 24
@@ -119,24 +119,24 @@ Stripes:
     Stream: column 4 section ROW_INDEX start: 91 length 24
     Stream: column 5 section ROW_INDEX start: 115 length 24
     Stream: column 6 section ROW_INDEX start: 139 length 11
-    Stream: column 7 section ROW_INDEX start: 150 length 30
-    Stream: column 7 section BLOOM_FILTER_UTF8 start: 180 length 112
-    Stream: column 8 section ROW_INDEX start: 292 length 30
-    Stream: column 8 section BLOOM_FILTER_UTF8 start: 322 length 112
-    Stream: column 9 section ROW_INDEX start: 434 length 27
-    Stream: column 10 section ROW_INDEX start: 461 length 27
-    Stream: column 10 section BLOOM_FILTER_UTF8 start: 488 length 110
-    Stream: column 1 section DATA start: 598 length 6
-    Stream: column 2 section DATA start: 604 length 6
-    Stream: column 3 section DATA start: 610 length 9
-    Stream: column 4 section DATA start: 619 length 6
-    Stream: column 5 section DATA start: 625 length 6
-    Stream: column 7 section DATA start: 631 length 7
-    Stream: column 7 section LENGTH start: 638 length 6
-    Stream: column 8 section DATA start: 644 length 7
-    Stream: column 8 section LENGTH start: 651 length 6
-    Stream: column 9 section DATA start: 657 length 7
-    Stream: column 10 section DATA start: 664 length 7
+    Stream: column 7 section ROW_INDEX start: 150 length 31
+    Stream: column 7 section BLOOM_FILTER_UTF8 start: 181 length 111
+    Stream: column 8 section ROW_INDEX start: 292 length 31
+    Stream: column 8 section BLOOM_FILTER_UTF8 start: 323 length 111
+    Stream: column 9 section ROW_INDEX start: 434 length 29
+    Stream: column 10 section ROW_INDEX start: 463 length 29
+    Stream: column 10 section BLOOM_FILTER_UTF8 start: 492 length 108
+    Stream: column 1 section DATA start: 600 length 6
+    Stream: column 2 section DATA start: 606 length 6
+    Stream: column 3 section DATA start: 612 length 9
+    Stream: column 4 section DATA start: 621 length 6
+    Stream: column 5 section DATA start: 627 length 6
+    Stream: column 7 section DATA start: 633 length 8
+    Stream: column 7 section LENGTH start: 641 length 6
+    Stream: column 8 section DATA start: 647 length 8
+    Stream: column 8 section LENGTH start: 655 length 6
+    Stream: column 9 section DATA start: 661 length 7
+    Stream: column 10 section DATA start: 668 length 7
     Encoding column 0: DIRECT
     Encoding column 1: DIRECT_V2
     Encoding column 2: DIRECT_V2
@@ -153,39 +153,39 @@ Stripes:
     Row group indices for column 1:
       Entry 0: count: 1 hasNull: false min: 0 max: 0 sum: 0 positions: 0,0,0
     Row group indices for column 2:
-      Entry 0: count: 1 hasNull: false min: 2 max: 2 sum: 2 positions: 0,0,0
+      Entry 0: count: 1 hasNull: false min: 1 max: 1 sum: 1 positions: 0,0,0
     Row group indices for column 3:
       Entry 0: count: 1 hasNull: false min: 536870912 max: 536870912 sum: 536870912 positions: 0,0,0
     Row group indices for column 4:
       Entry 0: count: 1 hasNull: false min: 0 max: 0 sum: 0 positions: 0,0,0
     Row group indices for column 5:
-      Entry 0: count: 1 hasNull: false min: 2 max: 2 sum: 2 positions: 0,0,0
+      Entry 0: count: 1 hasNull: false min: 1 max: 1 sum: 1 positions: 0,0,0
     Row group indices for column 6:
       Entry 0: count: 1 hasNull: false positions: 
     Row group indices for column 7:
-      Entry 0: count: 1 hasNull: false min: 2345 max: 2345 sum: 4 positions: 0,0,0,0,0
+      Entry 0: count: 1 hasNull: false min: 12345 max: 12345 sum: 5 positions: 0,0,0,0,0
     Bloom filters for column 7:
       Entry 0: numHashFunctions: 6 bitCount: 81472 popCount: 6 loadFactor: 0.0001 expectedFpp: 1.5953551E-25
       Stripe level merge: numHashFunctions: 6 bitCount: 81472 popCount: 6 loadFactor: 0.0001 expectedFpp: 1.5953551E-25
     Row group indices for column 8:
-      Entry 0: count: 1 hasNull: false min: 2345 max: 2345 sum: 4 positions: 0,0,0,0,0
+      Entry 0: count: 1 hasNull: false min: 12345 max: 12345 sum: 5 positions: 0,0,0,0,0
     Bloom filters for column 8:
       Entry 0: numHashFunctions: 6 bitCount: 81472 popCount: 6 loadFactor: 0.0001 expectedFpp: 1.5953551E-25
       Stripe level merge: numHashFunctions: 6 bitCount: 81472 popCount: 6 loadFactor: 0.0001 expectedFpp: 1.5953551E-25
     Row group indices for column 9:
-      Entry 0: count: 1 hasNull: false min: 2345 max: 2345 sum: 2345 positions: 0,0,0
+      Entry 0: count: 1 hasNull: false min: 12345 max: 12345 sum: 12345 positions: 0,0,0
     Row group indices for column 10:
-      Entry 0: count: 1 hasNull: false min: 2345 max: 2345 sum: 2345 positions: 0,0,0
+      Entry 0: count: 1 hasNull: false min: 12345 max: 12345 sum: 12345 positions: 0,0,0
     Bloom filters for column 10:
       Entry 0: numHashFunctions: 6 bitCount: 81472 popCount: 6 loadFactor: 0.0001 expectedFpp: 1.5953551E-25
       Stripe level merge: numHashFunctions: 6 bitCount: 81472 popCount: 6 loadFactor: 0.0001 expectedFpp: 1.5953551E-25
 
-File length: 1205 bytes
+File length: 1212 bytes
 Padding length: 0 bytes
 Padding ratio: 0%
 
 User Metadata:
-  hive.acid.key.index=2,536870912,0;
+  hive.acid.key.index=1,536870912,0;
   hive.acid.stats=1,0,0
   hive.acid.version=2
 ________________________________________________________________________________________________________________________
@@ -204,31 +204,31 @@ Stripe Statistics:
   Stripe 1:
     Column 0: count: 1 hasNull: false
     Column 1: count: 1 hasNull: false bytesOnDisk: 6 min: 0 max: 0 sum: 0
-    Column 2: count: 1 hasNull: false bytesOnDisk: 6 min: 1 max: 1 sum: 1
+    Column 2: count: 1 hasNull: false bytesOnDisk: 6 min: 2 max: 2 sum: 2
     Column 3: count: 1 hasNull: false bytesOnDisk: 9 min: 536870912 max: 536870912 sum: 536870912
     Column 4: count: 1 hasNull: false bytesOnDisk: 6 min: 0 max: 0 sum: 0
-    Column 5: count: 1 hasNull: false bytesOnDisk: 6 min: 1 max: 1 sum: 1
+    Column 5: count: 1 hasNull: false bytesOnDisk: 6 min: 2 max: 2 sum: 2
     Column 6: count: 1 hasNull: false
-    Column 7: count: 1 hasNull: false bytesOnDisk: 14 min: 12345 max: 12345 sum: 5
-    Column 8: count: 1 hasNull: false bytesOnDisk: 14 min: 12345 max: 12345 sum: 5
-    Column 9: count: 1 hasNull: false bytesOnDisk: 7 min: 12345 max: 12345 sum: 12345
-    Column 10: count: 1 hasNull: false bytesOnDisk: 7 min: 12345 max: 12345 sum: 12345
+    Column 7: count: 1 hasNull: false bytesOnDisk: 13 min: 2345 max: 2345 sum: 4
+    Column 8: count: 1 hasNull: false bytesOnDisk: 13 min: 2345 max: 2345 sum: 4
+    Column 9: count: 1 hasNull: false bytesOnDisk: 7 min: 2345 max: 2345 sum: 2345
+    Column 10: count: 1 hasNull: false bytesOnDisk: 7 min: 2345 max: 2345 sum: 2345
 
 File Statistics:
   Column 0: count: 1 hasNull: false
   Column 1: count: 1 hasNull: false bytesOnDisk: 6 min: 0 max: 0 sum: 0
-  Column 2: count: 1 hasNull: false bytesOnDisk: 6 min: 1 max: 1 sum: 1
+  Column 2: count: 1 hasNull: false bytesOnDisk: 6 min: 2 max: 2 sum: 2
   Column 3: count: 1 hasNull: false bytesOnDisk: 9 min: 536870912 max: 536870912 sum: 536870912
   Column 4: count: 1 hasNull: false bytesOnDisk: 6 min: 0 max: 0 sum: 0
-  Column 5: count: 1 hasNull: false bytesOnDisk: 6 min: 1 max: 1 sum: 1
+  Column 5: count: 1 hasNull: false bytesOnDisk: 6 min: 2 max: 2 sum: 2
   Column 6: count: 1 hasNull: false
-  Column 7: count: 1 hasNull: false bytesOnDisk: 14 min: 12345 max: 12345 sum: 5
-  Column 8: count: 1 hasNull: false bytesOnDisk: 14 min: 12345 max: 12345 sum: 5
-  Column 9: count: 1 hasNull: false bytesOnDisk: 7 min: 12345 max: 12345 sum: 12345
-  Column 10: count: 1 hasNull: false bytesOnDisk: 7 min: 12345 max: 12345 sum: 12345
+  Column 7: count: 1 hasNull: false bytesOnDisk: 13 min: 2345 max: 2345 sum: 4
+  Column 8: count: 1 hasNull: false bytesOnDisk: 13 min: 2345 max: 2345 sum: 4
+  Column 9: count: 1 hasNull: false bytesOnDisk: 7 min: 2345 max: 2345 sum: 2345
+  Column 10: count: 1 hasNull: false bytesOnDisk: 7 min: 2345 max: 2345 sum: 2345
 
 Stripes:
-  Stripe: offset: 3 data: 75 rows: 1 tail: 100 index: 597
+  Stripe: offset: 3 data: 73 rows: 1 tail: 103 index: 595
     Stream: column 0 section ROW_INDEX start: 3 length 11
     Stream: column 1 section ROW_INDEX start: 14 length 24
     Stream: column 2 section ROW_INDEX start: 38 length 24
@@ -236,24 +236,24 @@ Stripes:
     Stream: column 4 section ROW_INDEX start: 91 length 24
     Stream: column 5 section ROW_INDEX start: 115 length 24
     Stream: column 6 section ROW_INDEX start: 139 length 11
-    Stream: column 7 section ROW_INDEX start: 150 length 31
-    Stream: column 7 section BLOOM_FILTER_UTF8 start: 181 length 111
-    Stream: column 8 section ROW_INDEX start: 292 length 31
-    Stream: column 8 section BLOOM_FILTER_UTF8 start: 323 length 111
-    Stream: column 9 section ROW_INDEX start: 434 length 29
-    Stream: column 10 section ROW_INDEX start: 463 length 29
-    Stream: column 10 section BLOOM_FILTER_UTF8 start: 492 length 108
-    Stream: column 1 section DATA start: 600 length 6
-    Stream: column 2 section DATA start: 606 length 6
-    Stream: column 3 section DATA start: 612 length 9
-    Stream: column 4 section DATA start: 621 length 6
-    Stream: column 5 section DATA start: 627 length 6
-    Stream: column 7 section DATA start: 633 length 8
-    Stream: column 7 section LENGTH start: 641 length 6
-    Stream: column 8 section DATA start: 647 length 8
-    Stream: column 8 section LENGTH start: 655 length 6
-    Stream: column 9 section DATA start: 661 length 7
-    Stream: column 10 section DATA start: 668 length 7
+    Stream: column 7 section ROW_INDEX start: 150 length 30
+    Stream: column 7 section BLOOM_FILTER_UTF8 start: 180 length 112
+    Stream: column 8 section ROW_INDEX start: 292 length 30
+    Stream: column 8 section BLOOM_FILTER_UTF8 start: 322 length 112
+    Stream: column 9 section ROW_INDEX start: 434 length 27
+    Stream: column 10 section ROW_INDEX start: 461 length 27
+    Stream: column 10 section BLOOM_FILTER_UTF8 start: 488 length 110
+    Stream: column 1 section DATA start: 598 length 6
+    Stream: column 2 section DATA start: 604 length 6
+    Stream: column 3 section DATA start: 610 length 9
+    Stream: column 4 section DATA start: 619 length 6
+    Stream: column 5 section DATA start: 625 length 6
+    Stream: column 7 section DATA start: 631 length 7
+    Stream: column 7 section LENGTH start: 638 length 6
+    Stream: column 8 section DATA start: 644 length 7
+    Stream: column 8 section LENGTH start: 651 length 6
+    Stream: column 9 section DATA start: 657 length 7
+    Stream: column 10 section DATA start: 664 length 7
     Encoding column 0: DIRECT
     Encoding column 1: DIRECT_V2
     Encoding column 2: DIRECT_V2
@@ -270,39 +270,39 @@ Stripes:
     Row group indices for column 1:
       Entry 0: count: 1 hasNull: false min: 0 max: 0 sum: 0 positions: 0,0,0
     Row group indices for column 2:
-      Entry 0: count: 1 hasNull: false min: 1 max: 1 sum: 1 positions: 0,0,0
+      Entry 0: count: 1 hasNull: false min: 2 max: 2 sum: 2 positions: 0,0,0
     Row group indices for column 3:
       Entry 0: count: 1 hasNull: false min: 536870912 max: 536870912 sum: 536870912 positions: 0,0,0
     Row group indices for column 4:
       Entry 0: count: 1 hasNull: false min: 0 max: 0 sum: 0 positions: 0,0,0
     Row group indices for column 5:
-      Entry 0: count: 1 hasNull: false min: 1 max: 1 sum: 1 positions: 0,0,0
+      Entry 0: count: 1 hasNull: false min: 2 max: 2 sum: 2 positions: 0,0,0
     Row group indices for column 6:
       Entry 0: count: 1 hasNull: false positions: 
     Row group indices for column 7:
-      Entry 0: count: 1 hasNull: false min: 12345 max: 12345 sum: 5 positions: 0,0,0,0,0
+      Entry 0: count: 1 hasNull: false min: 2345 max: 2345 sum: 4 positions: 0,0,0,0,0
     Bloom filters for column 7:
       Entry 0: numHashFunctions: 6 bitCount: 81472 popCount: 6 loadFactor: 0.0001 expectedFpp: 1.5953551E-25
       Stripe level merge: numHashFunctions: 6 bitCount: 81472 popCount: 6 loadFactor: 0.0001 expectedFpp: 1.5953551E-25
     Row group indices for column 8:
-      Entry 0: count: 1 hasNull: false min: 12345 max: 12345 sum: 5 positions: 0,0,0,0,0
+      Entry 0: count: 1 hasNull: false min: 2345 max: 2345 sum: 4 positions: 0,0,0,0,0
     Bloom filters for column 8:
       Entry 0: numHashFunctions: 6 bitCount: 81472 popCount: 6 loadFactor: 0.0001 expectedFpp: 1.5953551E-25
       Stripe level merge: numHashFunctions: 6 bitCount: 81472 popCount: 6 loadFactor: 0.0001 expectedFpp: 1.5953551E-25
     Row group indices for column 9:
-      Entry 0: count: 1 hasNull: false min: 12345 max: 12345 sum: 12345 positions: 0,0,0
+      Entry 0: count: 1 hasNull: false min: 2345 max: 2345 sum: 2345 positions: 0,0,0
     Row group indices for column 10:
-      Entry 0: count: 1 hasNull: false min: 12345 max: 12345 sum: 12345 positions: 0,0,0
+      Entry 0: count: 1 hasNull: false min: 2345 max: 2345 sum: 2345 positions: 0,0,0
     Bloom filters for column 10:
       Entry 0: numHashFunctions: 6 bitCount: 81472 popCount: 6 loadFactor: 0.0001 expectedFpp: 1.5953551E-25
       Stripe level merge: numHashFunctions: 6 bitCount: 81472 popCount: 6 loadFactor: 0.0001 expectedFpp: 1.5953551E-25
 
-File length: 1212 bytes
+File length: 1205 bytes
 Padding length: 0 bytes
 Padding ratio: 0%
 
 User Metadata:
-  hive.acid.key.index=1,536870912,0;
+  hive.acid.key.index=2,536870912,0;
   hive.acid.stats=1,0,0
   hive.acid.version=2
 ________________________________________________________________________________________________________________________