mirror of https://github.com/apache/lucene.git
LUCENE-1195: Improve term lookup performance by adding a LRU cache to the TermInfosReader.
git-svn-id: https://svn.apache.org/repos/asf/lucene/java/trunk@659602 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
a7f1cbc5be
commit
eb0596c721
|
@ -183,6 +183,11 @@ Optimizations
|
|||
5. LUCENE-1183: Optimized TRStringDistance class (in contrib/spell) that uses
|
||||
less memory than the previous version. (Cédrik LIME via Otis Gospodnetic)
|
||||
|
||||
6. LUCENE-1195: Improve term lookup performance by adding a LRU cache to the
|
||||
TermInfosReader. In performance experiments the speedup was about 25% on
|
||||
average on mid-size indexes with ~500,000 documents for queries with 3
|
||||
terms and about 7% on larger indexes with ~4.3M documents. (Michael Busch)
|
||||
|
||||
Documentation
|
||||
|
||||
1. LUCENE-1236: Added some clarifying remarks to EdgeNGram*.java (Hiroaki Kawai via Grant Ingersoll)
|
||||
|
|
|
@ -150,10 +150,15 @@ final class SegmentTermEnum extends TermEnum implements Cloneable {
|
|||
return true;
|
||||
}
|
||||
|
||||
/** Optimized scan, without allocating new terms. */
|
||||
final void scanTo(Term term) throws IOException {
|
||||
/** Optimized scan, without allocating new terms.
|
||||
* Return number of invocations to next(). */
|
||||
final int scanTo(Term term) throws IOException {
|
||||
scanBuffer.set(term);
|
||||
while (scanBuffer.compareTo(termBuffer) > 0 && next()) {}
|
||||
int count = 0;
|
||||
while (scanBuffer.compareTo(termBuffer) > 0 && next()) {
|
||||
count++;
|
||||
}
|
||||
return count;
|
||||
}
|
||||
|
||||
/** Returns the current Term in the enumeration.
|
||||
|
|
|
@ -21,6 +21,8 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.BufferedIndexInput;
|
||||
import org.apache.lucene.util.cache.Cache;
|
||||
import org.apache.lucene.util.cache.SimpleLRUCache;
|
||||
|
||||
/** This stores a monotonically increasing set of <Term, TermInfo> pairs in a
|
||||
* Directory. Pairs are accessed either by Term or by ordinal position the
|
||||
|
@ -31,7 +33,7 @@ final class TermInfosReader {
|
|||
private String segment;
|
||||
private FieldInfos fieldInfos;
|
||||
|
||||
private ThreadLocal enumerators = new ThreadLocal();
|
||||
private ThreadLocal threadResources = new ThreadLocal();
|
||||
private SegmentTermEnum origEnum;
|
||||
private long size;
|
||||
|
||||
|
@ -44,6 +46,18 @@ final class TermInfosReader {
|
|||
private int indexDivisor = 1;
|
||||
private int totalIndexInterval;
|
||||
|
||||
private final static int DEFAULT_CACHE_SIZE = 1024;
|
||||
|
||||
/**
|
||||
* Per-thread resources managed by ThreadLocal
|
||||
*/
|
||||
private static final class ThreadResources {
|
||||
SegmentTermEnum termEnum;
|
||||
|
||||
// Used for caching the least recently looked-up Terms
|
||||
Cache termInfoCache;
|
||||
}
|
||||
|
||||
TermInfosReader(Directory dir, String seg, FieldInfos fis)
|
||||
throws CorruptIndexException, IOException {
|
||||
this(dir, seg, fis, BufferedIndexInput.BUFFER_SIZE);
|
||||
|
@ -129,7 +143,7 @@ final class TermInfosReader {
|
|||
origEnum.close();
|
||||
if (indexEnum != null)
|
||||
indexEnum.close();
|
||||
enumerators.set(null);
|
||||
threadResources.set(null);
|
||||
}
|
||||
|
||||
/** Returns the number of term/value pairs in the set. */
|
||||
|
@ -137,13 +151,16 @@ final class TermInfosReader {
|
|||
return size;
|
||||
}
|
||||
|
||||
private SegmentTermEnum getEnum() {
|
||||
SegmentTermEnum termEnum = (SegmentTermEnum)enumerators.get();
|
||||
if (termEnum == null) {
|
||||
termEnum = terms();
|
||||
enumerators.set(termEnum);
|
||||
private ThreadResources getThreadResources() {
|
||||
ThreadResources resources = (ThreadResources)threadResources.get();
|
||||
if (resources == null) {
|
||||
resources = new ThreadResources();
|
||||
resources.termEnum = terms();
|
||||
// Cache does not have to be thread-safe, it is only used by one thread at the same time
|
||||
resources.termInfoCache = new SimpleLRUCache(DEFAULT_CACHE_SIZE);
|
||||
threadResources.set(resources);
|
||||
}
|
||||
return termEnum;
|
||||
return resources;
|
||||
}
|
||||
|
||||
private synchronized void ensureIndexIsRead() throws IOException {
|
||||
|
@ -189,60 +206,94 @@ final class TermInfosReader {
|
|||
return hi;
|
||||
}
|
||||
|
||||
private final void seekEnum(int indexOffset) throws IOException {
|
||||
getEnum().seek(indexPointers[indexOffset],
|
||||
private final void seekEnum(SegmentTermEnum enumerator, int indexOffset) throws IOException {
|
||||
enumerator.seek(indexPointers[indexOffset],
|
||||
(indexOffset * totalIndexInterval) - 1,
|
||||
indexTerms[indexOffset], indexInfos[indexOffset]);
|
||||
}
|
||||
|
||||
/** Returns the TermInfo for a Term in the set, or null. */
|
||||
TermInfo get(Term term) throws IOException {
|
||||
return get(term, true);
|
||||
}
|
||||
|
||||
/** Returns the TermInfo for a Term in the set, or null. */
|
||||
private TermInfo get(Term term, boolean useCache) throws IOException {
|
||||
if (size == 0) return null;
|
||||
|
||||
ensureIndexIsRead();
|
||||
|
||||
|
||||
TermInfo ti;
|
||||
ThreadResources resources = getThreadResources();
|
||||
Cache cache = null;
|
||||
|
||||
if (useCache) {
|
||||
cache = resources.termInfoCache;
|
||||
// check the cache first if the term was recently looked up
|
||||
ti = (TermInfo) cache.get(term);
|
||||
if (ti != null) {
|
||||
return ti;
|
||||
}
|
||||
}
|
||||
|
||||
// optimize sequential access: first try scanning cached enum w/o seeking
|
||||
SegmentTermEnum enumerator = getEnum();
|
||||
SegmentTermEnum enumerator = resources.termEnum;
|
||||
if (enumerator.term() != null // term is at or past current
|
||||
&& ((enumerator.prev() != null && term.compareTo(enumerator.prev())> 0)
|
||||
|| term.compareTo(enumerator.term()) >= 0)) {
|
||||
int enumOffset = (int)(enumerator.position/totalIndexInterval)+1;
|
||||
if (indexTerms.length == enumOffset // but before end of block
|
||||
|| term.compareTo(indexTerms[enumOffset]) < 0)
|
||||
return scanEnum(term); // no need to seek
|
||||
|| term.compareTo(indexTerms[enumOffset]) < 0) {
|
||||
// no need to seek
|
||||
|
||||
int numScans = enumerator.scanTo(term);
|
||||
if (enumerator.term() != null && term.compareTo(enumerator.term()) == 0) {
|
||||
ti = enumerator.termInfo();
|
||||
if (cache != null && numScans > 1) {
|
||||
// we only want to put this TermInfo into the cache if
|
||||
// scanEnum skipped more than one dictionary entry.
|
||||
// This prevents RangeQueries or WildcardQueries to
|
||||
// wipe out the cache when they iterate over a large numbers
|
||||
// of terms in order
|
||||
cache.put(term, ti);
|
||||
}
|
||||
} else {
|
||||
ti = null;
|
||||
}
|
||||
|
||||
return ti;
|
||||
}
|
||||
}
|
||||
|
||||
// random-access: must seek
|
||||
seekEnum(getIndexOffset(term));
|
||||
return scanEnum(term);
|
||||
}
|
||||
|
||||
/** Scans within block for matching term. */
|
||||
private final TermInfo scanEnum(Term term) throws IOException {
|
||||
SegmentTermEnum enumerator = getEnum();
|
||||
seekEnum(enumerator, getIndexOffset(term));
|
||||
enumerator.scanTo(term);
|
||||
if (enumerator.term() != null && term.compareTo(enumerator.term()) == 0)
|
||||
return enumerator.termInfo();
|
||||
else
|
||||
return null;
|
||||
if (enumerator.term() != null && term.compareTo(enumerator.term()) == 0) {
|
||||
ti = enumerator.termInfo();
|
||||
if (cache != null) {
|
||||
cache.put(term, ti);
|
||||
}
|
||||
} else {
|
||||
ti = null;
|
||||
}
|
||||
return ti;
|
||||
}
|
||||
|
||||
/** Returns the nth term in the set. */
|
||||
final Term get(int position) throws IOException {
|
||||
if (size == 0) return null;
|
||||
|
||||
SegmentTermEnum enumerator = getEnum();
|
||||
SegmentTermEnum enumerator = getThreadResources().termEnum;
|
||||
if (enumerator != null && enumerator.term() != null &&
|
||||
position >= enumerator.position &&
|
||||
position < (enumerator.position + totalIndexInterval))
|
||||
return scanEnum(position); // can avoid seek
|
||||
return scanEnum(enumerator, position); // can avoid seek
|
||||
|
||||
seekEnum(position/totalIndexInterval); // must seek
|
||||
return scanEnum(position);
|
||||
seekEnum(enumerator, position/totalIndexInterval); // must seek
|
||||
return scanEnum(enumerator, position);
|
||||
}
|
||||
|
||||
private final Term scanEnum(int position) throws IOException {
|
||||
SegmentTermEnum enumerator = getEnum();
|
||||
private final Term scanEnum(SegmentTermEnum enumerator, int position) throws IOException {
|
||||
while(enumerator.position < position)
|
||||
if (!enumerator.next())
|
||||
return null;
|
||||
|
@ -256,9 +307,10 @@ final class TermInfosReader {
|
|||
|
||||
ensureIndexIsRead();
|
||||
int indexOffset = getIndexOffset(term);
|
||||
seekEnum(indexOffset);
|
||||
|
||||
SegmentTermEnum enumerator = getThreadResources().termEnum;
|
||||
seekEnum(enumerator, indexOffset);
|
||||
|
||||
SegmentTermEnum enumerator = getEnum();
|
||||
while(term.compareTo(enumerator.term()) > 0 && enumerator.next()) {}
|
||||
|
||||
if (term.compareTo(enumerator.term()) == 0)
|
||||
|
@ -274,7 +326,9 @@ final class TermInfosReader {
|
|||
|
||||
/** Returns an enumeration of terms starting at or after the named term. */
|
||||
public SegmentTermEnum terms(Term term) throws IOException {
|
||||
get(term);
|
||||
return (SegmentTermEnum)getEnum().clone();
|
||||
// don't use the cache in this call because we want to reposition the
|
||||
// enumeration
|
||||
get(term, false);
|
||||
return (SegmentTermEnum)getThreadResources().termEnum.clone();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,100 @@
|
|||
package org.apache.lucene.util.cache;
|
||||
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
|
||||
/**
|
||||
* Base class for cache implementations.
|
||||
*/
|
||||
public abstract class Cache {
|
||||
|
||||
/**
|
||||
* Simple Cache wrapper that synchronizes all
|
||||
* calls that access the cache.
|
||||
*/
|
||||
static class SynchronizedCache extends Cache {
|
||||
Object mutex;
|
||||
Cache cache;
|
||||
|
||||
SynchronizedCache(Cache cache) {
|
||||
this.cache = cache;
|
||||
this.mutex = this;
|
||||
}
|
||||
|
||||
SynchronizedCache(Cache cache, Object mutex) {
|
||||
this.cache = cache;
|
||||
this.mutex = mutex;
|
||||
}
|
||||
|
||||
public void put(Object key, Object value) {
|
||||
synchronized(mutex) {cache.put(key, value);}
|
||||
}
|
||||
|
||||
public Object get(Object key) {
|
||||
synchronized(mutex) {return cache.get(key);}
|
||||
}
|
||||
|
||||
public boolean containsKey(Object key) {
|
||||
synchronized(mutex) {return cache.containsKey(key);}
|
||||
}
|
||||
|
||||
public void close() {
|
||||
synchronized(mutex) {cache.close();}
|
||||
}
|
||||
|
||||
Cache getSynchronizedCache() {
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
public static Cache synchronizedCache(Cache cache) {
|
||||
return cache.getSynchronizedCache();
|
||||
}
|
||||
|
||||
/**
|
||||
* Called by {@link #synchronizedCache(Cache)}. This method
|
||||
* returns a {@link SynchronizedCache} instance that wraps
|
||||
* this instance by default and can be overridden to return
|
||||
* e. g. subclasses of {@link SynchronizedCache} or this
|
||||
* in case this cache is already synchronized.
|
||||
*/
|
||||
Cache getSynchronizedCache() {
|
||||
return new SynchronizedCache(this);
|
||||
}
|
||||
|
||||
/**
|
||||
* Puts a (key, value)-pair into the cache.
|
||||
*/
|
||||
public abstract void put(Object key, Object value);
|
||||
|
||||
/**
|
||||
* Returns the value for the given key.
|
||||
*/
|
||||
public abstract Object get(Object key);
|
||||
|
||||
/**
|
||||
* Returns whether the given key is in this cache.
|
||||
*/
|
||||
public abstract boolean containsKey(Object key);
|
||||
|
||||
/**
|
||||
* Closes the cache.
|
||||
*/
|
||||
public abstract void close();
|
||||
|
||||
}
|
|
@ -0,0 +1,49 @@
|
|||
package org.apache.lucene.util.cache;
|
||||
|
||||
/**
|
||||
* 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.util.LinkedHashMap;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Simple LRU cache implementation that uses a LinkedHashMap.
|
||||
* This cache is not synchronized, use {@link Cache#synchronizedCache(Cache)}
|
||||
* if needed.
|
||||
*
|
||||
*/
|
||||
public class SimpleLRUCache extends SimpleMapCache {
|
||||
private final static float LOADFACTOR = 0.75f;
|
||||
|
||||
private int cacheSize;
|
||||
|
||||
/**
|
||||
* Creates a last-recently-used cache with the specified size.
|
||||
*/
|
||||
public SimpleLRUCache(int cacheSize) {
|
||||
super(null);
|
||||
this.cacheSize = cacheSize;
|
||||
int capacity = (int) Math.ceil(cacheSize / LOADFACTOR) + 1;
|
||||
|
||||
super.map = new LinkedHashMap(capacity, LOADFACTOR, true) {
|
||||
protected boolean removeEldestEntry(Map.Entry eldest) {
|
||||
return size() > SimpleLRUCache.this.cacheSize;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,100 @@
|
|||
package org.apache.lucene.util.cache;
|
||||
|
||||
/**
|
||||
* 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.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Simple cache implementation that uses a HashMap to store (key, value) pairs.
|
||||
* This cache is not synchronized, use {@link Cache#synchronizedCache(Cache)}
|
||||
* if needed.
|
||||
*/
|
||||
public class SimpleMapCache extends Cache {
|
||||
Map map;
|
||||
|
||||
public SimpleMapCache() {
|
||||
this(new HashMap());
|
||||
}
|
||||
|
||||
public SimpleMapCache(Map map) {
|
||||
this.map = map;
|
||||
}
|
||||
|
||||
public Object get(Object key) {
|
||||
return map.get(key);
|
||||
}
|
||||
|
||||
public void put(Object key, Object value) {
|
||||
map.put(key, value);
|
||||
}
|
||||
|
||||
public void close() {
|
||||
// NOOP
|
||||
}
|
||||
|
||||
public boolean containsKey(Object key) {
|
||||
return map.containsKey(key);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a Set containing all keys in this cache.
|
||||
*/
|
||||
public Set keySet() {
|
||||
return map.keySet();
|
||||
}
|
||||
|
||||
Cache getSynchronizedCache() {
|
||||
return new SynchronizedSimpleMapCache(this);
|
||||
}
|
||||
|
||||
private static class SynchronizedSimpleMapCache extends SimpleMapCache {
|
||||
Object mutex;
|
||||
SimpleMapCache cache;
|
||||
|
||||
SynchronizedSimpleMapCache(SimpleMapCache cache) {
|
||||
this.cache = cache;
|
||||
this.mutex = this;
|
||||
}
|
||||
|
||||
public void put(Object key, Object value) {
|
||||
synchronized(mutex) {cache.put(key, value);}
|
||||
}
|
||||
|
||||
public Object get(Object key) {
|
||||
synchronized(mutex) {return cache.get(key);}
|
||||
}
|
||||
|
||||
public boolean containsKey(Object key) {
|
||||
synchronized(mutex) {return cache.containsKey(key);}
|
||||
}
|
||||
|
||||
public void close() {
|
||||
synchronized(mutex) {cache.close();}
|
||||
}
|
||||
|
||||
public Set keySet() {
|
||||
synchronized(mutex) {return cache.keySet();}
|
||||
}
|
||||
|
||||
Cache getSynchronizedCache() {
|
||||
return this;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,63 @@
|
|||
package org.apache.lucene.util.cache;
|
||||
|
||||
/**
|
||||
* 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 junit.framework.TestCase;
|
||||
|
||||
public class TestSimpleLRUCache extends TestCase {
|
||||
|
||||
public void testLRUCache() throws Exception {
|
||||
final int n = 100;
|
||||
Object dummy = new Object();
|
||||
|
||||
Cache cache = new SimpleLRUCache(n);
|
||||
|
||||
for (int i = 0; i < n; i++) {
|
||||
cache.put(new Integer(i), dummy);
|
||||
}
|
||||
|
||||
// access every 2nd item in cache
|
||||
for (int i = 0; i < n; i+=2) {
|
||||
assertNotNull(cache.get(new Integer(i)));
|
||||
}
|
||||
|
||||
// add n/2 elements to cache, the ones that weren't
|
||||
// touched in the previous loop should now be thrown away
|
||||
for (int i = n; i < n + (n / 2); i++) {
|
||||
cache.put(new Integer(i), dummy);
|
||||
}
|
||||
|
||||
// access every 4th item in cache
|
||||
for (int i = 0; i < n; i+=4) {
|
||||
assertNotNull(cache.get(new Integer(i)));
|
||||
}
|
||||
|
||||
// add 3/4n elements to cache, the ones that weren't
|
||||
// touched in the previous loops should now be thrown away
|
||||
for (int i = n; i < n + (n * 3 / 4); i++) {
|
||||
cache.put(new Integer(i), dummy);
|
||||
}
|
||||
|
||||
// access every 4th item in cache
|
||||
for (int i = 0; i < n; i+=4) {
|
||||
assertNotNull(cache.get(new Integer(i)));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
Loading…
Reference in New Issue