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

[44/50] [abbrv] hive git commit: HIVE-13027: Configuration changes to improve logging performance (Prasanth Jayachandran reviewed by Sergey Shelukhin)

HIVE-13027: Configuration changes to improve logging performance (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/b870d526
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/b870d526
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/b870d526

Branch: refs/heads/java8
Commit: b870d526edbac1831d66f2529cf1a854b57bddb2
Parents: 0cc4045
Author: Prasanth Jayachandran <pr...@apache.org>
Authored: Fri May 6 03:08:28 2016 -0500
Committer: Prasanth Jayachandran <pr...@apache.org>
Committed: Fri May 6 03:09:40 2016 -0500

----------------------------------------------------------------------
 cli/pom.xml                                     |  6 +++
 common/pom.xml                                  |  6 +++
 .../org/apache/hadoop/hive/common/LogUtils.java | 35 ++++++++++++--
 .../org/apache/hadoop/hive/conf/HiveConf.java   |  6 +++
 .../src/main/resources/hive-log4j2.properties   |  2 +-
 .../hadoop/hive/conf/TestHiveAsyncLogging.java  | 49 ++++++++++++++++++++
 data/conf/hive-log4j2.properties                |  2 +-
 hcatalog/core/pom.xml                           |  6 +++
 llap-server/bin/llapDaemon.sh                   |  2 +-
 .../hive/llap/daemon/impl/LlapDaemon.java       | 10 ++--
 .../main/resources/llap-cli-log4j2.properties   |  2 +-
 .../resources/llap-daemon-log4j2.properties     |  4 +-
 .../resources/llap-daemon-log4j2.properties     |  4 +-
 metastore/pom.xml                               |  6 +++
 pom.xml                                         |  2 +
 ql/pom.xml                                      |  6 +++
 .../main/resources/hive-exec-log4j2.properties  |  2 +-
 .../resources/tez-container-log4j2.properties   |  2 +-
 18 files changed, 136 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/b870d526/cli/pom.xml
----------------------------------------------------------------------
diff --git a/cli/pom.xml b/cli/pom.xml
index 76f6d11..6f2e664 100644
--- a/cli/pom.xml
+++ b/cli/pom.xml
@@ -82,6 +82,12 @@
       <scope>test</scope>
     </dependency>
     <dependency>
+      <groupId>com.lmax</groupId>
+      <artifactId>disruptor</artifactId>
+      <version>${disruptor.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>jline</groupId>
       <artifactId>jline</artifactId>
       <version>${jline.version}</version>

http://git-wip-us.apache.org/repos/asf/hive/blob/b870d526/common/pom.xml
----------------------------------------------------------------------
diff --git a/common/pom.xml b/common/pom.xml
index 67aab7c..9933072 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -164,6 +164,12 @@
       <scope>test</scope>
     </dependency>
     <dependency>
+      <groupId>com.lmax</groupId>
+      <artifactId>disruptor</artifactId>
+      <version>${disruptor.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>org.json</groupId>
       <artifactId>json</artifactId>
       <version>${json.version}</version>

http://git-wip-us.apache.org/repos/asf/hive/blob/b870d526/common/src/java/org/apache/hadoop/hive/common/LogUtils.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/LogUtils.java b/common/src/java/org/apache/hadoop/hive/common/LogUtils.java
index adcf805..599e798 100644
--- a/common/src/java/org/apache/hadoop/hive/common/LogUtils.java
+++ b/common/src/java/org/apache/hadoop/hive/common/LogUtils.java
@@ -21,12 +21,18 @@ package org.apache.hadoop.hive.common;
 import java.io.File;
 import java.net.URL;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.core.LoggerContext;
 import org.apache.logging.log4j.core.config.Configurator;
+import org.apache.logging.log4j.core.impl.Log4jContextFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * Utilities common to logging operations.
  */
@@ -66,8 +72,14 @@ public class LogUtils {
   }
 
   private static String initHiveLog4jCommon(ConfVars confVarName)
-    throws LogInitializationException {
+      throws LogInitializationException {
     HiveConf conf = new HiveConf();
+    return initHiveLog4jCommon(conf, confVarName);
+  }
+
+  @VisibleForTesting
+  public static String initHiveLog4jCommon(HiveConf conf, ConfVars confVarName)
+    throws LogInitializationException {
     if (HiveConf.getVar(conf, confVarName).equals("")) {
       // if log4j configuration file not set, or could not found, use default setting
       return initHiveLog4jDefault(conf, "", confVarName);
@@ -91,13 +103,28 @@ public class LogUtils {
           }
           System.setProperty(HiveConf.ConfVars.HIVEQUERYID.toString(), queryId);
         }
+        final boolean async = checkAndSetAsyncLogging(conf);
         Configurator.initialize(null, log4jFileName);
         logConfigLocation(conf);
-        return ("Logging initialized using configuration in " + log4jConfigFile);
+        return "Logging initialized using configuration in " + log4jConfigFile + " Async: " + async;
       }
     }
   }
 
+  public static boolean checkAndSetAsyncLogging(final Configuration conf) {
+    final boolean asyncLogging = HiveConf.getBoolVar(conf, ConfVars.HIVE_ASYNC_LOG_ENABLED);
+    if (asyncLogging) {
+      System.setProperty("Log4jContextSelector",
+          "org.apache.logging.log4j.core.async.AsyncLoggerContextSelector");
+      // default is ClassLoaderContextSelector which is created during automatic logging
+      // initialization in a static initialization block.
+      // Changing ContextSelector at runtime requires creating new context factory which will
+      // internally create new context selector based on system property.
+      LogManager.setFactory(new Log4jContextFactory());
+    }
+    return asyncLogging;
+  }
+
   private static String initHiveLog4jDefault(
     HiveConf conf, String logMessage, ConfVars confVarName)
     throws LogInitializationException {
@@ -118,9 +145,11 @@ public class LogUtils {
         break;
     }
     if (hive_l4j != null) {
+      final boolean async = checkAndSetAsyncLogging(conf);
       Configurator.initialize(null, hive_l4j.toString());
       logConfigLocation(conf);
-      return (logMessage + "\n" + "Logging initialized using configuration in " + hive_l4j);
+      return (logMessage + "\n" + "Logging initialized using configuration in " + hive_l4j +
+          " Async: " + async);
     } else {
       throw new LogInitializationException(
         logMessage + "Unable to initialize logging using "

http://git-wip-us.apache.org/repos/asf/hive/blob/b870d526/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index bb74d99..07dff08 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -1955,6 +1955,12 @@ public class HiveConf extends Configuration {
         "If the property is not set, then logging will be initialized using hive-exec-log4j2.properties found on the classpath.\n" +
         "If the property is set, the value must be a valid URI (java.net.URI, e.g. \"file:///tmp/my-logging.xml\"), \n" +
         "which you can then extract a URL from and pass to PropertyConfigurator.configure(URL)."),
+    HIVE_ASYNC_LOG_ENABLED("hive.async.log.enabled", true,
+        "Whether to enable Log4j2's asynchronous logging. Asynchronous logging can give\n" +
+        " significant performance improvement as logging will be handled in separate thread\n" +
+        " that uses LMAX disruptor queue for buffering log messages.\n" +
+        " Refer https://logging.apache.org/log4j/2.x/manual/async.html for benefits and\n" +
+        " drawbacks."),
 
     HIVE_LOG_EXPLAIN_OUTPUT("hive.log.explain.output", false,
         "Whether to log explain output for every query.\n" +

http://git-wip-us.apache.org/repos/asf/hive/blob/b870d526/common/src/main/resources/hive-log4j2.properties
----------------------------------------------------------------------
diff --git a/common/src/main/resources/hive-log4j2.properties b/common/src/main/resources/hive-log4j2.properties
index cf0369a..2f67be8 100644
--- a/common/src/main/resources/hive-log4j2.properties
+++ b/common/src/main/resources/hive-log4j2.properties
@@ -36,7 +36,7 @@ appender.console.layout.type = PatternLayout
 appender.console.layout.pattern = %d{yy/MM/dd HH:mm:ss} [%t]: %p %c{2}: %m%n
 
 # daily rolling file appender
-appender.DRFA.type = RollingFile
+appender.DRFA.type = RollingRandomAccessFile
 appender.DRFA.name = DRFA
 appender.DRFA.fileName = ${sys:hive.log.dir}/${sys:hive.log.file}
 # Use %pid in the filePattern to append <process-id>@<host-name> to the filename if you want separate log files for different CLI session

http://git-wip-us.apache.org/repos/asf/hive/blob/b870d526/common/src/test/org/apache/hadoop/hive/conf/TestHiveAsyncLogging.java
----------------------------------------------------------------------
diff --git a/common/src/test/org/apache/hadoop/hive/conf/TestHiveAsyncLogging.java b/common/src/test/org/apache/hadoop/hive/conf/TestHiveAsyncLogging.java
new file mode 100644
index 0000000..e2631cf
--- /dev/null
+++ b/common/src/test/org/apache/hadoop/hive/conf/TestHiveAsyncLogging.java
@@ -0,0 +1,49 @@
+/**
+ * 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.
+ */
+package org.apache.hadoop.hive.conf;
+
+import org.apache.hadoop.hive.common.LogUtils;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.core.async.AsyncLoggerContextSelector;
+import org.apache.logging.log4j.core.impl.Log4jContextFactory;
+import org.apache.logging.log4j.core.selector.ClassLoaderContextSelector;
+import org.apache.logging.log4j.core.selector.ContextSelector;
+import org.junit.Test;
+
+import junit.framework.TestCase;
+
+public class TestHiveAsyncLogging extends TestCase {
+
+  // this test requires disruptor jar in classpath
+  @Test
+  public void testAsyncLoggingInitialization() throws Exception {
+    HiveConf conf = new HiveConf();
+    conf.setBoolVar(ConfVars.HIVE_ASYNC_LOG_ENABLED, false);
+    LogUtils.initHiveLog4jCommon(conf, ConfVars.HIVE_LOG4J_FILE);
+    Log4jContextFactory log4jContextFactory = (Log4jContextFactory) LogManager.getFactory();
+    ContextSelector contextSelector = log4jContextFactory.getSelector();
+    assertTrue(contextSelector instanceof ClassLoaderContextSelector);
+
+    conf.setBoolVar(ConfVars.HIVE_ASYNC_LOG_ENABLED, true);
+    LogUtils.initHiveLog4jCommon(conf, ConfVars.HIVE_LOG4J_FILE);
+    log4jContextFactory = (Log4jContextFactory) LogManager.getFactory();
+    contextSelector = log4jContextFactory.getSelector();
+    assertTrue(contextSelector instanceof AsyncLoggerContextSelector);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/b870d526/data/conf/hive-log4j2.properties
----------------------------------------------------------------------
diff --git a/data/conf/hive-log4j2.properties b/data/conf/hive-log4j2.properties
index 6bace1f..f60d5be 100644
--- a/data/conf/hive-log4j2.properties
+++ b/data/conf/hive-log4j2.properties
@@ -35,7 +35,7 @@ appender.console.layout.type = PatternLayout
 appender.console.layout.pattern = %d{yy/MM/dd HH:mm:ss} [%t]: %p %c{2}: %m%n
 
 # daily rolling file appender
-appender.DRFA.type = RollingFile
+appender.DRFA.type = RollingRandomAccessFile
 appender.DRFA.name = DRFA
 appender.DRFA.fileName = ${sys:hive.log.dir}/${sys:hive.log.file}
 appender.DRFA.filePattern = ${sys:hive.log.dir}/${sys:hive.log.file}.%d{yyyy-MM-dd}

http://git-wip-us.apache.org/repos/asf/hive/blob/b870d526/hcatalog/core/pom.xml
----------------------------------------------------------------------
diff --git a/hcatalog/core/pom.xml b/hcatalog/core/pom.xml
index 1e970bf..c9a6c01 100644
--- a/hcatalog/core/pom.xml
+++ b/hcatalog/core/pom.xml
@@ -131,6 +131,12 @@
       <scope>test</scope>
     </dependency>
     <dependency>
+      <groupId>com.lmax</groupId>
+      <artifactId>disruptor</artifactId>
+      <version>${disruptor.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-common</artifactId>
       <version>${hadoop.version}</version>

http://git-wip-us.apache.org/repos/asf/hive/blob/b870d526/llap-server/bin/llapDaemon.sh
----------------------------------------------------------------------
diff --git a/llap-server/bin/llapDaemon.sh b/llap-server/bin/llapDaemon.sh
index 6f57998..566bbc8 100755
--- a/llap-server/bin/llapDaemon.sh
+++ b/llap-server/bin/llapDaemon.sh
@@ -113,7 +113,7 @@ case $startStop in
     #rotate_log $logOut
     echo starting llapdaemon, logging to $logLog and $logOut
     export LLAP_DAEMON_LOGFILE=${LLAP_DAEMON_LOG_BASE}.log
-    nohup nice -n $LLAP_DAEMON_NICENESS "$LLAP_DAEMON_BIN_HOME"/runLlapDaemon.sh run  > "$logOut" 2>&1 < /dev/null &
+    nohup nice -n $LLAP_DAEMON_NICENESS "$LLAP_DAEMON_BIN_HOME"/runLlapDaemon.sh run  >> "$logOut" 2>&1 < /dev/null &
     echo $! > $pid
     ;;
           

http://git-wip-us.apache.org/repos/asf/hive/blob/b870d526/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 e662de9..2e07a8c 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
@@ -31,6 +31,7 @@ import java.util.regex.Pattern;
 import javax.management.ObjectName;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.common.LogUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.llap.DaemonId;
@@ -119,7 +120,8 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
       int mngPort, int shufflePort, int webPort, String appName) {
     super("LlapDaemon");
 
-    initializeLogging();
+    initializeLogging(daemonConf);
+
     printAsciiArt();
 
     Preconditions.checkArgument(numExecutors > 0);
@@ -264,13 +266,15 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
     addIfService(amReporter);
   }
 
-  private void initializeLogging() {
+  private void initializeLogging(final Configuration conf) {
     long start = System.currentTimeMillis();
     URL llap_l4j2 = LlapDaemon.class.getClassLoader().getResource(LOG4j2_PROPERTIES_FILE);
     if (llap_l4j2 != null) {
+      final boolean async = LogUtils.checkAndSetAsyncLogging(conf);
       Configurator.initialize("LlapDaemonLog4j2", llap_l4j2.toString());
       long end = System.currentTimeMillis();
-      LOG.warn("LLAP daemon logging initialized from {} in {} ms", llap_l4j2, (end - start));
+      LOG.warn("LLAP daemon logging initialized from {} in {} ms. Async: {}",
+          llap_l4j2, (end - start), async);
     } else {
       throw new RuntimeException("Log initialization failed." +
           " Unable to locate " + LOG4j2_PROPERTIES_FILE + " file in classpath");

http://git-wip-us.apache.org/repos/asf/hive/blob/b870d526/llap-server/src/main/resources/llap-cli-log4j2.properties
----------------------------------------------------------------------
diff --git a/llap-server/src/main/resources/llap-cli-log4j2.properties b/llap-server/src/main/resources/llap-cli-log4j2.properties
index 2f27b5e..c6b8f20 100644
--- a/llap-server/src/main/resources/llap-cli-log4j2.properties
+++ b/llap-server/src/main/resources/llap-cli-log4j2.properties
@@ -36,7 +36,7 @@ appender.console.layout.type = PatternLayout
 appender.console.layout.pattern = %p %c{2}: %m%n
 
 # daily rolling file appender
-appender.DRFA.type = RollingFile
+appender.DRFA.type = RollingRandomAccessFile
 appender.DRFA.name = DRFA
 appender.DRFA.fileName = ${sys:hive.log.dir}/${sys:hive.log.file}
 # Use %pid in the filePattern to append <process-id>@<host-name> to the filename if you want separate log files for different CLI session

http://git-wip-us.apache.org/repos/asf/hive/blob/b870d526/llap-server/src/main/resources/llap-daemon-log4j2.properties
----------------------------------------------------------------------
diff --git a/llap-server/src/main/resources/llap-daemon-log4j2.properties b/llap-server/src/main/resources/llap-daemon-log4j2.properties
index 268eb59..c5166e3 100644
--- a/llap-server/src/main/resources/llap-daemon-log4j2.properties
+++ b/llap-server/src/main/resources/llap-daemon-log4j2.properties
@@ -38,7 +38,7 @@ appender.console.layout.type = PatternLayout
 appender.console.layout.pattern = %d{yy/MM/dd HH:mm:ss} [%t%x] %p %c{2} : %m%n
 
 # rolling file appender
-appender.RFA.type = RollingFile
+appender.RFA.type = RollingRandomAccessFile
 appender.RFA.name = RFA
 appender.RFA.fileName = ${sys:llap.daemon.log.dir}/${sys:llap.daemon.log.file}
 appender.RFA.filePattern = ${sys:llap.daemon.log.dir}/${sys:llap.daemon.log.file}_%i
@@ -51,7 +51,7 @@ appender.RFA.strategy.type = DefaultRolloverStrategy
 appender.RFA.strategy.max = ${sys:llap.daemon.log.maxbackupindex}
 
 # history file appender
-appender.HISTORYAPPENDER.type = RollingFile
+appender.HISTORYAPPENDER.type = RollingRandomAccessFile
 appender.HISTORYAPPENDER.name = HISTORYAPPENDER
 appender.HISTORYAPPENDER.fileName = ${sys:llap.daemon.log.dir}/${sys:llap.daemon.historylog.file}
 appender.HISTORYAPPENDER.filePattern = ${sys:llap.daemon.log.dir}/${sys:llap.daemon.historylog.file}_%i

http://git-wip-us.apache.org/repos/asf/hive/blob/b870d526/llap-server/src/test/resources/llap-daemon-log4j2.properties
----------------------------------------------------------------------
diff --git a/llap-server/src/test/resources/llap-daemon-log4j2.properties b/llap-server/src/test/resources/llap-daemon-log4j2.properties
index 7b5f4ed..2714dbd 100644
--- a/llap-server/src/test/resources/llap-daemon-log4j2.properties
+++ b/llap-server/src/test/resources/llap-daemon-log4j2.properties
@@ -38,7 +38,7 @@ appender.console.layout.type = PatternLayout
 appender.console.layout.pattern = %d{yy/MM/dd HH:mm:ss} [%t%x] %p %c{2} : %m%n
 
 # rolling file appender
-appender.RFA.type = RollingFile
+appender.RFA.type = RollingRandomAccessFile
 appender.RFA.name = RFA
 appender.RFA.fileName = ${sys:llap.daemon.log.dir}/${sys:llap.daemon.log.file}
 appender.RFA.filePattern = ${sys:llap.daemon.log.dir}/${sys:llap.daemon.log.file}_%i
@@ -51,7 +51,7 @@ appender.RFA.strategy.type = DefaultRolloverStrategy
 appender.RFA.strategy.max = ${sys:llap.daemon.log.maxbackupindex}
 
 # history file appender
-appender.HISTORYAPPENDER.type = RollingFile
+appender.HISTORYAPPENDER.type = RollingRandomAccessFile
 appender.HISTORYAPPENDER.name = HISTORYAPPENDER
 appender.HISTORYAPPENDER.fileName = ${sys:llap.daemon.log.dir}/${sys:llap.daemon.historylog.file}
 appender.HISTORYAPPENDER.filePattern = ${sys:llap.daemon.log.dir}/${sys:llap.daemon.historylog.file}_%i

http://git-wip-us.apache.org/repos/asf/hive/blob/b870d526/metastore/pom.xml
----------------------------------------------------------------------
diff --git a/metastore/pom.xml b/metastore/pom.xml
index 8816829..3827a51 100644
--- a/metastore/pom.xml
+++ b/metastore/pom.xml
@@ -228,6 +228,12 @@
       <version>${mockito-all.version}</version>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>com.lmax</groupId>
+      <artifactId>disruptor</artifactId>
+      <version>${disruptor.version}</version>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <profiles>

http://git-wip-us.apache.org/repos/asf/hive/blob/b870d526/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index dff2a72..bde6857 100644
--- a/pom.xml
+++ b/pom.xml
@@ -134,6 +134,8 @@
     <hadoop.version>2.6.0</hadoop.version>
     <hadoop.bin.path>${basedir}/${hive.path.to.root}/testutils/hadoop</hadoop.bin.path>
     <hbase.version>1.1.1</hbase.version>
+    <!-- required for logging test to avoid including hbase which pulls disruptor transitively -->
+    <disruptor.version>3.3.0</disruptor.version>
     <!-- httpcomponents are not always in version sync -->
     <httpcomponents.client.version>4.4</httpcomponents.client.version>
     <httpcomponents.core.version>4.4</httpcomponents.core.version>

http://git-wip-us.apache.org/repos/asf/hive/blob/b870d526/ql/pom.xml
----------------------------------------------------------------------
diff --git a/ql/pom.xml b/ql/pom.xml
index aaa3271..8b2d0e6 100644
--- a/ql/pom.xml
+++ b/ql/pom.xml
@@ -325,6 +325,12 @@
       <scope>test</scope>
     </dependency>
     <dependency>
+      <groupId>com.lmax</groupId>
+      <artifactId>disruptor</artifactId>
+      <version>${disruptor.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>org.codehaus.groovy</groupId>
       <artifactId>groovy-all</artifactId>
       <version>${groovy.version}</version>

http://git-wip-us.apache.org/repos/asf/hive/blob/b870d526/ql/src/main/resources/hive-exec-log4j2.properties
----------------------------------------------------------------------
diff --git a/ql/src/main/resources/hive-exec-log4j2.properties b/ql/src/main/resources/hive-exec-log4j2.properties
index 4fba04c..21e24fd 100644
--- a/ql/src/main/resources/hive-exec-log4j2.properties
+++ b/ql/src/main/resources/hive-exec-log4j2.properties
@@ -36,7 +36,7 @@ appender.console.layout.type = PatternLayout
 appender.console.layout.pattern = %d{yy/MM/dd HH:mm:ss} [%t]: %p %c{2}: %m%n
 
 # simple file appender
-appender.FA.type = File
+appender.FA.type = RandomAccessFile
 appender.FA.name = FA
 appender.FA.fileName = ${sys:hive.log.dir}/${sys:hive.log.file}
 appender.FA.layout.type = PatternLayout

http://git-wip-us.apache.org/repos/asf/hive/blob/b870d526/ql/src/main/resources/tez-container-log4j2.properties
----------------------------------------------------------------------
diff --git a/ql/src/main/resources/tez-container-log4j2.properties b/ql/src/main/resources/tez-container-log4j2.properties
index 5d2b138..a048b17 100644
--- a/ql/src/main/resources/tez-container-log4j2.properties
+++ b/ql/src/main/resources/tez-container-log4j2.properties
@@ -28,7 +28,7 @@ property.tez.container.log.file = syslog
 appenders = CLA
 
 # daily rolling file appender
-appender.CLA.type = RollingFile
+appender.CLA.type = RollingRandomAccessFile
 appender.CLA.name = CLA
 appender.CLA.fileName = ${sys:tez.container.log.dir}/${sys:tez.container.log.file}
 appender.CLA.filePattern = ${sys:tez.container.log.dir}/${sys:tez.container.log.file}.%d{yyyy-MM-dd}