|
| 1 | +/* |
| 2 | + * Licensed to Elastic Search and Shay Banon under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. Elastic Search licenses this |
| 6 | + * file to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, |
| 13 | + * software distributed under the License is distributed on an |
| 14 | + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY |
| 15 | + * KIND, either express or implied. See the License for the |
| 16 | + * specific language governing permissions and limitations |
| 17 | + * under the License. |
| 18 | + */ |
| 19 | + |
| 20 | +package org.elasticsearch.index.cache.filter.support; |
| 21 | + |
| 22 | +import org.apache.lucene.index.IndexReader; |
| 23 | +import org.apache.lucene.search.DocIdSet; |
| 24 | +import org.apache.lucene.search.Filter; |
| 25 | +import org.elasticsearch.common.collect.MapMaker; |
| 26 | +import org.elasticsearch.common.lucene.docset.DocSet; |
| 27 | +import org.elasticsearch.common.settings.Settings; |
| 28 | +import org.elasticsearch.index.AbstractIndexComponent; |
| 29 | +import org.elasticsearch.index.Index; |
| 30 | +import org.elasticsearch.index.cache.filter.FilterCache; |
| 31 | +import org.elasticsearch.index.settings.IndexSettings; |
| 32 | + |
| 33 | +import java.io.IOException; |
| 34 | +import java.util.concurrent.ConcurrentMap; |
| 35 | + |
| 36 | +import static org.elasticsearch.common.lucene.docset.DocSets.*; |
| 37 | +import static org.elasticsearch.common.util.concurrent.ConcurrentCollections.*; |
| 38 | + |
| 39 | +/** |
| 40 | + * A base concurrent filter cache that accepts the actual cache to use. |
| 41 | + * |
| 42 | + * @author kimchy (shay.banon) |
| 43 | + */ |
| 44 | +public abstract class AbstractDoubleConcurrentMapFilterCache extends AbstractIndexComponent implements FilterCache { |
| 45 | + |
| 46 | + final ConcurrentMap<Object, ConcurrentMap<Filter, DocSet>> cache; |
| 47 | + final ConcurrentMap<Object, ConcurrentMap<Filter, DocSet>> weakCache; |
| 48 | + |
| 49 | + protected AbstractDoubleConcurrentMapFilterCache(Index index, @IndexSettings Settings indexSettings) { |
| 50 | + super(index, indexSettings); |
| 51 | + // weak keys is fine, it will only be cleared once IndexReader references will be removed |
| 52 | + // (assuming clear(...) will not be called) |
| 53 | + this.cache = new MapMaker().weakKeys().makeMap(); |
| 54 | + this.weakCache = new MapMaker().weakKeys().makeMap(); |
| 55 | + } |
| 56 | + |
| 57 | + @Override public void close() { |
| 58 | + cache.clear(); |
| 59 | + } |
| 60 | + |
| 61 | + @Override public void clear() { |
| 62 | + cache.clear(); |
| 63 | + } |
| 64 | + |
| 65 | + @Override public void clear(IndexReader reader) { |
| 66 | + ConcurrentMap<Filter, DocSet> map = cache.remove(reader.getFieldCacheKey()); |
| 67 | + // help soft/weak handling GC |
| 68 | + if (map != null) { |
| 69 | + map.clear(); |
| 70 | + } |
| 71 | + map = weakCache.remove(reader.getFieldCacheKey()); |
| 72 | + // help soft/weak handling GC |
| 73 | + if (map != null) { |
| 74 | + map.clear(); |
| 75 | + } |
| 76 | + } |
| 77 | + |
| 78 | + @Override public void clearUnreferenced() { |
| 79 | + } |
| 80 | + |
| 81 | + @Override public Filter cache(Filter filterToCache) { |
| 82 | + if (isCached(filterToCache)) { |
| 83 | + return filterToCache; |
| 84 | + } |
| 85 | + return new FilterCacheFilterWrapper(filterToCache, this); |
| 86 | + } |
| 87 | + |
| 88 | + @Override public Filter weakCache(Filter filterToCache) { |
| 89 | + if (isCached(filterToCache)) { |
| 90 | + return filterToCache; |
| 91 | + } |
| 92 | + return new FilterWeakCacheFilterWrapper(filterToCache, this); |
| 93 | + } |
| 94 | + |
| 95 | + @Override public boolean isCached(Filter filter) { |
| 96 | + return filter instanceof CacheMarker; |
| 97 | + } |
| 98 | + |
| 99 | + protected ConcurrentMap<Filter, DocSet> buildCacheMap() { |
| 100 | + return newConcurrentMap(); |
| 101 | + } |
| 102 | + |
| 103 | + protected ConcurrentMap<Filter, DocSet> buildWeakCacheMap() { |
| 104 | + return newConcurrentMap(); |
| 105 | + } |
| 106 | + |
| 107 | + static abstract class CacheMarker extends Filter { |
| 108 | + |
| 109 | + } |
| 110 | + |
| 111 | + // LUCENE MONITOR: Check next version Lucene for CachingWrapperFilter, consider using that logic |
| 112 | + // and not use the DeletableConstantScoreQuery, instead pass the DeletesMode enum to the cache method |
| 113 | + // see: https://issues.apache.org/jira/browse/LUCENE-2468 |
| 114 | + |
| 115 | + static class FilterCacheFilterWrapper extends CacheMarker { |
| 116 | + |
| 117 | + private final Filter filter; |
| 118 | + |
| 119 | + private final AbstractDoubleConcurrentMapFilterCache cache; |
| 120 | + |
| 121 | + FilterCacheFilterWrapper(Filter filter, AbstractDoubleConcurrentMapFilterCache cache) { |
| 122 | + this.filter = filter; |
| 123 | + this.cache = cache; |
| 124 | + } |
| 125 | + |
| 126 | + @Override public DocIdSet getDocIdSet(IndexReader reader) throws IOException { |
| 127 | + ConcurrentMap<Filter, DocSet> cachedFilters = cache.cache.get(reader.getFieldCacheKey()); |
| 128 | + if (cachedFilters == null) { |
| 129 | + cachedFilters = cache.buildCacheMap(); |
| 130 | + cache.cache.putIfAbsent(reader.getFieldCacheKey(), cachedFilters); |
| 131 | + } |
| 132 | + DocSet docSet = cachedFilters.get(filter); |
| 133 | + if (docSet != null) { |
| 134 | + return docSet; |
| 135 | + } |
| 136 | + |
| 137 | + // check if its in the weak cache, if so, move it from weak to soft |
| 138 | + ConcurrentMap<Filter, DocSet> weakCachedFilters = cache.weakCache.get(reader.getFieldCacheKey()); |
| 139 | + if (weakCachedFilters != null) { |
| 140 | + docSet = weakCachedFilters.get(filter); |
| 141 | + if (docSet != null) { |
| 142 | + cachedFilters.put(filter, docSet); |
| 143 | + weakCachedFilters.remove(filter); |
| 144 | + return docSet; |
| 145 | + } |
| 146 | + } |
| 147 | + |
| 148 | + DocIdSet docIdSet = filter.getDocIdSet(reader); |
| 149 | + docSet = cacheable(reader, docIdSet); |
| 150 | + cachedFilters.putIfAbsent(filter, docSet); |
| 151 | + return docIdSet; |
| 152 | + } |
| 153 | + |
| 154 | + public String toString() { |
| 155 | + return "FilterCacheFilterWrapper(" + filter + ")"; |
| 156 | + } |
| 157 | + |
| 158 | + public boolean equals(Object o) { |
| 159 | + if (!(o instanceof FilterCacheFilterWrapper)) return false; |
| 160 | + return this.filter.equals(((FilterCacheFilterWrapper) o).filter); |
| 161 | + } |
| 162 | + |
| 163 | + public int hashCode() { |
| 164 | + return filter.hashCode() ^ 0x1117BF25; |
| 165 | + } |
| 166 | + } |
| 167 | + |
| 168 | + static class FilterWeakCacheFilterWrapper extends CacheMarker { |
| 169 | + |
| 170 | + private final Filter filter; |
| 171 | + |
| 172 | + private final AbstractDoubleConcurrentMapFilterCache cache; |
| 173 | + |
| 174 | + FilterWeakCacheFilterWrapper(Filter filter, AbstractDoubleConcurrentMapFilterCache cache) { |
| 175 | + this.filter = filter; |
| 176 | + this.cache = cache; |
| 177 | + } |
| 178 | + |
| 179 | + @Override public DocIdSet getDocIdSet(IndexReader reader) throws IOException { |
| 180 | + DocSet docSet; |
| 181 | + // first check if its in the actual cache |
| 182 | + ConcurrentMap<Filter, DocSet> cachedFilters = cache.cache.get(reader.getFieldCacheKey()); |
| 183 | + if (cachedFilters != null) { |
| 184 | + docSet = cachedFilters.get(filter); |
| 185 | + if (docSet != null) { |
| 186 | + return docSet; |
| 187 | + } |
| 188 | + } |
| 189 | + |
| 190 | + // now, handle it in the weak cache |
| 191 | + ConcurrentMap<Filter, DocSet> weakCacheFilters = cache.weakCache.get(reader.getFieldCacheKey()); |
| 192 | + if (weakCacheFilters == null) { |
| 193 | + weakCacheFilters = cache.buildWeakCacheMap(); |
| 194 | + cache.weakCache.putIfAbsent(reader.getFieldCacheKey(), weakCacheFilters); |
| 195 | + } |
| 196 | + |
| 197 | + docSet = weakCacheFilters.get(filter); |
| 198 | + if (docSet != null) { |
| 199 | + return docSet; |
| 200 | + } |
| 201 | + |
| 202 | + DocIdSet docIdSet = filter.getDocIdSet(reader); |
| 203 | + docSet = cacheable(reader, docIdSet); |
| 204 | + weakCacheFilters.putIfAbsent(filter, docSet); |
| 205 | + return docIdSet; |
| 206 | + } |
| 207 | + |
| 208 | + public String toString() { |
| 209 | + return "FilterCacheFilterWrapper(" + filter + ")"; |
| 210 | + } |
| 211 | + |
| 212 | + public boolean equals(Object o) { |
| 213 | + if (!(o instanceof FilterCacheFilterWrapper)) return false; |
| 214 | + return this.filter.equals(((FilterCacheFilterWrapper) o).filter); |
| 215 | + } |
| 216 | + |
| 217 | + public int hashCode() { |
| 218 | + return filter.hashCode() ^ 0x1117BF25; |
| 219 | + } |
| 220 | + } |
| 221 | +} |
0 commit comments