You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jx...@apache.org on 2015/11/06 18:32:52 UTC

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

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/6fda3b55
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/6fda3b55
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/6fda3b55

Branch: refs/heads/master-fixed
Commit: 6fda3b55e9ae680f47c55395f90be762285f2760
Parents: c9246f4
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/6fda3b55/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/6fda3b55/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/6fda3b55/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/6fda3b55/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/6fda3b55/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/6fda3b55/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/6fda3b55/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/6fda3b55/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/6fda3b55/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/6fda3b55/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;
+  }
+}