mirror of https://github.com/apache/lucene.git
LUCENE-3531: Fix cache lookup with a new WeakIdentityHashMap, also add ReaderClosedListeners.
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1214274 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
d7c7117eda
commit
a2d8c13021
|
@ -25,11 +25,14 @@ Jean-Philippe Barrette-LaPierre. This library is available under an MIT license,
|
||||||
see http://sites.google.com/site/rrettesite/moman and
|
see http://sites.google.com/site/rrettesite/moman and
|
||||||
http://bitbucket.org/jpbarrette/moman/overview/
|
http://bitbucket.org/jpbarrette/moman/overview/
|
||||||
|
|
||||||
The class org.apache.lucene.SorterTemplate was inspired by CGLIB's class with
|
The class org.apache.lucene.util.SorterTemplate was inspired by CGLIB's class
|
||||||
the same name. The implementation part is mainly done using pre-existing
|
with the same name. The implementation part is mainly done using pre-existing
|
||||||
Lucene sorting code. In-place stable mergesort was borrowed from CGLIB,
|
Lucene sorting code. In-place stable mergesort was borrowed from CGLIB,
|
||||||
which is Apache-licensed.
|
which is Apache-licensed.
|
||||||
|
|
||||||
|
The class org.apache.lucene.util.WeakIdentityHashMap was derived from
|
||||||
|
the Apache CXF project and is Apache License 2.0.
|
||||||
|
|
||||||
The Google Code Prettify is Apache License 2.0.
|
The Google Code Prettify is Apache License 2.0.
|
||||||
See http://code.google.com/p/google-code-prettify/
|
See http://code.google.com/p/google-code-prettify/
|
||||||
|
|
||||||
|
|
|
@ -18,13 +18,15 @@ package org.apache.lucene.search;
|
||||||
*/
|
*/
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Map;
|
import java.lang.ref.SoftReference;
|
||||||
import java.util.WeakHashMap;
|
import java.util.WeakHashMap;
|
||||||
|
|
||||||
import org.apache.lucene.index.IndexReader;
|
import org.apache.lucene.index.IndexReader;
|
||||||
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
|
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
|
||||||
|
import org.apache.lucene.index.SegmentReader;
|
||||||
import org.apache.lucene.util.FixedBitSet;
|
import org.apache.lucene.util.FixedBitSet;
|
||||||
import org.apache.lucene.util.Bits;
|
import org.apache.lucene.util.Bits;
|
||||||
|
import org.apache.lucene.util.WeakIdentityHashMap;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Wraps another filter's result and caches it. The purpose is to allow
|
* Wraps another filter's result and caches it. The purpose is to allow
|
||||||
|
@ -35,30 +37,45 @@ public class CachingWrapperFilter extends Filter {
|
||||||
// TODO: make this filter aware of ReaderContext. a cached filter could
|
// TODO: make this filter aware of ReaderContext. a cached filter could
|
||||||
// specify the actual readers key or something similar to indicate on which
|
// specify the actual readers key or something similar to indicate on which
|
||||||
// level of the readers hierarchy it should be cached.
|
// level of the readers hierarchy it should be cached.
|
||||||
Filter filter;
|
private final Filter filter;
|
||||||
|
private final FilterCache cache = new FilterCache();
|
||||||
protected final FilterCache<DocIdSet> cache;
|
|
||||||
private final boolean recacheDeletes;
|
private final boolean recacheDeletes;
|
||||||
|
|
||||||
private static class FilterCache<T> {
|
private static class FilterCache implements SegmentReader.CoreClosedListener, IndexReader.ReaderClosedListener {
|
||||||
|
private final WeakHashMap<Object,WeakIdentityHashMap<Bits,SoftReference<DocIdSet>>> cache =
|
||||||
|
new WeakHashMap<Object,WeakIdentityHashMap<Bits,SoftReference<DocIdSet>>>();
|
||||||
|
|
||||||
/**
|
public synchronized DocIdSet get(IndexReader reader, Bits acceptDocs) throws IOException {
|
||||||
* A transient Filter cache (package private because of test)
|
final Object coreKey = reader.getCoreCacheKey();
|
||||||
*/
|
WeakIdentityHashMap<Bits,SoftReference<DocIdSet>> innerCache = cache.get(coreKey);
|
||||||
private final Map<Object,Map<Object,T>> cache = new WeakHashMap<Object,Map<Object,T>>();
|
|
||||||
|
|
||||||
public synchronized T get(IndexReader reader, Object coreKey, Object coreSubKey) throws IOException {
|
|
||||||
Map<Object,T> innerCache = cache.get(coreKey);
|
|
||||||
if (innerCache == null) {
|
if (innerCache == null) {
|
||||||
innerCache = new WeakHashMap<Object,T>();
|
if (reader instanceof SegmentReader) {
|
||||||
|
((SegmentReader) reader).addCoreClosedListener(this);
|
||||||
|
} else {
|
||||||
|
assert reader.getSequentialSubReaders() == null :
|
||||||
|
"we only operate on AtomicContext, so all cached readers must be atomic";
|
||||||
|
reader.addReaderClosedListener(this);
|
||||||
|
}
|
||||||
|
innerCache = new WeakIdentityHashMap<Bits,SoftReference<DocIdSet>>();
|
||||||
cache.put(coreKey, innerCache);
|
cache.put(coreKey, innerCache);
|
||||||
}
|
}
|
||||||
|
|
||||||
return innerCache.get(coreSubKey);
|
final SoftReference<DocIdSet> innerRef = innerCache.get(acceptDocs);
|
||||||
|
return innerRef == null ? null : innerRef.get();
|
||||||
}
|
}
|
||||||
|
|
||||||
public synchronized void put(Object coreKey, Object coreSubKey, T value) {
|
public synchronized void put(IndexReader reader, Bits acceptDocs, DocIdSet value) {
|
||||||
cache.get(coreKey).put(coreSubKey, value);
|
cache.get(reader.getCoreCacheKey()).put(acceptDocs, new SoftReference<DocIdSet>(value));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public synchronized void onClose(IndexReader reader) {
|
||||||
|
cache.remove(reader.getCoreCacheKey());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public synchronized void onClose(SegmentReader reader) {
|
||||||
|
cache.remove(reader.getCoreCacheKey());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -79,7 +96,6 @@ public class CachingWrapperFilter extends Filter {
|
||||||
public CachingWrapperFilter(Filter filter, boolean recacheDeletes) {
|
public CachingWrapperFilter(Filter filter, boolean recacheDeletes) {
|
||||||
this.filter = filter;
|
this.filter = filter;
|
||||||
this.recacheDeletes = recacheDeletes;
|
this.recacheDeletes = recacheDeletes;
|
||||||
cache = new FilterCache<DocIdSet>();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Provide the DocIdSet to be cached, using the DocIdSet provided
|
/** Provide the DocIdSet to be cached, using the DocIdSet provided
|
||||||
|
@ -115,7 +131,6 @@ public class CachingWrapperFilter extends Filter {
|
||||||
@Override
|
@Override
|
||||||
public DocIdSet getDocIdSet(AtomicReaderContext context, final Bits acceptDocs) throws IOException {
|
public DocIdSet getDocIdSet(AtomicReaderContext context, final Bits acceptDocs) throws IOException {
|
||||||
final IndexReader reader = context.reader;
|
final IndexReader reader = context.reader;
|
||||||
final Object coreKey = reader.getCoreCacheKey();
|
|
||||||
|
|
||||||
// Only cache if incoming acceptDocs is == live docs;
|
// Only cache if incoming acceptDocs is == live docs;
|
||||||
// if Lucene passes in more interesting acceptDocs in
|
// if Lucene passes in more interesting acceptDocs in
|
||||||
|
@ -129,13 +144,13 @@ public class CachingWrapperFilter extends Filter {
|
||||||
subAcceptDocs = null;
|
subAcceptDocs = null;
|
||||||
}
|
}
|
||||||
|
|
||||||
DocIdSet docIdSet = cache.get(reader, coreKey, subAcceptDocs);
|
DocIdSet docIdSet = cache.get(reader, subAcceptDocs);
|
||||||
if (docIdSet != null) {
|
if (docIdSet != null) {
|
||||||
hitCount++;
|
hitCount++;
|
||||||
} else {
|
} else {
|
||||||
missCount++;
|
missCount++;
|
||||||
docIdSet = docIdSetToCache(filter.getDocIdSet(context, subAcceptDocs), reader);
|
docIdSet = docIdSetToCache(filter.getDocIdSet(context, subAcceptDocs), reader);
|
||||||
cache.put(coreKey, subAcceptDocs, docIdSet);
|
cache.put(reader, subAcceptDocs, docIdSet);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (doCacheSubAcceptDocs) {
|
if (doCacheSubAcceptDocs) {
|
||||||
|
|
|
@ -0,0 +1,144 @@
|
||||||
|
package org.apache.lucene.util;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
import java.lang.ref.Reference;
|
||||||
|
import java.lang.ref.ReferenceQueue;
|
||||||
|
import java.lang.ref.WeakReference;
|
||||||
|
import java.util.HashMap;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Implements a combination of {@link java.util.WeakHashMap} and
|
||||||
|
* {@link java.util.IdentityHashMap}.
|
||||||
|
* Useful for caches that need to key off of a {@code ==} comparison
|
||||||
|
* instead of a {@code .equals}.
|
||||||
|
*
|
||||||
|
* <p>This class is not a general-purpose {@link java.util.Map}
|
||||||
|
* implementation! It intentionally violates
|
||||||
|
* Map's general contract, which mandates the use of the equals method
|
||||||
|
* when comparing objects. This class is designed for use only in the
|
||||||
|
* rare cases wherein reference-equality semantics are required.
|
||||||
|
*
|
||||||
|
* <p><b>Note that this implementation is not synchronized.</b>
|
||||||
|
*
|
||||||
|
* <p>This implementation was forked from <a href="http://cxf.apache.org/">Apache CXF</a>
|
||||||
|
* but modified to <b>not</b> implement the {@link java.util.Map} interface and
|
||||||
|
* without any set/iterator views on it, as those are error-prone
|
||||||
|
* and inefficient, if not implemented carefully. Lucene's implementation also
|
||||||
|
* supports {@code null} keys, but those are never weak!
|
||||||
|
*
|
||||||
|
* @lucene.internal
|
||||||
|
*/
|
||||||
|
public class WeakIdentityHashMap<K,V> {
|
||||||
|
final ReferenceQueue<Object> queue = new ReferenceQueue<Object>(); // pkg-private for inner class
|
||||||
|
private final HashMap<IdentityWeakReference, V> backingStore;
|
||||||
|
|
||||||
|
public WeakIdentityHashMap() {
|
||||||
|
backingStore = new HashMap<IdentityWeakReference, V>();
|
||||||
|
}
|
||||||
|
|
||||||
|
public WeakIdentityHashMap(int initialCapacity) {
|
||||||
|
backingStore = new HashMap<IdentityWeakReference,V>(initialCapacity);
|
||||||
|
}
|
||||||
|
|
||||||
|
public WeakIdentityHashMap(int initialCapacity, float loadFactor) {
|
||||||
|
backingStore = new HashMap<IdentityWeakReference,V>(initialCapacity, loadFactor);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void clear() {
|
||||||
|
backingStore.clear();
|
||||||
|
reap();
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean containsKey(Object key) {
|
||||||
|
reap();
|
||||||
|
return backingStore.containsKey(new IdentityWeakReference(key));
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean containsValue(Object value) {
|
||||||
|
reap();
|
||||||
|
return backingStore.containsValue(value);
|
||||||
|
}
|
||||||
|
|
||||||
|
public V get(Object key) {
|
||||||
|
reap();
|
||||||
|
return backingStore.get(new IdentityWeakReference(key));
|
||||||
|
}
|
||||||
|
|
||||||
|
public V put(K key, V value) {
|
||||||
|
reap();
|
||||||
|
return backingStore.put(new IdentityWeakReference(key), value);
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean isEmpty() {
|
||||||
|
return size() == 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
public V remove(Object key) {
|
||||||
|
try {
|
||||||
|
reap();
|
||||||
|
return backingStore.remove(new IdentityWeakReference(key));
|
||||||
|
} finally {
|
||||||
|
reap();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public int size() {
|
||||||
|
if (backingStore.isEmpty())
|
||||||
|
return 0;
|
||||||
|
reap();
|
||||||
|
return backingStore.size();
|
||||||
|
}
|
||||||
|
|
||||||
|
private void reap() {
|
||||||
|
Reference<?> zombie;
|
||||||
|
while ((zombie = queue.poll()) != null) {
|
||||||
|
backingStore.remove(zombie);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
final class IdentityWeakReference extends WeakReference<Object> {
|
||||||
|
private final int hash;
|
||||||
|
|
||||||
|
IdentityWeakReference(Object obj) {
|
||||||
|
super(obj == null ? NULL : obj, queue);
|
||||||
|
hash = System.identityHashCode(obj);
|
||||||
|
}
|
||||||
|
|
||||||
|
public int hashCode() {
|
||||||
|
return hash;
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean equals(Object o) {
|
||||||
|
if (this == o) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
if (o instanceof WeakReference) {
|
||||||
|
final WeakReference ref = (WeakReference)o;
|
||||||
|
if (this.get() == ref.get()) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// we keep a hard reference to our NULL key, so this map supports null keys that never get GCed:
|
||||||
|
static final Object NULL = new Object();
|
||||||
|
}
|
||||||
|
|
|
@ -47,7 +47,7 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
|
||||||
CachingWrapperFilter cacher = new CachingWrapperFilter(filter);
|
CachingWrapperFilter cacher = new CachingWrapperFilter(filter);
|
||||||
|
|
||||||
// first time, nested filter is called
|
// first time, nested filter is called
|
||||||
cacher.getDocIdSet(context, context.reader.getLiveDocs());
|
DocIdSet strongRef = cacher.getDocIdSet(context, context.reader.getLiveDocs());
|
||||||
assertTrue("first time", filter.wasCalled());
|
assertTrue("first time", filter.wasCalled());
|
||||||
|
|
||||||
// make sure no exception if cache is holding the wrong docIdSet
|
// make sure no exception if cache is holding the wrong docIdSet
|
||||||
|
|
|
@ -0,0 +1,78 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.lucene.util;
|
||||||
|
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.WeakHashMap;
|
||||||
|
|
||||||
|
public class TestWeakIdentityHashMap extends LuceneTestCase {
|
||||||
|
|
||||||
|
public void test() {
|
||||||
|
final WeakIdentityHashMap<String,String> map =
|
||||||
|
new WeakIdentityHashMap<String,String>();
|
||||||
|
// we keep strong references to the keys,
|
||||||
|
// so WeakIdentityHashMap will not forget about them:
|
||||||
|
String key1 = new String("foo");
|
||||||
|
String key2 = new String("foo");
|
||||||
|
String key3 = new String("foo");
|
||||||
|
|
||||||
|
assertNotSame(key1, key2);
|
||||||
|
assertEquals(key1, key2);
|
||||||
|
assertNotSame(key1, key3);
|
||||||
|
assertEquals(key1, key3);
|
||||||
|
assertNotSame(key2, key3);
|
||||||
|
assertEquals(key2, key3);
|
||||||
|
|
||||||
|
map.put(key1, "bar1");
|
||||||
|
map.put(key2, "bar2");
|
||||||
|
map.put(null, "null");
|
||||||
|
|
||||||
|
assertEquals("bar1", map.get(key1));
|
||||||
|
assertEquals("bar2", map.get(key2));
|
||||||
|
assertEquals(null, map.get(key3));
|
||||||
|
assertEquals("null", map.get(null));
|
||||||
|
|
||||||
|
assertTrue(map.containsKey(key1));
|
||||||
|
assertTrue(map.containsKey(key2));
|
||||||
|
assertFalse(map.containsKey(key3));
|
||||||
|
assertTrue(map.containsKey(null));
|
||||||
|
|
||||||
|
assertEquals(3, map.size());
|
||||||
|
map.remove(null);
|
||||||
|
assertEquals(2, map.size());
|
||||||
|
map.remove(key1);
|
||||||
|
assertEquals(1, map.size());
|
||||||
|
map.put(key1, "bar1");
|
||||||
|
map.put(key2, "bar2");
|
||||||
|
map.put(key3, "bar3");
|
||||||
|
assertEquals(3, map.size());
|
||||||
|
|
||||||
|
// clear strong refs
|
||||||
|
key1 = key2 = key3 = null;
|
||||||
|
|
||||||
|
// check that GC does not cause problems in reap() method:
|
||||||
|
for (int i = 0; !map.isEmpty(); i++) try {
|
||||||
|
if (i > 40)
|
||||||
|
fail("The garbage collector did not reclaim all keys after 2 seconds, failing test!");
|
||||||
|
System.runFinalization();
|
||||||
|
System.gc();
|
||||||
|
Thread.currentThread().sleep(50L);
|
||||||
|
} catch (InterruptedException ie) {}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -64,11 +64,14 @@ Jean-Philippe Barrette-LaPierre. This library is available under an MIT license,
|
||||||
see http://sites.google.com/site/rrettesite/moman and
|
see http://sites.google.com/site/rrettesite/moman and
|
||||||
http://bitbucket.org/jpbarrette/moman/overview/
|
http://bitbucket.org/jpbarrette/moman/overview/
|
||||||
|
|
||||||
The class org.apache.lucene.SorterTemplate was inspired by CGLIB's class with
|
The class org.apache.lucene.util.SorterTemplate was inspired by CGLIB's class
|
||||||
the same name. The implementation part is mainly done using pre-existing
|
with the same name. The implementation part is mainly done using pre-existing
|
||||||
Lucene sorting code. In-place stable mergesort was borrowed from CGLIB,
|
Lucene sorting code. In-place stable mergesort was borrowed from CGLIB,
|
||||||
which is Apache-licensed.
|
which is Apache-licensed.
|
||||||
|
|
||||||
|
The class org.apache.lucene.util.WeakIdentityHashMap was derived from
|
||||||
|
the Apache CXF project and is Apache License 2.0.
|
||||||
|
|
||||||
The Google Code Prettify is Apache License 2.0.
|
The Google Code Prettify is Apache License 2.0.
|
||||||
See http://code.google.com/p/google-code-prettify/
|
See http://code.google.com/p/google-code-prettify/
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue