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 2017/04/11 19:01:22 UTC

hive git commit: HIVE-16037 : add IO memory usage report to LLAP UI (Sergey Shelukhin, reviewed by Prasanth Jayachandran)

Repository: hive
Updated Branches:
  refs/heads/master a4fd2ea23 -> 5facfbb86


HIVE-16037 : add IO memory usage report to LLAP UI (Sergey Shelukhin, reviewed by Prasanth Jayachandran)


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

Branch: refs/heads/master
Commit: 5facfbb863366d7a661c21c57011b8dbe43f52e0
Parents: a4fd2ea
Author: Sergey Shelukhin <se...@apache.org>
Authored: Tue Apr 11 11:43:37 2017 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Tue Apr 11 11:43:37 2017 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hive/llap/io/api/LlapIo.java  |  1 +
 .../hadoop/hive/llap/cache/BuddyAllocator.java  | 72 ++++++++++++++--
 .../hive/llap/cache/EvictionDispatcher.java     | 25 +++++-
 .../hive/llap/cache/LlapOomDebugDump.java       |  1 +
 .../hadoop/hive/llap/cache/LowLevelCache.java   |  2 +-
 .../hive/llap/cache/LowLevelCacheImpl.java      | 39 +++++++++
 .../llap/cache/LowLevelCacheMemoryManager.java  |  8 +-
 .../llap/cache/LowLevelFifoCachePolicy.java     | 14 ++++
 .../llap/cache/LowLevelLrfuCachePolicy.java     | 22 +++++
 .../hive/llap/cache/SerDeLowLevelCacheImpl.java | 51 ++++++++++++
 .../hive/llap/cache/SimpleBufferManager.java    | 10 +++
 .../services/impl/LlapIoMemoryServlet.java      | 88 ++++++++++++++++++++
 .../daemon/services/impl/LlapWebServices.java   |  1 +
 .../hive/llap/io/api/impl/LlapIoImpl.java       | 23 +++--
 .../hive/llap/io/metadata/OrcMetadataCache.java | 17 +++-
 .../hive/llap/cache/TestBuddyAllocator.java     |  6 +-
 .../hive/llap/cache/TestLowLevelCacheImpl.java  |  4 +
 .../hive/llap/cache/TestOrcMetadataCache.java   |  8 ++
 18 files changed, 375 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/5facfbb8/llap-client/src/java/org/apache/hadoop/hive/llap/io/api/LlapIo.java
----------------------------------------------------------------------
diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/io/api/LlapIo.java b/llap-client/src/java/org/apache/hadoop/hive/llap/io/api/LlapIo.java
index e5ab601..42129b7 100644
--- a/llap-client/src/java/org/apache/hadoop/hive/llap/io/api/LlapIo.java
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/io/api/LlapIo.java
@@ -25,4 +25,5 @@ import org.apache.hadoop.mapred.InputFormat;
 public interface LlapIo<T> {
   InputFormat<NullWritable, T> getInputFormat(InputFormat sourceInputFormat, Deserializer serde);
   void close();
+  String getMemoryInfo();
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/5facfbb8/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 e71a09e..c049d27 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
@@ -45,7 +45,8 @@ import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.llap.io.api.impl.LlapIoImpl;
 import org.apache.hadoop.hive.llap.metrics.LlapDaemonCacheMetrics;
 
-public final class BuddyAllocator implements EvictionAwareAllocator, BuddyAllocatorMXBean {
+public final class BuddyAllocator
+  implements EvictionAwareAllocator, BuddyAllocatorMXBean, LlapOomDebugDump {
   private final Arena[] arenas;
   private final AtomicInteger allocatedArenas = new AtomicInteger(0);
 
@@ -299,7 +300,6 @@ public final class BuddyAllocator implements EvictionAwareAllocator, BuddyAlloca
         memoryManager.releaseMemory(forceReserved);
       }
     }
-
   }
 
   private void logOomErrorMessage(String msg) {
@@ -312,8 +312,7 @@ public final class BuddyAllocator implements EvictionAwareAllocator, BuddyAlloca
         continue;
       }
       if (shouldLog) {
-        LlapIoImpl.LOG.error(msg + "\nALLOCATOR STATE:\n" + debugDump()
-            + "\nPARENT STATE:\n" + memoryManager.debugDumpForOom());
+        LlapIoImpl.LOG.error(msg + debugDumpForOom());
       } else {
         LlapIoImpl.LOG.error(msg);
       }
@@ -321,6 +320,33 @@ public final class BuddyAllocator implements EvictionAwareAllocator, BuddyAlloca
     }
   }
 
+  /**
+   * Arbitrarily, we start getting the state from Allocator. Allocator calls MM which calls
+   * the policies that call the eviction dispatcher that calls the caches. See init - these all
+   * are connected in a cycle, so we need to make sure the who-calls-whom order is definite.
+   */
+  @Override
+  public void debugDumpShort(StringBuilder sb) {
+    memoryManager.debugDumpShort(sb);
+    sb.append("\nAllocator state:");
+    int unallocCount = 0, fullCount = 0;
+    long totalFree = 0;
+    for (Arena arena : arenas) {
+      Integer result = arena.debugDumpShort(sb);
+      if (result == null) {
+        ++unallocCount;
+      } else if (result == 0) {
+        ++fullCount;
+      } else {
+        totalFree += result;
+      }
+    }
+    sb.append("\nTotal available and allocated: ").append(totalFree).append(
+        "; unallocated arenas: ").append(unallocCount).append(
+        "; full arenas ").append(fullCount);
+    sb.append("\n");
+  }
+
   @Override
   public void deallocate(MemoryBuffer buffer) {
     deallocateInternal(buffer, true);
@@ -345,7 +371,7 @@ public final class BuddyAllocator implements EvictionAwareAllocator, BuddyAlloca
     return isDirect;
   }
 
-  public String debugDump() {
+  public String debugDumpForOomInternal() {
     StringBuilder result = new StringBuilder(
         "NOTE: with multiple threads the dump is not guaranteed to be consistent");
     for (Arena arena : arenas) {
@@ -442,6 +468,36 @@ public final class BuddyAllocator implements EvictionAwareAllocator, BuddyAlloca
       }
     }
 
+    public Integer debugDumpShort(StringBuilder result) {
+      if (data == null) {
+        return null;
+      }
+      int allocSize = minAllocation;
+      int total = 0;
+      for (int i = 0; i < freeLists.length; ++i, allocSize <<= 1) {
+        FreeList freeList = freeLists[i];
+        freeList.lock.lock();
+        try {
+          int nextHeaderIx = freeList.listHead;
+          int count = 0;
+          while (nextHeaderIx >= 0) {
+            ++count;
+            nextHeaderIx = getNextFreeListItem(offsetFromHeaderIndex(nextHeaderIx));
+          }
+          if (count > 0) {
+            if (total == 0) {
+              result.append("\nArena with free list lengths by size: ");
+            }
+            total += (allocSize * count);
+            result.append(allocSize).append(" => ").append(count).append(", ");
+          }
+        } finally {
+          freeList.lock.unlock();
+        }
+      }
+      return total;
+    }
+
     public void debugDump(StringBuilder result) {
       result.append("\nArena: ");
       if (data == null) {
@@ -723,4 +779,10 @@ public final class BuddyAllocator implements EvictionAwareAllocator, BuddyAlloca
   public MemoryBuffer createUnallocated() {
     return new LlapDataBuffer();
   }
+
+  @Override
+  public String debugDumpForOom() {
+    return "\nALLOCATOR STATE:\n" + debugDumpForOomInternal()
+        + "\nPARENT STATE:\n" + memoryManager.debugDumpForOom();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/5facfbb8/llap-server/src/java/org/apache/hadoop/hive/llap/cache/EvictionDispatcher.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/EvictionDispatcher.java b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/EvictionDispatcher.java
index a6b0abd..c73f1a1 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/EvictionDispatcher.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/EvictionDispatcher.java
@@ -25,7 +25,7 @@ import org.apache.hadoop.hive.llap.io.metadata.OrcStripeMetadata;
 /**
  * Eviction dispatcher - uses double dispatch to route eviction notifications to correct caches.
  */
-public final class EvictionDispatcher implements EvictionListener {
+public final class EvictionDispatcher implements EvictionListener, LlapOomDebugDump {
   private final LowLevelCache dataCache;
   private final SerDeLowLevelCacheImpl serdeCache;
   private final OrcMetadataCache metadataCache;
@@ -65,4 +65,27 @@ public final class EvictionDispatcher implements EvictionListener {
   public void notifyEvicted(OrcFileEstimateErrors buffer) {
     metadataCache.notifyEvicted(buffer);
   }
+
+  @Override
+  public String debugDumpForOom() {
+    StringBuilder sb = new StringBuilder(dataCache.debugDumpForOom());
+    if (serdeCache != null) {
+      sb.append(serdeCache.debugDumpForOom());
+    }
+    if (metadataCache != null) {
+      sb.append(metadataCache.debugDumpForOom());
+    }
+    return sb.toString();
+  }
+
+  @Override
+  public void debugDumpShort(StringBuilder sb) {
+    dataCache.debugDumpShort(sb);
+    if (serdeCache != null) {
+      serdeCache.debugDumpShort(sb);
+    }
+    if (metadataCache != null) {
+      metadataCache.debugDumpShort(sb);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/5facfbb8/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LlapOomDebugDump.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LlapOomDebugDump.java b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LlapOomDebugDump.java
index 30bf5a9..e861a7e 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LlapOomDebugDump.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LlapOomDebugDump.java
@@ -20,4 +20,5 @@ package org.apache.hadoop.hive.llap.cache;
 
 public interface LlapOomDebugDump {
   String debugDumpForOom();
+  void debugDumpShort(StringBuilder sb);
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/5facfbb8/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelCache.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelCache.java b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelCache.java
index 19c589a..c5d0c84 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelCache.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelCache.java
@@ -24,7 +24,7 @@ import org.apache.hadoop.hive.common.io.DataCache.BooleanRef;
 import org.apache.hadoop.hive.common.io.DataCache.DiskRangeListFactory;
 import org.apache.hadoop.hive.common.io.encoded.MemoryBuffer;
 
-public interface LowLevelCache {
+public interface LowLevelCache extends LlapOomDebugDump {
   public enum Priority {
     NORMAL,
     HIGH

http://git-wip-us.apache.org/repos/asf/hive/blob/5facfbb8/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 4dc1c23..23796f6 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
@@ -466,4 +466,43 @@ public class LowLevelCacheImpl implements LowLevelCache, BufferUsageManager, Lla
     }
     return sb.toString();
   }
+
+  @Override
+  public void debugDumpShort(StringBuilder sb) {
+    sb.append("\nORC cache state ");
+    int allLocked = 0, allUnlocked = 0, allEvicted = 0;
+    for (Map.Entry<Object, FileCache<ConcurrentSkipListMap<Long, LlapDataBuffer>>> e :
+      cache.entrySet()) {
+      if (!e.getValue().incRef()) continue;
+      try {
+        int fileLocked = 0, fileUnlocked = 0, fileEvicted = 0;
+        if (e.getValue().getCache().isEmpty()) continue;
+        for (Map.Entry<Long, LlapDataBuffer> e2 : e.getValue().getCache().entrySet()) {
+          int newRc = e2.getValue().incRef();
+          if (newRc < 0) {
+            ++fileEvicted;
+            continue;
+          }
+          try {
+            if (newRc > 1) { // We hold one refcount.
+              ++fileLocked;
+            } else {
+              ++fileUnlocked;
+            }
+          } finally {
+            e2.getValue().decRef();
+          }
+        }
+        allLocked += fileLocked;
+        allUnlocked += fileUnlocked;
+        allEvicted += fileEvicted;
+        sb.append("\n  file " + e.getKey() + ": " + fileLocked + " locked, "
+            + fileUnlocked + " unlocked, " + fileEvicted + " evicted");
+      } finally {
+        e.getValue().decRef();
+      }
+    }
+    sb.append("\nORC cache summary: " + allLocked + " locked, "
+        + allUnlocked + " unlocked, " + allEvicted + " evicted");
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/5facfbb8/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 198c31c..2132574 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
@@ -139,7 +139,13 @@ public class LowLevelCacheMemoryManager implements MemoryManager {
   @Override
   public String debugDumpForOom() {
     if (evictor == null) return null;
-    return "cache state\n" + evictor.debugDumpForOom();
+    return "\ncache state\n" + evictor.debugDumpForOom();
+  }
+
+  @Override
+  public void debugDumpShort(StringBuilder sb) {
+    if (evictor == null) return;
+    evictor.debugDumpShort(sb);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/5facfbb8/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 31c4505..761fd00 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
@@ -116,6 +116,20 @@ public class LowLevelFifoCachePolicy implements LowLevelCachePolicy {
   }
 
   @Override
+  public void debugDumpShort(StringBuilder sb) {
+    sb.append("\nFIFO eviction list: ");
+    lock.lock();
+    try {
+      sb.append(buffers.size()).append(" elements)");
+    } finally {
+      lock.unlock();
+    }
+    if (parentDebugDump != null) {
+      parentDebugDump.debugDumpShort(sb);
+    }
+  }
+
+  @Override
   public long tryEvictContiguousData(int allocationSize, int count) {
     long evicted = evictInternal(allocationSize * count, allocationSize);
     int remainingCount = count - (int)(evicted / allocationSize);

http://git-wip-us.apache.org/repos/asf/hive/blob/5facfbb8/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 bf5566d..3973c8a 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
@@ -568,4 +568,26 @@ public class LowLevelLrfuCachePolicy implements LowLevelCachePolicy {
     }
     return result;
   }
+
+  @Override
+  public void debugDumpShort(StringBuilder sb) {
+    sb.append("\nLRFU eviction list: ");
+    LlapCacheableBuffer listHeadLocal = listHead, listTailLocal = listTail;
+    if (listHeadLocal == null) {
+      sb.append("0 items");
+    } else {
+      LlapCacheableBuffer listItem = listHeadLocal;
+      int c = 0;
+      while (listItem != null) {
+        ++c;
+        if (listItem == listTailLocal) break;
+        listItem = listItem.next;
+      }
+      sb.append(c + " items");
+    }
+    sb.append("\nLRFU eviction heap: " + heapSize + " items");
+    if (parentDebugDump != null) {
+      parentDebugDump.debugDumpShort(sb);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/5facfbb8/llap-server/src/java/org/apache/hadoop/hive/llap/cache/SerDeLowLevelCacheImpl.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/SerDeLowLevelCacheImpl.java b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/SerDeLowLevelCacheImpl.java
index 917c4a3..cd5bc9b 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/SerDeLowLevelCacheImpl.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/SerDeLowLevelCacheImpl.java
@@ -706,4 +706,55 @@ public class SerDeLowLevelCacheImpl implements LlapOomDebugDump {
     }
     return sb.toString();
   }
+
+
+  @Override
+  public void debugDumpShort(StringBuilder sb) {
+    sb.append("\nSerDe cache state ");
+    int allLocked = 0, allUnlocked = 0, allEvicted = 0;
+    for (Map.Entry<Object, FileCache<FileData>> e : cache.entrySet()) {
+      if (!e.getValue().incRef()) continue;
+      try {
+        FileData fd = e.getValue().getCache();
+        int fileLocked = 0, fileUnlocked = 0, fileEvicted = 0;
+        sb.append(fd.colCount).append(" columns, ").append(fd.stripes.size()).append(" stripes; ");
+        for (StripeData stripe : fd.stripes) {
+          if (stripe.data == null) continue;
+          for (int i = 0; i < stripe.data.length; ++i) {
+            LlapDataBuffer[][] colData = stripe.data[i];
+            if (colData == null) continue;
+            for (int j = 0; j < colData.length; ++j) {
+              LlapDataBuffer[] streamData = colData[j];
+              if (streamData == null) continue;
+              for (int k = 0; k < streamData.length; ++k) {
+                int newRc = streamData[k].incRef();
+                if (newRc < 0) {
+                  ++fileEvicted;
+                  continue;
+                }
+                try {
+                  if (newRc > 1) { // We hold one refcount.
+                    ++fileLocked;
+                  } else {
+                    ++fileUnlocked;
+                  }
+                } finally {
+                  streamData[k].decRef();
+                }
+              }
+            }
+          }
+        }
+        allLocked += fileLocked;
+        allUnlocked += fileUnlocked;
+        allEvicted += fileEvicted;
+        sb.append("\n  file " + e.getKey() + ": " + fileLocked + " locked, "
+            + fileUnlocked + " unlocked, " + fileEvicted + " evicted");
+      } finally {
+        e.getValue().decRef();
+      }
+    }
+    sb.append("\nSerDe cache summary: " + allLocked + " locked, "
+        + allUnlocked + " unlocked, " + allEvicted + " evicted");
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/5facfbb8/llap-server/src/java/org/apache/hadoop/hive/llap/cache/SimpleBufferManager.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/SimpleBufferManager.java b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/SimpleBufferManager.java
index d1eee04..af7cf3d 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/SimpleBufferManager.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/SimpleBufferManager.java
@@ -102,4 +102,14 @@ public class SimpleBufferManager implements BufferUsageManager, LowLevelCache {
   public void notifyEvicted(MemoryBuffer buffer) {
     throw new UnsupportedOperationException("Buffer manager doesn't have cache");
   }
+
+  @Override
+  public String debugDumpForOom() {
+    return "";
+  }
+
+  @Override
+  public void debugDumpShort(StringBuilder sb) {
+    // TODO Auto-generated method stub
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/5facfbb8/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/services/impl/LlapIoMemoryServlet.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/services/impl/LlapIoMemoryServlet.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/services/impl/LlapIoMemoryServlet.java
new file mode 100644
index 0000000..3386cb4
--- /dev/null
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/services/impl/LlapIoMemoryServlet.java
@@ -0,0 +1,88 @@
+/**
+ * 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.llap.daemon.services.impl;
+
+import java.io.PrintWriter;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.llap.io.api.LlapIo;
+import org.apache.hadoop.hive.llap.io.api.LlapProxy;
+import org.apache.hive.http.HttpServer;
+
+@SuppressWarnings("serial")
+public class LlapIoMemoryServlet extends HttpServlet {
+
+  private static final Log LOG = LogFactory.getLog(LlapIoMemoryServlet.class);
+  static final String ACCESS_CONTROL_ALLOW_METHODS = "Access-Control-Allow-Methods";
+  static final String ACCESS_CONTROL_ALLOW_ORIGIN = "Access-Control-Allow-Origin";
+
+  /**
+   * Initialize this servlet.
+   */
+  @Override
+  public void init() throws ServletException {
+  }
+
+  /**
+   * Process a GET request for the specified resource.
+   *
+   * @param request
+   *          The servlet request we are processing
+   * @param response
+   *          The servlet response we are creating
+   */
+  @Override
+  public void doGet(HttpServletRequest request, HttpServletResponse response) {
+    try {
+      if (!HttpServer.isInstrumentationAccessAllowed(getServletContext(), request, response)) {
+        return;
+      }
+      PrintWriter writer = null;
+ 
+      try {
+        response.setContentType("text/plain; charset=utf8");
+        response.setHeader(ACCESS_CONTROL_ALLOW_METHODS, "GET");
+        response.setHeader(ACCESS_CONTROL_ALLOW_ORIGIN, "*");
+        response.setHeader("Cache-Control", "no-transform,public,max-age=60,s-maxage=60");
+
+        writer = response.getWriter();
+
+        LlapIo<?> llapIo = LlapProxy.getIo();
+        if (llapIo == null) {
+          writer.write("LLAP IO not found");
+        } else {
+          writer.write(llapIo.getMemoryInfo());
+        }
+
+      } finally {
+        if (writer != null) {
+          writer.close();
+        }
+      }
+    } catch (Exception e) {
+      LOG.error("Caught exception while processing llap status request", e);
+      response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/5facfbb8/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/services/impl/LlapWebServices.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/services/impl/LlapWebServices.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/services/impl/LlapWebServices.java
index 028daa1..e896df2 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/services/impl/LlapWebServices.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/services/impl/LlapWebServices.java
@@ -102,6 +102,7 @@ public class LlapWebServices extends AbstractService {
       this.http = builder.build();
       this.http.addServlet("status", "/status", LlapStatusServlet.class);
       this.http.addServlet("peers", "/peers", LlapPeerRegistryServlet.class);
+      this.http.addServlet("iomem", "/iomem", LlapIoMemoryServlet.class);
     } catch (IOException e) {
       LOG.warn("LLAP web service failed to come up", e);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/5facfbb8/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 5a86114..294fb2b 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
@@ -23,9 +23,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
 import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 
 import javax.management.ObjectName;
@@ -40,8 +38,8 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.llap.cache.BuddyAllocator;
 import org.apache.hadoop.hive.llap.cache.BufferUsageManager;
-import org.apache.hadoop.hive.llap.cache.EvictionAwareAllocator;
 import org.apache.hadoop.hive.llap.cache.EvictionDispatcher;
+import org.apache.hadoop.hive.llap.cache.LlapOomDebugDump;
 import org.apache.hadoop.hive.llap.cache.LowLevelCache;
 import org.apache.hadoop.hive.llap.cache.LowLevelCacheImpl;
 import org.apache.hadoop.hive.llap.cache.LowLevelCacheMemoryManager;
@@ -67,8 +65,6 @@ import org.apache.hadoop.mapred.InputFormat;
 import org.apache.hadoop.metrics2.util.MBeans;
 
 import com.google.common.primitives.Ints;
-import com.google.common.util.concurrent.ListeningExecutorService;
-import com.google.common.util.concurrent.MoreExecutors;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 
 public class LlapIoImpl implements LlapIo<VectorizedRowBatch> {
@@ -86,6 +82,7 @@ public class LlapIoImpl implements LlapIo<VectorizedRowBatch> {
   private final LlapDaemonIOMetrics ioMetrics;
   private ObjectName buddyAllocatorMXBean;
   private final Allocator allocator;
+  private final LlapOomDebugDump memoryDump;
 
   private LlapIoImpl(Configuration conf) throws IOException {
     String ioMode = HiveConf.getVar(conf, HiveConf.ConfVars.LLAP_IO_MEMORY_MODE);
@@ -149,8 +146,9 @@ public class LlapIoImpl implements LlapIo<VectorizedRowBatch> {
       }
       cacheMetrics.setCacheCapacityTotal(totalMemorySize + metaMem);
       // Cache uses allocator to allocate and deallocate, create allocator and then caches.
-      EvictionAwareAllocator allocator = new BuddyAllocator(conf, memManager, cacheMetrics);
+      BuddyAllocator allocator = new BuddyAllocator(conf, memManager, cacheMetrics);
       this.allocator = allocator;
+      this.memoryDump = allocator;
       LowLevelCacheImpl cacheImpl = new LowLevelCacheImpl(
           cacheMetrics, cachePolicy, allocator, true);
       cache = cacheImpl;
@@ -165,13 +163,16 @@ public class LlapIoImpl implements LlapIo<VectorizedRowBatch> {
       EvictionDispatcher e = new EvictionDispatcher(cache, serdeCache, metadataCache, allocator);
       if (isSplitCache) {
         metaCachePolicy.setEvictionListener(e);
+        metaCachePolicy.setParentDebugDumper(e);
       }
       cachePolicy.setEvictionListener(e);
-      cachePolicy.setParentDebugDumper(cacheImpl);
+      cachePolicy.setParentDebugDumper(e);
+
       cacheImpl.startThreads(); // Start the cache threads.
       bufferManager = cacheImpl; // Cache also serves as buffer manager.
     } else {
       this.allocator = new SimpleAllocator(conf);
+      memoryDump = null;
       SimpleBufferManager sbm = new SimpleBufferManager(allocator, cacheMetrics);
       bufferManager = sbm;
       cache = sbm;
@@ -196,6 +197,14 @@ public class LlapIoImpl implements LlapIo<VectorizedRowBatch> {
     buddyAllocatorMXBean = MBeans.register("LlapDaemon", "BuddyAllocatorInfo", allocator);
   }
 
+  @Override
+  public String getMemoryInfo() {
+    if (memoryDump == null) return "\nNot using the allocator";
+    StringBuilder sb = new StringBuilder();
+    memoryDump.debugDumpShort(sb);
+    return sb.toString();
+  }
+
   @SuppressWarnings("rawtypes")
   @Override
   public InputFormat<NullWritable, VectorizedRowBatch> getInputFormat(

http://git-wip-us.apache.org/repos/asf/hive/blob/5facfbb8/llap-server/src/java/org/apache/hadoop/hive/llap/io/metadata/OrcMetadataCache.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/metadata/OrcMetadataCache.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/metadata/OrcMetadataCache.java
index 2645428..601b622 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/metadata/OrcMetadataCache.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/metadata/OrcMetadataCache.java
@@ -27,12 +27,13 @@ import org.apache.hadoop.hive.common.io.DiskRange;
 import org.apache.hadoop.hive.common.io.DiskRangeList;
 import org.apache.hadoop.hive.common.io.DataCache.BooleanRef;
 import org.apache.hadoop.hive.common.io.DataCache.DiskRangeListFactory;
+import org.apache.hadoop.hive.llap.cache.LlapOomDebugDump;
 import org.apache.hadoop.hive.llap.cache.LowLevelCachePolicy;
 import org.apache.hadoop.hive.llap.cache.MemoryManager;
 import org.apache.hadoop.hive.llap.cache.LowLevelCache.Priority;
 import org.apache.hadoop.hive.ql.io.orc.encoded.OrcBatchKey;
 
-public class OrcMetadataCache {
+public class OrcMetadataCache implements LlapOomDebugDump {
   private final ConcurrentHashMap<Object, OrcFileMetadata> metadata = new ConcurrentHashMap<>();
   private final ConcurrentHashMap<OrcBatchKey, OrcStripeMetadata> stripeMetadata =
       new ConcurrentHashMap<>();
@@ -145,4 +146,18 @@ public class OrcMetadataCache {
   public void notifyEvicted(OrcFileEstimateErrors buffer) {
     estimateErrors.remove(buffer.getFileKey());
   }
+
+  @Override
+  public String debugDumpForOom() {
+    StringBuilder sb = new StringBuilder();
+    debugDumpShort(sb);
+    return sb.toString();
+  }
+
+  @Override
+  public void debugDumpShort(StringBuilder sb) {
+    sb.append("\nORC metadata cache state: ").append(metadata.size()).append(" files, ")
+      .append(stripeMetadata.size()).append(" stripes, ").append(estimateErrors.size())
+      .append(" files w/ORC estimate");
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/5facfbb8/llap-server/src/test/org/apache/hadoop/hive/llap/cache/TestBuddyAllocator.java
----------------------------------------------------------------------
diff --git a/llap-server/src/test/org/apache/hadoop/hive/llap/cache/TestBuddyAllocator.java b/llap-server/src/test/org/apache/hadoop/hive/llap/cache/TestBuddyAllocator.java
index f621005..a6080e6 100644
--- a/llap-server/src/test/org/apache/hadoop/hive/llap/cache/TestBuddyAllocator.java
+++ b/llap-server/src/test/org/apache/hadoop/hive/llap/cache/TestBuddyAllocator.java
@@ -78,6 +78,10 @@ public class TestBuddyAllocator {
     public long forceReservedMemory(int allocationSize, int count) {
       return allocationSize * count;
     }
+
+    @Override
+    public void debugDumpShort(StringBuilder sb) {
+    }
   }
 
   @Test
@@ -248,7 +252,7 @@ public class TestBuddyAllocator {
     try {
       a.allocateMultiple(allocs[index], size);
     } catch (AllocatorOutOfMemoryException ex) {
-      LOG.error("Failed to allocate " + allocCount + " of " + size + "; " + a.debugDump());
+      LOG.error("Failed to allocate " + allocCount + " of " + size + "; " + a.debugDumpForOomInternal());
       throw ex;
     }
     // LOG.info("Allocated " + allocCount + " of " + size + "; " + a.debugDump());

http://git-wip-us.apache.org/repos/asf/hive/blob/5facfbb8/llap-server/src/test/org/apache/hadoop/hive/llap/cache/TestLowLevelCacheImpl.java
----------------------------------------------------------------------
diff --git a/llap-server/src/test/org/apache/hadoop/hive/llap/cache/TestLowLevelCacheImpl.java b/llap-server/src/test/org/apache/hadoop/hive/llap/cache/TestLowLevelCacheImpl.java
index 42c4c56..e95f807 100644
--- a/llap-server/src/test/org/apache/hadoop/hive/llap/cache/TestLowLevelCacheImpl.java
+++ b/llap-server/src/test/org/apache/hadoop/hive/llap/cache/TestLowLevelCacheImpl.java
@@ -119,6 +119,10 @@ public class TestLowLevelCacheImpl {
     public long tryEvictContiguousData(int allocationSize, int count) {
       return count * allocationSize;
     }
+
+    @Override
+    public void debugDumpShort(StringBuilder sb) {
+    }
   }
 
 /*

http://git-wip-us.apache.org/repos/asf/hive/blob/5facfbb8/llap-server/src/test/org/apache/hadoop/hive/llap/cache/TestOrcMetadataCache.java
----------------------------------------------------------------------
diff --git a/llap-server/src/test/org/apache/hadoop/hive/llap/cache/TestOrcMetadataCache.java b/llap-server/src/test/org/apache/hadoop/hive/llap/cache/TestOrcMetadataCache.java
index 4855ed7..3059382 100644
--- a/llap-server/src/test/org/apache/hadoop/hive/llap/cache/TestOrcMetadataCache.java
+++ b/llap-server/src/test/org/apache/hadoop/hive/llap/cache/TestOrcMetadataCache.java
@@ -66,6 +66,10 @@ public class TestOrcMetadataCache {
     public long tryEvictContiguousData(int allocationSize, int count) {
       return 0;
     }
+
+    @Override
+    public void debugDumpShort(StringBuilder sb) {
+    }
   }
 
   private static class DummyMemoryManager implements MemoryManager {
@@ -94,6 +98,10 @@ public class TestOrcMetadataCache {
     public long forceReservedMemory(int allocationSize, int count) {
       return allocationSize * count;
     }
+
+    @Override
+    public void debugDumpShort(StringBuilder sb) {
+    }
   }
 
   @Test