Field Cache: Allow to set `index.cache.field.expire` when using `resident` type, closes #957.

This commit is contained in:
kimchy 2011-05-22 07:22:44 +03:00
parent 404a8f23de
commit 6f80e306d6
2 changed files with 69 additions and 7 deletions

View File

@ -22,7 +22,7 @@ package org.elasticsearch.index.cache.field.data;
import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.inject.Scopes;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.cache.field.data.soft.SoftFieldDataCache;
import org.elasticsearch.index.cache.field.data.resident.ResidentFieldDataCache;
/**
* @author kimchy (shay.banon)
@ -41,7 +41,7 @@ public class FieldDataCacheModule extends AbstractModule {
@Override protected void configure() {
bind(FieldDataCache.class)
.to(settings.getAsClass(FieldDataCacheSettings.FIELD_DATA_CACHE_TYPE, SoftFieldDataCache.class, "org.elasticsearch.index.cache.field.data.", "FieldDataCache"))
.to(settings.getAsClass(FieldDataCacheSettings.FIELD_DATA_CACHE_TYPE, ResidentFieldDataCache.class, "org.elasticsearch.index.cache.field.data.", "FieldDataCache"))
.in(Scopes.SINGLETON);
}
}

View File

@ -19,27 +19,64 @@
package org.elasticsearch.index.cache.field.data.resident;
import org.elasticsearch.ElasticSearchException;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.base.Objects;
import org.elasticsearch.common.collect.MapEvictionListener;
import org.elasticsearch.common.collect.MapMaker;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.cache.field.data.support.AbstractConcurrentMapFieldDataCache;
import org.elasticsearch.index.field.data.FieldData;
import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.settings.IndexSettingsService;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
/**
* @author kimchy (shay.banon)
*/
public class ResidentFieldDataCache extends AbstractConcurrentMapFieldDataCache {
public class ResidentFieldDataCache extends AbstractConcurrentMapFieldDataCache implements MapEvictionListener<String, FieldData> {
@Inject public ResidentFieldDataCache(Index index, @IndexSettings Settings indexSettings) {
private final IndexSettingsService indexSettingsService;
private volatile int maxSize;
private volatile TimeValue expire;
private final AtomicLong evictions = new AtomicLong();
private final ApplySettings applySettings = new ApplySettings();
@Inject public ResidentFieldDataCache(Index index, @IndexSettings Settings indexSettings, IndexSettingsService indexSettingsService) {
super(index, indexSettings);
this.indexSettingsService = indexSettingsService;
this.maxSize = indexSettings.getAsInt("index.cache.field.max_size", componentSettings.getAsInt("max_size", -1));
this.expire = indexSettings.getAsTime("index.cache.field.expire", componentSettings.getAsTime("expire", null));
logger.debug("using [resident] field cache with max_size [{}], expire [{}]", maxSize, expire);
indexSettingsService.addListener(applySettings);
}
@Override public void close() throws ElasticSearchException {
indexSettingsService.removeListener(applySettings);
super.close();
}
@Override protected ConcurrentMap<String, FieldData> buildFieldDataMap() {
return new ConcurrentHashMap<String, FieldData>();
MapMaker mapMaker = new MapMaker();
if (maxSize != -1) {
mapMaker.maximumSize(maxSize);
}
if (expire != null) {
mapMaker.expireAfterAccess(expire.nanos(), TimeUnit.NANOSECONDS);
}
mapMaker.evictionListener(this);
return mapMaker.makeMap();
}
@Override public String type() {
@ -47,6 +84,31 @@ public class ResidentFieldDataCache extends AbstractConcurrentMapFieldDataCache
}
@Override public long evictions() {
return 0;
return evictions.get();
}
@Override public void onEviction(@Nullable String s, @Nullable FieldData fieldData) {
evictions.incrementAndGet();
}
class ApplySettings implements IndexSettingsService.Listener {
@Override public void onRefreshSettings(Settings settings) {
int maxSize = settings.getAsInt("index.cache.field.max_size", ResidentFieldDataCache.this.maxSize);
TimeValue expire = settings.getAsTime("index.cache.field.expire", ResidentFieldDataCache.this.expire);
boolean changed = false;
if (maxSize != ResidentFieldDataCache.this.maxSize) {
logger.info("updating index.cache.field.max_size from [{}] to [{}]", ResidentFieldDataCache.this.maxSize, maxSize);
changed = true;
ResidentFieldDataCache.this.maxSize = maxSize;
}
if (!Objects.equal(expire, ResidentFieldDataCache.this.expire)) {
logger.info("updating index.cache.field.expire from [{}] to [{}]", ResidentFieldDataCache.this.expire, expire);
changed = true;
ResidentFieldDataCache.this.expire = expire;
}
if (changed) {
clear();
}
}
}
}