LRU cache guarantee to keep size under limit (#3510)

* LRU cache guarantee to keep size under limit

* address comments

* fix failed tests in jdk7
This commit is contained in:
Xiaoyao 2016-09-27 19:13:06 -05:00 committed by Charles Allen
parent 56b0586097
commit 91e6ab4fcf
2 changed files with 35 additions and 27 deletions

View File

@ -22,10 +22,14 @@ package io.druid.client.cache;
import com.metamx.common.logger.Logger; import com.metamx.common.logger.Logger;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Collections; import java.util.Collections;
import java.util.Iterator;
import java.util.LinkedHashMap; import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.concurrent.atomic.AtomicLong;
/** /**
*/ */
@ -37,8 +41,8 @@ class ByteCountingLRUMap extends LinkedHashMap<ByteBuffer, byte[]>
private final int logEvictionCount; private final int logEvictionCount;
private final long sizeInBytes; private final long sizeInBytes;
private volatile long numBytes; private final AtomicLong numBytes;
private volatile long evictionCount; private final AtomicLong evictionCount;
public ByteCountingLRUMap( public ByteCountingLRUMap(
final long sizeInBytes final long sizeInBytes
@ -58,46 +62,49 @@ class ByteCountingLRUMap extends LinkedHashMap<ByteBuffer, byte[]>
this.sizeInBytes = sizeInBytes; this.sizeInBytes = sizeInBytes;
logEvictions = logEvictionCount != 0; logEvictions = logEvictionCount != 0;
numBytes = 0; numBytes = new AtomicLong(0L);
evictionCount = 0; evictionCount = new AtomicLong(0L);
} }
public long getNumBytes() public long getNumBytes()
{ {
return numBytes; return numBytes.get();
} }
public long getEvictionCount() public long getEvictionCount()
{ {
return evictionCount; return evictionCount.get();
} }
@Override @Override
public byte[] put(ByteBuffer key, byte[] value) public byte[] put(ByteBuffer key, byte[] value)
{ {
numBytes += key.remaining() + value.length; numBytes.addAndGet(key.remaining() + value.length);
return super.put(key, value); Iterator<Map.Entry<ByteBuffer, byte[]>> it = entrySet().iterator();
} List<ByteBuffer> keysToRemove = new ArrayList<>();
long totalEvictionSize = 0L;
@Override while (numBytes.get() - totalEvictionSize > sizeInBytes && it.hasNext()) {
protected boolean removeEldestEntry(Map.Entry<ByteBuffer, byte[]> eldest) evictionCount.incrementAndGet();
{ if (logEvictions && evictionCount.get() % logEvictionCount == 0) {
if (numBytes > sizeInBytes) {
++evictionCount;
if (logEvictions && evictionCount % logEvictionCount == 0) {
log.info( log.info(
"Evicting %,dth element. Size[%,d], numBytes[%,d], averageSize[%,d]", "Evicting %,dth element. Size[%,d], numBytes[%,d], averageSize[%,d]",
evictionCount, evictionCount,
size(), size(),
numBytes, numBytes.get(),
numBytes / size() numBytes.get() / size()
); );
} }
numBytes -= eldest.getKey().remaining() + eldest.getValue().length; Map.Entry<ByteBuffer, byte[]> next = it.next();
return true; totalEvictionSize += next.getKey().remaining() + next.getValue().length;
keysToRemove.add(next.getKey());
} }
return false;
for (ByteBuffer keyToRemove : keysToRemove) {
remove(keyToRemove);
}
return super.put(key, value);
} }
@Override @Override
@ -105,7 +112,8 @@ class ByteCountingLRUMap extends LinkedHashMap<ByteBuffer, byte[]>
{ {
byte[] value = super.remove(key); byte[] value = super.remove(key);
if(value != null) { if(value != null) {
numBytes -= ((ByteBuffer)key).remaining() + value.length; long delta = -((ByteBuffer)key).remaining() - value.length;
numBytes.addAndGet(delta);
} }
return value; return value;
} }
@ -123,7 +131,7 @@ class ByteCountingLRUMap extends LinkedHashMap<ByteBuffer, byte[]>
@Override @Override
public void clear() public void clear()
{ {
numBytes = 0; numBytes.set(0L);
super.clear(); super.clear();
} }
} }

View File

@ -65,8 +65,8 @@ public class ByteCountingLRUMapTest
Assert.assertEquals(ByteBuffer.wrap(eightyEightVal), ByteBuffer.wrap(map.get(tenKey))); Assert.assertEquals(ByteBuffer.wrap(eightyEightVal), ByteBuffer.wrap(map.get(tenKey)));
map.put(twoByte, oneByte.array()); map.put(twoByte, oneByte.array());
assertMapValues(2, 101, 2); assertMapValues(1, 3, 3);
Assert.assertEquals(ByteBuffer.wrap(eightyEightVal), ByteBuffer.wrap(map.get(tenKey))); Assert.assertEquals(null, map.get(tenKey));
Assert.assertEquals(oneByte, ByteBuffer.wrap(map.get(twoByte))); Assert.assertEquals(oneByte, ByteBuffer.wrap(map.get(twoByte)));
Iterator<ByteBuffer> it = map.keySet().iterator(); Iterator<ByteBuffer> it = map.keySet().iterator();
@ -80,10 +80,10 @@ public class ByteCountingLRUMapTest
for(ByteBuffer buf : toRemove) { for(ByteBuffer buf : toRemove) {
map.remove(buf); map.remove(buf);
} }
assertMapValues(1, 3, 2); assertMapValues(1, 3, 3);
map.remove(twoByte); map.remove(twoByte);
assertMapValues(0, 0, 2); assertMapValues(0, 0, 3);
} }
private void assertMapValues(final int size, final int numBytes, final int evictionCount) private void assertMapValues(final int size, final int numBytes, final int evictionCount)