SOLR-7355: Switch from ConcurrentLinkedHashMap to Caffeine. Trunk-only change, as it requires java 8.

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1672133 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Shawn Heisey 2015-04-08 18:23:19 +00:00
parent eaf4659bfe
commit 7e9b7448d1
9 changed files with 32 additions and 23 deletions

View File

@ -58,6 +58,9 @@ Other Changes
* SOLR-6954: Deprecated SolrClient.shutdown() method removed (Alan Woodward)
* SOLR-7355: Switch from Google's ConcurrentLinkedHashMap to Caffeine. Only
affects HDFS support. (Ben Manes via Shawn Heisey)
================== 5.2.0 ==================
Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release

View File

@ -66,7 +66,7 @@
<dependency org="commons-collections" name="commons-collections" rev="${/commons-collections/commons-collections}" conf="compile.hadoop"/>
<dependency org="com.google.protobuf" name="protobuf-java" rev="${/com.google.protobuf/protobuf-java}" conf="compile.hadoop"/>
<dependency org="com.googlecode.concurrentlinkedhashmap" name="concurrentlinkedhashmap-lru" rev="${/com.googlecode.concurrentlinkedhashmap/concurrentlinkedhashmap-lru}" conf="compile.hadoop"/>
<dependency org="com.github.ben-manes.caffeine" name="caffeine" rev="${/com.github.ben-manes.caffeine/caffeine}" conf="compile.hadoop"/>
<dependency org="org.htrace" name="htrace-core" rev="${/org.htrace/htrace-core}" conf="compile.hadoop"/>
<!-- Hadoop DfsMiniCluster Dependencies-->

View File

@ -21,8 +21,9 @@ import java.nio.ByteBuffer;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.atomic.AtomicInteger;
import com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap;
import com.googlecode.concurrentlinkedhashmap.EvictionListener;
import com.github.benmanes.caffeine.cache.Cache;
import com.github.benmanes.caffeine.cache.Caffeine;
import com.github.benmanes.caffeine.cache.RemovalListener;
/**
* @lucene.experimental
@ -31,7 +32,7 @@ public class BlockCache {
public static final int _128M = 134217728;
public static final int _32K = 32768;
private final ConcurrentMap<BlockCacheKey,BlockCacheLocation> cache;
private final Cache<BlockCacheKey,BlockCacheLocation> cache;
private final ByteBuffer[] banks;
private final BlockLocks[] locks;
private final AtomicInteger[] lockCounters;
@ -68,20 +69,18 @@ public class BlockCache {
locks[i] = new BlockLocks(numberOfBlocksPerBank);
lockCounters[i] = new AtomicInteger();
}
EvictionListener<BlockCacheKey,BlockCacheLocation> listener = new EvictionListener<BlockCacheKey,BlockCacheLocation>() {
@Override
public void onEviction(BlockCacheKey key, BlockCacheLocation location) {
releaseLocation(location);
}
};
cache = new ConcurrentLinkedHashMap.Builder<BlockCacheKey,BlockCacheLocation>()
.maximumWeightedCapacity(maxEntries).listener(listener).build();
RemovalListener<BlockCacheKey,BlockCacheLocation> listener =
notification -> releaseLocation(notification.getValue());
cache = Caffeine.newBuilder()
.removalListener(listener)
.maximumSize(maxEntries)
.build();
this.blockSize = blockSize;
}
public void release(BlockCacheKey key) {
releaseLocation(cache.remove(key));
cache.invalidate(key);
}
private void releaseLocation(BlockCacheLocation location) {
@ -104,7 +103,7 @@ public class BlockCache {
+ blockSize + "] got length [" + length + "] with blockOffset ["
+ blockOffset + "]");
}
BlockCacheLocation location = cache.get(blockCacheKey);
BlockCacheLocation location = cache.getIfPresent(blockCacheKey);
boolean newLocation = false;
if (location == null) {
newLocation = true;
@ -122,7 +121,7 @@ public class BlockCache {
bank.position(bankOffset + blockOffset);
bank.put(data, offset, length);
if (newLocation) {
releaseLocation(cache.put(blockCacheKey.clone(), location));
cache.put(blockCacheKey.clone(), location);
metrics.blockCacheSize.incrementAndGet();
}
return true;
@ -130,7 +129,7 @@ public class BlockCache {
public boolean fetch(BlockCacheKey blockCacheKey, byte[] buffer,
int blockOffset, int off, int length) {
BlockCacheLocation location = cache.get(blockCacheKey);
BlockCacheLocation location = cache.getIfPresent(blockCacheKey);
if (location == null) {
return false;
}
@ -201,6 +200,6 @@ public class BlockCache {
}
public int getSize() {
return cache.size();
return cache.asMap().size();
}
}

View File

@ -22,6 +22,8 @@ import java.io.IOException;
import java.util.Map;
import java.util.Random;
import com.github.benmanes.caffeine.cache.Caffeine;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FSDirectory;
import org.apache.lucene.store.IOContext;
@ -34,12 +36,13 @@ import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap;
public class BlockDirectoryTest extends SolrTestCaseJ4 {
private class MapperCache implements Cache {
public Map<String, byte[]> map = new ConcurrentLinkedHashMap.Builder<String, byte[]>().maximumWeightedCapacity(8).build();
public Map<String, byte[]> map = Caffeine.newBuilder()
.maximumSize(8)
.<String, byte[]>build()
.asMap();
@Override
public void update(String name, long blockId, int blockOffset, byte[] buffer, int offset, int length) {
@ -167,7 +170,10 @@ public class BlockDirectoryTest extends SolrTestCaseJ4 {
@Test
public void testRandomAccessWritesLargeCache() throws IOException {
mapperCache.map = new ConcurrentLinkedHashMap.Builder<String, byte[]>().maximumWeightedCapacity(10000).build();
mapperCache.map = Caffeine.newBuilder()
.maximumSize(10_000)
.<String, byte[]>build()
.asMap();
testRandomAccessWrites();
}

View File

@ -0,0 +1 @@
eb95a1eb55cb02018b8e0bc1609ce569b455ea98

View File

@ -0,0 +1 @@
Copyright 2015 by Ben Manes

View File

@ -1 +0,0 @@
4316d710b6619ffe210c98deb2b0893587dad454