Skip to content

Commit 6f80e30

Browse files
committed
Field Cache: Allow to set index.cache.field.expire when using resident type, closes #957.
1 parent 404a8f2 commit 6f80e30

File tree

2 files changed

+69
-7
lines changed

2 files changed

+69
-7
lines changed

modules/elasticsearch/src/main/java/org/elasticsearch/index/cache/field/data/FieldDataCacheModule.java

+2-2
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,7 @@
2222
import org.elasticsearch.common.inject.AbstractModule;
2323
import org.elasticsearch.common.inject.Scopes;
2424
import org.elasticsearch.common.settings.Settings;
25-
import org.elasticsearch.index.cache.field.data.soft.SoftFieldDataCache;
25+
import org.elasticsearch.index.cache.field.data.resident.ResidentFieldDataCache;
2626

2727
/**
2828
* @author kimchy (shay.banon)
@@ -41,7 +41,7 @@ public FieldDataCacheModule(Settings settings) {
4141

4242
@Override protected void configure() {
4343
bind(FieldDataCache.class)
44-
.to(settings.getAsClass(FieldDataCacheSettings.FIELD_DATA_CACHE_TYPE, SoftFieldDataCache.class, "org.elasticsearch.index.cache.field.data.", "FieldDataCache"))
44+
.to(settings.getAsClass(FieldDataCacheSettings.FIELD_DATA_CACHE_TYPE, ResidentFieldDataCache.class, "org.elasticsearch.index.cache.field.data.", "FieldDataCache"))
4545
.in(Scopes.SINGLETON);
4646
}
4747
}

modules/elasticsearch/src/main/java/org/elasticsearch/index/cache/field/data/resident/ResidentFieldDataCache.java

+67-5
Original file line numberDiff line numberDiff line change
@@ -19,34 +19,96 @@
1919

2020
package org.elasticsearch.index.cache.field.data.resident;
2121

22+
import org.elasticsearch.ElasticSearchException;
23+
import org.elasticsearch.common.Nullable;
24+
import org.elasticsearch.common.base.Objects;
25+
import org.elasticsearch.common.collect.MapEvictionListener;
26+
import org.elasticsearch.common.collect.MapMaker;
2227
import org.elasticsearch.common.inject.Inject;
2328
import org.elasticsearch.common.settings.Settings;
29+
import org.elasticsearch.common.unit.TimeValue;
2430
import org.elasticsearch.index.Index;
2531
import org.elasticsearch.index.cache.field.data.support.AbstractConcurrentMapFieldDataCache;
2632
import org.elasticsearch.index.field.data.FieldData;
2733
import org.elasticsearch.index.settings.IndexSettings;
34+
import org.elasticsearch.index.settings.IndexSettingsService;
2835

29-
import java.util.concurrent.ConcurrentHashMap;
3036
import java.util.concurrent.ConcurrentMap;
37+
import java.util.concurrent.TimeUnit;
38+
import java.util.concurrent.atomic.AtomicLong;
3139

3240
/**
3341
* @author kimchy (shay.banon)
3442
*/
35-
public class ResidentFieldDataCache extends AbstractConcurrentMapFieldDataCache {
43+
public class ResidentFieldDataCache extends AbstractConcurrentMapFieldDataCache implements MapEvictionListener<String, FieldData> {
3644

37-
@Inject public ResidentFieldDataCache(Index index, @IndexSettings Settings indexSettings) {
45+
private final IndexSettingsService indexSettingsService;
46+
47+
private volatile int maxSize;
48+
private volatile TimeValue expire;
49+
50+
private final AtomicLong evictions = new AtomicLong();
51+
52+
private final ApplySettings applySettings = new ApplySettings();
53+
54+
@Inject public ResidentFieldDataCache(Index index, @IndexSettings Settings indexSettings, IndexSettingsService indexSettingsService) {
3855
super(index, indexSettings);
56+
this.indexSettingsService = indexSettingsService;
57+
58+
this.maxSize = indexSettings.getAsInt("index.cache.field.max_size", componentSettings.getAsInt("max_size", -1));
59+
this.expire = indexSettings.getAsTime("index.cache.field.expire", componentSettings.getAsTime("expire", null));
60+
logger.debug("using [resident] field cache with max_size [{}], expire [{}]", maxSize, expire);
61+
62+
indexSettingsService.addListener(applySettings);
63+
}
64+
65+
@Override public void close() throws ElasticSearchException {
66+
indexSettingsService.removeListener(applySettings);
67+
super.close();
3968
}
4069

4170
@Override protected ConcurrentMap<String, FieldData> buildFieldDataMap() {
42-
return new ConcurrentHashMap<String, FieldData>();
71+
MapMaker mapMaker = new MapMaker();
72+
if (maxSize != -1) {
73+
mapMaker.maximumSize(maxSize);
74+
}
75+
if (expire != null) {
76+
mapMaker.expireAfterAccess(expire.nanos(), TimeUnit.NANOSECONDS);
77+
}
78+
mapMaker.evictionListener(this);
79+
return mapMaker.makeMap();
4380
}
4481

4582
@Override public String type() {
4683
return "resident";
4784
}
4885

4986
@Override public long evictions() {
50-
return 0;
87+
return evictions.get();
88+
}
89+
90+
@Override public void onEviction(@Nullable String s, @Nullable FieldData fieldData) {
91+
evictions.incrementAndGet();
92+
}
93+
94+
class ApplySettings implements IndexSettingsService.Listener {
95+
@Override public void onRefreshSettings(Settings settings) {
96+
int maxSize = settings.getAsInt("index.cache.field.max_size", ResidentFieldDataCache.this.maxSize);
97+
TimeValue expire = settings.getAsTime("index.cache.field.expire", ResidentFieldDataCache.this.expire);
98+
boolean changed = false;
99+
if (maxSize != ResidentFieldDataCache.this.maxSize) {
100+
logger.info("updating index.cache.field.max_size from [{}] to [{}]", ResidentFieldDataCache.this.maxSize, maxSize);
101+
changed = true;
102+
ResidentFieldDataCache.this.maxSize = maxSize;
103+
}
104+
if (!Objects.equal(expire, ResidentFieldDataCache.this.expire)) {
105+
logger.info("updating index.cache.field.expire from [{}] to [{}]", ResidentFieldDataCache.this.expire, expire);
106+
changed = true;
107+
ResidentFieldDataCache.this.expire = expire;
108+
}
109+
if (changed) {
110+
clear();
111+
}
112+
}
51113
}
52114
}

0 commit comments

Comments
 (0)