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 21:07:03 UTC

[1/2] hive git commit: Revert "HIVE-13396: LLAP: Include hadoop-metrics2.properties file LlapServiceDriver (Prasanth Jayachandran reviewed by Sergey Shelukhin)"

Repository: hive
Updated Branches:
  refs/heads/branch-2.0 4826b0435 -> 88e2237d3


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

This reverts commit 5333268b4140f08a69f7849f00011017cd956cba.


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

Branch: refs/heads/branch-2.0
Commit: 962f49a52727b1c6098a44007bfa70eeeb8f0c3f
Parents: 4826b04
Author: Prasanth Jayachandran <pr...@apache.org>
Authored: Tue Apr 5 14:05:24 2016 -0500
Committer: Prasanth Jayachandran <pr...@apache.org>
Committed: Tue Apr 5 14:05:24 2016 -0500

----------------------------------------------------------------------
 .../hadoop/hive/llap/cli/LlapServiceDriver.java | 19 +++-----
 .../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, 14 insertions(+), 69 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/962f49a5/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 691fb87..68c6b29 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
@@ -304,18 +304,13 @@ 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) {
-      udfStream.println(udfClass);
+    String java_home = System.getenv("JAVA_HOME");
+    String jre_home = System.getProperty("java.home");
+    if (java_home == null) {
+      java_home = jre_home;
+    } else if (!java_home.equals(jre_home)) {
+      LOG.warn("Java versions might not match : JAVA_HOME=%s,process jre=%s", 
+          java_home, jre_home);
     }
 
     // extract configs for processing by the python fragments in Slider

http://git-wip-us.apache.org/repos/asf/hive/blob/962f49a5/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 be5fbd6..9b7c163 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
@@ -67,7 +67,6 @@ 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/962f49a5/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 b89c6c4..52057e4 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 = "cache")
+@Metrics(about = "LlapDaemon Cache Metrics", context = MetricsUtils.METRICS_CONTEXT)
 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("cache")
+        .setContext(MetricsUtils.METRICS_CONTEXT)
         .tag(ProcessName, MetricsUtils.METRICS_PROCESS_NAME)
         .tag(SessionId, sessionId);
     getCacheStats(rb);

http://git-wip-us.apache.org/repos/asf/hive/blob/962f49a5/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 7919200..33b8f9d 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 = "executors")
+@Metrics(about = "LlapDaemon Executor Metrics", context = MetricsUtils.METRICS_CONTEXT)
 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("executors")
+        .setContext(MetricsUtils.METRICS_CONTEXT)
         .tag(ProcessName, MetricsUtils.METRICS_PROCESS_NAME)
         .tag(SessionId, sessionId);
     getExecutorStats(rb);

http://git-wip-us.apache.org/repos/asf/hive/blob/962f49a5/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 10a0124..5905cae 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 = "queue")
+@Metrics(about = "LlapDaemon Queue Metrics", context = MetricsUtils.METRICS_CONTEXT)
 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("queue")
+        .setContext(MetricsUtils.METRICS_CONTEXT)
         .tag(ProcessName, MetricsUtils.METRICS_PROCESS_NAME)
         .tag(SessionId, sessionId);
     getQueueStats(rb);

http://git-wip-us.apache.org/repos/asf/hive/blob/962f49a5/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 c22ce4e..ce1c965 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,6 +26,7 @@ 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/962f49a5/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
deleted file mode 100644
index 994acaa..0000000
--- a/llap-server/src/main/resources/hadoop-metrics2.properties.template
+++ /dev/null
@@ -1,50 +0,0 @@
-#
-# 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


[2/2] hive git commit: Revert "HIVE-13086: LLAP: Programmatically initialize log4j2 to print out the properties location (Prasanth Jayachandran reviewed by Gopal V)"

Posted by pr...@apache.org.
Revert "HIVE-13086: LLAP: Programmatically initialize log4j2 to print out the properties location (Prasanth Jayachandran reviewed by Gopal V)"

This reverts commit 9c0549a48ab3b5bdfa429dfbcfd986a52de5417e.


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

Branch: refs/heads/branch-2.0
Commit: 88e2237d323776d4f51c7b03536dd67f9f872d7d
Parents: 962f49a
Author: Prasanth Jayachandran <pr...@apache.org>
Authored: Tue Apr 5 14:05:46 2016 -0500
Committer: Prasanth Jayachandran <pr...@apache.org>
Committed: Tue Apr 5 14:06:31 2016 -0500

----------------------------------------------------------------------
 .../hadoop/hive/llap/cli/LlapServiceDriver.java   |  6 ++----
 .../hadoop/hive/llap/daemon/impl/LlapDaemon.java  | 18 ------------------
 2 files changed, 2 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/88e2237d/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 68c6b29..1c1a521 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
@@ -26,10 +26,9 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Collection;
 
-import org.apache.hadoop.hive.llap.configuration.LlapDaemonConfiguration;
-import org.apache.hadoop.hive.llap.daemon.impl.LlapDaemon;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos;
 import org.apache.hadoop.hive.llap.tezplugins.LlapTezUtils;
+import org.eclipse.jetty.server.ssl.SslSocketConnector;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -194,8 +193,7 @@ public class LlapServiceDriver {
       conf.set((String) props.getKey(), (String) props.getValue());
     }
 
-
-    URL logger = conf.getResource(LlapDaemon.LOG4j2_PROPERTIES_FILE);
+    URL logger = conf.getResource("llap-daemon-log4j2.properties");
 
     if (null == logger) {
       throw new Exception("Unable to find required config file: llap-daemon-log4j2.properties");

http://git-wip-us.apache.org/repos/asf/hive/blob/88e2237d/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 9b7c163..37120d2 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
@@ -19,7 +19,6 @@ import java.lang.management.ManagementFactory;
 import java.lang.management.MemoryPoolMXBean;
 import java.lang.management.MemoryType;
 import java.net.InetSocketAddress;
-import java.net.URL;
 import java.util.Arrays;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicLong;
@@ -54,7 +53,6 @@ import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.util.JvmPauseMonitor;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hive.common.util.ShutdownHookManager;
-import org.apache.logging.log4j.core.config.Configurator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -66,7 +64,6 @@ 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";
   private final Configuration shuffleHandlerConf;
   private final LlapProtocolServerImpl server;
   private final ContainerRunnerImpl containerRunner;
@@ -96,8 +93,6 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
       int mngPort, int shufflePort) {
     super("LlapDaemon");
 
-    initializeLogging();
-
     printAsciiArt();
 
     Preconditions.checkArgument(numExecutors > 0);
@@ -199,19 +194,6 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
     addIfService(amReporter);
   }
 
-  private void initializeLogging() {
-    long start = System.currentTimeMillis();
-    URL llap_l4j2 = LlapDaemon.class.getClassLoader().getResource(LOG4j2_PROPERTIES_FILE);
-    if (llap_l4j2 != null) {
-      Configurator.initialize("LlapDaemonLog4j2", llap_l4j2.toString());
-      long end = System.currentTimeMillis();
-      LOG.info("LLAP daemon logging initialized from {} in {} ms", llap_l4j2, (end - start));
-    } else {
-      throw new RuntimeException("Log initialization failed." +
-          " Unable to locate " + LOG4j2_PROPERTIES_FILE + " file in classpath");
-    }
-  }
-
   public static long getTotalHeapSize() {
     // runtime.getMax() gives a very different number from the actual Xmx sizing.
     // you can iterate through the