mirror of https://github.com/apache/druid.git
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:
parent
56b0586097
commit
91e6ab4fcf
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
|
|
Loading…
Reference in New Issue