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 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();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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)
|
||||||
|
|
Loading…
Reference in New Issue