You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by pr...@apache.org on 2016/04/05 02:15:19 UTC

hive git commit: HIVE-13396: LLAP: Include hadoop-metrics2.properties file LlapServiceDriver (Prasanth Jayachandran reviewed by Sergey Shelukhin)

Repository: hive
Updated Branches:
  refs/heads/master 91ab819a1 -> f3358b036


HIVE-13396: LLAP: Include hadoop-metrics2.properties file LlapServiceDriver (Prasanth Jayachandran reviewed by Sergey Shelukhin)


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

Branch: refs/heads/master
Commit: f3358b036f1d14d369711e8863b904d04ef75a04
Parents: 91ab819
Author: Prasanth Jayachandran <pr...@apache.org>
Authored: Mon Apr 4 19:15:07 2016 -0500
Committer: Prasanth Jayachandran <pr...@apache.org>
Committed: Mon Apr 4 19:15:07 2016 -0500

----------------------------------------------------------------------
 .../hadoop/hive/llap/cli/LlapServiceDriver.java |  8 ++++
 .../hive/llap/daemon/impl/LlapDaemon.java       |  1 +
 .../llap/metrics/LlapDaemonCacheMetrics.java    |  4 +-
 .../llap/metrics/LlapDaemonExecutorMetrics.java |  4 +-
 .../llap/metrics/LlapDaemonQueueMetrics.java    |  4 +-
 .../hadoop/hive/llap/metrics/MetricsUtils.java  |  1 -
 .../hadoop-metrics2.properties.template         | 50 ++++++++++++++++++++
 7 files changed, 65 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/f3358b03/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java b/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java
index 1f3b930..8cd6df7 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java
@@ -399,6 +399,14 @@ public class LlapServiceDriver {
     IOUtils.copyBytes(loggerContent,
         lfs.create(new Path(confPath, "llap-daemon-log4j2.properties"), true), conf, true);
 
+    URL metrics2 = conf.getResource(LlapDaemon.HADOOP_METRICS2_PROPERTIES_FILE);
+    if (metrics2 != null) {
+      InputStream metrics2FileStream = metrics2.openStream();
+      IOUtils.copyBytes(metrics2FileStream,
+          lfs.create(new Path(confPath, LlapDaemon.HADOOP_METRICS2_PROPERTIES_FILE), true),
+          conf, true);
+    }
+
     PrintWriter udfStream =
         new PrintWriter(lfs.create(new Path(confPath, StaticPermanentFunctionChecker.PERMANENT_FUNCTIONS_LIST)));
     for (String udfClass : allowedUdfs) {

http://git-wip-us.apache.org/repos/asf/hive/blob/f3358b03/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
index 2fe59a2..8600832 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
@@ -73,6 +73,7 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
   private static final Logger LOG = LoggerFactory.getLogger(LlapDaemon.class);
 
   public static final String LOG4j2_PROPERTIES_FILE = "llap-daemon-log4j2.properties";
+  public static final String HADOOP_METRICS2_PROPERTIES_FILE = "hadoop-metrics2.properties";
   private final Configuration shuffleHandlerConf;
   private final LlapProtocolServerImpl server;
   private final ContainerRunnerImpl containerRunner;

http://git-wip-us.apache.org/repos/asf/hive/blob/f3358b03/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonCacheMetrics.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonCacheMetrics.java b/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonCacheMetrics.java
index 52057e4..b89c6c4 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonCacheMetrics.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonCacheMetrics.java
@@ -45,7 +45,7 @@ import com.google.common.annotations.VisibleForTesting;
 /**
  * Llap daemon cache metrics source.
  */
-@Metrics(about = "LlapDaemon Cache Metrics", context = MetricsUtils.METRICS_CONTEXT)
+@Metrics(about = "LlapDaemon Cache Metrics", context = "cache")
 public class LlapDaemonCacheMetrics implements MetricsSource {
   final String name;
   private String sessionId;
@@ -127,7 +127,7 @@ public class LlapDaemonCacheMetrics implements MetricsSource {
   @Override
   public void getMetrics(MetricsCollector collector, boolean b) {
     MetricsRecordBuilder rb = collector.addRecord(CacheMetrics)
-        .setContext(MetricsUtils.METRICS_CONTEXT)
+        .setContext("cache")
         .tag(ProcessName, MetricsUtils.METRICS_PROCESS_NAME)
         .tag(SessionId, sessionId);
     getCacheStats(rb);

http://git-wip-us.apache.org/repos/asf/hive/blob/f3358b03/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonExecutorMetrics.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonExecutorMetrics.java b/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonExecutorMetrics.java
index 33b8f9d..7919200 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonExecutorMetrics.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonExecutorMetrics.java
@@ -52,7 +52,7 @@ import org.apache.hadoop.metrics2.source.JvmMetrics;
 /**
  * Metrics about the llap daemon executors.
  */
-@Metrics(about = "LlapDaemon Executor Metrics", context = MetricsUtils.METRICS_CONTEXT)
+@Metrics(about = "LlapDaemon Executor Metrics", context = "executors")
 public class LlapDaemonExecutorMetrics implements MetricsSource {
 
   private final String name;
@@ -117,7 +117,7 @@ public class LlapDaemonExecutorMetrics implements MetricsSource {
   @Override
   public void getMetrics(MetricsCollector collector, boolean b) {
     MetricsRecordBuilder rb = collector.addRecord(ExecutorMetrics)
-        .setContext(MetricsUtils.METRICS_CONTEXT)
+        .setContext("executors")
         .tag(ProcessName, MetricsUtils.METRICS_PROCESS_NAME)
         .tag(SessionId, sessionId);
     getExecutorStats(rb);

http://git-wip-us.apache.org/repos/asf/hive/blob/f3358b03/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonQueueMetrics.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonQueueMetrics.java b/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonQueueMetrics.java
index 5905cae..10a0124 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonQueueMetrics.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/LlapDaemonQueueMetrics.java
@@ -38,7 +38,7 @@ import org.apache.hadoop.metrics2.lib.MutableRate;
 /**
  *
  */
-@Metrics(about = "LlapDaemon Queue Metrics", context = MetricsUtils.METRICS_CONTEXT)
+@Metrics(about = "LlapDaemon Queue Metrics", context = "queue")
 public class LlapDaemonQueueMetrics implements MetricsSource {
   private final String name;
   private final String sessionId;
@@ -78,7 +78,7 @@ public class LlapDaemonQueueMetrics implements MetricsSource {
   @Override
   public void getMetrics(MetricsCollector collector, boolean b) {
     MetricsRecordBuilder rb = collector.addRecord(QueueMetrics)
-        .setContext(MetricsUtils.METRICS_CONTEXT)
+        .setContext("queue")
         .tag(ProcessName, MetricsUtils.METRICS_PROCESS_NAME)
         .tag(SessionId, sessionId);
     getQueueStats(rb);

http://git-wip-us.apache.org/repos/asf/hive/blob/f3358b03/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/MetricsUtils.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/MetricsUtils.java b/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/MetricsUtils.java
index ce1c965..c22ce4e 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/MetricsUtils.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/metrics/MetricsUtils.java
@@ -26,7 +26,6 @@ import java.util.UUID;
  */
 public class MetricsUtils {
   private static final String LOCALHOST = "localhost";
-  public static final String METRICS_CONTEXT = "llap";
   public static final String METRICS_PROCESS_NAME = "LlapDaemon";
 
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f3358b03/llap-server/src/main/resources/hadoop-metrics2.properties.template
----------------------------------------------------------------------
diff --git a/llap-server/src/main/resources/hadoop-metrics2.properties.template b/llap-server/src/main/resources/hadoop-metrics2.properties.template
new file mode 100644
index 0000000..994acaa
--- /dev/null
+++ b/llap-server/src/main/resources/hadoop-metrics2.properties.template
@@ -0,0 +1,50 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#}
+
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+# syntax: [prefix].[source|sink].[instance].[options]
+# See javadoc of package-info.java for org.apache.hadoop.metrics2 for details
+
+#*.sink.file.class=org.apache.hadoop.metrics2.sink.FileSink
+# default sampling period, in seconds
+#*.sink.file.period=10
+
+# *.sink.timeline.class=org.apache.hadoop.metrics2.sink.timeline.HadoopTimelineMetricsSink
+# *.sink.timeline.period=60
+
+# llapdeamon metrics for all contexts (jvm,queue,executors,cache) will go to this file
+# llapdaemon.sink.file.filename=llapdaemon-metrics.out
+
+# to configure separate files per context define following for each context
+# llapdaemon.sink.file_jvm.class=org.apache.hadoop.metrics2.sink.FileSink
+# llapdaemon.sink.file_jvm.context=jvm
+# llapdaemon.sink.file_jvm.filename=llapdaemon-jvm-metrics.out