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 java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
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 long sizeInBytes;
private volatile long numBytes;
private volatile long evictionCount;
private final AtomicLong numBytes;
private final AtomicLong evictionCount;
public ByteCountingLRUMap(
final long sizeInBytes
@ -58,46 +62,49 @@ class ByteCountingLRUMap extends LinkedHashMap<ByteBuffer, byte[]>
this.sizeInBytes = sizeInBytes;
logEvictions = logEvictionCount != 0;
numBytes = 0;
evictionCount = 0;
numBytes = new AtomicLong(0L);
evictionCount = new AtomicLong(0L);
}
public long getNumBytes()
{
return numBytes;
return numBytes.get();
}
public long getEvictionCount()
{
return evictionCount;
return evictionCount.get();
}
@Override
public byte[] put(ByteBuffer key, byte[] value)
{
numBytes += key.remaining() + value.length;
return super.put(key, value);
}
@Override
protected boolean removeEldestEntry(Map.Entry<ByteBuffer, byte[]> eldest)
{
if (numBytes > sizeInBytes) {
++evictionCount;
if (logEvictions && evictionCount % logEvictionCount == 0) {
numBytes.addAndGet(key.remaining() + value.length);
Iterator<Map.Entry<ByteBuffer, byte[]>> it = entrySet().iterator();
List<ByteBuffer> keysToRemove = new ArrayList<>();
long totalEvictionSize = 0L;
while (numBytes.get() - totalEvictionSize > sizeInBytes && it.hasNext()) {
evictionCount.incrementAndGet();
if (logEvictions && evictionCount.get() % logEvictionCount == 0) {
log.info(
"Evicting %,dth element. Size[%,d], numBytes[%,d], averageSize[%,d]",
evictionCount,
size(),
numBytes,
numBytes / size()
numBytes.get(),
numBytes.get() / size()
);
}
numBytes -= eldest.getKey().remaining() + eldest.getValue().length;
return true;
Map.Entry<ByteBuffer, byte[]> next = it.next();
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
@ -105,7 +112,8 @@ class ByteCountingLRUMap extends LinkedHashMap<ByteBuffer, byte[]>
{
byte[] value = super.remove(key);
if(value != null) {
numBytes -= ((ByteBuffer)key).remaining() + value.length;
long delta = -((ByteBuffer)key).remaining() - value.length;
numBytes.addAndGet(delta);
}
return value;
}
@ -123,7 +131,7 @@ class ByteCountingLRUMap extends LinkedHashMap<ByteBuffer, byte[]>
@Override
public void clear()
{
numBytes = 0;
numBytes.set(0L);
super.clear();
}
}

View File

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