You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jc...@apache.org on 2018/05/11 16:20:43 UTC

[1/2] hive git commit: HIVE-19483: Metastore cleaner tasks that run periodically are created more than once (Jesus Camacho Rodriguez, reviewed by Alan Gates)

Repository: hive
Updated Branches:
  refs/heads/master 8028ce8a4 -> 598dcf40f


HIVE-19483: Metastore cleaner tasks that run periodically are created more than once (Jesus Camacho Rodriguez, reviewed by Alan Gates)


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

Branch: refs/heads/master
Commit: e71c469c4f61a97dbfb8f9a6934fd92a6b38098c
Parents: 8028ce8
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Fri May 11 08:53:53 2018 -0700
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Fri May 11 08:53:53 2018 -0700

----------------------------------------------------------------------
 .../hadoop/hive/metastore/HiveMetaStore.java    | 33 ++++++++++++--------
 1 file changed, 20 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/e71c469c/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index 269798c..a2b8743 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -241,6 +241,11 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     private final Configuration conf; // stores datastore (jpox) properties,
                                      // right now they come from jpox.properties
 
+    // Flag to control that always threads are initialized only once
+    // instead of multiple times
+    private final static AtomicBoolean alwaysThreadsInitialized =
+        new AtomicBoolean(false);
+
     private static String currentUrl;
     private FileMetadataManager fileMetadataManager;
     private PartitionExpressionProxy expressionProxy;
@@ -558,19 +563,21 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         partitionValidationPattern = null;
       }
 
-      ThreadPool.initialize(conf);
-      Collection<String> taskNames =
-          MetastoreConf.getStringCollection(conf, ConfVars.TASK_THREADS_ALWAYS);
-      for (String taskName : taskNames) {
-        MetastoreTaskThread task =
-            JavaUtils.newInstance(JavaUtils.getClass(taskName, MetastoreTaskThread.class));
-        task.setConf(conf);
-        long freq = task.runFrequency(TimeUnit.MILLISECONDS);
-        // For backwards compatibility, since some threads used to be hard coded but only run if
-        // frequency was > 0
-        if (freq > 0) {
-          ThreadPool.getPool().scheduleAtFixedRate(task, freq, freq, TimeUnit.MILLISECONDS);
-
+      // We only initialize once the tasks that need to be run periodically
+      if (alwaysThreadsInitialized.compareAndSet(false, true)) {
+        ThreadPool.initialize(conf);
+        Collection<String> taskNames =
+            MetastoreConf.getStringCollection(conf, ConfVars.TASK_THREADS_ALWAYS);
+        for (String taskName : taskNames) {
+          MetastoreTaskThread task =
+              JavaUtils.newInstance(JavaUtils.getClass(taskName, MetastoreTaskThread.class));
+          task.setConf(conf);
+          long freq = task.runFrequency(TimeUnit.MILLISECONDS);
+          // For backwards compatibility, since some threads used to be hard coded but only run if
+          // frequency was > 0
+          if (freq > 0) {
+            ThreadPool.getPool().scheduleAtFixedRate(task, freq, freq, TimeUnit.MILLISECONDS);
+          }
         }
       }
       expressionProxy = PartFilterExprUtil.createExpressionProxy(conf);


[2/2] hive git commit: HIVE-19477: Hiveserver2 in http mode not emitting metric default.General.open_connections (Jesus Camacho Rodriguez, reviewed by Vaibhav Gumashta)

Posted by jc...@apache.org.
HIVE-19477: Hiveserver2 in http mode not emitting metric default.General.open_connections (Jesus Camacho Rodriguez, reviewed by Vaibhav Gumashta)


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

Branch: refs/heads/master
Commit: 598dcf40f3c69dfc1ca52bc28282b58ad30507fb
Parents: e71c469
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Fri May 11 09:00:51 2018 -0700
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Fri May 11 09:00:51 2018 -0700

----------------------------------------------------------------------
 .../cli/thrift/ThriftHttpCLIService.java        | 34 ++++++++++++++++++--
 1 file changed, 32 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/598dcf40/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 a02f13c..0b3f2c3 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
@@ -23,14 +23,17 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.SynchronousQueue;
 import java.util.concurrent.TimeUnit;
 
+import javax.servlet.ServletContextEvent;
+import javax.servlet.ServletContextListener;
 import javax.ws.rs.HttpMethod;
 
+import org.apache.hadoop.hive.common.metrics.common.Metrics;
+import org.apache.hadoop.hive.common.metrics.common.MetricsConstant;
+import org.apache.hadoop.hive.common.metrics.common.MetricsFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.shims.ShimLoader;
-import org.apache.hadoop.hive.shims.Utils;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.util.Shell;
 import org.apache.hive.service.auth.HiveAuthFactory;
 import org.apache.hive.service.cli.CLIService;
 import org.apache.hive.service.rpc.thrift.TCLIService;
@@ -153,6 +156,33 @@ public class ThriftHttpCLIService extends ThriftCLIService {
         LOG.warn("XSRF filter disabled");
       }
 
+      context.addEventListener(new ServletContextListener() {
+        @Override
+        public void contextInitialized(ServletContextEvent servletContextEvent) {
+          Metrics metrics = MetricsFactory.getInstance();
+          if (metrics != null) {
+            try {
+              metrics.incrementCounter(MetricsConstant.OPEN_CONNECTIONS);
+              metrics.incrementCounter(MetricsConstant.CUMULATIVE_CONNECTION_COUNT);
+            } catch (Exception e) {
+              LOG.warn("Error reporting HS2 open connection operation to Metrics system", e);
+            }
+          }
+        }
+
+        @Override
+        public void contextDestroyed(ServletContextEvent servletContextEvent) {
+          Metrics metrics = MetricsFactory.getInstance();
+          if (metrics != null) {
+            try {
+              metrics.decrementCounter(MetricsConstant.OPEN_CONNECTIONS);
+            } catch (Exception e) {
+              LOG.warn("Error reporting HS2 close connection operation to Metrics system", e);
+            }
+          }
+        }
+      });
+
       final String httpPath = getHttpPath(hiveConf
           .getVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_HTTP_PATH));