You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by go...@apache.org on 2018/07/31 20:52:10 UTC

[1/3] hive git commit: HIVE-20177: Vectorization: Reduce KeyWrapper allocation in GroupBy Streaming mode (Gopal V, reviewed by Matt McCline)

Repository: hive
Updated Branches:
  refs/heads/master 3104d4756 -> 3e46515d3


HIVE-20177: Vectorization: Reduce KeyWrapper allocation in GroupBy Streaming mode (Gopal V, reviewed by Matt McCline)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/4d251514
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/4d251514
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/4d251514

Branch: refs/heads/master
Commit: 4d251514fde46a28f1d59d439097918576c26560
Parents: 3104d47
Author: Gopal V <go...@apache.org>
Authored: Tue Jul 31 13:48:07 2018 -0700
Committer: Gopal V <go...@apache.org>
Committed: Tue Jul 31 13:48:07 2018 -0700

----------------------------------------------------------------------
 .../hive/ql/exec/vector/VectorGroupByOperator.java | 17 +++++++++--------
 1 file changed, 9 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/4d251514/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java
index 75efc29..43f1162 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java
@@ -725,10 +725,11 @@ public class VectorGroupByOperator extends Operator<GroupByDesc>
 
       VectorHashKeyWrapper[] batchKeys = keyWrappersBatch.getVectorHashKeyWrappers();
 
+      final VectorHashKeyWrapper prevKey = streamingKey;
       if (streamingKey == null) {
         // This is the first batch we process after switching from hash mode
         currentStreamingAggregators = streamAggregationBufferRowPool.getFromPool();
-        streamingKey = (VectorHashKeyWrapper) batchKeys[0].copyKey();
+        streamingKey = batchKeys[0];
       }
 
       aggregationBatchInfo.startBatch();
@@ -739,14 +740,9 @@ public class VectorGroupByOperator extends Operator<GroupByDesc>
           // We've encountered a new key, must save current one
           // We can't forward yet, the aggregators have not been evaluated
           rowsToFlush[flushMark] = currentStreamingAggregators;
-          if (keysToFlush[flushMark] == null) {
-            keysToFlush[flushMark] = (VectorHashKeyWrapper) streamingKey.copyKey();
-          } else {
-            streamingKey.duplicateTo(keysToFlush[flushMark]);
-          }
-
+          keysToFlush[flushMark] = streamingKey;
           currentStreamingAggregators = streamAggregationBufferRowPool.getFromPool();
-          batchKeys[i].duplicateTo(streamingKey);
+          streamingKey = batchKeys[i];
           ++flushMark;
         }
         aggregationBatchInfo.mapAggregationBufferSet(currentStreamingAggregators, i);
@@ -759,8 +755,13 @@ public class VectorGroupByOperator extends Operator<GroupByDesc>
       for (int i = 0; i < flushMark; ++i) {
         writeSingleRow(keysToFlush[i], rowsToFlush[i]);
         rowsToFlush[i].reset();
+        keysToFlush[i] = null;
         streamAggregationBufferRowPool.putInPool(rowsToFlush[i]);
       }
+
+      if (streamingKey != prevKey) {
+        streamingKey = (VectorHashKeyWrapper) streamingKey.copyKey();
+      }
     }
 
     @Override


[2/3] hive git commit: HIVE-20040: JDBC: HTTP listen queue is 50 and SYNs are lost (Gopal V, reviewed by Prasanth Jayachandran)

Posted by go...@apache.org.
HIVE-20040: JDBC: HTTP listen queue is 50 and SYNs are lost (Gopal V, reviewed by Prasanth Jayachandran)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/80ae3332
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/80ae3332
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/80ae3332

Branch: refs/heads/master
Commit: 80ae333272803945e5a44e34bb0370aac30357da
Parents: 4d25151
Author: Gopal V <go...@apache.org>
Authored: Tue Jul 31 13:49:14 2018 -0700
Committer: Gopal V <go...@apache.org>
Committed: Tue Jul 31 13:49:14 2018 -0700

----------------------------------------------------------------------
 .../org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java    | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/80ae3332/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java b/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java
index 95d78f8..89271d7 100644
--- a/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java
+++ b/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java
@@ -143,6 +143,7 @@ public class ThriftHttpCLIService extends ThriftCLIService {
       int maxIdleTime = (int) hiveConf.getTimeVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_MAX_IDLE_TIME,
           TimeUnit.MILLISECONDS);
       connector.setIdleTimeout(maxIdleTime);
+      connector.setAcceptQueueSize(maxWorkerThreads);
 
       server.addConnector(connector);
 


[3/3] hive git commit: HIVE-19199: ACID: DbTxnManager heartbeat-service needs static sync init (Gopal V, reviewed by Eugene Koifman)

Posted by go...@apache.org.
HIVE-19199: ACID: DbTxnManager heartbeat-service needs static sync init (Gopal V, reviewed by Eugene Koifman)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/3e46515d
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/3e46515d
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/3e46515d

Branch: refs/heads/master
Commit: 3e46515d3d556205cb643908565a24f6634c27c1
Parents: 80ae333
Author: Gopal V <go...@apache.org>
Authored: Tue Jul 31 13:51:53 2018 -0700
Committer: Gopal V <go...@apache.org>
Committed: Tue Jul 31 13:51:53 2018 -0700

----------------------------------------------------------------------
 .../hadoop/hive/ql/lockmgr/DbTxnManager.java    | 31 +++++++++++---------
 1 file changed, 17 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/3e46515d/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
index 27abaf5..f746d2d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
@@ -785,21 +785,24 @@ public final class DbTxnManager extends HiveTxnManagerImpl {
   }
 
   private synchronized void initHeartbeatExecutorService() {
-    if (heartbeatExecutorService != null && !heartbeatExecutorService.isShutdown()
-        && !heartbeatExecutorService.isTerminated()) {
-      return;
+    synchronized (DbTxnManager.class) {
+      if (heartbeatExecutorService != null && !heartbeatExecutorService.isShutdown()
+          && !heartbeatExecutorService.isTerminated()) {
+        return;
+      }
+      heartbeatExecutorService =
+          Executors.newScheduledThreadPool(
+              conf.getIntVar(HiveConf.ConfVars.HIVE_TXN_HEARTBEAT_THREADPOOL_SIZE),
+              new ThreadFactory() {
+                private final AtomicInteger threadCounter = new AtomicInteger();
+
+                @Override
+                public Thread newThread(Runnable r) {
+                  return new HeartbeaterThread(r, "Heartbeater-" + threadCounter.getAndIncrement());
+                }
+              });
+      ((ScheduledThreadPoolExecutor) heartbeatExecutorService).setRemoveOnCancelPolicy(true);
     }
-    heartbeatExecutorService =
-        Executors.newScheduledThreadPool(
-          conf.getIntVar(HiveConf.ConfVars.HIVE_TXN_HEARTBEAT_THREADPOOL_SIZE), new ThreadFactory() {
-          private final AtomicInteger threadCounter = new AtomicInteger();
-
-          @Override
-          public Thread newThread(Runnable r) {
-            return new HeartbeaterThread(r, "Heartbeater-" + threadCounter.getAndIncrement());
-          }
-        });
-    ((ScheduledThreadPoolExecutor) heartbeatExecutorService).setRemoveOnCancelPolicy(true);
   }
 
   public static class HeartbeaterThread extends Thread {