Update Settings: Allow to dynamically set filter cache settings, closes #874.

This commit is contained in:
kimchy 2011-04-21 14:40:38 +03:00
parent add6db352d
commit dedbedbb77
5 changed files with 121 additions and 14 deletions

View File

@ -20,6 +20,7 @@
package org.elasticsearch.index.cache.filter.resident;
import org.apache.lucene.search.Filter;
import org.elasticsearch.common.base.Objects;
import org.elasticsearch.common.collect.MapEvictionListener;
import org.elasticsearch.common.collect.MapMaker;
import org.elasticsearch.common.inject.Inject;
@ -29,29 +30,41 @@ import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.cache.filter.support.AbstractConcurrentMapFilterCache;
import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.settings.IndexSettingsService;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
/**
* A resident reference based filter cache that has soft keys on the <tt>IndexReader</tt>.
* A resident reference based filter cache that has weak keys on the <tt>IndexReader</tt>.
*
* @author kimchy (shay.banon)
*/
public class ResidentFilterCache extends AbstractConcurrentMapFilterCache implements MapEvictionListener<Filter, DocSet> {
private final int maxSize;
private final IndexSettingsService indexSettingsService;
private final TimeValue expire;
private volatile int maxSize;
private volatile TimeValue expire;
private final AtomicLong evictions = new AtomicLong();
@Inject public ResidentFilterCache(Index index, @IndexSettings Settings indexSettings) {
private final ApplySettings applySettings = new ApplySettings();
@Inject public ResidentFilterCache(Index index, @IndexSettings Settings indexSettings, IndexSettingsService indexSettingsService) {
super(index, indexSettings);
this.indexSettingsService = indexSettingsService;
this.maxSize = indexSettings.getAsInt("index.cache.filter.max_size", componentSettings.getAsInt("max_size", 1000));
this.expire = indexSettings.getAsTime("index.cache.filter.expire", componentSettings.getAsTime("expire", null));
logger.debug("using [resident] filter cache with max_size [{}], expire [{}]", maxSize, expire);
indexSettingsService.addListener(applySettings);
}
@Override public void close() {
indexSettingsService.removeListener(applySettings);
super.close();
}
@Override protected ConcurrentMap<Filter, DocSet> buildFilterMap() {
@ -67,7 +80,7 @@ public class ResidentFilterCache extends AbstractConcurrentMapFilterCache implem
}
@Override public String type() {
return "soft";
return "resident";
}
@Override public long evictions() {
@ -81,4 +94,25 @@ public class ResidentFilterCache extends AbstractConcurrentMapFilterCache implem
@Override public void onEviction(Filter filter, DocSet docSet) {
evictions.incrementAndGet();
}
class ApplySettings implements IndexSettingsService.Listener {
@Override public void onRefreshSettings(Settings settings) {
int maxSize = settings.getAsInt("index.cache.filter.max_size", ResidentFilterCache.this.maxSize);
TimeValue expire = settings.getAsTime("index.cache.filter.expire", ResidentFilterCache.this.expire);
boolean changed = false;
if (maxSize != ResidentFilterCache.this.maxSize) {
logger.info("updating index.cache.filter.max_size from [{}] to [{}]", ResidentFilterCache.this.maxSize, maxSize);
changed = true;
ResidentFilterCache.this.maxSize = maxSize;
}
if (!Objects.equal(expire, ResidentFilterCache.this.expire)) {
logger.info("updating index.cache.filter.expire from [{}] to [{}]", ResidentFilterCache.this.expire, expire);
changed = true;
ResidentFilterCache.this.expire = expire;
}
if (changed) {
clear();
}
}
}
}

View File

@ -20,6 +20,7 @@
package org.elasticsearch.index.cache.filter.soft;
import org.apache.lucene.search.Filter;
import org.elasticsearch.common.base.Objects;
import org.elasticsearch.common.collect.MapEvictionListener;
import org.elasticsearch.common.collect.MapMaker;
import org.elasticsearch.common.inject.Inject;
@ -29,6 +30,7 @@ import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.cache.filter.support.AbstractConcurrentMapFilterCache;
import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.settings.IndexSettingsService;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.TimeUnit;
@ -41,18 +43,29 @@ import java.util.concurrent.atomic.AtomicLong;
*/
public class SoftFilterCache extends AbstractConcurrentMapFilterCache implements MapEvictionListener<Filter, DocSet> {
private final int maxSize;
private final IndexSettingsService indexSettingsService;
private final TimeValue expire;
private volatile int maxSize;
private volatile TimeValue expire;
private final AtomicLong evictions = new AtomicLong();
private AtomicLong memEvictions;
@Inject public SoftFilterCache(Index index, @IndexSettings Settings indexSettings) {
private final ApplySettings applySettings = new ApplySettings();
@Inject public SoftFilterCache(Index index, @IndexSettings Settings indexSettings, IndexSettingsService indexSettingsService) {
super(index, indexSettings);
this.indexSettingsService = indexSettingsService;
this.maxSize = indexSettings.getAsInt("index.cache.filter.max_size", componentSettings.getAsInt("max_size", -1));
this.expire = indexSettings.getAsTime("index.cache.filter.expire", componentSettings.getAsTime("expire", null));
logger.debug("using [soft] filter cache with max_size [{}], expire [{}]", maxSize, expire);
indexSettingsService.addListener(applySettings);
}
@Override public void close() {
indexSettingsService.removeListener(applySettings);
super.close();
}
@Override protected ConcurrentMap<Object, ReaderValue> buildCache() {
@ -70,7 +83,7 @@ public class SoftFilterCache extends AbstractConcurrentMapFilterCache implements
if (maxSize != -1) {
mapMaker.maximumSize(maxSize);
}
if (expire != null) {
if (expire != null && expire.nanos() > 0) {
mapMaker.expireAfterAccess(expire.nanos(), TimeUnit.NANOSECONDS);
}
mapMaker.evictionListener(this);
@ -92,4 +105,25 @@ public class SoftFilterCache extends AbstractConcurrentMapFilterCache implements
@Override public void onEviction(Filter filter, DocSet docSet) {
evictions.incrementAndGet();
}
class ApplySettings implements IndexSettingsService.Listener {
@Override public void onRefreshSettings(Settings settings) {
int maxSize = settings.getAsInt("index.cache.filter.max_size", SoftFilterCache.this.maxSize);
TimeValue expire = settings.getAsTime("index.cache.filter.expire", SoftFilterCache.this.expire);
boolean changed = false;
if (maxSize != SoftFilterCache.this.maxSize) {
logger.info("updating index.cache.filter.max_size from [{}] to [{}]", SoftFilterCache.this.maxSize, maxSize);
changed = true;
SoftFilterCache.this.maxSize = maxSize;
}
if (!Objects.equal(expire, SoftFilterCache.this.expire)) {
logger.info("updating index.cache.filter.expire from [{}] to [{}]", SoftFilterCache.this.expire, expire);
changed = true;
SoftFilterCache.this.expire = expire;
}
if (changed) {
clear();
}
}
}
}

View File

@ -20,6 +20,7 @@
package org.elasticsearch.index.cache.filter.weak;
import org.apache.lucene.search.Filter;
import org.elasticsearch.common.base.Objects;
import org.elasticsearch.common.collect.MapEvictionListener;
import org.elasticsearch.common.collect.MapMaker;
import org.elasticsearch.common.inject.Inject;
@ -29,6 +30,7 @@ import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.cache.filter.support.AbstractConcurrentMapFilterCache;
import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.settings.IndexSettingsService;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.TimeUnit;
@ -41,18 +43,29 @@ import java.util.concurrent.atomic.AtomicLong;
*/
public class WeakFilterCache extends AbstractConcurrentMapFilterCache implements MapEvictionListener<Filter, DocSet> {
private final int maxSize;
private final IndexSettingsService indexSettingsService;
private final TimeValue expire;
private volatile int maxSize;
private volatile TimeValue expire;
private final AtomicLong evictions = new AtomicLong();
private AtomicLong memEvictions;
@Inject public WeakFilterCache(Index index, @IndexSettings Settings indexSettings) {
private final ApplySettings applySettings = new ApplySettings();
@Inject public WeakFilterCache(Index index, @IndexSettings Settings indexSettings, IndexSettingsService indexSettingsService) {
super(index, indexSettings);
this.indexSettingsService = indexSettingsService;
this.maxSize = indexSettings.getAsInt("index.cache.filter.max_size", componentSettings.getAsInt("max_size", -1));
this.expire = indexSettings.getAsTime("index.cache.filter.expire", componentSettings.getAsTime("expire", null));
logger.debug("using [weak] filter cache with max_size [{}], expire [{}]", maxSize, expire);
indexSettingsService.addListener(applySettings);
}
@Override public void close() {
indexSettingsService.removeListener(applySettings);
super.close();
}
@Override protected ConcurrentMap<Object, ReaderValue> buildCache() {
@ -90,4 +103,25 @@ public class WeakFilterCache extends AbstractConcurrentMapFilterCache implements
@Override public void onEviction(Filter filter, DocSet docSet) {
evictions.incrementAndGet();
}
class ApplySettings implements IndexSettingsService.Listener {
@Override public void onRefreshSettings(Settings settings) {
int maxSize = settings.getAsInt("index.cache.filter.max_size", WeakFilterCache.this.maxSize);
TimeValue expire = settings.getAsTime("index.cache.filter.expire", WeakFilterCache.this.expire);
boolean changed = false;
if (maxSize != WeakFilterCache.this.maxSize) {
logger.info("updating index.cache.filter.max_size from [{}] to [{}]", WeakFilterCache.this.maxSize, maxSize);
changed = true;
WeakFilterCache.this.maxSize = maxSize;
}
if (!Objects.equal(expire, WeakFilterCache.this.expire)) {
logger.info("updating index.cache.filter.expire from [{}] to [{}]", WeakFilterCache.this.expire, expire);
changed = true;
WeakFilterCache.this.expire = expire;
}
if (changed) {
clear();
}
}
}
}

View File

@ -56,4 +56,8 @@ public class TimeValueTests {
assertThat(new TimeValue(65, TimeUnit.MINUTES).format(PeriodType.dayTime()), equalTo("1 hour and 5 minutes"));
assertThat(new TimeValue(24 * 600 + 85, TimeUnit.MINUTES).format(PeriodType.dayTime()), equalTo("241 hours and 25 minutes"));
}
@Test public void testMinusOne() {
assertThat(new TimeValue(-1).nanos(), lessThan(0l));
}
}

View File

@ -31,6 +31,7 @@ import org.elasticsearch.index.Index;
import org.elasticsearch.index.cache.filter.none.NoneFilterCache;
import org.elasticsearch.index.cache.filter.soft.SoftFilterCache;
import org.elasticsearch.index.cache.filter.weak.WeakFilterCache;
import org.elasticsearch.index.settings.IndexSettingsService;
import org.testng.annotations.Test;
import java.io.IOException;
@ -51,11 +52,11 @@ public class FilterCacheTests {
}
@Test public void testSoftCache() throws Exception {
verifyCache(new SoftFilterCache(new Index("test"), EMPTY_SETTINGS));
verifyCache(new SoftFilterCache(new Index("test"), EMPTY_SETTINGS, new IndexSettingsService(new Index("test"), EMPTY_SETTINGS)));
}
@Test public void testWeakCache() throws Exception {
verifyCache(new WeakFilterCache(new Index("test"), EMPTY_SETTINGS));
verifyCache(new WeakFilterCache(new Index("test"), EMPTY_SETTINGS, new IndexSettingsService(new Index("test"), EMPTY_SETTINGS)));
}
private void verifyCache(FilterCache filterCache) throws Exception {