You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by se...@apache.org on 2015/11/02 22:08:35 UTC

[1/2] hive git commit: HIVE-12292 : revert the if removal from HIVE-12237 (Sergey Shelukhin, reviewed by Ashutosh Chauhan)

Repository: hive
Updated Branches:
  refs/heads/master b7986a8fb -> 71da33a6a


HIVE-12292 : revert the if removal from HIVE-12237 (Sergey Shelukhin, reviewed by Ashutosh Chauhan)


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

Branch: refs/heads/master
Commit: 526ae444100a618feeb4dc180a38b8b76d54179b
Parents: b7986a8
Author: Sergey Shelukhin <se...@apache.org>
Authored: Mon Nov 2 13:03:01 2015 -0800
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Mon Nov 2 13:03:01 2015 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/cli/CliDriver.java   |  4 +-
 .../hadoop/hive/llap/cache/BuddyAllocator.java  |  9 ++--
 .../hive/llap/cache/LowLevelCacheImpl.java      |  5 +-
 .../llap/cache/LowLevelCacheMemoryManager.java  |  5 +-
 .../llap/cache/LowLevelFifoCachePolicy.java     |  4 +-
 .../llap/cache/LowLevelLrfuCachePolicy.java     |  7 +--
 .../hive/llap/io/api/impl/LlapIoImpl.java       | 17 +++++--
 .../llap/io/decode/OrcColumnVectorProducer.java |  9 ++--
 .../llap/io/encoded/OrcEncodedDataReader.java   |  7 +--
 .../org/apache/hadoop/hive/llap/LogLevels.java  | 53 ++++++++++++++++++++
 10 files changed, 95 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/526ae444/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java
----------------------------------------------------------------------
diff --git a/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java b/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java
index 82d064d..b359850 100644
--- a/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java
+++ b/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java
@@ -107,7 +107,9 @@ public class CliDriver {
     SessionState ss = SessionState.get();
     conf = (ss != null) ? ss.getConf() : new Configuration();
     Logger LOG = LoggerFactory.getLogger("CliDriver");
-    LOG.debug("CliDriver inited with classpath {}", System.getProperty("java.class.path"));
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("CliDriver inited with classpath {}", System.getProperty("java.class.path"));
+    }
     console = new LogHelper(LOG);
     originalThreadName = Thread.currentThread().getName();
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/526ae444/llap-server/src/java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java
index f69ac5b..2aca68d 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java
@@ -48,10 +48,11 @@ public final class BuddyAllocator implements EvictionAwareAllocator, BuddyAlloca
     maxAllocation = HiveConf.getIntVar(conf, ConfVars.LLAP_ORC_CACHE_MAX_ALLOC);
     arenaSize = HiveConf.getIntVar(conf, ConfVars.LLAP_ORC_CACHE_ARENA_SIZE);
     long maxSizeVal = HiveConf.getLongVar(conf, ConfVars.LLAP_ORC_CACHE_MAX_SIZE);
-    LlapIoImpl.LOG.info("Buddy allocator with {}", (isDirect ? "direct" : "byte")
-        , " buffers; allocation sizes {} ", minAllocation, " - {}", maxAllocation
-        , ", arena size {}", arenaSize, ". total size {}", maxSizeVal);
-
+    if (LlapIoImpl.LOGL.isInfoEnabled()) {
+      LlapIoImpl.LOG.info("Buddy allocator with " + (isDirect ? "direct" : "byte")
+          + " buffers; allocation sizes " + minAllocation + " - " + maxAllocation
+          + ", arena size " + arenaSize + ". total size " + maxSizeVal);
+    }
 
     if (minAllocation < 8) {
       throw new AssertionError("Min allocation must be at least 8: " + minAllocation);

http://git-wip-us.apache.org/repos/asf/hive/blob/526ae444/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelCacheImpl.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelCacheImpl.java b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelCacheImpl.java
index e7b8f1a..c2a130a 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelCacheImpl.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelCacheImpl.java
@@ -58,8 +58,9 @@ public class LowLevelCacheImpl implements LowLevelCache, LlapOomDebugDump {
   @VisibleForTesting
   LowLevelCacheImpl(LlapDaemonCacheMetrics metrics, LowLevelCachePolicy cachePolicy,
       EvictionAwareAllocator allocator, boolean doAssumeGranularBlocks, long cleanupInterval) {
-      LlapIoImpl.LOG.info("Low level cache; cleanup interval {}", cleanupInterval, "sec");
-
+    if (LlapIoImpl.LOGL.isInfoEnabled()) {
+      LlapIoImpl.LOG.info("Low level cache; cleanup interval " + cleanupInterval + "sec");
+    }
     this.cachePolicy = cachePolicy;
     this.allocator = allocator;
     this.cleanupInterval = cleanupInterval;

http://git-wip-us.apache.org/repos/asf/hive/blob/526ae444/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelCacheMemoryManager.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelCacheMemoryManager.java b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelCacheMemoryManager.java
index 8a39e35..4a256ee 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelCacheMemoryManager.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelCacheMemoryManager.java
@@ -44,8 +44,9 @@ public class LowLevelCacheMemoryManager implements MemoryManager {
     this.usedMemory = new AtomicLong(0);
     this.metrics = metrics;
     metrics.incrCacheCapacityTotal(maxSize);
-    LlapIoImpl.LOG.info("Cache memory manager initialized with max size {}", maxSize);
-
+    if (LlapIoImpl.LOGL.isInfoEnabled()) {
+      LlapIoImpl.LOG.info("Cache memory manager initialized with max size " + maxSize);
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/526ae444/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelFifoCachePolicy.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelFifoCachePolicy.java b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelFifoCachePolicy.java
index 0838682..1430eae 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelFifoCachePolicy.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelFifoCachePolicy.java
@@ -35,7 +35,9 @@ public class LowLevelFifoCachePolicy implements LowLevelCachePolicy {
   private LlapOomDebugDump parentDebugDump;
 
   public LowLevelFifoCachePolicy(Configuration conf) {
-    LlapIoImpl.LOG.info("FIFO cache policy");
+    if (LlapIoImpl.LOGL.isInfoEnabled()) {
+      LlapIoImpl.LOG.info("FIFO cache policy");
+    }
     buffers = new LinkedList<LlapCacheableBuffer>();
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/526ae444/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelLrfuCachePolicy.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelLrfuCachePolicy.java b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelLrfuCachePolicy.java
index 49e1b59..76e7605 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelLrfuCachePolicy.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelLrfuCachePolicy.java
@@ -78,9 +78,10 @@ public class LowLevelLrfuCachePolicy implements LowLevelCachePolicy {
       int lrfuThreshold = (int)((Math.log(1 - Math.pow(0.5, lambda)) / Math.log(0.5)) / lambda);
       maxHeapSize = Math.min(lrfuThreshold, maxBuffers);
     }
-    LlapIoImpl.LOG.info("LRFU cache policy with min buffer size {}", minBufferSize
-        , " and lambda {}", lambda, " (heap size {} ", maxHeapSize + ")");
-
+    if (LlapIoImpl.LOGL.isInfoEnabled()) {
+      LlapIoImpl.LOG.info("LRFU cache policy with min buffer size " + minBufferSize
+          + " and lambda " + lambda + " (heap size " + maxHeapSize + ")");
+    }
 
     heap = new LlapCacheableBuffer[maxHeapSize];
     listHead = listTail = null;

http://git-wip-us.apache.org/repos/asf/hive/blob/526ae444/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapIoImpl.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapIoImpl.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapIoImpl.java
index 83a88f5..b38f472 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapIoImpl.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapIoImpl.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.hive.llap.io.api.impl;
 
+import org.apache.hadoop.hive.llap.LogLevels;
+
 import java.io.IOException;
 import java.util.concurrent.Executors;
 
@@ -56,19 +58,21 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
 
 public class LlapIoImpl implements LlapIo<VectorizedRowBatch> {
   public static final Logger LOG = LoggerFactory.getLogger(LlapIoImpl.class);
+  public static final LogLevels LOGL = new LogLevels(LOG);
 
   private final ColumnVectorProducer cvp;
   private final ListeningExecutorService executor;
-  private final LlapDaemonCacheMetrics cacheMetrics;
-  private final LlapDaemonQueueMetrics queueMetrics;
+  private LlapDaemonCacheMetrics cacheMetrics;
+  private LlapDaemonQueueMetrics queueMetrics;
   private ObjectName buddyAllocatorMXBean;
   private EvictionAwareAllocator allocator;
 
   private LlapIoImpl(Configuration conf) throws IOException {
     boolean useLowLevelCache = HiveConf.getBoolVar(conf, HiveConf.ConfVars.LLAP_LOW_LEVEL_CACHE);
     // High-level cache not supported yet.
-    LOG.info("Initializing LLAP IO {}", useLowLevelCache ? " with low level cache" : "");
-
+    if (LOGL.isInfoEnabled()) {
+      LOG.info("Initializing LLAP IO" + (useLowLevelCache ? " with low level cache" : ""));
+    }
 
     String displayName = "LlapDaemonCacheMetrics-" + MetricsUtils.getHostName();
     String sessionId = conf.get("llap.daemon.metrics.sessionid");
@@ -111,7 +115,10 @@ public class LlapIoImpl implements LlapIo<VectorizedRowBatch> {
     // TODO: this should depends on input format and be in a map, or something.
     this.cvp = new OrcColumnVectorProducer(metadataCache, orcCache, cache, conf, cacheMetrics,
         queueMetrics);
-    LOG.info("LLAP IO initialized");
+    if (LOGL.isInfoEnabled()) {
+      LOG.info("LLAP IO initialized");
+    }
+
     registerMXBeans();
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/526ae444/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcColumnVectorProducer.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcColumnVectorProducer.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcColumnVectorProducer.java
index 38c31d3..259c483 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcColumnVectorProducer.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcColumnVectorProducer.java
@@ -43,15 +43,16 @@ public class OrcColumnVectorProducer implements ColumnVectorProducer {
   private final Cache<OrcCacheKey> cache;
   private final LowLevelCache lowLevelCache;
   private final Configuration conf;
-  private final boolean _skipCorrupt; // TODO: get rid of this
-  private final LlapDaemonCacheMetrics cacheMetrics;
-  private final LlapDaemonQueueMetrics queueMetrics;
+  private boolean _skipCorrupt; // TODO: get rid of this
+  private LlapDaemonCacheMetrics cacheMetrics;
+  private LlapDaemonQueueMetrics queueMetrics;
 
   public OrcColumnVectorProducer(OrcMetadataCache metadataCache,
       LowLevelCacheImpl lowLevelCache, Cache<OrcCacheKey> cache, Configuration conf,
       LlapDaemonCacheMetrics metrics, LlapDaemonQueueMetrics queueMetrics) {
+    if (LlapIoImpl.LOGL.isInfoEnabled()) {
       LlapIoImpl.LOG.info("Initializing ORC column vector producer");
-
+    }
 
     this.metadataCache = metadataCache;
     this.lowLevelCache = lowLevelCache;

http://git-wip-us.apache.org/repos/asf/hive/blob/526ae444/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java
index e625490..9bdafc9 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java
@@ -198,8 +198,9 @@ public class OrcEncodedDataReader extends CallableWithNdc<Void>
 
   protected Void performDataRead() throws IOException {
     long startTime = counters.startTimeCounter();
-    LlapIoImpl.LOG.info("Processing data for {}", split.getPath());
-
+    if (LlapIoImpl.LOGL.isInfoEnabled()) {
+      LlapIoImpl.LOG.info("Processing data for " + split.getPath());
+    }
     if (processStop()) {
       recordReaderTime(startTime);
       return null;
@@ -744,7 +745,7 @@ public class OrcEncodedDataReader extends CallableWithNdc<Void>
     long offset = split.getStart(), maxOffset = offset + split.getLength();
     stripeIxFrom = -1;
     int stripeIxTo = -1;
-    if (LlapIoImpl.LOG.isDebugEnabled()) {
+    if (LlapIoImpl.LOGL.isDebugEnabled()) {
       String tmp = "FileSplit {" + split.getStart() + ", " + split.getLength() + "}; stripes ";
       for (StripeInformation stripe : stripes) {
         tmp += "{" + stripe.getOffset() + ", " + stripe.getLength() + "}, ";

http://git-wip-us.apache.org/repos/asf/hive/blob/526ae444/ql/src/java/org/apache/hadoop/hive/llap/LogLevels.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/llap/LogLevels.java b/ql/src/java/org/apache/hadoop/hive/llap/LogLevels.java
new file mode 100644
index 0000000..300230f
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/llap/LogLevels.java
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional debugrmation
+ * 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.llap;
+
+import org.slf4j.Logger;
+
+public class LogLevels {
+  private final boolean isT, isD, isI, isW, isE;
+
+  public LogLevels(Logger log) {
+    isT = log.isTraceEnabled();
+    isD = log.isDebugEnabled();
+    isI = log.isInfoEnabled();
+    isW = log.isWarnEnabled();
+    isE = log.isErrorEnabled();
+  }
+
+  public boolean isTraceEnabled() {
+    return isT;
+  }
+
+  public boolean isDebugEnabled() {
+    return isD;
+  }
+
+  public boolean isInfoEnabled() {
+    return isI;
+  }
+
+  public boolean isWarnEnabled() {
+    return isW;
+  }
+
+  public boolean isErrorEnabled() {
+    return isE;
+  }
+}


[2/2] hive git commit: HIVE-12295 : change some logs from info to debug (Sergey Shelukhin, reviewed by Ashutosh Chauhan)

Posted by se...@apache.org.
HIVE-12295 : change some logs from info to debug (Sergey Shelukhin, reviewed by Ashutosh Chauhan)


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

Branch: refs/heads/master
Commit: 71da33a6a4e878914299616c7c9d5d2ea181b066
Parents: 526ae44
Author: Sergey Shelukhin <se...@apache.org>
Authored: Mon Nov 2 13:07:04 2015 -0800
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Mon Nov 2 13:07:04 2015 -0800

----------------------------------------------------------------------
 .../llap/daemon/impl/ContainerRunnerImpl.java   |  2 +-
 .../llap/daemon/impl/TaskExecutorService.java   | 21 +++++-----
 .../llap/daemon/impl/TaskRunnerCallable.java    |  4 +-
 .../llap/io/encoded/OrcEncodedDataReader.java   |  4 +-
 .../llap/shufflehandler/ShuffleHandler.java     | 13 +++---
 .../dag/app/rm/LlapTaskSchedulerService.java    |  6 +--
 .../hive/metastore/AggregateStatsCache.java     |  2 +-
 .../hadoop/hive/ql/exec/MapJoinOperator.java    |  8 ++--
 .../hadoop/hive/ql/exec/MapredContext.java      |  2 +-
 .../apache/hadoop/hive/ql/exec/Operator.java    | 42 ++++++++++----------
 .../apache/hadoop/hive/ql/exec/Utilities.java   |  6 +--
 .../hadoop/hive/ql/exec/mr/ObjectCache.java     | 10 ++---
 .../hive/ql/exec/tez/LlapObjectCache.java       | 18 +++++----
 .../hadoop/hive/ql/exec/tez/TezProcessor.java   | 10 +++--
 .../ql/exec/vector/VectorGroupByOperator.java   |  2 +-
 .../ql/exec/vector/VectorizationContext.java    | 11 +++--
 .../ql/io/HiveContextAwareRecordReader.java     |  2 +-
 .../hadoop/hive/ql/io/HiveInputFormat.java      |  4 +-
 .../physical/NullScanTaskDispatcher.java        |  4 +-
 .../hive/ql/optimizer/physical/Vectorizer.java  | 11 +++--
 .../hadoop/hive/ql/ppd/OpProcFactory.java       | 18 ++++++---
 21 files changed, 114 insertions(+), 86 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/71da33a6/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/ContainerRunnerImpl.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/ContainerRunnerImpl.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/ContainerRunnerImpl.java
index fad2d2c..4b28b53 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/ContainerRunnerImpl.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/ContainerRunnerImpl.java
@@ -191,7 +191,7 @@ public class ContainerRunnerImpl extends CompositeService implements ContainerRu
 
       Token<JobTokenIdentifier> jobToken = TokenCache.getSessionToken(credentials);
 
-      LOG.info("DEBUG: Registering request with the ShuffleHandler");
+      LOG.debug("Registering request with the ShuffleHandler");
       ShuffleHandler.get()
           .registerDag(request.getApplicationIdString(), dagIdentifier, jobToken,
               request.getUser(), localDirs);

http://git-wip-us.apache.org/repos/asf/hive/blob/71da33a6/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorService.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorService.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorService.java
index 875aef6..5e2c6dd 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorService.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorService.java
@@ -331,8 +331,8 @@ public class TaskExecutorService extends AbstractService implements Scheduler<Ta
       if (evictedTask != taskWrapper) {
         knownTasks.put(taskWrapper.getRequestId(), taskWrapper);
         taskWrapper.setIsInWaitQueue(true);
-        if (isInfoEnabled) {
-          LOG.info("{} added to wait queue. Current wait queue size={}", task.getRequestId(),
+        if (isDebugEnabled) {
+          LOG.debug("{} added to wait queue. Current wait queue size={}", task.getRequestId(),
               waitQueue.size());
         }
       } else {
@@ -413,8 +413,9 @@ public class TaskExecutorService extends AbstractService implements Scheduler<Ta
         // is actually available for execution and will not potentially result in a RejectedExecution
         Futures.addCallback(future, wrappedCallback, executionCompletionExecutorService);
 
-        if (isInfoEnabled) {
-          LOG.info("{} scheduled for execution. canFinish={}", taskWrapper.getRequestId(), canFinish);
+        if (isDebugEnabled) {
+          LOG.debug("{} scheduled for execution. canFinish={}",
+              taskWrapper.getRequestId(), canFinish);
         }
 
         // only tasks that cannot finish immediately are pre-emptable. In other words, if all inputs
@@ -465,7 +466,7 @@ public class TaskExecutorService extends AbstractService implements Scheduler<Ta
     synchronized (lock) {
       if (taskWrapper.isInWaitQueue()) {
         // Re-order the wait queue
-        LOG.info("DEBUG: Re-ordering the wait queue since {} finishable state moved to {}",
+        LOG.debug("Re-ordering the wait queue since {} finishable state moved to {}",
             taskWrapper.getRequestId(), newFinishableState);
         if (waitQueue.remove(taskWrapper)) {
           // Put element back only if it existed.
@@ -477,12 +478,12 @@ public class TaskExecutorService extends AbstractService implements Scheduler<Ta
       }
 
       if (newFinishableState == true && taskWrapper.isInPreemptionQueue()) {
-        LOG.info("DEBUG: Removing {} from preemption queue because it's state changed to {}",
+        LOG.debug("Removing {} from preemption queue because it's state changed to {}",
             taskWrapper.getRequestId(), newFinishableState);
         preemptionQueue.remove(taskWrapper.getTaskRunnerCallable());
       } else if (newFinishableState == false && !taskWrapper.isInPreemptionQueue() &&
           !taskWrapper.isInWaitQueue()) {
-        LOG.info("DEBUG: Adding {} to preemption queue since finishable state changed to {}",
+        LOG.debug("Adding {} to preemption queue since finishable state changed to {}",
             taskWrapper.getRequestId(), newFinishableState);
         preemptionQueue.offer(taskWrapper);
       }
@@ -554,9 +555,11 @@ public class TaskExecutorService extends AbstractService implements Scheduler<Ta
       }
 
       numSlotsAvailable.incrementAndGet();
-      LOG.info("Task {} complete. WaitQueueSize={}, numSlotsAvailable={}, preemptionQueueSize={}",
+      if (isDebugEnabled) {
+        LOG.debug("Task {} complete. WaitQueueSize={}, numSlotsAvailable={}, preemptionQueueSize={}",
           taskWrapper.getRequestId(), waitQueue.size(), numSlotsAvailable.get(),
           preemptionQueue.size());
+      }
       synchronized (lock) {
         if (!waitQueue.isEmpty()) {
           lock.notify();
@@ -707,7 +710,7 @@ public class TaskExecutorService extends AbstractService implements Scheduler<Ta
     public void finishableStateUpdated(boolean finishableState) {
       // This method should not by synchronized. Can lead to deadlocks since it calls a sync method.
       // Meanwhile the scheduler could try updating states via a synchronized method.
-      LOG.info("DEBUG: Received finishable state update for {}, state={}",
+      LOG.info("Received finishable state update for {}, state={}",
           taskRunnerCallable.getRequestId(), finishableState);
       taskExecutorService.finishableStateUpdated(this, finishableState);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/71da33a6/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java
index 3b38597..30a38c8 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java
@@ -272,7 +272,7 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
             shouldRunTask = false;
           } else {
             // If the task hasn't started, and it is killed - report back to the AM that the task has been killed.
-            LOG.info("DBG: Reporting taskKilled for non-started fragment {}", getRequestId());
+            LOG.debug("Reporting taskKilled for non-started fragment {}", getRequestId());
             reportTaskKilled();
           }
           if (!isStarted.get()) {
@@ -398,7 +398,7 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
       switch(result.getEndReason()) {
         // Only the KILLED case requires a message to be sent out to the AM.
         case SUCCESS:
-          LOG.info("Successfully finished {}", requestId);
+          LOG.debug("Successfully finished {}", requestId);
           metrics.incrExecutorTotalSuccess();
           break;
         case CONTAINER_STOP_REQUESTED:

http://git-wip-us.apache.org/repos/asf/hive/blob/71da33a6/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java
index 9bdafc9..1d0fdf0 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java
@@ -168,8 +168,8 @@ public class OrcEncodedDataReader extends CallableWithNdc<Void>
 
   @Override
   public void stop() {
-    if (LOG.isInfoEnabled()) {
-      LOG.info("Encoded reader is being stopped");
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Encoded reader is being stopped");
     }
     isStopped = true;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/71da33a6/llap-server/src/java/org/apache/hadoop/hive/llap/shufflehandler/ShuffleHandler.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/shufflehandler/ShuffleHandler.java b/llap-server/src/java/org/apache/hadoop/hive/llap/shufflehandler/ShuffleHandler.java
index 762f069..b042455 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/shufflehandler/ShuffleHandler.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/shufflehandler/ShuffleHandler.java
@@ -417,7 +417,7 @@ public class ShuffleHandler implements AttemptRegistrationListener {
     // TODO Fix this. There's a race here, where an app may think everything is registered, finish really fast, send events and the consumer will not find the registration.
     Boolean registered = registeredApps.putIfAbsent(applicationIdString, Boolean.valueOf(true));
     if (registered == null) {
-      LOG.info("DEBUG: Registering watches for AppDirs: appId=" + applicationIdString);
+      LOG.debug("Registering watches for AppDirs: appId=" + applicationIdString);
       recordJobShuffleInfo(applicationIdString, user, appToken);
       if (dirWatcher != null) {
         for (String appDir : appDirs) {
@@ -538,7 +538,7 @@ public class ShuffleHandler implements AttemptRegistrationListener {
               @Override
               public void onRemoval(
                   RemovalNotification<AttemptPathIdentifier, AttemptPathInfo> notification) {
-                LOG.info("DEBUG: PathCacheEviction: " + notification.getKey() + ", Reason=" +
+                LOG.debug("PathCacheEviction: " + notification.getKey() + ", Reason=" +
                     notification.getCause());
               }
             })
@@ -561,7 +561,7 @@ public class ShuffleHandler implements AttemptRegistrationListener {
             Path mapOutputFileName =
                 lDirAlloc.getLocalPathToRead(attemptBase + "/" + DATA_FILE_NAME, conf);
 
-            LOG.info("DEBUG: Loaded : " + key + " via loader");
+            LOG.debug("Loaded : " + key + " via loader");
             if (dirWatcher != null) {
               dirWatcher.attemptInfoFound(key);
             }
@@ -582,7 +582,7 @@ public class ShuffleHandler implements AttemptRegistrationListener {
 
     void registerAttemptDirs(AttemptPathIdentifier identifier,
                                     AttemptPathInfo pathInfo) {
-      LOG.info("DEBUG: Registering " + identifier + " via watcher");
+      LOG.debug("Registering " + identifier + " via watcher");
       pathCache.put(identifier, pathInfo);
     }
 
@@ -748,7 +748,10 @@ public class ShuffleHandler implements AttemptRegistrationListener {
       try {
         AttemptPathIdentifier identifier = new AttemptPathIdentifier(jobId, dagId, user, mapId);
         pathInfo = pathCache.get(identifier);
-        LOG.info("DEBUG: Retrieved pathInfo for " + identifier + " check for corresponding loaded messages to determine whether it was loaded or cached");
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Retrieved pathInfo for " + identifier + " check for corresponding "
+              + "loaded messages to determine whether it was loaded or cached");
+        }
       } catch (ExecutionException e) {
         if (e.getCause() instanceof IOException) {
           throw (IOException) e.getCause();

http://git-wip-us.apache.org/repos/asf/hive/blob/71da33a6/llap-server/src/java/org/apache/tez/dag/app/rm/LlapTaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/tez/dag/app/rm/LlapTaskSchedulerService.java b/llap-server/src/java/org/apache/tez/dag/app/rm/LlapTaskSchedulerService.java
index 6fd01f9..e920f86 100644
--- a/llap-server/src/java/org/apache/tez/dag/app/rm/LlapTaskSchedulerService.java
+++ b/llap-server/src/java/org/apache/tez/dag/app/rm/LlapTaskSchedulerService.java
@@ -359,12 +359,12 @@ public class LlapTaskSchedulerService extends TaskScheduler {
 
   @Override
   public void blacklistNode(NodeId nodeId) {
-    LOG.info("DEBUG: BlacklistNode not supported");
+    LOG.info("BlacklistNode not supported");
   }
 
   @Override
   public void unblacklistNode(NodeId nodeId) {
-    LOG.info("DEBUG: unBlacklistNode not supported");
+    LOG.info("unBlacklistNode not supported");
   }
 
   @Override
@@ -494,7 +494,7 @@ public class LlapTaskSchedulerService extends TaskScheduler {
 
   @Override
   public Object deallocateContainer(ContainerId containerId) {
-    LOG.info("DEBUG: Ignoring deallocateContainer for containerId: " + containerId);
+    LOG.debug("Ignoring deallocateContainer for containerId: " + containerId);
     // Containers are not being tracked for re-use.
     // This is safe to ignore since a deallocate task will come in.
     return null;

http://git-wip-us.apache.org/repos/asf/hive/blob/71da33a6/metastore/src/java/org/apache/hadoop/hive/metastore/AggregateStatsCache.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/AggregateStatsCache.java b/metastore/src/java/org/apache/hadoop/hive/metastore/AggregateStatsCache.java
index 58c9f9e..4ab178c 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/AggregateStatsCache.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/AggregateStatsCache.java
@@ -171,7 +171,7 @@ public class AggregateStatsCache {
     AggrColStatsList candidateList = cacheStore.get(key);
     // No key, or no nodes in candidate list
     if ((candidateList == null) || (candidateList.nodes.size() == 0)) {
-      LOG.info("No aggregate stats cached for " + key.toString());
+      LOG.debug("No aggregate stats cached for " + key.toString());
       return null;
     }
     // Find the value object

http://git-wip-us.apache.org/repos/asf/hive/blob/71da33a6/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java
index 95fd1bf..4af98e5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java
@@ -160,8 +160,8 @@ public class MapJoinOperator extends AbstractMapJoinOperator<MapJoinDesc> implem
        * requires changes in the Tez API with regard to finding bucket id and
        * also ability to schedule tasks to re-use containers that have cached the specific bucket.
        */
-      if (isLogInfoEnabled) {
-        LOG.info("This is not bucket map join, so cache");
+      if (isLogDebugEnabled) {
+        LOG.debug("This is not bucket map join, so cache");
       }
 
       Future<Pair<MapJoinTableContainer[], MapJoinTableContainerSerDe[]>> future =
@@ -200,12 +200,12 @@ public class MapJoinOperator extends AbstractMapJoinOperator<MapJoinDesc> implem
 
         loadHashTable(getExecContext(), MapredContext.get());
       } else {
-        if (LOG.isInfoEnabled()) {
+        if (LOG.isDebugEnabled()) {
           String s = "Using tables from cache: [";
           for (MapJoinTableContainer c : pair.getLeft()) {
             s += ((c == null) ? "null" : c.getClass().getSimpleName()) + ", ";
           }
-          LOG.info(s + "]");
+          LOG.debug(s + "]");
         }
         // let's use the table from the cache.
         mapJoinTables = pair.getLeft();

http://git-wip-us.apache.org/repos/asf/hive/blob/71da33a6/ql/src/java/org/apache/hadoop/hive/ql/exec/MapredContext.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/MapredContext.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/MapredContext.java
index caf4aa3..6ce84ac 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/MapredContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/MapredContext.java
@@ -52,7 +52,7 @@ public class MapredContext {
         HiveConf.getVar(jobConf, ConfVars.HIVE_EXECUTION_ENGINE).equals("tez") ?
             new TezContext(isMap, jobConf) : new MapredContext(isMap, jobConf);
     contexts.set(context);
-    logger.info("MapredContext initialized.");
+    logger.debug("MapredContext initialized.");
     return context;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/71da33a6/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java
index 7282228..e2630ad 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java
@@ -373,8 +373,8 @@ public abstract class Operator<T extends OperatorDesc> implements Serializable,C
           || childOperatorsArray.length != childOperators.size()) {
         throw new AssertionError("Internal error during operator initialization");
       }
-      if (isLogInfoEnabled) {
-        LOG.info("Initialization Done " + id + " " + getName());
+      if (isLogDebugEnabled) {
+        LOG.debug("Initialization Done " + id + " " + getName());
       }
 
       initializeChildren(hconf);
@@ -386,8 +386,8 @@ public abstract class Operator<T extends OperatorDesc> implements Serializable,C
       }
     }
 
-    if (isLogInfoEnabled) {
-      LOG.info("Initialization Done " + id + " " + getName() + " done is reset.");
+    if (isLogDebugEnabled) {
+      LOG.debug("Initialization Done " + id + " " + getName() + " done is reset.");
     }
 
     // let's wait on the async ops before continuing
@@ -459,14 +459,14 @@ public abstract class Operator<T extends OperatorDesc> implements Serializable,C
    */
   protected void initializeChildren(Configuration hconf) throws HiveException {
     state = State.INIT;
-    if (isLogInfoEnabled) {
-      LOG.info("Operator " + id + " " + getName() + " initialized");
+    if (isLogDebugEnabled) {
+      LOG.debug("Operator " + id + " " + getName() + " initialized");
     }
     if (childOperators == null || childOperators.isEmpty()) {
       return;
     }
-    if (isLogInfoEnabled) {
-      LOG.info("Initializing children of " + id + " " + getName());
+    if (isLogDebugEnabled) {
+      LOG.debug("Initializing children of " + id + " " + getName());
     }
     for (int i = 0; i < childOperatorsArray.length; i++) {
       childOperatorsArray[i].initialize(hconf, outputObjInspector, childOperatorsTag[i]);
@@ -503,8 +503,8 @@ public abstract class Operator<T extends OperatorDesc> implements Serializable,C
    */
   protected void initialize(Configuration hconf, ObjectInspector inputOI,
       int parentId) throws HiveException {
-    if (isLogInfoEnabled) {
-      LOG.info("Initializing child " + id + " " + getName());
+    if (isLogDebugEnabled) {
+      LOG.debug("Initializing child " + id + " " + getName());
     }
     // Double the size of the array if needed
     if (parentId >= inputObjInspectors.length) {
@@ -646,8 +646,8 @@ public abstract class Operator<T extends OperatorDesc> implements Serializable,C
     // set state as CLOSE as long as all parents are closed
     // state == CLOSE doesn't mean all children are also in state CLOSE
     state = State.CLOSE;
-    if (isLogInfoEnabled) {
-      LOG.info(id + " finished. closing... ");
+    if (isLogDebugEnabled) {
+      LOG.debug(id + " finished. closing... ");
     }
 
     abort |= abortOp.get();
@@ -664,14 +664,14 @@ public abstract class Operator<T extends OperatorDesc> implements Serializable,C
       }
 
       for (Operator<? extends OperatorDesc> op : childOperators) {
-	if (isLogDebugEnabled) {
-	  LOG.debug("Closing child = " + op);
-	}
+        if (isLogDebugEnabled) {
+          LOG.debug("Closing child = " + op);
+        }
         op.close(abort);
       }
 
-      if (isLogInfoEnabled) {
-	LOG.info(id + " Close done");
+      if (isLogDebugEnabled) {
+        LOG.debug(id + " Close done");
       }
     } catch (HiveException e) {
       e.printStackTrace();
@@ -893,10 +893,12 @@ public abstract class Operator<T extends OperatorDesc> implements Serializable,C
   }
 
   public void logStats() {
-    if (isLogInfoEnabled) {
-      for (String e : statsMap.keySet()) {
-        LOG.info(e.toString() + ":" + statsMap.get(e).toString());
+    if (isLogInfoEnabled && !statsMap.isEmpty()) {
+      StringBuilder sb = new StringBuilder();
+      for (Map.Entry<String, LongWritable> e : statsMap.entrySet()) {
+        sb.append(e.getKey()).append(":").append(statsMap.get(e).toString()).append(", ");
       }
+      LOG.info(sb.toString());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/71da33a6/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
index 0618077..665b3f7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
@@ -399,11 +399,11 @@ public final class Utilities {
         } else if (ShimLoader.getHadoopShims().isLocalMode(conf)) {
           localPath = path;
         } else {
-          LOG.info("***************non-local mode***************");
+          LOG.debug("***************non-local mode***************");
           localPath = new Path(name);
         }
         localPath = path;
-        LOG.info("local path = " + localPath);
+        LOG.debug("local path = " + localPath);
         if (HiveConf.getBoolVar(conf, ConfVars.HIVE_RPC_QUERY_PLAN)) {
           LOG.debug("Loading plan from string: "+path.toUri().getPath());
           String planString = conf.getRaw(path.toUri().getPath());
@@ -415,7 +415,7 @@ public final class Utilities {
           in = new ByteArrayInputStream(planBytes);
           in = new InflaterInputStream(in);
         } else {
-          LOG.info("Open file to read in plan: " + localPath);
+          LOG.debug("Open file to read in plan: " + localPath);
           in = localPath.getFileSystem(conf).open(localPath);
         }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/71da33a6/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ObjectCache.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ObjectCache.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ObjectCache.java
index 7baf9b2..008f8a4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ObjectCache.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ObjectCache.java
@@ -36,21 +36,21 @@ import org.apache.hadoop.hive.ql.metadata.HiveException;
 public class ObjectCache implements org.apache.hadoop.hive.ql.exec.ObjectCache {
 
   private static final Logger LOG = LoggerFactory.getLogger(ObjectCache.class.getName());
-  private static final boolean isInfoEnabled = LOG.isInfoEnabled();
+  private static final boolean isDebugEnabled = LOG.isDebugEnabled();
 
   @Override
   public void release(String key) {
     // nothing to do
-    if (isInfoEnabled) {
-      LOG.info(key + " no longer needed");
+    if (isDebugEnabled) {
+      LOG.debug(key + " no longer needed");
     }
   }
 
   @Override
   public <T> T retrieve(String key, Callable<T> fn) throws HiveException {
     try {
-      if (isInfoEnabled) {
-        LOG.info("Creating " + key);
+      if (isDebugEnabled) {
+        LOG.debug("Creating " + key);
       }
       return fn.call();
     } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/hive/blob/71da33a6/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/LlapObjectCache.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/LlapObjectCache.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/LlapObjectCache.java
index 6f77453..0141230 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/LlapObjectCache.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/LlapObjectCache.java
@@ -44,7 +44,7 @@ public class LlapObjectCache implements org.apache.hadoop.hive.ql.exec.ObjectCac
 
   private static ExecutorService staticPool = Executors.newCachedThreadPool();
 
-  private static final boolean isLogInfoEnabled = LOG.isInfoEnabled();
+  private static final boolean isLogDebugEnabled = LOG.isDebugEnabled();
 
   private final Cache<String, Object> registry = CacheBuilder.newBuilder().softValues().build();
 
@@ -69,8 +69,8 @@ public class LlapObjectCache implements org.apache.hadoop.hive.ql.exec.ObjectCac
     try {
       value = (T) registry.getIfPresent(key);
       if (value != null) {
-        if (isLogInfoEnabled) {
-          LOG.info("Found " + key + " in cache");
+        if (isLogDebugEnabled) {
+          LOG.debug("Found " + key + " in cache");
         }
         return value;
       }
@@ -91,8 +91,8 @@ public class LlapObjectCache implements org.apache.hadoop.hive.ql.exec.ObjectCac
       try {
         value = (T) registry.getIfPresent(key);
         if (value != null) {
-          if (isLogInfoEnabled) {
-            LOG.info("Found " + key + " in cache");
+          if (isLogDebugEnabled) {
+            LOG.debug("Found " + key + " in cache");
           }
           return value;
         }
@@ -108,8 +108,8 @@ public class LlapObjectCache implements org.apache.hadoop.hive.ql.exec.ObjectCac
 
       lock.lock();
       try {
-        if (isLogInfoEnabled) {
-          LOG.info("Caching new object for key: " + key);
+        if (isLogDebugEnabled) {
+          LOG.debug("Caching new object for key: " + key);
         }
 
         registry.put(key, value);
@@ -135,7 +135,9 @@ public class LlapObjectCache implements org.apache.hadoop.hive.ql.exec.ObjectCac
 
   @Override
   public void remove(String key) {
-    LOG.info("Removing key: " + key);
+    if (isLogDebugEnabled) {
+      LOG.debug("Removing key: " + key);
+    }
     registry.invalidate(key);
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/71da33a6/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezProcessor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezProcessor.java
index 23f2487..c560f37 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezProcessor.java
@@ -140,11 +140,13 @@ public class TezProcessor extends AbstractLogicalIOProcessor {
       return;
     }
 
-      perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.TEZ_RUN_PROCESSOR);
-      // in case of broadcast-join read the broadcast edge inputs
-      // (possibly asynchronously)
+    perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.TEZ_RUN_PROCESSOR);
+    // in case of broadcast-join read the broadcast edge inputs
+    // (possibly asynchronously)
 
-      LOG.info("Running task: " + getContext().getUniqueIdentifier());
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Running task: " + getContext().getUniqueIdentifier());
+    }
 
     synchronized (this) {
       if (isMap) {

http://git-wip-us.apache.org/repos/asf/hive/blob/71da33a6/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 0bea5ff..35bbaef 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
@@ -318,7 +318,7 @@ public class VectorGroupByOperator extends Operator<GroupByDesc> implements
 
       mapKeysAggregationBuffers = new HashMap<KeyWrapper, VectorAggregationBufferRow>();
       computeMemoryLimits();
-      LOG.info("using hash aggregation processing mode");
+      LOG.debug("using hash aggregation processing mode");
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/71da33a6/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
index f00804e..3489c9c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
@@ -157,7 +157,10 @@ public class VectorizationContext {
   public VectorizationContext(String contextName, List<String> initialColumnNames) {
     this.contextName = contextName;
     level = 0;
-    LOG.info("VectorizationContext consructor contextName " + contextName + " level " + level + " initialColumnNames " + initialColumnNames.toString());
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("VectorizationContext consructor contextName " + contextName + " level "
+          + level + " initialColumnNames " + initialColumnNames);
+    }
     this.projectionColumnNames = initialColumnNames;
 
     projectedColumns = new ArrayList<Integer>();
@@ -177,8 +180,10 @@ public class VectorizationContext {
   public VectorizationContext(String contextName) {
     this.contextName = contextName;
     level = 0;
-    LOG.info("VectorizationContext consructor contextName " + contextName + " level " + level);
-      projectedColumns = new ArrayList<Integer>();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("VectorizationContext consructor contextName " + contextName + " level " + level);
+    }
+    projectedColumns = new ArrayList<Integer>();
     projectionColumnNames = new ArrayList<String>();
     projectionColumnMap = new HashMap<String, Integer>();
     this.ocm = new OutputColumnManager(0);

http://git-wip-us.apache.org/repos/asf/hive/blob/71da33a6/ql/src/java/org/apache/hadoop/hive/ql/io/HiveContextAwareRecordReader.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveContextAwareRecordReader.java b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveContextAwareRecordReader.java
index 7d36e42..4a05a62 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveContextAwareRecordReader.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveContextAwareRecordReader.java
@@ -171,7 +171,7 @@ public abstract class HiveContextAwareRecordReader<K, V> implements RecordReader
     ioCxtRef.setCurrentBlockStart(startPos);
     ioCxtRef.setBlockPointer(isBlockPointer);
     ioCxtRef.setInputPath(inputPath);
-    LOG.info("Processing file " + inputPath);
+    LOG.debug("Processing file " + inputPath); // Logged at INFO in multiple other places.
     initDone = true;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/71da33a6/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
index 29c4b61..a883124 100755
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
@@ -210,7 +210,9 @@ public class HiveInputFormat<K extends WritableComparable, V extends Writable>
       LOG.info("Not using llap for " + inputFormat + ": " + isSupported + ", " + isVector);
       return inputFormat;
     }
-    LOG.info("Wrapping " + inputFormat);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Wrapping " + inputFormat);
+    }
     @SuppressWarnings("unchecked")
     LlapIo<VectorizedRowBatch> llapIo = LlapIoProxy.getIo();
     if (llapIo == null) {

http://git-wip-us.apache.org/repos/asf/hive/blob/71da33a6/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/NullScanTaskDispatcher.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/NullScanTaskDispatcher.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/NullScanTaskDispatcher.java
index 2e924fb..0c4519c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/NullScanTaskDispatcher.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/NullScanTaskDispatcher.java
@@ -172,7 +172,7 @@ public class NullScanTaskDispatcher implements Dispatcher {
         return null;
       }
 
-      LOG.info("Looking for table scans where optimization is applicable");
+      LOG.debug("Looking for table scans where optimization is applicable");
 
       // The dispatcher fires the processor corresponding to the closest
       // matching rule and passes the context along
@@ -196,7 +196,7 @@ public class NullScanTaskDispatcher implements Dispatcher {
 
       ogw.startWalking(topNodes, null);
 
-      LOG.info(String.format("Found %d null table scans",
+      LOG.debug(String.format("Found %d null table scans",
           walkerCtx.getMetadataOnlyTableScans().size()));
       if (walkerCtx.getMetadataOnlyTableScans().size() > 0)
         processAlias(mapWork, walkerCtx.getMetadataOnlyTableScans());

http://git-wip-us.apache.org/repos/asf/hive/blob/71da33a6/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
index 97e7013..a8ebf8f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
@@ -744,7 +744,7 @@ public class Vectorizer implements PhysicalPlanResolver {
         }
         vContext = taskVectorizationContext;
       } else {
-        LOG.info("MapWorkVectorizationNodeProcessor process going to walk the operator stack to get vectorization context for " + op.getName());
+        LOG.debug("MapWorkVectorizationNodeProcessor process going to walk the operator stack to get vectorization context for " + op.getName());
         vContext = walkStackToFindVectorizationContext(stack, op);
         if (vContext == null) {
           // No operator has "pushed" a new context -- so use the task vectorization context.
@@ -753,7 +753,10 @@ public class Vectorizer implements PhysicalPlanResolver {
       }
 
       assert vContext != null;
-      LOG.info("MapWorkVectorizationNodeProcessor process operator " + op.getName() + " using vectorization context" + vContext.toString());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("MapWorkVectorizationNodeProcessor process operator " + op.getName()
+            + " using vectorization context" + vContext.toString());
+      }
 
       // When Vectorized GROUPBY outputs rows instead of vectorized row batchs, we don't
       // vectorize the operators below it.
@@ -1986,8 +1989,8 @@ public class Vectorizer implements PhysicalPlanResolver {
         break;
     }
 
-    LOG.info("vectorizeOperator " + (vectorOp == null ? "NULL" : vectorOp.getClass().getName()));
-    LOG.info("vectorizeOperator " + (vectorOp == null || vectorOp.getConf() == null ? "NULL" : vectorOp.getConf().getClass().getName()));
+    LOG.debug("vectorizeOperator " + (vectorOp == null ? "NULL" : vectorOp.getClass().getName()));
+    LOG.debug("vectorizeOperator " + (vectorOp == null || vectorOp.getConf() == null ? "NULL" : vectorOp.getConf().getClass().getName()));
 
     if (vectorOp != op) {
       fixupParentChildOperators(op, vectorOp);

http://git-wip-us.apache.org/repos/asf/hive/blob/71da33a6/ql/src/java/org/apache/hadoop/hive/ql/ppd/OpProcFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ppd/OpProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/ppd/OpProcFactory.java
index 5b85c93..3605484 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ppd/OpProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ppd/OpProcFactory.java
@@ -127,7 +127,7 @@ public final class OpProcFactory {
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
         Object... nodeOutputs) throws SemanticException {
-      LOG.info("Processing for " + nd.getName() + "("
+      LOG.debug("Processing for " + nd.getName() + "("
           + ((Operator) nd).getIdentifier() + ")");
       // script operator is a black-box to hive so no optimization here
       // assuming that nothing can be pushed above the script op
@@ -709,13 +709,19 @@ public final class OpProcFactory {
      * @param ewi
      */
     protected void logExpr(Node nd, ExprWalkerInfo ewi) {
-      for (Entry<String, List<ExprNodeDesc>> e : ewi.getFinalCandidates()
-          .entrySet()) {
-        LOG.info("Pushdown Predicates of " + nd.getName() + " For Alias : "
-            + e.getKey());
+      if (!LOG.isInfoEnabled()) return;
+      for (Entry<String, List<ExprNodeDesc>> e : ewi.getFinalCandidates().entrySet()) {
+        StringBuilder sb = new StringBuilder("Pushdown predicates of ").append(nd.getName())
+            .append(" for alias ").append(e.getKey()).append(": ");
+        boolean isFirst = true;
         for (ExprNodeDesc n : e.getValue()) {
-          LOG.info("\t" + n.getExprString());
+          if (!isFirst) {
+            sb.append("; ");
+          }
+          isFirst = false;
+          sb.append(n.getExprString());
         }
+        LOG.info(sb.toString());
       }
     }