|
19 | 19 |
|
20 | 20 | package org.elasticsearch.index.cache.field.data.resident;
|
21 | 21 |
|
| 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; |
22 | 27 | import org.elasticsearch.common.inject.Inject;
|
23 | 28 | import org.elasticsearch.common.settings.Settings;
|
| 29 | +import org.elasticsearch.common.unit.TimeValue; |
24 | 30 | import org.elasticsearch.index.Index;
|
25 | 31 | import org.elasticsearch.index.cache.field.data.support.AbstractConcurrentMapFieldDataCache;
|
26 | 32 | import org.elasticsearch.index.field.data.FieldData;
|
27 | 33 | import org.elasticsearch.index.settings.IndexSettings;
|
| 34 | +import org.elasticsearch.index.settings.IndexSettingsService; |
28 | 35 |
|
29 |
| -import java.util.concurrent.ConcurrentHashMap; |
30 | 36 | import java.util.concurrent.ConcurrentMap;
|
| 37 | +import java.util.concurrent.TimeUnit; |
| 38 | +import java.util.concurrent.atomic.AtomicLong; |
31 | 39 |
|
32 | 40 | /**
|
33 | 41 | * @author kimchy (shay.banon)
|
34 | 42 | */
|
35 |
| -public class ResidentFieldDataCache extends AbstractConcurrentMapFieldDataCache { |
| 43 | +public class ResidentFieldDataCache extends AbstractConcurrentMapFieldDataCache implements MapEvictionListener<String, FieldData> { |
36 | 44 |
|
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) { |
38 | 55 | 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(); |
39 | 68 | }
|
40 | 69 |
|
41 | 70 | @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(); |
43 | 80 | }
|
44 | 81 |
|
45 | 82 | @Override public String type() {
|
46 | 83 | return "resident";
|
47 | 84 | }
|
48 | 85 |
|
49 | 86 | @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 | + } |
51 | 113 | }
|
52 | 114 | }
|
0 commit comments