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 2016/07/13 21:27:30 UTC

hive git commit: HIVE-14172 : force evict blocks by size to handle memory fragmentation (Sergey Shelukhin, reviewed by Gopal V)

Repository: hive
Updated Branches:
  refs/heads/branch-2.1 e9479f0f2 -> 62f5a3f2f


HIVE-14172 : force evict blocks by size to handle memory fragmentation (Sergey Shelukhin, reviewed by Gopal V)


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

Branch: refs/heads/branch-2.1
Commit: 62f5a3f2fdaff55ff4a823a9f1755d45bb0ee2b7
Parents: e9479f0
Author: Sergey Shelukhin <se...@apache.org>
Authored: Wed Jul 13 14:23:21 2016 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Wed Jul 13 14:23:44 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hive/llap/cache/BuddyAllocator.java  |   2 +-
 .../llap/cache/LowLevelCacheMemoryManager.java  |  10 +-
 .../hive/llap/cache/LowLevelCachePolicy.java    |   2 +
 .../llap/cache/LowLevelFifoCachePolicy.java     |  22 ++-
 .../llap/cache/LowLevelLrfuCachePolicy.java     | 162 +++++++++++++++----
 .../hadoop/hive/llap/cache/MemoryManager.java   |   3 +-
 .../hive/llap/cache/TestBuddyAllocator.java     |   2 +-
 .../hive/llap/cache/TestLowLevelCacheImpl.java  |   5 +
 .../llap/cache/TestLowLevelLrfuCachePolicy.java |  51 +++++-
 .../hive/llap/cache/TestOrcMetadataCache.java   |   7 +-
 10 files changed, 218 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/62f5a3f2/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 47325ad..b03927a 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
@@ -265,7 +265,7 @@ public final class BuddyAllocator implements EvictionAwareAllocator, BuddyAlloca
           if (destAllocIx == dest.length) return;
         }
       }
-      memoryManager.forceReservedMemory(allocationSize * (dest.length - destAllocIx));
+      memoryManager.forceReservedMemory(allocationSize, dest.length - destAllocIx);
       LlapIoImpl.LOG.warn("Failed to allocate despite reserved memory; will retry " + attempt);
     }
     String msg = "Failed to allocate " + size + "; at " + destAllocIx + " out of " + dest.length;

http://git-wip-us.apache.org/repos/asf/hive/blob/62f5a3f2/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 4def4a1..88bfa8b 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
@@ -114,12 +114,12 @@ public class LowLevelCacheMemoryManager implements MemoryManager {
 
 
   @Override
-  public void forceReservedMemory(int memoryToEvict) {
+  public void forceReservedMemory(int allocationSize, int count) {
     if (evictor == null) return;
-    while (memoryToEvict > 0) {
-      long evicted = evictor.evictSomeBlocks(memoryToEvict);
-      if (evicted == 0) return;
-      memoryToEvict -= evicted;
+    while (count > 0) {
+      int evictedCount = evictor.tryEvictContiguousData(allocationSize, count);
+      if (evictedCount == 0) return;
+      count -= evictedCount;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/62f5a3f2/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelCachePolicy.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelCachePolicy.java b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelCachePolicy.java
index acbaf85..bb1d1b0 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelCachePolicy.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelCachePolicy.java
@@ -27,4 +27,6 @@ public interface LowLevelCachePolicy extends LlapOomDebugDump {
   long evictSomeBlocks(long memoryToReserve);
   void setEvictionListener(EvictionListener listener);
   void setParentDebugDumper(LlapOomDebugDump dumper);
+  /** TODO: temporary method until we have a better allocator */
+  int tryEvictContiguousData(int allocationSize, int count);
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/62f5a3f2/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..430a5f8 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
@@ -72,16 +72,22 @@ public class LowLevelFifoCachePolicy implements LowLevelCachePolicy {
 
   @Override
   public long evictSomeBlocks(long memoryToReserve) {
+    return evictInternal(memoryToReserve, -1);
+  }
+
+  private long evictInternal(long memoryToReserve, int minSize) {
     long evicted = 0;
     lock.lock();
     try {
       Iterator<LlapCacheableBuffer> iter = buffers.iterator();
       while (evicted < memoryToReserve && iter.hasNext()) {
-        LlapCacheableBuffer candidate = iter.next();
-        if (candidate.invalidate()) {
+        LlapCacheableBuffer buffer = iter.next();
+        long memUsage = buffer.getMemoryUsage();
+        if (memUsage < minSize || (minSize > 0  && !(buffer instanceof LlapDataBuffer))) continue;
+        if (buffer.invalidate()) {
           iter.remove();
-          evicted += candidate.getMemoryUsage();
-          evictionListener.notifyEvicted(candidate);
+          evicted += memUsage;
+          evictionListener.notifyEvicted(buffer);
         }
       }
     } finally {
@@ -108,4 +114,12 @@ public class LowLevelFifoCachePolicy implements LowLevelCachePolicy {
     }
     return sb.toString();
   }
+
+  @Override
+  public int tryEvictContiguousData(int allocationSize, int count) {
+    long evicted = evictInternal(allocationSize * count, allocationSize);
+    // This makes granularity assumptions.
+    assert evicted % allocationSize == 0;
+    return (int)(evicted / allocationSize);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/62f5a3f2/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 5a0b27f..17be8a0 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
@@ -19,6 +19,9 @@
 package org.apache.hadoop.hive.llap.cache;
 
 import com.google.common.annotations.VisibleForTesting;
+
+import java.util.ArrayList;
+import java.util.List;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.ReentrantLock;
 import org.apache.commons.lang.StringUtils;
@@ -188,10 +191,41 @@ public class LowLevelLrfuCachePolicy implements LowLevelCachePolicy {
 
   @Override
   public long evictSomeBlocks(long memoryToReserve) {
-    long evicted = 0;
     // In normal case, we evict the items from the list.
-    LlapCacheableBuffer nextCandidate, firstCandidate;
+    long evicted = evictFromList(memoryToReserve);
+    if (evicted >= memoryToReserve) return evicted;
+    // This should not happen unless we are evicting a lot at once, or buffers are large (so
+    // there's a small number of buffers and they all live in the heap).
+    long time = timer.get();
+    while (evicted < memoryToReserve) {
+      LlapCacheableBuffer buffer = null;
+      synchronized (heap) {
+        buffer = evictFromHeapUnderLock(time);
+      }
+      if (buffer == null) return evicted;
+      evicted += buffer.getMemoryUsage();
+      evictionListener.notifyEvicted(buffer);
+    }
+    return evicted;
+  }
+
+  @Override
+  public int tryEvictContiguousData(int allocationSize, int count) {
+    int evicted = evictDataFromList(allocationSize, count);
+    count -= evicted;
+    if (count > 0) {
+      evicted += evictDataFromHeap(timer.get(), count, allocationSize);
+    }
+    return evicted;
+  }
+
+  private long evictFromList(long memoryToReserve) {
+    long evicted = 0;
+    LlapCacheableBuffer nextCandidate = null, firstCandidate = null;
     listLock.lock();
+    // We assume that there are no locked blocks in the list; or if they are, they can be dropped.
+    // Therefore we always evict one contiguous sequence from the tail. We can find it in one pass,
+    // splice it out and then finalize the eviction outside of the list lock.
     try {
       nextCandidate = firstCandidate = listTail;
       while (evicted < memoryToReserve && nextCandidate != null) {
@@ -226,22 +260,86 @@ public class LowLevelLrfuCachePolicy implements LowLevelCachePolicy {
       evictionListener.notifyEvicted(firstCandidate);
       firstCandidate = firstCandidate.prev;
     }
-    if (evicted >= memoryToReserve) return evicted;
-    // This should not happen unless we are evicting a lot at once, or buffers are large (so
-    // there's a small number of buffers and they all live in the heap).
-    long time = timer.get();
-    while (evicted < memoryToReserve) {
-      LlapCacheableBuffer buffer = null;
-      synchronized (heap) {
-        buffer = evictFromHeapUnderLock(time);
+    return evicted;
+  }
+
+
+  private int evictDataFromList(int minSize, int count) {
+    int evictedCount = 0;
+    // Unlike the normal evictFromList, we don't necessarily evict a sequence of blocks. We won't
+    // bother with optimization here and will just evict blocks one by one.
+    List<LlapCacheableBuffer> evictedBuffers = new ArrayList<>(count);
+    listLock.lock();
+    try {
+      LlapCacheableBuffer candidate = listTail;
+      while (evictedCount < count && candidate != null) {
+        LlapCacheableBuffer current = candidate;
+        candidate = candidate.prev;
+        long memUsage = current.getMemoryUsage();
+        if (memUsage < minSize || !(current instanceof LlapDataBuffer)) continue;
+        if (!current.invalidate()) {
+          // Locked buffer was in the list - just drop it; will be re-added on unlock.
+          removeFromListUnderLock(current);
+          continue;
+        }
+        // Remove the buffer from the list.
+        removeFromListUnderLock(current);
+        // This makes granularity assumptions.
+        assert memUsage % minSize == 0;
+        evictedCount += (memUsage / minSize);
+        evictedBuffers.add(current);
       }
-      if (buffer == null) return evicted;
-      evicted += buffer.getMemoryUsage();
+    } finally {
+      listLock.unlock();
+    }
+    for (LlapCacheableBuffer buffer : evictedBuffers) {
       evictionListener.notifyEvicted(buffer);
     }
-    return evicted;
+    return evictedCount;
+  }
+
+
+  // Note: rarely called (unless buffers are very large or we evict a lot, or in LFU case).
+  private LlapCacheableBuffer evictFromHeapUnderLock(long time) {
+    while (true) {
+      if (heapSize == 0) return null;
+      LlapCacheableBuffer result = evictHeapElementUnderLock(time, 0);
+      if (result != null) return result;
+    }
   }
 
+  // Note: almost never called (unless buffers are very large or we evict a lot, or LFU).
+  private int evictDataFromHeap(long time, int count, int minSize) {
+    LlapCacheableBuffer evicted = null;
+    int evictedCount = 0;
+    synchronized (heap) {
+      // Priorities go out of the window here.
+      int index = 0;
+      while (index < heapSize && evictedCount < count) {
+        LlapCacheableBuffer buffer = heap[index];
+        long memUsage = buffer.getMemoryUsage();
+        if (memUsage < minSize || !(buffer instanceof LlapDataBuffer)) {
+          ++index;
+          continue;
+        }
+        LlapCacheableBuffer result = evictHeapElementUnderLock(time, index);
+        // Don't advance the index - the buffer has been removed either way.
+        if (result != null) {
+          assert memUsage % minSize == 0;
+          evictedCount += (memUsage / minSize);
+          if (evicted != null) {
+            evictionListener.notifyEvicted(evicted);
+          }
+          evicted = result;
+        }
+      }
+    }
+    if (evicted != null) {
+      evictionListener.notifyEvicted(evicted);
+    }
+    return evictedCount;
+  }
+  
   private void heapifyUpUnderLock(LlapCacheableBuffer buffer, long time) {
     // See heapifyDown comment.
     int ix = buffer.indexInHeap;
@@ -260,29 +358,25 @@ public class LowLevelLrfuCachePolicy implements LowLevelCachePolicy {
     heap[ix] = buffer;
   }
 
-  // Note: almost never called (unless buffers are very large or we evict a lot, or LFU).
-  private LlapCacheableBuffer evictFromHeapUnderLock(long time) {
-    while (true) {
-      if (heapSize == 0) return null;
-      LlapCacheableBuffer result = heap[0];
-      if (LlapIoImpl.CACHE_LOGGER.isTraceEnabled()) {
-        LlapIoImpl.CACHE_LOGGER.info("Evicting {} at {}", result, time);
-      }
-      result.indexInHeap = LlapCacheableBuffer.NOT_IN_CACHE;
-      --heapSize;
-      boolean canEvict = result.invalidate();
-      if (heapSize > 0) {
-        LlapCacheableBuffer newRoot = heap[heapSize];
-        newRoot.indexInHeap = 0;
-        if (newRoot.lastUpdate != time) {
-          newRoot.priority = expirePriority(time, newRoot.lastUpdate, newRoot.priority);
-          newRoot.lastUpdate = time;
-        }
-        heapifyDownUnderLock(newRoot, time);
+  private LlapCacheableBuffer evictHeapElementUnderLock(long time, int ix) {
+    LlapCacheableBuffer result = heap[ix];
+    if (LlapIoImpl.CACHE_LOGGER.isTraceEnabled()) {
+      LlapIoImpl.CACHE_LOGGER.info("Evicting {} at {}", result, time);
+    }
+    result.indexInHeap = LlapCacheableBuffer.NOT_IN_CACHE;
+    --heapSize;
+    boolean canEvict = result.invalidate();
+    if (heapSize > 0) {
+      LlapCacheableBuffer newRoot = heap[heapSize];
+      newRoot.indexInHeap = ix;
+      if (newRoot.lastUpdate != time) {
+        newRoot.priority = expirePriority(time, newRoot.lastUpdate, newRoot.priority);
+        newRoot.lastUpdate = time;
       }
-      if (canEvict) return result;
-      // Otherwise we just removed a locked item from heap; unlock will re-add it, we continue.
+      heapifyDownUnderLock(newRoot, time);
     }
+    // Otherwise we just removed a locked item from heap; unlock will re-add it, we continue.
+    return canEvict ? result : null;
   }
 
   private void heapifyDownUnderLock(LlapCacheableBuffer buffer, long time) {

http://git-wip-us.apache.org/repos/asf/hive/blob/62f5a3f2/llap-server/src/java/org/apache/hadoop/hive/llap/cache/MemoryManager.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/MemoryManager.java b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/MemoryManager.java
index 6cc262e..ca41825 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/MemoryManager.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/MemoryManager.java
@@ -22,5 +22,6 @@ public interface MemoryManager extends LlapOomDebugDump {
   boolean reserveMemory(long memoryToReserve, boolean waitForEviction);
   void releaseMemory(long memUsage);
   void updateMaxSize(long maxSize);
-  void forceReservedMemory(int memoryToEvict);
+  /** TODO: temporary method until we get a better allocator. */
+  void forceReservedMemory(int allocationSize, int count);
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/62f5a3f2/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 345f5b1..04ba273 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
@@ -76,7 +76,7 @@ public class TestBuddyAllocator {
     }
 
     @Override
-    public void forceReservedMemory(int memoryToEvict) {
+    public void forceReservedMemory(int allocationSize, int count) {
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/62f5a3f2/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 0846db9..0ac0a0d 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
@@ -114,6 +114,11 @@ public class TestLowLevelCacheImpl {
 
     public void setParentDebugDumper(LlapOomDebugDump dumper) {
     }
+
+    @Override
+    public int tryEvictContiguousData(int allocationSize, int count) {
+      return count;
+    }
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hive/blob/62f5a3f2/llap-server/src/test/org/apache/hadoop/hive/llap/cache/TestLowLevelLrfuCachePolicy.java
----------------------------------------------------------------------
diff --git a/llap-server/src/test/org/apache/hadoop/hive/llap/cache/TestLowLevelLrfuCachePolicy.java b/llap-server/src/test/org/apache/hadoop/hive/llap/cache/TestLowLevelLrfuCachePolicy.java
index 616c040..f0de7c4 100644
--- a/llap-server/src/test/org/apache/hadoop/hive/llap/cache/TestLowLevelLrfuCachePolicy.java
+++ b/llap-server/src/test/org/apache/hadoop/hive/llap/cache/TestLowLevelLrfuCachePolicy.java
@@ -29,6 +29,7 @@ import static org.mockito.Mockito.doAnswer;
 import static org.mockito.Mockito.mock;
 
 import java.lang.reflect.Field;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
@@ -204,11 +205,59 @@ public class TestLowLevelLrfuCachePolicy {
     assertNotSame(locked, evicted);
     unlock(lrfu, locked);
   }
+  
+
+  @Test
+  public void testForceEvictBySize() {
+    int heapSize = 12;
+    LOG.info("Testing force-eviction out of order");
+    Configuration conf = new Configuration();
+    ArrayList<LlapDataBuffer> sizeTwo = new ArrayList<LlapDataBuffer>(4),
+        sizeOne = new ArrayList<LlapDataBuffer>(4);
+    conf.setFloat(HiveConf.ConfVars.LLAP_LRFU_LAMBDA.varname, 0.45f); // size of LFU heap is 4
+    EvictionTracker et = new EvictionTracker();
+    LowLevelLrfuCachePolicy lrfu = new LowLevelLrfuCachePolicy(1, heapSize, conf);
+    lrfu.setEvictionListener(et);
+    for (int i = 0; i < 2; ++i) {
+      sizeTwo.add(cacheSizeTwoFake(et, lrfu));
+      for (int j = 0; j < 2; ++j) {
+        LlapDataBuffer fake = LowLevelCacheImpl.allocateFake();
+        assertTrue(cache(null, lrfu, et, fake));
+        sizeOne.add(fake);
+      }
+      sizeTwo.add(cacheSizeTwoFake(et, lrfu));
+    }
+    // Now we should have two in the heap and two in the list, which is an implementation detail.
+    // Evict only big blocks.
+    et.evicted.clear();
+    assertEquals(4, lrfu.tryEvictContiguousData(2, 4));
+    for (int i = 0; i < sizeTwo.size(); ++i) {
+      LlapDataBuffer block = et.evicted.get(i);
+      assertTrue(block.isInvalid());
+      assertSame(sizeTwo.get(i), block);
+    }
+    et.evicted.clear();
+    // Cannot evict any more size 2.
+    assertEquals(0, lrfu.tryEvictContiguousData(2, 1));
+    assertEquals(4, lrfu.evictSomeBlocks(4));
+    for (int i = 0; i < sizeOne.size(); ++i) {
+      LlapDataBuffer block = et.evicted.get(i);
+      assertTrue(block.isInvalid());
+      assertSame(sizeOne.get(i), block);
+    }
+  }
+
+  private LlapDataBuffer cacheSizeTwoFake(EvictionTracker et, LowLevelLrfuCachePolicy lrfu) {
+    LlapDataBuffer fake = new LlapDataBuffer();
+    fake.initialize(-1, ByteBuffer.wrap(new byte[2]), 0, 2);
+    assertTrue(cache(null, lrfu, et, fake));
+    return fake;
+  }
 
   // Buffers in test are fakes not linked to cache; notify cache policy explicitly.
   public boolean cache(LowLevelCacheMemoryManager mm,
       LowLevelLrfuCachePolicy lrfu, EvictionTracker et, LlapDataBuffer buffer) {
-    if (!mm.reserveMemory(1, false)) {
+    if (mm != null && !mm.reserveMemory(1, false)) {
       return false;
     }
     buffer.incRef();

http://git-wip-us.apache.org/repos/asf/hive/blob/62f5a3f2/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 40edb28..3408dff 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
@@ -61,6 +61,11 @@ public class TestOrcMetadataCache {
       assertEquals(i, lockCount);
       assertEquals(i, unlockCount);
     }
+
+    @Override
+    public int tryEvictContiguousData(int allocationSize, int count) {
+      return 0;
+    }
   }
 
   private static class DummyMemoryManager implements MemoryManager {
@@ -87,7 +92,7 @@ public class TestOrcMetadataCache {
     }
 
     @Override
-    public void forceReservedMemory(int memoryToEvict) {
+    public void forceReservedMemory(int allocationSize, int count) {
     }
   }