LUCENE-3892: merge trunk take 2

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/pforcodec_3892@1373365 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael McCandless 2012-08-15 12:22:02 +00:00
commit 1c637c5e9b
121 changed files with 1241 additions and 1210 deletions

View File

@ -31,6 +31,25 @@ API Changes
and exposes Iterator, so you can iterate over field names with
for (String field : fields) instead. (Robert Muir)
* LUCENE-4152: added IndexReader.leaves(), which lets you enumerate
the leaf atomic reader contexts for all readers in the tree.
(Uwe Schindler, Robert Muir)
* LUCENE-4304: removed PayloadProcessorProvider. If you want to change
payloads (or other things) when merging indexes, its recommended
to just use a FilterAtomicReader + IndexWriter.addIndexes. See the
OrdinalMappingAtomicReader and TaxonomyMergeUtils in the facets
module if you want an example of this.
(Mike McCandless, Uwe Schindler, Shai Erera, Robert Muir)
* LUCENE-4304: Make CompositeReader.getSequentialSubReaders()
protected. To get atomic leaves of any IndexReader use the new method
leaves() (LUCENE-4152), which lists AtomicReaderContexts including
the doc base of each leaf. (Uwe Schindler, Robert Muir)
* LUCENE-4307: Renamed IndexReader.getTopReaderContext to
IndexReader.getContext. (Robert Muir)
Bug Fixes
* LUCENE-4297: BooleanScorer2 would multiply the coord() factor
@ -52,11 +71,20 @@ Bug Fixes
upgraded to 3.x, then to Lucene 4.0-ALPHA or -BETA, you should run
CheckIndex. If it fails, then you need to upgrade again to 4.0 (Robert Muir)
* LUCENE-4303: PhoneticFilterFactory and SnowballPorterFilterFactory load their
encoders / stemmers via the ResourceLoader now instead of Class.forName().
Solr users should now no longer have to embed these in its war. (David Smiley)
Build
* LUCENE-3985: Upgrade to randomizedtesting 2.0.0. Added support for
thread leak detection. Added support for suite timeouts. (Dawid Weiss)
Documentation
* LUCENE-4302: Fix facet userguide to have HTML loose doctype like
all other javadocs. (Karl Nicholas via Uwe Schindler)
======================= Lucene 4.0.0-BETA =======================
New features

View File

@ -314,6 +314,16 @@ an AtomicReader. Note: using "atomicity emulators" can cause serious
slowdowns due to the need to merge terms, postings, DocValues, and
FieldCache, use them with care!
## LUCENE-4306: getSequentialSubReaders(), ReaderUtil.Gather
The method IndexReader#getSequentialSubReaders() was moved to CompositeReader
(see LUCENE-2858, LUCENE-3733) and made protected. It is solely used by
CompositeReader itsself to build its reader tree. To get all atomic leaves
of a reader, use IndexReader#leaves(), which also provides the doc base
of each leave. Readers that are already atomic return itsself as leaf with
doc base 0. To emulate Lucene 3.x getSequentialSubReaders(),
use getContext().children().
## LUCENE-2413,LUCENE-3396: Analyzer package changes
Lucene's core and contrib analyzers, along with Solr's analyzers,

View File

@ -44,7 +44,7 @@ public final class SnowballFilter extends TokenFilter {
private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
private final KeywordAttribute keywordAttr = addAttribute(KeywordAttribute.class);
public SnowballFilter(TokenStream input, SnowballProgram stemmer) {
super(input);
this.stemmer = stemmer;
@ -62,7 +62,9 @@ public final class SnowballFilter extends TokenFilter {
*/
public SnowballFilter(TokenStream in, String name) {
super(in);
try {
//Class.forName is frowned upon in place of the ResourceLoader but in this case,
// the factory will use the other constructor so that the program is already loaded.
try {
Class<? extends SnowballProgram> stemClass =
Class.forName("org.tartarus.snowball.ext." + name + "Stemmer").asSubclass(SnowballProgram.class);
stemmer = stemClass.newInstance();

View File

@ -46,35 +46,28 @@ public class SnowballPorterFilterFactory extends TokenFilterFactory implements R
public static final String PROTECTED_TOKENS = "protected";
private String language = "English";
private Class<?> stemClass;
private Class<? extends SnowballProgram> stemClass;
private CharArraySet protectedWords = null;
@Override
public void inform(ResourceLoader loader) throws IOException {
String cfgLanguage = args.get("language");
if (cfgLanguage != null)
language = cfgLanguage;
String className = "org.tartarus.snowball.ext." + language + "Stemmer";
stemClass = loader.newInstance(className, SnowballProgram.class).getClass();
String wordFiles = args.get(PROTECTED_TOKENS);
if (wordFiles != null) {
protectedWords = getWordSet(loader, wordFiles, false);
}
}
private CharArraySet protectedWords = null;
@Override
public void init(Map<String, String> args) {
super.init(args);
final String cfgLanguage = args.get("language");
if(cfgLanguage!=null) language = cfgLanguage;
try {
stemClass = Class.forName("org.tartarus.snowball.ext." + language + "Stemmer");
} catch (ClassNotFoundException e) {
throw new IllegalArgumentException("Can't find class for stemmer language " + language, e);
}
}
public TokenFilter create(TokenStream input) {
SnowballProgram program;
try {
program = (SnowballProgram)stemClass.newInstance();
program = stemClass.newInstance();
} catch (Exception e) {
throw new RuntimeException("Error instantiating stemmer for language " + language + "from class " + stemClass, e);
}

View File

@ -27,6 +27,7 @@ import org.apache.lucene.analysis.MockTokenizer;
import org.apache.lucene.analysis.Tokenizer;
import org.apache.lucene.analysis.util.AbstractAnalysisFactory;
import org.apache.lucene.analysis.util.CharFilterFactory;
import org.apache.lucene.analysis.util.ClasspathResourceLoader;
import org.apache.lucene.analysis.util.MultiTermAwareComponent;
import org.apache.lucene.analysis.util.ResourceLoaderAware;
import org.apache.lucene.analysis.util.StringMockResourceLoader;
@ -114,11 +115,15 @@ public class TestFactories extends BaseTokenStreamTestCase {
}
/** tries to initialize a factory with no arguments */
private boolean initialize(AbstractAnalysisFactory factory) {
private boolean initialize(AbstractAnalysisFactory factory) throws IOException {
boolean success = false;
try {
factory.setLuceneMatchVersion(TEST_VERSION_CURRENT);
factory.init(Collections.<String,String>emptyMap());
if (factory instanceof ResourceLoaderAware) {
ResourceLoaderAware resourceLoaderAware = (ResourceLoaderAware) factory;
resourceLoaderAware.inform(new ClasspathResourceLoader(factory.getClass()));
}
success = true;
} catch (IllegalArgumentException ignored) {
// its ok if we dont provide the right parameters to throw this

View File

@ -17,6 +17,7 @@ package org.apache.lucene.analysis.phonetic;
* limitations under the License.
*/
import java.io.IOException;
import java.lang.reflect.Method;
import java.lang.reflect.InvocationTargetException;
import java.util.HashMap;
@ -26,7 +27,8 @@ import java.util.Map;
import org.apache.commons.codec.Encoder;
import org.apache.commons.codec.language.*;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.phonetic.PhoneticFilter;
import org.apache.lucene.analysis.util.ResourceLoader;
import org.apache.lucene.analysis.util.ResourceLoaderAware;
import org.apache.lucene.analysis.util.TokenFilterFactory;
/**
@ -57,6 +59,7 @@ import org.apache.lucene.analysis.util.TokenFilterFactory;
* @see PhoneticFilter
*/
public class PhoneticFilterFactory extends TokenFilterFactory
implements ResourceLoaderAware
{
public static final String ENCODER = "encoder";
public static final String INJECT = "inject"; // boolean
@ -75,18 +78,17 @@ public class PhoneticFilterFactory extends TokenFilterFactory
registry.put("ColognePhonetic".toUpperCase(Locale.ROOT), ColognePhonetic.class);
}
protected boolean inject = true;
protected String name = null;
protected Class<? extends Encoder> clazz = null;
protected Method setMaxCodeLenMethod = null;
protected Integer maxCodeLength = null;
boolean inject = true; //accessed by the test
private String name = null;
private Class<? extends Encoder> clazz = null;
private Method setMaxCodeLenMethod = null;
private Integer maxCodeLength = null;
@Override
public void init(Map<String,String> args) {
super.init( args );
public void inform(ResourceLoader loader) throws IOException {
inject = getBoolean(INJECT, true);
String name = args.get( ENCODER );
if( name == null ) {
throw new IllegalArgumentException("Missing required parameter: " + ENCODER
@ -94,7 +96,7 @@ public class PhoneticFilterFactory extends TokenFilterFactory
}
clazz = registry.get(name.toUpperCase(Locale.ROOT));
if( clazz == null ) {
clazz = resolveEncoder(name);
clazz = resolveEncoder(name, loader);
}
String v = args.get(MAX_CODE_LENGTH);
@ -110,17 +112,15 @@ public class PhoneticFilterFactory extends TokenFilterFactory
getEncoder();//trigger initialization for potential problems to be thrown now
}
private Class<? extends Encoder> resolveEncoder(String name) {
private Class<? extends Encoder> resolveEncoder(String name, ResourceLoader loader) {
String lookupName = name;
if (name.indexOf('.') == -1) {
lookupName = PACKAGE_CONTAINING_ENCODERS + name;
}
try {
return Class.forName(lookupName).asSubclass(Encoder.class);
} catch (ClassNotFoundException cnfe) {
throw new IllegalArgumentException("Unknown encoder: " + name + " must be full class name or one of " + registry.keySet(), cnfe);
} catch (ClassCastException e) {
throw new IllegalArgumentException("Not an encoder: " + name + " must be full class name or one of " + registry.keySet(), e);
return loader.newInstance(lookupName, Encoder.class).getClass();
} catch (RuntimeException e) {
throw new IllegalArgumentException("Error loading encoder '" + name + "': must be full class name or one of " + registry.keySet(), e);
}
}

View File

@ -17,6 +17,7 @@ package org.apache.lucene.analysis.phonetic;
* limitations under the License.
*/
import java.io.IOException;
import java.io.StringReader;
import java.util.HashMap;
import java.util.Map;
@ -27,6 +28,7 @@ import org.apache.lucene.analysis.BaseTokenStreamTestCase;
import org.apache.lucene.analysis.MockTokenizer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.Tokenizer;
import org.apache.lucene.analysis.util.ClasspathResourceLoader;
import org.apache.lucene.util.LuceneTestCase.Slow;
@ -41,48 +43,54 @@ public class TestPhoneticFilterFactory extends BaseTokenStreamTestCase {
/**
* Case: default
*/
public void testFactory()
{
public void testFactory() throws IOException {
Map<String,String> args = new HashMap<String, String>();
PhoneticFilterFactory ff = new PhoneticFilterFactory();
args.put( PhoneticFilterFactory.ENCODER, "Metaphone" );
ff.init( args );
ff.inform(new ClasspathResourceLoader(ff.getClass()));
assertTrue( ff.getEncoder() instanceof Metaphone );
assertTrue( ff.inject ); // default
args.put( PhoneticFilterFactory.INJECT, "false" );
ff.init( args );
ff.inform(new ClasspathResourceLoader(ff.getClass()));
assertFalse( ff.inject );
args.put( PhoneticFilterFactory.MAX_CODE_LENGTH, "2");
ff.init( args );
assertEquals(2,((Metaphone) ff.getEncoder()).getMaxCodeLen());
ff.init(args);
ff.inform(new ClasspathResourceLoader(ff.getClass()));
assertEquals(2, ((Metaphone) ff.getEncoder()).getMaxCodeLen());
}
/**
* Case: Failures and Exceptions
*/
public void testFactoryCaseFailure()
{
public void testFactoryCaseFailure() throws IOException {
Map<String,String> args = new HashMap<String, String>();
PhoneticFilterFactory ff = new PhoneticFilterFactory();
ClasspathResourceLoader loader = new ClasspathResourceLoader(ff.getClass());
try {
ff.init( args );
ff.inform( loader );
fail( "missing encoder parameter" );
}
catch( Exception ex ) {}
args.put( PhoneticFilterFactory.ENCODER, "XXX" );
try {
ff.init( args );
ff.inform( loader );
fail( "unknown encoder parameter" );
}
catch( Exception ex ) {}
args.put( PhoneticFilterFactory.ENCODER, "org.apache.commons.codec.language.NonExistence" );
try {
ff.init( args );
ff.inform( loader );
fail( "unknown encoder parameter" );
}
catch( Exception ex ) {}
@ -91,14 +99,15 @@ public class TestPhoneticFilterFactory extends BaseTokenStreamTestCase {
/**
* Case: Reflection
*/
public void testFactoryCaseReflection()
{
public void testFactoryCaseReflection() throws IOException {
Map<String,String> args = new HashMap<String, String>();
PhoneticFilterFactory ff = new PhoneticFilterFactory();
ClasspathResourceLoader loader = new ClasspathResourceLoader(ff.getClass());
args.put( PhoneticFilterFactory.ENCODER, "org.apache.commons.codec.language.Metaphone" );
ff.init( args );
ff.inform( loader );
assertTrue( ff.getEncoder() instanceof Metaphone );
assertTrue( ff.inject ); // default
@ -106,12 +115,14 @@ public class TestPhoneticFilterFactory extends BaseTokenStreamTestCase {
// so this effectively tests reflection without package name
args.put( PhoneticFilterFactory.ENCODER, "Caverphone2" );
ff.init( args );
ff.inform( loader );
assertTrue( ff.getEncoder() instanceof Caverphone2 );
assertTrue( ff.inject ); // default
// cross check with registry
args.put( PhoneticFilterFactory.ENCODER, "Caverphone" );
ff.init( args );
ff.inform( loader );
assertTrue( ff.getEncoder() instanceof Caverphone2 );
assertTrue( ff.inject ); // default
}
@ -158,6 +169,7 @@ public class TestPhoneticFilterFactory extends BaseTokenStreamTestCase {
args.put("inject", inject);
PhoneticFilterFactory factory = new PhoneticFilterFactory();
factory.init(args);
factory.inform(new ClasspathResourceLoader(factory.getClass()));
TokenStream stream = factory.create(tokenizer);
assertTokenStreamContents(stream, expected);
}

View File

@ -123,18 +123,7 @@ public final class MappingMultiDocsAndPositionsEnum extends DocsAndPositionsEnum
@Override
public BytesRef getPayload() throws IOException {
BytesRef payload = current.getPayload();
if (mergeState.currentPayloadProcessor[upto] != null && payload != null) {
// to not violate the D&P api, we must give the processor a private copy
// TODO: reuse a BytesRef if there is a PPP
payload = BytesRef.deepCopyOf(payload);
mergeState.currentPayloadProcessor[upto].processPayload(payload);
if (payload.length == 0) {
// don't let PayloadProcessors corrumpt the index
return null;
}
}
return payload;
return current.getPayload();
}
}

View File

@ -27,8 +27,6 @@ import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.PayloadProcessorProvider.PayloadProcessor;
import org.apache.lucene.index.PayloadProcessorProvider.ReaderPayloadProcessor;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.DocIdSetIterator;
@ -170,12 +168,7 @@ public abstract class TermVectorsWriter implements Closeable {
final AtomicReader reader = mergeState.readers.get(i);
final int maxDoc = reader.maxDoc();
final Bits liveDocs = reader.getLiveDocs();
// set PayloadProcessor
if (mergeState.payloadProcessorProvider != null) {
mergeState.currentReaderPayloadProcessor = mergeState.readerPayloadProcessor[i];
} else {
mergeState.currentReaderPayloadProcessor = null;
}
for (int docID = 0; docID < maxDoc; docID++) {
if (liveDocs != null && !liveDocs.get(docID)) {
// skip deleted docs
@ -215,9 +208,6 @@ public abstract class TermVectorsWriter implements Closeable {
TermsEnum termsEnum = null;
DocsAndPositionsEnum docsAndPositionsEnum = null;
final ReaderPayloadProcessor readerPayloadProcessor = mergeState.currentReaderPayloadProcessor;
PayloadProcessor payloadProcessor = null;
for(String fieldName : vectors) {
final FieldInfo fieldInfo = mergeState.fieldInfos.fieldInfo(fieldName);
@ -250,10 +240,6 @@ public abstract class TermVectorsWriter implements Closeable {
final int freq = (int) termsEnum.totalTermFreq();
startTerm(termsEnum.term(), freq);
if (hasPayloads && readerPayloadProcessor != null) {
payloadProcessor = readerPayloadProcessor.getProcessor(fieldName, termsEnum.term());
}
if (hasPositions || hasOffsets) {
docsAndPositionsEnum = termsEnum.docsAndPositions(null, docsAndPositionsEnum);
@ -268,17 +254,7 @@ public abstract class TermVectorsWriter implements Closeable {
final int startOffset = docsAndPositionsEnum.startOffset();
final int endOffset = docsAndPositionsEnum.endOffset();
BytesRef payload = docsAndPositionsEnum.getPayload();
if (payloadProcessor != null && payload != null) {
// to not violate the D&P api, we must give the processor a private copy
payload = BytesRef.deepCopyOf(payload);
payloadProcessor.processPayload(payload);
if (payload.length == 0) {
// don't let PayloadProcessors corrumpt the index
payload = null;
}
}
final BytesRef payload = docsAndPositionsEnum.getPayload();
assert !hasPositions || pos >= 0;
addPosition(pos, startOffset, endOffset, payload);

View File

@ -154,14 +154,7 @@ public abstract class TermsConsumer {
postingsEnumIn = (MultiDocsAndPositionsEnum) termsEnum.docsAndPositions(null, postingsEnumIn, DocsAndPositionsEnum.FLAG_PAYLOADS);
assert postingsEnumIn != null;
postingsEnum.reset(postingsEnumIn);
// set PayloadProcessor
if (mergeState.payloadProcessorProvider != null) {
for (int i = 0; i < mergeState.readers.size(); i++) {
if (mergeState.readerPayloadProcessor[i] != null) {
mergeState.currentPayloadProcessor[i] = mergeState.readerPayloadProcessor[i].getProcessor(mergeState.fieldInfo.name, term);
}
}
}
final PostingsConsumer postingsConsumer = startTerm(term);
final TermStats stats = postingsConsumer.merge(mergeState, postingsEnum, visitedDocs);
if (stats.docFreq > 0) {
@ -188,14 +181,7 @@ public abstract class TermsConsumer {
postingsEnumIn = (MultiDocsAndPositionsEnum) termsEnum.docsAndPositions(null, postingsEnumIn);
assert postingsEnumIn != null;
postingsEnum.reset(postingsEnumIn);
// set PayloadProcessor
if (mergeState.payloadProcessorProvider != null) {
for (int i = 0; i < mergeState.readers.size(); i++) {
if (mergeState.readerPayloadProcessor[i] != null) {
mergeState.currentPayloadProcessor[i] = mergeState.readerPayloadProcessor[i].getProcessor(mergeState.fieldInfo.name, term);
}
}
}
final PostingsConsumer postingsConsumer = startTerm(term);
final TermStats stats = postingsConsumer.merge(mergeState, postingsEnum, visitedDocs);
if (stats.docFreq > 0) {

View File

@ -315,12 +315,7 @@ public final class Lucene40TermVectorsWriter extends TermVectorsWriter {
int numDocs = 0;
for (int i = 0; i < mergeState.readers.size(); i++) {
final AtomicReader reader = mergeState.readers.get(i);
// set PayloadProcessor
if (mergeState.payloadProcessorProvider != null) {
mergeState.currentReaderPayloadProcessor = mergeState.readerPayloadProcessor[i];
} else {
mergeState.currentReaderPayloadProcessor = null;
}
final SegmentReader matchingSegmentReader = mergeState.matchingSegmentReaders[idx++];
Lucene40TermVectorsReader matchingVectorsReader = null;
if (matchingSegmentReader != null) {
@ -353,8 +348,8 @@ public final class Lucene40TermVectorsWriter extends TermVectorsWriter {
final int maxDoc = reader.maxDoc();
final Bits liveDocs = reader.getLiveDocs();
int totalNumDocs = 0;
if (matchingVectorsReader != null && mergeState.currentReaderPayloadProcessor == null) {
// We can bulk-copy because the fieldInfos are "congruent" and there is no payload processor
if (matchingVectorsReader != null) {
// We can bulk-copy because the fieldInfos are "congruent"
for (int docNum = 0; docNum < maxDoc;) {
if (!liveDocs.get(docNum)) {
// skip deleted docs
@ -404,8 +399,8 @@ public final class Lucene40TermVectorsWriter extends TermVectorsWriter {
int rawDocLengths2[])
throws IOException {
final int maxDoc = reader.maxDoc();
if (matchingVectorsReader != null && mergeState.currentReaderPayloadProcessor == null) {
// We can bulk-copy because the fieldInfos are "congruent" and there is no payload processor
if (matchingVectorsReader != null) {
// We can bulk-copy because the fieldInfos are "congruent"
int docCount = 0;
while (docCount < maxDoc) {
int len = Math.min(MAX_RAW_MERGE_DOCS, maxDoc - docCount);

View File

@ -54,7 +54,7 @@ public abstract class AtomicReader extends IndexReader {
}
@Override
public final AtomicReaderContext getTopReaderContext() {
public final AtomicReaderContext getContext() {
ensureOpen();
return readerContext;
}

View File

@ -21,8 +21,7 @@ import java.util.Collections;
import java.util.List;
/**
* {@link IndexReaderContext} for {@link AtomicReader} instances
* @lucene.experimental
* {@link IndexReaderContext} for {@link AtomicReader} instances.
*/
public final class AtomicReaderContext extends IndexReaderContext {
/** The readers ord in the top-level's leaves array */

View File

@ -439,7 +439,7 @@ class BufferedDeletesStream {
// Delete by query
private static long applyQueryDeletes(Iterable<QueryAndLimit> queriesIter, ReadersAndLiveDocs rld, final SegmentReader reader) throws IOException {
long delCount = 0;
final AtomicReaderContext readerContext = reader.getTopReaderContext();
final AtomicReaderContext readerContext = reader.getContext();
boolean any = false;
for (QueryAndLimit ent : queriesIter) {
Query query = ent.query;

View File

@ -24,7 +24,6 @@ import java.util.List;
import java.util.Map;
import org.apache.lucene.search.Query;
import org.apache.lucene.util.MergedIterator;
import org.apache.lucene.index.BufferedDeletesStream.QueryAndLimit;
class CoalescedDeletes {

View File

@ -78,16 +78,18 @@ public abstract class CompositeReader extends IndexReader {
}
/** Expert: returns the sequential sub readers that this
* reader is logically composed of. It contrast to previous
* Lucene versions may not return null.
* If this method returns an empty array, that means this
* reader is a null reader (for example a MultiReader
* that has no sub readers).
* reader is logically composed of. This method may not
* return {@code null}.
*
* <p><b>NOTE:</b> In contrast to previous Lucene versions this method
* is no longer public, code that wants to get all {@link AtomicReader}s
* this composite is composed of should use {@link IndexReader#leaves()}.
* @see IndexReader#leaves()
*/
public abstract List<? extends IndexReader> getSequentialSubReaders();
protected abstract List<? extends IndexReader> getSequentialSubReaders();
@Override
public final CompositeReaderContext getTopReaderContext() {
public final CompositeReaderContext getContext() {
ensureOpen();
// lazy init without thread safety for perf reasons: Building the readerContext twice does not hurt!
if (readerContext == null) {

View File

@ -24,7 +24,6 @@ import java.util.List;
/**
* {@link IndexReaderContext} for {@link CompositeReader} instance.
* @lucene.experimental
*/
public final class CompositeReaderContext extends IndexReaderContext {
private final List<IndexReaderContext> children;

View File

@ -21,6 +21,7 @@ import java.io.Closeable;
import java.io.IOException;
import java.util.Collections;
import java.util.LinkedHashSet;
import java.util.List;
import java.util.WeakHashMap;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
@ -377,9 +378,11 @@ public abstract class IndexReader implements Closeable {
protected abstract void doClose() throws IOException;
/**
* Expert: Returns a the root {@link IndexReaderContext} for this
* {@link IndexReader}'s sub-reader tree. Iff this reader is composed of sub
* readers ,ie. this reader being a composite reader, this method returns a
* Expert: Returns the root {@link IndexReaderContext} for this
* {@link IndexReader}'s sub-reader tree.
* <p>
* Iff this reader is composed of sub
* readers, i.e. this reader being a composite reader, this method returns a
* {@link CompositeReaderContext} holding the reader's direct children as well as a
* view of the reader tree's atomic leaf contexts. All sub-
* {@link IndexReaderContext} instances referenced from this readers top-level
@ -388,14 +391,21 @@ public abstract class IndexReader implements Closeable {
* atomic leaf reader at a time. If this reader is not composed of child
* readers, this method returns an {@link AtomicReaderContext}.
* <p>
* Note: Any of the sub-{@link CompositeReaderContext} instances reference from this
* top-level context holds a <code>null</code> {@link CompositeReaderContext#leaves()}
* reference. Only the top-level context maintains the convenience leaf-view
* Note: Any of the sub-{@link CompositeReaderContext} instances referenced
* from this top-level context do not support {@link CompositeReaderContext#leaves()}.
* Only the top-level context maintains the convenience leaf-view
* for performance reasons.
*
* @lucene.experimental
*/
public abstract IndexReaderContext getTopReaderContext();
public abstract IndexReaderContext getContext();
/**
* Returns the reader's leaves, or itself if this reader is atomic.
* This is a convenience method calling {@code this.getContext().leaves()}.
* @see IndexReaderContext#leaves()
*/
public final List<AtomicReaderContext> leaves() {
return getContext().leaves();
}
/** Expert: Returns a key for this IndexReader, so FieldCache/CachingWrapperFilter can find
* it again.

View File

@ -22,7 +22,6 @@ import java.util.List;
/**
* A struct like class that represents a hierarchical relationship between
* {@link IndexReader} instances.
* @lucene.experimental
*/
public abstract class IndexReaderContext {
/** The reader context for this reader's immediate parent, or null if none */

View File

@ -260,9 +260,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
// to allow users to query an IndexWriter settings.
private final LiveIndexWriterConfig config;
// The PayloadProcessorProvider to use when segments are merged
private PayloadProcessorProvider payloadProcessorProvider;
DirectoryReader getReader() throws IOException {
return getReader(true);
}
@ -763,8 +760,15 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
}
/**
* Commits all changes to an index and closes all
* associated files. Note that this may be a costly
* Commits all changes to an index, waits for pending merges
* to complete, and closes all associated files.
* <p>
* This is a "slow graceful shutdown" which may take a long time
* especially if a big merge is pending: If you only want to close
* resources use {@link #rollback()}. If you only want to commit
* pending changes and close resources see {@link #close(boolean)}.
* <p>
* Note that this may be a costly
* operation, so, try to re-use a single writer instead of
* closing and opening a new one. See {@link #commit()} for
* caveats about write caching done by some IO devices.
@ -1263,7 +1267,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
reader = (AtomicReader) readerIn;
} else {
// Composite reader: lookup sub-reader and re-base docID:
List<AtomicReaderContext> leaves = readerIn.getTopReaderContext().leaves();
List<AtomicReaderContext> leaves = readerIn.leaves();
int subIndex = ReaderUtil.subIndex(docID, leaves);
reader = leaves.get(subIndex).reader();
docID -= leaves.get(subIndex).docBase;
@ -2399,8 +2403,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
false, codec, null, null);
SegmentMerger merger = new SegmentMerger(info, infoStream, trackingDir, config.getTermIndexInterval(),
MergeState.CheckAbort.NONE, payloadProcessorProvider,
globalFieldNumberMap, context);
MergeState.CheckAbort.NONE, globalFieldNumberMap, context);
for (IndexReader reader : readers) { // add new indexes
merger.add(reader);
@ -3503,7 +3506,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
final TrackingDirectoryWrapper dirWrapper = new TrackingDirectoryWrapper(directory);
SegmentMerger merger = new SegmentMerger(merge.info.info, infoStream, dirWrapper, config.getTermIndexInterval(), checkAbort,
payloadProcessorProvider, globalFieldNumberMap, context);
globalFieldNumberMap, context);
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "merging " + segString(merge.segments));
@ -4058,38 +4061,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
synchronized void deletePendingFiles() throws IOException {
deleter.deletePendingFiles();
}
/**
* Sets the {@link PayloadProcessorProvider} to use when merging payloads.
* Note that the given <code>pcp</code> will be invoked for every segment that
* is merged, not only external ones that are given through
* {@link #addIndexes}. If you want only the payloads of the external segments
* to be processed, you can return <code>null</code> whenever a
* {@link PayloadProcessorProvider.ReaderPayloadProcessor} is requested for the {@link Directory} of the
* {@link IndexWriter}.
* <p>
* The default is <code>null</code> which means payloads are processed
* normally (copied) during segment merges. You can also unset it by passing
* <code>null</code>.
* <p>
* <b>NOTE:</b> the set {@link PayloadProcessorProvider} will be in effect
* immediately, potentially for already running merges too. If you want to be
* sure it is used for further operations only, such as {@link #addIndexes} or
* {@link #forceMerge}, you can call {@link #waitForMerges()} before.
*/
public void setPayloadProcessorProvider(PayloadProcessorProvider pcp) {
ensureOpen();
payloadProcessorProvider = pcp;
}
/**
* Returns the {@link PayloadProcessorProvider} that is used during segment
* merges to process payloads.
*/
public PayloadProcessorProvider getPayloadProcessorProvider() {
ensureOpen();
return payloadProcessorProvider;
}
/**
* NOTE: this method creates a compound file for all files returned by

View File

@ -19,8 +19,6 @@ package org.apache.lucene.index;
import java.util.List;
import org.apache.lucene.index.PayloadProcessorProvider.PayloadProcessor;
import org.apache.lucene.index.PayloadProcessorProvider.ReaderPayloadProcessor;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.InfoStream;
@ -194,14 +192,6 @@ public class MergeState {
// Updated per field;
public FieldInfo fieldInfo;
// Used to process payloads
// TODO: this is a FactoryFactory here basically
// and we could make a codec(wrapper) to do all of this privately so IW is uninvolved
public PayloadProcessorProvider payloadProcessorProvider;
public ReaderPayloadProcessor[] readerPayloadProcessor;
public ReaderPayloadProcessor currentReaderPayloadProcessor;
public PayloadProcessor[] currentPayloadProcessor;
// TODO: get rid of this? it tells you which segments are 'aligned' (e.g. for bulk merging)
// but is this really so expensive to compute again in different components, versus once in SM?
public SegmentReader[] matchingSegmentReaders;

View File

@ -1,4 +1,4 @@
package org.apache.lucene.util;
package org.apache.lucene.index;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -20,6 +20,8 @@ package org.apache.lucene.util;
import java.util.Iterator;
import java.util.NoSuchElementException;
import org.apache.lucene.util.PriorityQueue;
/**
* Provides a merged sorted view from several sorted iterators, each
* iterating over a unique set of elements.
@ -39,13 +41,13 @@ import java.util.NoSuchElementException;
* </ul>
* @lucene.internal
*/
public class MergedIterator<T extends Comparable<T>> implements Iterator<T> {
final class MergedIterator<T extends Comparable<T>> implements Iterator<T> {
private T current;
private final TermMergeQueue<T> queue;
private final SubIterator<T>[] top;
private int numTop;
@SuppressWarnings("unchecked")
@SuppressWarnings({"unchecked","rawtypes"})
public MergedIterator(Iterator<T>... iterators) {
queue = new TermMergeQueue<T>(iterators.length);
top = new SubIterator[iterators.length];

View File

@ -36,7 +36,7 @@ import org.apache.lucene.util.packed.PackedInts.Reader;
*
* <p><b>NOTE</b>: for multi readers, you'll get better
* performance by gathering the sub readers using
* {@link IndexReader#getTopReaderContext()} to get the
* {@link IndexReader#getContext()} to get the
* atomic leaves and then operate per-AtomicReader,
* instead of using this class.
*
@ -128,7 +128,7 @@ public class MultiDocValues extends DocValues {
return puller.pull((AtomicReader) reader, field);
}
assert reader instanceof CompositeReader;
final List<AtomicReaderContext> leaves = reader.getTopReaderContext().leaves();
final List<AtomicReaderContext> leaves = reader.leaves();
switch (leaves.size()) {
case 0:
// no fields

View File

@ -28,7 +28,6 @@ import java.util.concurrent.ConcurrentHashMap;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.MergedIterator;
/**
* Exposes flex API, merged from flex API of sub-segments.
@ -39,7 +38,7 @@ import org.apache.lucene.util.MergedIterator;
*
* <p><b>NOTE</b>: for composite readers, you'll get better
* performance by gathering the sub readers using
* {@link IndexReader#getTopReaderContext()} to get the
* {@link IndexReader#getContext()} to get the
* atomic leaves and then operate per-AtomicReader,
* instead of using this class.
*
@ -60,7 +59,7 @@ public final class MultiFields extends Fields {
* It's better to get the sub-readers and iterate through them
* yourself. */
public static Fields getFields(IndexReader reader) throws IOException {
final List<AtomicReaderContext> leaves = reader.getTopReaderContext().leaves();
final List<AtomicReaderContext> leaves = reader.leaves();
switch (leaves.size()) {
case 0:
// no fields
@ -92,7 +91,7 @@ public final class MultiFields extends Fields {
public static Bits getLiveDocs(IndexReader reader) {
if (reader.hasDeletions()) {
final List<AtomicReaderContext> leaves = reader.getTopReaderContext().leaves();
final List<AtomicReaderContext> leaves = reader.leaves();
final int size = leaves.size();
assert size > 0 : "A reader with deletions must have at least one leave";
if (size == 1) {
@ -182,7 +181,7 @@ public final class MultiFields extends Fields {
this.subSlices = subSlices;
}
@SuppressWarnings("unchecked")
@SuppressWarnings({"unchecked","rawtypes"})
@Override
public Iterator<String> iterator() {
Iterator<String> subIterators[] = new Iterator[subs.length];
@ -251,7 +250,7 @@ public final class MultiFields extends Fields {
*/
public static FieldInfos getMergedFieldInfos(IndexReader reader) {
final FieldInfos.Builder builder = new FieldInfos.Builder();
for(final AtomicReaderContext ctx : reader.getTopReaderContext().leaves()) {
for(final AtomicReaderContext ctx : reader.leaves()) {
builder.add(ctx.reader().getFieldInfos());
}
return builder.finish();

View File

@ -1,81 +0,0 @@
package org.apache.lucene.index;
/*
* 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.io.IOException;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
/**
* Provides a {@link ReaderPayloadProcessor} to be used for a {@link Directory}.
* This allows using different {@link ReaderPayloadProcessor}s for different
* source {@link AtomicReader}, for e.g. to perform different processing of payloads of
* different directories.
* <p>
* <b>NOTE:</b> to avoid processing payloads of certain directories, you can
* return <code>null</code> in {@link #getReaderProcessor}.
* <p>
* <b>NOTE:</b> it is possible that the same {@link ReaderPayloadProcessor} will be
* requested for the same {@link Directory} concurrently. Therefore, to avoid
* concurrency issues you should return different instances for different
* threads. Usually, if your {@link ReaderPayloadProcessor} does not maintain state
* this is not a problem. The merge code ensures that the
* {@link ReaderPayloadProcessor} instance you return will be accessed by one
* thread to obtain the {@link PayloadProcessor}s for different terms.
*
* @lucene.experimental
*/
public abstract class PayloadProcessorProvider {
/**
* Returns a {@link PayloadProcessor} for a given {@link Term} which allows
* processing the payloads of different terms differently. If you intent to
* process all your payloads the same way, then you can ignore the given term.
* <p>
* <b>NOTE:</b> if you protect your {@link ReaderPayloadProcessor} from
* concurrency issues, then you shouldn't worry about any such issues when
* {@link PayloadProcessor}s are requested for different terms.
*/
public static abstract class ReaderPayloadProcessor {
/** Returns a {@link PayloadProcessor} for the given term. */
public abstract PayloadProcessor getProcessor(String field, BytesRef text) throws IOException;
}
/**
* Processes the given payload.
*
* @lucene.experimental
*/
public static abstract class PayloadProcessor {
/** Process the incoming payload and stores the result in the given {@link BytesRef}. */
public abstract void processPayload(BytesRef payload) throws IOException;
}
/**
* Returns a {@link ReaderPayloadProcessor} for the given {@link Directory},
* through which {@link PayloadProcessor}s can be obtained for each
* {@link Term}, or <code>null</code> if none should be used.
*/
public abstract ReaderPayloadProcessor getReaderProcessor(AtomicReader reader) throws IOException;
}

View File

@ -56,13 +56,11 @@ final class SegmentMerger {
// note, just like in codec apis Directory 'dir' is NOT the same as segmentInfo.dir!!
SegmentMerger(SegmentInfo segmentInfo, InfoStream infoStream, Directory dir, int termIndexInterval,
MergeState.CheckAbort checkAbort, PayloadProcessorProvider payloadProcessorProvider,
FieldInfos.FieldNumbers fieldNumbers, IOContext context) {
MergeState.CheckAbort checkAbort, FieldInfos.FieldNumbers fieldNumbers, IOContext context) {
mergeState.segmentInfo = segmentInfo;
mergeState.infoStream = infoStream;
mergeState.readers = new ArrayList<AtomicReader>();
mergeState.checkAbort = checkAbort;
mergeState.payloadProcessorProvider = payloadProcessorProvider;
directory = dir;
this.termIndexInterval = termIndexInterval;
this.codec = segmentInfo.getCodec();
@ -75,7 +73,7 @@ final class SegmentMerger {
* @param reader
*/
final void add(IndexReader reader) {
for (final AtomicReaderContext ctx : reader.getTopReaderContext().leaves()) {
for (final AtomicReaderContext ctx : reader.leaves()) {
final AtomicReader r = ctx.reader();
mergeState.readers.add(r);
}
@ -274,8 +272,6 @@ final class SegmentMerger {
// Remap docIDs
mergeState.docMaps = new MergeState.DocMap[numReaders];
mergeState.docBase = new int[numReaders];
mergeState.readerPayloadProcessor = new PayloadProcessorProvider.ReaderPayloadProcessor[numReaders];
mergeState.currentPayloadProcessor = new PayloadProcessorProvider.PayloadProcessor[numReaders];
int docBase = 0;
@ -289,10 +285,6 @@ final class SegmentMerger {
mergeState.docMaps[i] = docMap;
docBase += docMap.numDocs();
if (mergeState.payloadProcessorProvider != null) {
mergeState.readerPayloadProcessor[i] = mergeState.payloadProcessorProvider.getReaderProcessor(reader);
}
i++;
}

View File

@ -37,7 +37,7 @@ import org.apache.lucene.index.MultiReader; // javadoc
* <p><b>NOTE</b>: this class almost always results in a
* performance hit. If this is important to your use case,
* you'll get better performance by gathering the sub readers using
* {@link IndexReader#getTopReaderContext()} to get the
* {@link IndexReader#getContext()} to get the
* atomic leaves and then operate per-AtomicReader,
* instead of using this class.
*/

View File

@ -122,7 +122,7 @@ public class IndexSearcher {
*
* @lucene.experimental */
public IndexSearcher(IndexReader r, ExecutorService executor) {
this(r.getTopReaderContext(), executor);
this(r.getContext(), executor);
}
/**
@ -138,7 +138,7 @@ public class IndexSearcher {
* href="https://issues.apache.org/jira/browse/LUCENE-2239">LUCENE-2239</a>).
*
* @see IndexReaderContext
* @see IndexReader#getTopReaderContext()
* @see IndexReader#getContext()
* @lucene.experimental
*/
public IndexSearcher(IndexReaderContext context, ExecutorService executor) {
@ -154,7 +154,7 @@ public class IndexSearcher {
* Creates a searcher searching the provided top-level {@link IndexReaderContext}.
*
* @see IndexReaderContext
* @see IndexReader#getTopReaderContext()
* @see IndexReader#getContext()
* @lucene.experimental
*/
public IndexSearcher(IndexReaderContext context) {
@ -639,7 +639,7 @@ public class IndexSearcher {
/**
* Returns this searchers the top-level {@link IndexReaderContext}.
* @see IndexReader#getTopReaderContext()
* @see IndexReader#getContext()
*/
/* sugar for #getReader().getTopReaderContext() */
public IndexReaderContext getTopReaderContext() {

View File

@ -50,7 +50,7 @@ public class QueryWrapperFilter extends Filter {
@Override
public DocIdSet getDocIdSet(final AtomicReaderContext context, final Bits acceptDocs) throws IOException {
// get a private context that is used to rewrite, createWeight and score eventually
final AtomicReaderContext privateContext = context.reader().getTopReaderContext();
final AtomicReaderContext privateContext = context.reader().getContext();
final Weight weight = new IndexSearcher(privateContext).createNormalizedWeight(query);
return new DocIdSet() {
@Override

View File

@ -46,7 +46,7 @@ abstract class TermCollectingRewrite<Q extends Query> extends MultiTermQuery.Rew
final void collectTerms(IndexReader reader, MultiTermQuery query, TermCollector collector) throws IOException {
IndexReaderContext topReaderContext = reader.getTopReaderContext();
IndexReaderContext topReaderContext = reader.getContext();
Comparator<BytesRef> lastTermComp = null;
for (AtomicReaderContext context : topReaderContext.leaves()) {
final Fields fields = context.reader().fields();

View File

@ -60,7 +60,7 @@ public class PayloadSpanUtil {
* @param context
* that contains doc with payloads to extract
*
* @see IndexReader#getTopReaderContext()
* @see IndexReader#getContext()
*/
public PayloadSpanUtil(IndexReaderContext context) {
this.context = context;

View File

@ -25,8 +25,10 @@ import java.util.Set;
import org.apache.lucene.index.CompositeReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReaderContext;
import org.apache.lucene.search.FieldCache;
import org.apache.lucene.search.FieldCache.CacheEntry;
import org.apache.lucene.store.AlreadyClosedException;
/**
* Provides methods for sanity checking that entries in the FieldCache
@ -272,20 +274,28 @@ public final class FieldCacheSanityChecker {
/**
* Checks if the seed is an IndexReader, and if so will walk
* the hierarchy of subReaders building up a list of the objects
* returned by obj.getFieldCacheKey()
* returned by {@code seed.getCoreCacheKey()}
*/
private List<Object> getAllDescendantReaderKeys(Object seed) {
List<Object> all = new ArrayList<Object>(17); // will grow as we iter
all.add(seed);
for (int i = 0; i < all.size(); i++) {
Object obj = all.get(i);
if (obj instanceof CompositeReader) {
List<? extends IndexReader> subs = ((CompositeReader)obj).getSequentialSubReaders();
for (int j = 0; (null != subs) && (j < subs.size()); j++) {
all.add(subs.get(j).getCoreCacheKey());
final Object obj = all.get(i);
// TODO: We don't check closed readers here (as getTopReaderContext
// throws AlreadyClosedException), what should we do? Reflection?
if (obj instanceof IndexReader) {
try {
final List<IndexReaderContext> childs =
((IndexReader) obj).getContext().children();
if (childs != null) { // it is composite reader
for (final IndexReaderContext ctx : childs) {
all.add(ctx.reader().getCoreCacheKey());
}
}
} catch (AlreadyClosedException ace) {
// ignore this reader
}
}
}
// need to skip the first, because it was the seed
return all.subList(1, all.size());

View File

@ -23,9 +23,9 @@ import java.util.Random;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
@ -50,7 +50,8 @@ public class TestReuseDocsEnum extends LuceneTestCase {
writer.commit();
DirectoryReader open = DirectoryReader.open(dir);
for (AtomicReader indexReader : open.getSequentialSubReaders()) {
for (AtomicReaderContext ctx : open.leaves()) {
AtomicReader indexReader = ctx.reader();
Terms terms = indexReader.terms("body");
TermsEnum iterator = terms.iterator(null);
IdentityHashMap<DocsEnum, Boolean> enums = new IdentityHashMap<DocsEnum, Boolean>();
@ -76,8 +77,8 @@ public class TestReuseDocsEnum extends LuceneTestCase {
writer.commit();
DirectoryReader open = DirectoryReader.open(dir);
for (AtomicReader indexReader : open.getSequentialSubReaders()) {
Terms terms = indexReader.terms("body");
for (AtomicReaderContext ctx : open.leaves()) {
Terms terms = ctx.reader().terms("body");
TermsEnum iterator = terms.iterator(null);
IdentityHashMap<DocsEnum, Boolean> enums = new IdentityHashMap<DocsEnum, Boolean>();
MatchNoBits bits = new Bits.MatchNoBits(open.maxDoc());
@ -121,11 +122,11 @@ public class TestReuseDocsEnum extends LuceneTestCase {
DirectoryReader firstReader = DirectoryReader.open(dir);
DirectoryReader secondReader = DirectoryReader.open(dir);
List<? extends AtomicReader> sequentialSubReaders = firstReader.getSequentialSubReaders();
List<? extends AtomicReader> sequentialSubReaders2 = secondReader.getSequentialSubReaders();
List<AtomicReaderContext> leaves = firstReader.leaves();
List<AtomicReaderContext> leaves2 = secondReader.leaves();
for (IndexReader indexReader : sequentialSubReaders) {
Terms terms = ((AtomicReader) indexReader).terms("body");
for (AtomicReaderContext ctx : leaves) {
Terms terms = ctx.reader().terms("body");
TermsEnum iterator = terms.iterator(null);
IdentityHashMap<DocsEnum, Boolean> enums = new IdentityHashMap<DocsEnum, Boolean>();
MatchNoBits bits = new Bits.MatchNoBits(firstReader.maxDoc());
@ -133,7 +134,7 @@ public class TestReuseDocsEnum extends LuceneTestCase {
DocsEnum docs = null;
BytesRef term = null;
while ((term = iterator.next()) != null) {
docs = iterator.docs(null, randomDocsEnum("body", term, sequentialSubReaders2, bits), random().nextBoolean() ? DocsEnum.FLAG_FREQS : 0);
docs = iterator.docs(null, randomDocsEnum("body", term, leaves2, bits), random().nextBoolean() ? DocsEnum.FLAG_FREQS : 0);
enums.put(docs, true);
}
assertEquals(terms.size(), enums.size());
@ -142,7 +143,7 @@ public class TestReuseDocsEnum extends LuceneTestCase {
enums.clear();
docs = null;
while ((term = iterator.next()) != null) {
docs = iterator.docs(bits, randomDocsEnum("body", term, sequentialSubReaders2, bits), random().nextBoolean() ? DocsEnum.FLAG_FREQS : 0);
docs = iterator.docs(bits, randomDocsEnum("body", term, leaves2, bits), random().nextBoolean() ? DocsEnum.FLAG_FREQS : 0);
enums.put(docs, true);
}
assertEquals(terms.size(), enums.size());
@ -150,11 +151,11 @@ public class TestReuseDocsEnum extends LuceneTestCase {
IOUtils.close(writer, firstReader, secondReader, dir);
}
public DocsEnum randomDocsEnum(String field, BytesRef term, List<? extends AtomicReader> readers, Bits bits) throws IOException {
public DocsEnum randomDocsEnum(String field, BytesRef term, List<AtomicReaderContext> readers, Bits bits) throws IOException {
if (random().nextInt(10) == 0) {
return null;
}
AtomicReader indexReader = (AtomicReader) readers.get(random().nextInt(readers.size()));
AtomicReader indexReader = readers.get(random().nextInt(readers.size())).reader();
return indexReader.termDocsEnum(bits, field, term, random().nextBoolean() ? DocsEnum.FLAG_FREQS : 0);
}

View File

@ -143,7 +143,7 @@ public class TestCustomNorms extends LuceneTestCase {
IndexReader reader = writer.getReader();
writer.close();
assertEquals(numAdded, reader.numDocs());
IndexReaderContext topReaderContext = reader.getTopReaderContext();
IndexReaderContext topReaderContext = reader.getContext();
for (final AtomicReaderContext ctx : topReaderContext.leaves()) {
AtomicReader atomicReader = ctx.reader();
Source source = random().nextBoolean() ? atomicReader.normValues("foo").getSource() : atomicReader.normValues("foo").getDirectSource();

View File

@ -68,7 +68,7 @@ public class TestDeletionPolicy extends LuceneTestCase {
public void onCommit(List<? extends IndexCommit> commits) throws IOException {
IndexCommit lastCommit = commits.get(commits.size()-1);
DirectoryReader r = DirectoryReader.open(dir);
assertEquals("lastCommit.segmentCount()=" + lastCommit.getSegmentCount() + " vs IndexReader.segmentCount=" + r.getSequentialSubReaders().size(), r.getSequentialSubReaders().size(), lastCommit.getSegmentCount());
assertEquals("lastCommit.segmentCount()=" + lastCommit.getSegmentCount() + " vs IndexReader.segmentCount=" + r.leaves().size(), r.leaves().size(), lastCommit.getSegmentCount());
r.close();
verifyCommitOrder(commits);
numOnCommit++;
@ -318,7 +318,7 @@ public class TestDeletionPolicy extends LuceneTestCase {
final boolean needsMerging;
{
DirectoryReader r = DirectoryReader.open(dir);
needsMerging = r.getSequentialSubReaders().size() != 1;
needsMerging = r.leaves().size() != 1;
r.close();
}
if (needsMerging) {
@ -435,7 +435,7 @@ public class TestDeletionPolicy extends LuceneTestCase {
DirectoryReader r = DirectoryReader.open(dir);
// Still merged, still 11 docs
assertEquals(1, r.getSequentialSubReaders().size());
assertEquals(1, r.leaves().size());
assertEquals(11, r.numDocs());
r.close();
@ -451,7 +451,7 @@ public class TestDeletionPolicy extends LuceneTestCase {
r = DirectoryReader.open(dir);
// Not fully merged because we rolled it back, and now only
// 10 docs
assertTrue(r.getSequentialSubReaders().size() > 1);
assertTrue(r.leaves().size() > 1);
assertEquals(10, r.numDocs());
r.close();
@ -461,7 +461,7 @@ public class TestDeletionPolicy extends LuceneTestCase {
writer.close();
r = DirectoryReader.open(dir);
assertEquals(1, r.getSequentialSubReaders().size());
assertEquals(1, r.leaves().size());
assertEquals(10, r.numDocs());
r.close();
@ -473,7 +473,7 @@ public class TestDeletionPolicy extends LuceneTestCase {
// Reader still sees fully merged index, because writer
// opened on the prior commit has not yet committed:
r = DirectoryReader.open(dir);
assertEquals(1, r.getSequentialSubReaders().size());
assertEquals(1, r.leaves().size());
assertEquals(10, r.numDocs());
r.close();
@ -481,7 +481,7 @@ public class TestDeletionPolicy extends LuceneTestCase {
// Now reader sees not-fully-merged index:
r = DirectoryReader.open(dir);
assertTrue(r.getSequentialSubReaders().size() > 1);
assertTrue(r.leaves().size() > 1);
assertEquals(10, r.numDocs());
r.close();

View File

@ -549,7 +549,7 @@ public void testFilesOpenClose() throws IOException {
assertEquals("IndexReaders have different values for numDocs.", index1.numDocs(), index2.numDocs());
assertEquals("IndexReaders have different values for maxDoc.", index1.maxDoc(), index2.maxDoc());
assertEquals("Only one IndexReader has deletions.", index1.hasDeletions(), index2.hasDeletions());
assertEquals("Single segment test differs.", index1.getSequentialSubReaders().size() == 1, index2.getSequentialSubReaders().size() == 1);
assertEquals("Single segment test differs.", index1.leaves().size() == 1, index2.leaves().size() == 1);
// check field names
FieldInfos fieldInfos1 = MultiFields.getMergedFieldInfos(index1);
@ -785,7 +785,7 @@ public void testFilesOpenClose() throws IOException {
DirectoryReader r2 = DirectoryReader.openIfChanged(r);
assertNotNull(r2);
r.close();
AtomicReader sub0 = r2.getSequentialSubReaders().get(0);
AtomicReader sub0 = r2.leaves().get(0).reader();
final int[] ints2 = FieldCache.DEFAULT.getInts(sub0, "number", false);
r2.close();
assertTrue(ints == ints2);
@ -814,9 +814,8 @@ public void testFilesOpenClose() throws IOException {
assertNotNull(r2);
r.close();
List<? extends AtomicReader> subs = r2.getSequentialSubReaders();
for(AtomicReader s : subs) {
assertEquals(36, s.getUniqueTermCount());
for(AtomicReaderContext s : r2.leaves()) {
assertEquals(36, s.reader().getUniqueTermCount());
}
r2.close();
writer.close();
@ -842,7 +841,7 @@ public void testFilesOpenClose() throws IOException {
// expected
}
assertEquals(-1, ((SegmentReader) r.getSequentialSubReaders().get(0)).getTermInfosIndexDivisor());
assertEquals(-1, ((SegmentReader) r.leaves().get(0).reader()).getTermInfosIndexDivisor());
writer = new IndexWriter(
dir,
newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())).
@ -857,11 +856,11 @@ public void testFilesOpenClose() throws IOException {
assertNotNull(r2);
assertNull(DirectoryReader.openIfChanged(r2));
r.close();
List<? extends AtomicReader> subReaders = r2.getSequentialSubReaders();
assertEquals(2, subReaders.size());
for(AtomicReader s : subReaders) {
List<AtomicReaderContext> leaves = r2.leaves();
assertEquals(2, leaves.size());
for(AtomicReaderContext ctx : leaves) {
try {
s.docFreq(new Term("field", "f"));
ctx.reader().docFreq(new Term("field", "f"));
fail("did not hit expected exception");
} catch (IllegalStateException ise) {
// expected

View File

@ -171,8 +171,8 @@ public class TestDirectoryReaderReopen extends LuceneTestCase {
TestDirectoryReader.assertIndexEquals(index1, index2_refreshed);
index2_refreshed.close();
assertReaderClosed(index2, true, true);
assertReaderClosed(index2_refreshed, true, true);
assertReaderClosed(index2, true);
assertReaderClosed(index2_refreshed, true);
index2 = test.openReader();
@ -190,28 +190,8 @@ public class TestDirectoryReaderReopen extends LuceneTestCase {
index1.close();
index2.close();
assertReaderClosed(index1, true, true);
assertReaderClosed(index2, true, true);
}
private void performTestsWithExceptionInReopen(TestReopen test) throws Exception {
DirectoryReader index1 = test.openReader();
DirectoryReader index2 = test.openReader();
TestDirectoryReader.assertIndexEquals(index1, index2);
try {
refreshReader(index1, test, 0, true);
fail("Expected exception not thrown.");
} catch (Exception e) {
// expected exception
}
// index2 should still be usable and unaffected by the failed reopen() call
TestDirectoryReader.assertIndexEquals(index1, index2);
index1.close();
index2.close();
assertReaderClosed(index1, true);
assertReaderClosed(index2, true);
}
public void testThreadSafety() throws Exception {
@ -355,11 +335,11 @@ public class TestDirectoryReaderReopen extends LuceneTestCase {
reader.close();
for (final DirectoryReader readerToClose : readersToClose) {
assertReaderClosed(readerToClose, true, true);
assertReaderClosed(readerToClose, true);
}
assertReaderClosed(reader, true, true);
assertReaderClosed(firstReader, true, true);
assertReaderClosed(reader, true);
assertReaderClosed(firstReader, true);
dir.close();
}
@ -374,7 +354,7 @@ public class TestDirectoryReaderReopen extends LuceneTestCase {
DirectoryReader refreshedReader;
}
private abstract static class ReaderThreadTask {
abstract static class ReaderThreadTask {
protected volatile boolean stopped;
public void stop() {
this.stopped = true;
@ -384,8 +364,8 @@ public class TestDirectoryReaderReopen extends LuceneTestCase {
}
private static class ReaderThread extends Thread {
private ReaderThreadTask task;
private Throwable error;
ReaderThreadTask task;
Throwable error;
ReaderThread(ReaderThreadTask task) {
@ -469,9 +449,9 @@ public class TestDirectoryReaderReopen extends LuceneTestCase {
DirectoryReader r = DirectoryReader.open(dir);
if (multiSegment) {
assertTrue(r.getSequentialSubReaders().size() > 1);
assertTrue(r.leaves().size() > 1);
} else {
assertTrue(r.getSequentialSubReaders().size() == 1);
assertTrue(r.leaves().size() == 1);
}
r.close();
}
@ -533,46 +513,25 @@ public class TestDirectoryReaderReopen extends LuceneTestCase {
}
}
static void assertReaderClosed(IndexReader reader, boolean checkSubReaders, boolean checkNormsClosed) {
static void assertReaderClosed(IndexReader reader, boolean checkSubReaders) {
assertEquals(0, reader.getRefCount());
if (checkNormsClosed && reader instanceof AtomicReader) {
// TODO: should we really assert something here? we check for open files and this is obselete...
// assertTrue(((SegmentReader) reader).normsClosed());
}
if (checkSubReaders && reader instanceof CompositeReader) {
// we cannot use reader context here, as reader is
// already closed and calling getTopReaderContext() throws AlreadyClosed!
List<? extends IndexReader> subReaders = ((CompositeReader) reader).getSequentialSubReaders();
for (IndexReader r : subReaders) {
assertReaderClosed(r, checkSubReaders, checkNormsClosed);
for (final IndexReader r : subReaders) {
assertReaderClosed(r, checkSubReaders);
}
}
}
/*
private void assertReaderOpen(DirectoryReader reader) {
reader.ensureOpen();
if (reader instanceof DirectoryReader) {
DirectoryReader[] subReaders = reader.getSequentialSubReaders();
for (int i = 0; i < subReaders.length; i++) {
assertReaderOpen(subReaders[i]);
}
}
}
*/
private void assertRefCountEquals(int refCount, DirectoryReader reader) {
assertEquals("Reader has wrong refCount value.", refCount, reader.getRefCount());
}
private abstract static class TestReopen {
abstract static class TestReopen {
protected abstract DirectoryReader openReader() throws IOException;
protected abstract void modifyIndex(int i) throws IOException;
}
private static class KeepAllCommits implements IndexDeletionPolicy {
static class KeepAllCommits implements IndexDeletionPolicy {
public void onInit(List<? extends IndexCommit> commits) {
}
public void onCommit(List<? extends IndexCommit> commits) {

View File

@ -204,7 +204,7 @@ public class TestDoc extends LuceneTestCase {
final SegmentInfo si = new SegmentInfo(si1.info.dir, Constants.LUCENE_MAIN_VERSION, merged, -1, false, codec, null, null);
SegmentMerger merger = new SegmentMerger(si, InfoStream.getDefault(), trackingDir, IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL,
MergeState.CheckAbort.NONE, null, new FieldInfos.FieldNumbers(), context);
MergeState.CheckAbort.NONE, new FieldInfos.FieldNumbers(), context);
merger.add(r1);
merger.add(r2);

View File

@ -154,11 +154,11 @@ public class TestDocTermOrds extends LuceneTestCase {
System.out.println("TEST: reader=" + r);
}
for(IndexReader subR : r.getSequentialSubReaders()) {
for(AtomicReaderContext ctx : r.leaves()) {
if (VERBOSE) {
System.out.println("\nTEST: sub=" + subR);
System.out.println("\nTEST: sub=" + ctx.reader());
}
verify((AtomicReader) subR, idToOrds, termsArray, null);
verify(ctx.reader(), idToOrds, termsArray, null);
}
// Also test top-level reader: its enum does not support
@ -273,11 +273,11 @@ public class TestDocTermOrds extends LuceneTestCase {
idToOrdsPrefix[id] = newOrdsArray;
}
for(IndexReader subR : r.getSequentialSubReaders()) {
for(AtomicReaderContext ctx : r.leaves()) {
if (VERBOSE) {
System.out.println("\nTEST: sub=" + subR);
System.out.println("\nTEST: sub=" + ctx.reader());
}
verify((AtomicReader) subR, idToOrdsPrefix, termsArray, prefixRef);
verify(ctx.reader(), idToOrdsPrefix, termsArray, prefixRef);
}
// Also test top-level reader: its enum does not support

View File

@ -91,7 +91,7 @@ public class TestDocValuesIndexing extends LuceneTestCase {
writer.close(true);
DirectoryReader reader = DirectoryReader.open(dir, 1);
assertEquals(1, reader.getSequentialSubReaders().size());
assertEquals(1, reader.leaves().size());
IndexSearcher searcher = new IndexSearcher(reader);

View File

@ -63,7 +63,7 @@ public class TestDocsAndPositions extends LuceneTestCase {
int num = atLeast(13);
for (int i = 0; i < num; i++) {
BytesRef bytes = new BytesRef("1");
IndexReaderContext topReaderContext = reader.getTopReaderContext();
IndexReaderContext topReaderContext = reader.getContext();
for (AtomicReaderContext atomicReaderContext : topReaderContext.leaves()) {
DocsAndPositionsEnum docsAndPosEnum = getDocsAndPositions(
atomicReaderContext.reader(), bytes, null);
@ -138,7 +138,7 @@ public class TestDocsAndPositions extends LuceneTestCase {
int num = atLeast(13);
for (int i = 0; i < num; i++) {
BytesRef bytes = new BytesRef("" + term);
IndexReaderContext topReaderContext = reader.getTopReaderContext();
IndexReaderContext topReaderContext = reader.getContext();
for (AtomicReaderContext atomicReaderContext : topReaderContext.leaves()) {
DocsAndPositionsEnum docsAndPosEnum = getDocsAndPositions(
atomicReaderContext.reader(), bytes, null);
@ -214,7 +214,7 @@ public class TestDocsAndPositions extends LuceneTestCase {
int num = atLeast(13);
for (int i = 0; i < num; i++) {
BytesRef bytes = new BytesRef("" + term);
IndexReaderContext topReaderContext = reader.getTopReaderContext();
IndexReaderContext topReaderContext = reader.getContext();
for (AtomicReaderContext context : topReaderContext.leaves()) {
int maxDoc = context.reader().maxDoc();
DocsEnum docsEnum = _TestUtil.docs(random(), context.reader(), fieldName, bytes, null, null, DocsEnum.FLAG_FREQS);
@ -292,7 +292,7 @@ public class TestDocsAndPositions extends LuceneTestCase {
for (int i = 0; i < num; i++) {
BytesRef bytes = new BytesRef("even");
IndexReaderContext topReaderContext = reader.getTopReaderContext();
IndexReaderContext topReaderContext = reader.getContext();
for (AtomicReaderContext atomicReaderContext : topReaderContext.leaves()) {
DocsAndPositionsEnum docsAndPosEnum = getDocsAndPositions(
atomicReaderContext.reader(), bytes, null);

View File

@ -289,8 +289,9 @@ public class TestFieldsReader extends LuceneTestCase {
assertEquals(numDocs, r.numDocs());
for(IndexReader sub : r.getSequentialSubReaders()) {
final int[] ids = FieldCache.DEFAULT.getInts((AtomicReader) sub, "id", false);
for(AtomicReaderContext ctx : r.leaves()) {
final AtomicReader sub = ctx.reader();
final int[] ids = FieldCache.DEFAULT.getInts(sub, "id", false);
for(int docID=0;docID<sub.numDocs();docID++) {
final Document doc = sub.document(docID);
final Field f = (Field) doc.getField("nf");

View File

@ -1498,9 +1498,9 @@ public class TestIndexWriter extends LuceneTestCase {
assertNoUnreferencedFiles(dir, "no tv files");
DirectoryReader r0 = DirectoryReader.open(dir);
for (IndexReader r : r0.getSequentialSubReaders()) {
SegmentInfoPerCommit s = ((SegmentReader) r).getSegmentInfo();
assertFalse(((SegmentReader) r).getFieldInfos().hasVectors());
for (AtomicReaderContext ctx : r0.leaves()) {
SegmentReader sr = (SegmentReader) ctx.reader();
assertFalse(sr.getFieldInfos().hasVectors());
}
r0.close();

View File

@ -286,7 +286,7 @@ public class TestIndexWriterCommit extends LuceneTestCase {
// Reader should see index as multi-seg at this
// point:
assertTrue("Reader incorrectly sees one segment", reader.getSequentialSubReaders().size() > 1);
assertTrue("Reader incorrectly sees one segment", reader.leaves().size() > 1);
reader.close();
// Abort the writer:
@ -297,7 +297,7 @@ public class TestIndexWriterCommit extends LuceneTestCase {
reader = DirectoryReader.open(dir);
// Reader should still see index as multi-segment
assertTrue("Reader incorrectly sees one segment", reader.getSequentialSubReaders().size() > 1);
assertTrue("Reader incorrectly sees one segment", reader.leaves().size() > 1);
reader.close();
if (VERBOSE) {
@ -316,7 +316,7 @@ public class TestIndexWriterCommit extends LuceneTestCase {
reader = DirectoryReader.open(dir);
// Reader should see index as one segment
assertEquals("Reader incorrectly sees more than one segment", 1, reader.getSequentialSubReaders().size());
assertEquals("Reader incorrectly sees more than one segment", 1, reader.leaves().size());
reader.close();
dir.close();
}

View File

@ -1293,7 +1293,7 @@ public class TestIndexWriterExceptions extends LuceneTestCase {
assertTrue(reader.numDocs() > 0);
SegmentInfos sis = new SegmentInfos();
sis.read(dir);
for(AtomicReaderContext context : reader.getTopReaderContext().leaves()) {
for(AtomicReaderContext context : reader.leaves()) {
assertFalse(context.reader().getFieldInfos().hasVectors());
}
reader.close();

View File

@ -187,7 +187,7 @@ public class TestIndexWriterForceMerge extends LuceneTestCase {
if (0 == pass) {
writer.close();
DirectoryReader reader = DirectoryReader.open(dir);
assertEquals(1, reader.getSequentialSubReaders().size());
assertEquals(1, reader.leaves().size());
reader.close();
} else {
// Get another segment to flush so we can verify it is
@ -197,7 +197,7 @@ public class TestIndexWriterForceMerge extends LuceneTestCase {
writer.close();
DirectoryReader reader = DirectoryReader.open(dir);
assertTrue(reader.getSequentialSubReaders().size() > 1);
assertTrue(reader.leaves().size() > 1);
reader.close();
SegmentInfos infos = new SegmentInfos();

View File

@ -315,7 +315,7 @@ public class TestIndexWriterUnicode extends LuceneTestCase {
IndexReader r = writer.getReader();
// Test each sub-segment
for (AtomicReaderContext ctx : r.getTopReaderContext().leaves()) {
for (AtomicReaderContext ctx : r.leaves()) {
checkTermsOrder(ctx.reader(), allTerms, false);
}
checkTermsOrder(r, allTerms, true);

View File

@ -339,13 +339,13 @@ public class TestParallelCompositeReader extends LuceneTestCase {
if (compositeComposite) {
rd1 = new MultiReader(DirectoryReader.open(dir1), DirectoryReader.open(dir1));
rd2 = new MultiReader(DirectoryReader.open(dir2), DirectoryReader.open(dir2));
assertEquals(2, rd1.getSequentialSubReaders().size());
assertEquals(2, rd2.getSequentialSubReaders().size());
assertEquals(2, rd1.getContext().children().size());
assertEquals(2, rd2.getContext().children().size());
} else {
rd1 = DirectoryReader.open(dir1);
rd2 = DirectoryReader.open(dir2);
assertEquals(3, rd1.getSequentialSubReaders().size());
assertEquals(3, rd2.getSequentialSubReaders().size());
assertEquals(3, rd1.getContext().children().size());
assertEquals(3, rd2.getContext().children().size());
}
ParallelCompositeReader pr = new ParallelCompositeReader(rd1, rd2);
return newSearcher(pr);

View File

@ -1,271 +0,0 @@
package org.apache.lucene.index;
/*
* 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.io.IOException;
import java.util.HashMap;
import java.util.Map;
import java.util.Random;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.analysis.MockTokenizer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.PayloadProcessorProvider.ReaderPayloadProcessor;
import org.apache.lucene.index.PayloadProcessorProvider.PayloadProcessor;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
import org.junit.Test;
public class TestPayloadProcessorProvider extends LuceneTestCase {
private static final class PerDirPayloadProcessor extends PayloadProcessorProvider {
private final Map<Directory, ReaderPayloadProcessor> processors;
public PerDirPayloadProcessor(Map<Directory, ReaderPayloadProcessor> processors) {
this.processors = processors;
}
@Override
public ReaderPayloadProcessor getReaderProcessor(AtomicReader reader) {
if (reader instanceof SegmentReader) {
return processors.get(((SegmentReader) reader).directory());
} else {
throw new UnsupportedOperationException("This shouldnot happen in this test: Reader is no SegmentReader");
}
}
}
private static final class PerTermPayloadProcessor extends ReaderPayloadProcessor {
@Override
public PayloadProcessor getProcessor(String field, BytesRef text) {
// don't process payloads of terms other than "p:p1"
if (!field.equals("p") || !text.bytesEquals(new BytesRef("p1"))) {
return null;
}
// All other terms are processed the same way
return new DeletePayloadProcessor();
}
}
/** deletes the incoming payload */
private static final class DeletePayloadProcessor extends PayloadProcessor {
@Override
public void processPayload(BytesRef payload) {
payload.length = 0;
}
}
private static final class PayloadTokenStream extends TokenStream {
private final PayloadAttribute payload = addAttribute(PayloadAttribute.class);
private final CharTermAttribute term = addAttribute(CharTermAttribute.class);
private boolean called = false;
private String t;
public PayloadTokenStream(String t) {
this.t = t;
}
@Override
public boolean incrementToken() {
if (called) {
return false;
}
called = true;
byte[] p = new byte[] { 1 };
payload.setPayload(new BytesRef(p));
term.append(t);
return true;
}
@Override
public void reset() throws IOException {
super.reset();
called = false;
term.setEmpty();
}
}
private static final int NUM_DOCS = 10;
private void populateDirs(Random random, Directory[] dirs, boolean multipleCommits)
throws IOException {
for (int i = 0; i < dirs.length; i++) {
dirs[i] = newDirectory();
populateDocs(random, dirs[i], multipleCommits);
verifyPayloadExists(dirs[i], "p", new BytesRef("p1"), NUM_DOCS);
verifyPayloadExists(dirs[i], "p", new BytesRef("p2"), NUM_DOCS);
}
}
private void populateDocs(Random random, Directory dir, boolean multipleCommits)
throws IOException {
IndexWriter writer = new IndexWriter(
dir,
newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random, MockTokenizer.WHITESPACE, false)).
setMergePolicy(newLogMergePolicy(10))
);
TokenStream payloadTS1 = new PayloadTokenStream("p1");
TokenStream payloadTS2 = new PayloadTokenStream("p2");
FieldType customType = new FieldType(TextField.TYPE_NOT_STORED);
customType.setOmitNorms(true);
for (int i = 0; i < NUM_DOCS; i++) {
Document doc = new Document();
doc.add(newField("id", "doc" + i, customType));
doc.add(newTextField("content", "doc content " + i, Field.Store.NO));
if (random.nextBoolean()) {
doc.add(new TextField("p", payloadTS1));
doc.add(new TextField("p", payloadTS2));
} else {
FieldType type = new FieldType(TextField.TYPE_NOT_STORED);
type.setStoreTermVectors(true);
type.setStoreTermVectorPositions(true);
type.setStoreTermVectorPayloads(true);
type.setStoreTermVectorOffsets(random.nextBoolean());
doc.add(new Field("p", payloadTS1, type));
doc.add(new Field("p", payloadTS2, type));
}
writer.addDocument(doc);
if (multipleCommits && (i % 4 == 0)) {
writer.commit();
}
}
writer.close();
}
private void verifyPayloadExists(Directory dir, String field, BytesRef text, int numExpected)
throws IOException {
IndexReader reader = DirectoryReader.open(dir);
try {
int numPayloads = 0;
DocsAndPositionsEnum tpe = MultiFields.getTermPositionsEnum(reader, null, field, text);
while (tpe.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
tpe.nextPosition();
BytesRef payload = tpe.getPayload();
if (payload != null) {
assertEquals(1, payload.length);
assertEquals(1, payload.bytes[0]);
++numPayloads;
}
}
assertEquals(numExpected, numPayloads);
} finally {
reader.close();
}
}
private void doTest(Random random, boolean addToEmptyIndex,
int numExpectedPayloads, boolean multipleCommits) throws IOException {
Directory[] dirs = new Directory[2];
populateDirs(random, dirs, multipleCommits);
Directory dir = newDirectory();
if (!addToEmptyIndex) {
populateDocs(random, dir, multipleCommits);
verifyPayloadExists(dir, "p", new BytesRef("p1"), NUM_DOCS);
verifyPayloadExists(dir, "p", new BytesRef("p2"), NUM_DOCS);
}
// Add two source dirs. By not adding the dest dir, we ensure its payloads
// won't get processed.
Map<Directory, ReaderPayloadProcessor> processors = new HashMap<Directory, ReaderPayloadProcessor>();
for (Directory d : dirs) {
processors.put(d, new PerTermPayloadProcessor());
}
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random, MockTokenizer.WHITESPACE, false)));
writer.setPayloadProcessorProvider(new PerDirPayloadProcessor(processors));
IndexReader[] readers = new IndexReader[dirs.length];
for (int i = 0; i < readers.length; i++) {
readers[i] = DirectoryReader.open(dirs[i]);
}
try {
writer.addIndexes(readers);
} finally {
for (IndexReader r : readers) {
r.close();
}
}
writer.close();
verifyPayloadExists(dir, "p", new BytesRef("p1"), numExpectedPayloads);
// the second term should always have all payloads
numExpectedPayloads = NUM_DOCS * dirs.length
+ (addToEmptyIndex ? 0 : NUM_DOCS);
verifyPayloadExists(dir, "p", new BytesRef("p2"), numExpectedPayloads);
for (Directory d : dirs)
d.close();
dir.close();
}
@Test
public void testAddIndexes() throws Exception {
// addIndexes - single commit in each
doTest(random(), true, 0, false);
// addIndexes - multiple commits in each
doTest(random(), true, 0, true);
}
@Test
public void testAddIndexesIntoExisting() throws Exception {
// addIndexes - single commit in each
doTest(random(), false, NUM_DOCS, false);
// addIndexes - multiple commits in each
doTest(random(), false, NUM_DOCS, true);
}
@Test
public void testRegularMerges() throws Exception {
Directory dir = newDirectory();
populateDocs(random(), dir, true);
verifyPayloadExists(dir, "p", new BytesRef("p1"), NUM_DOCS);
verifyPayloadExists(dir, "p", new BytesRef("p2"), NUM_DOCS);
// Add two source dirs. By not adding the dest dir, we ensure its payloads
// won't get processed.
Map<Directory, ReaderPayloadProcessor> processors = new HashMap<Directory, ReaderPayloadProcessor>();
processors.put(dir, new PerTermPayloadProcessor());
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false)));
writer.setPayloadProcessorProvider(new PerDirPayloadProcessor(processors));
writer.forceMerge(1);
writer.close();
verifyPayloadExists(dir, "p", new BytesRef("p1"), 0);
verifyPayloadExists(dir, "p", new BytesRef("p2"), NUM_DOCS);
dir.close();
}
}

View File

@ -604,7 +604,7 @@ public class TestPayloads extends LuceneTestCase {
field.setTokenStream(ts);
writer.addDocument(doc);
DirectoryReader reader = writer.getReader();
AtomicReader sr = reader.getSequentialSubReaders().get(0);
AtomicReader sr = SlowCompositeReaderWrapper.wrap(reader);
DocsAndPositionsEnum de = sr.termPositionsEnum(null, "field", new BytesRef("withPayload"));
de.nextDoc();
de.nextPosition();

View File

@ -289,9 +289,9 @@ public class TestPostingsOffsets extends LuceneTestCase {
w.close();
final String[] terms = new String[] {"a", "b", "c", "d"};
for(IndexReader reader : r.getSequentialSubReaders()) {
for(AtomicReaderContext ctx : r.leaves()) {
// TODO: improve this
AtomicReader sub = (AtomicReader) reader;
AtomicReader sub = ctx.reader();
//System.out.println("\nsub=" + sub);
final TermsEnum termsEnum = sub.fields().terms("content").iterator(null);
DocsEnum docs = null;

View File

@ -25,7 +25,6 @@ import java.util.Set;
import java.util.TreeSet;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.MergedIterator;
import org.apache.lucene.util._TestUtil;
public class TestPrefixCodedTerms extends LuceneTestCase {
@ -70,14 +69,10 @@ public class TestPrefixCodedTerms extends LuceneTestCase {
@SuppressWarnings("unchecked")
public void testMergeEmpty() {
List<Iterator<Term>> subs = Collections.emptyList();
Iterator<Term> merged = new MergedIterator<Term>(subs.toArray(new Iterator[0]));
Iterator<Term> merged = new MergedIterator<Term>();
assertFalse(merged.hasNext());
subs = new ArrayList<Iterator<Term>>();
subs.add(new PrefixCodedTerms.Builder().finish().iterator());
subs.add(new PrefixCodedTerms.Builder().finish().iterator());
merged = new MergedIterator<Term>(subs.toArray(new Iterator[0]));
merged = new MergedIterator<Term>(new PrefixCodedTerms.Builder().finish().iterator(), new PrefixCodedTerms.Builder().finish().iterator());
assertFalse(merged.hasNext());
}
@ -93,18 +88,14 @@ public class TestPrefixCodedTerms extends LuceneTestCase {
b2.add(t2);
PrefixCodedTerms pb2 = b2.finish();
List<Iterator<Term>> subs = new ArrayList<Iterator<Term>>();
subs.add(pb1.iterator());
subs.add(pb2.iterator());
Iterator<Term> merged = new MergedIterator<Term>(subs.toArray(new Iterator[0]));
Iterator<Term> merged = new MergedIterator<Term>(pb1.iterator(), pb2.iterator());
assertTrue(merged.hasNext());
assertEquals(t1, merged.next());
assertTrue(merged.hasNext());
assertEquals(t2, merged.next());
}
@SuppressWarnings("unchecked")
@SuppressWarnings({"unchecked","rawtypes"})
public void testMergeRandom() {
PrefixCodedTerms pb[] = new PrefixCodedTerms[_TestUtil.nextInt(random(), 2, 10)];
Set<Term> superSet = new TreeSet<Term>();

View File

@ -82,7 +82,7 @@ public class TestSegmentMerger extends LuceneTestCase {
final SegmentInfo si = new SegmentInfo(mergedDir, Constants.LUCENE_MAIN_VERSION, mergedSegment, -1, false, codec, null, null);
SegmentMerger merger = new SegmentMerger(si, InfoStream.getDefault(), mergedDir, IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL,
MergeState.CheckAbort.NONE, null, new FieldInfos.FieldNumbers(), newIOContext(random()));
MergeState.CheckAbort.NONE, new FieldInfos.FieldNumbers(), newIOContext(random()));
merger.add(reader1);
merger.add(reader2);
MergeState mergeState = merger.merge();

View File

@ -281,10 +281,10 @@ public class TestStressIndexing2 extends LuceneTestCase {
}
private static void printDocs(DirectoryReader r) throws Throwable {
List<? extends AtomicReader> subs = r.getSequentialSubReaders();
for(IndexReader sub : subs) {
for(AtomicReaderContext ctx : r.leaves()) {
// TODO: improve this
Bits liveDocs = ((AtomicReader)sub).getLiveDocs();
AtomicReader sub = ctx.reader();
Bits liveDocs = sub.getLiveDocs();
System.out.println(" " + ((SegmentReader) sub).getSegmentInfo());
for(int docID=0;docID<sub.maxDoc();docID++) {
Document doc = sub.document(docID);

View File

@ -184,9 +184,9 @@ public class TestTermVectorsReader extends LuceneTestCase {
public void test() throws IOException {
//Check to see the files were created properly in setup
DirectoryReader reader = DirectoryReader.open(dir);
for (IndexReader r : reader.getSequentialSubReaders()) {
SegmentInfoPerCommit s = ((SegmentReader) r).getSegmentInfo();
assertTrue(((SegmentReader) r).getFieldInfos().hasVectors());
for (AtomicReaderContext ctx : reader.leaves()) {
SegmentReader sr = (SegmentReader) ctx.reader();
assertTrue(sr.getFieldInfos().hasVectors());
}
reader.close();
}

View File

@ -130,7 +130,7 @@ public class TestThreadedForceMerge extends LuceneTestCase {
OpenMode.APPEND).setMaxBufferedDocs(2));
DirectoryReader reader = DirectoryReader.open(directory);
assertEquals("reader=" + reader, 1, reader.getSequentialSubReaders().size());
assertEquals("reader=" + reader, 1, reader.leaves().size());
assertEquals(expectedDocCount, reader.numDocs());
reader.close();
}

View File

@ -120,8 +120,8 @@ public class TestTypePromotion extends LuceneTestCase {
private void assertValues(TestType type, Directory dir, long[] values, Type[] sourceType)
throws IOException {
DirectoryReader reader = DirectoryReader.open(dir);
assertEquals(1, reader.getSequentialSubReaders().size());
IndexReaderContext topReaderContext = reader.getTopReaderContext();
assertEquals(1, reader.leaves().size());
IndexReaderContext topReaderContext = reader.getContext();
List<AtomicReaderContext> leaves = topReaderContext.leaves();
assertEquals(1, leaves.size());
DocValues docValues = leaves.get(0).reader().docValues("promote");
@ -371,8 +371,8 @@ public class TestTypePromotion extends LuceneTestCase {
writer.forceMerge(1);
writer.close();
DirectoryReader reader = DirectoryReader.open(dir);
assertEquals(1, reader.getSequentialSubReaders().size());
IndexReaderContext topReaderContext = reader.getTopReaderContext();
assertEquals(1, reader.leaves().size());
IndexReaderContext topReaderContext = reader.getContext();
List<AtomicReaderContext> leaves = topReaderContext.leaves();
DocValues docValues = leaves.get(0).reader().docValues("promote");
assertNotNull(docValues);

View File

@ -43,7 +43,7 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
writer.close();
IndexReader reader = SlowCompositeReaderWrapper.wrap(DirectoryReader.open(dir));
AtomicReaderContext context = (AtomicReaderContext) reader.getTopReaderContext();
AtomicReaderContext context = (AtomicReaderContext) reader.getContext();
MockFilter filter = new MockFilter();
CachingWrapperFilter cacher = new CachingWrapperFilter(filter);
@ -69,7 +69,7 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
writer.close();
IndexReader reader = SlowCompositeReaderWrapper.wrap(DirectoryReader.open(dir));
AtomicReaderContext context = (AtomicReaderContext) reader.getTopReaderContext();
AtomicReaderContext context = (AtomicReaderContext) reader.getContext();
final Filter filter = new Filter() {
@Override
@ -92,7 +92,7 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
writer.close();
IndexReader reader = SlowCompositeReaderWrapper.wrap(DirectoryReader.open(dir));
AtomicReaderContext context = (AtomicReaderContext) reader.getTopReaderContext();
AtomicReaderContext context = (AtomicReaderContext) reader.getContext();
final Filter filter = new Filter() {
@Override
@ -115,8 +115,8 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
}
private static void assertDocIdSetCacheable(IndexReader reader, Filter filter, boolean shouldCacheable) throws IOException {
assertTrue(reader.getTopReaderContext() instanceof AtomicReaderContext);
AtomicReaderContext context = (AtomicReaderContext) reader.getTopReaderContext();
assertTrue(reader.getContext() instanceof AtomicReaderContext);
AtomicReaderContext context = (AtomicReaderContext) reader.getContext();
final CachingWrapperFilter cacher = new CachingWrapperFilter(filter);
final DocIdSet originalSet = filter.getDocIdSet(context, context.reader().getLiveDocs());
final DocIdSet cachedSet = cacher.getDocIdSet(context, context.reader().getLiveDocs());

View File

@ -197,7 +197,7 @@ public class TestNumericRangeQuery32 extends LuceneTestCase {
@Test
public void testInverseRange() throws Exception {
AtomicReaderContext context = SlowCompositeReaderWrapper.wrap(reader).getTopReaderContext();
AtomicReaderContext context = SlowCompositeReaderWrapper.wrap(reader).getContext();
NumericRangeFilter<Integer> f = NumericRangeFilter.newIntRange("field8", 8, 1000, -1000, true, true);
assertSame("A inverse range should return the EMPTY_DOCIDSET instance", DocIdSet.EMPTY_DOCIDSET, f.getDocIdSet(context, context.reader().getLiveDocs()));
f = NumericRangeFilter.newIntRange("field8", 8, Integer.MAX_VALUE, null, false, false);

View File

@ -211,7 +211,7 @@ public class TestNumericRangeQuery64 extends LuceneTestCase {
@Test
public void testInverseRange() throws Exception {
AtomicReaderContext context = SlowCompositeReaderWrapper.wrap(searcher.getIndexReader()).getTopReaderContext();
AtomicReaderContext context = SlowCompositeReaderWrapper.wrap(searcher.getIndexReader()).getContext();
NumericRangeFilter<Long> f = NumericRangeFilter.newLongRange("field8", 8, 1000L, -1000L, true, true);
assertSame("A inverse range should return the EMPTY_DOCIDSET instance", DocIdSet.EMPTY_DOCIDSET,
f.getDocIdSet(context, context.reader().getLiveDocs()));

View File

@ -22,8 +22,8 @@ import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import org.apache.lucene.index.CompositeReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReaderContext;
import org.apache.lucene.index.MultiFields;
import org.apache.lucene.index.MultiReader;
import org.apache.lucene.index.Term;
@ -310,13 +310,11 @@ public class TestShardSearching extends ShardSearchingTestBase {
final int numNodes = shardSearcher.nodeVersions.length;
int[] base = new int[numNodes];
final List<? extends IndexReader> subs = ((CompositeReader) mockSearcher.getIndexReader()).getSequentialSubReaders();
final List<IndexReaderContext> subs = mockSearcher.getTopReaderContext().children();
assertEquals(numNodes, subs.size());
int docCount = 0;
for(int nodeID=0;nodeID<numNodes;nodeID++) {
base[nodeID] = docCount;
docCount += subs.get(nodeID).maxDoc();
base[nodeID] = subs.get(nodeID).docBaseInParent;
}
if (VERBOSE) {

View File

@ -67,12 +67,12 @@ public class TestPayloadSpans extends LuceneTestCase {
SpanTermQuery stq;
Spans spans;
stq = new SpanTermQuery(new Term(PayloadHelper.FIELD, "seventy"));
spans = MultiSpansWrapper.wrap(indexReader.getTopReaderContext(), stq);
spans = MultiSpansWrapper.wrap(indexReader.getContext(), stq);
assertTrue("spans is null and it shouldn't be", spans != null);
checkSpans(spans, 100, 1, 1, 1);
stq = new SpanTermQuery(new Term(PayloadHelper.NO_PAYLOAD_FIELD, "seventy"));
spans = MultiSpansWrapper.wrap(indexReader.getTopReaderContext(), stq);
spans = MultiSpansWrapper.wrap(indexReader.getContext(), stq);
assertTrue("spans is null and it shouldn't be", spans != null);
checkSpans(spans, 100, 0, 0, 0);
}
@ -83,7 +83,7 @@ public class TestPayloadSpans extends LuceneTestCase {
SpanFirstQuery sfq;
match = new SpanTermQuery(new Term(PayloadHelper.FIELD, "one"));
sfq = new SpanFirstQuery(match, 2);
Spans spans = MultiSpansWrapper.wrap(indexReader.getTopReaderContext(), sfq);
Spans spans = MultiSpansWrapper.wrap(indexReader.getContext(), sfq);
checkSpans(spans, 109, 1, 1, 1);
//Test more complicated subclause
SpanQuery[] clauses = new SpanQuery[2];
@ -91,11 +91,11 @@ public class TestPayloadSpans extends LuceneTestCase {
clauses[1] = new SpanTermQuery(new Term(PayloadHelper.FIELD, "hundred"));
match = new SpanNearQuery(clauses, 0, true);
sfq = new SpanFirstQuery(match, 2);
checkSpans(MultiSpansWrapper.wrap(indexReader.getTopReaderContext(), sfq), 100, 2, 1, 1);
checkSpans(MultiSpansWrapper.wrap(indexReader.getContext(), sfq), 100, 2, 1, 1);
match = new SpanNearQuery(clauses, 0, false);
sfq = new SpanFirstQuery(match, 2);
checkSpans(MultiSpansWrapper.wrap(indexReader.getTopReaderContext(), sfq), 100, 2, 1, 1);
checkSpans(MultiSpansWrapper.wrap(indexReader.getContext(), sfq), 100, 2, 1, 1);
}
@ -119,7 +119,7 @@ public class TestPayloadSpans extends LuceneTestCase {
writer.close();
checkSpans(MultiSpansWrapper.wrap(reader.getTopReaderContext(), snq), 1,new int[]{2});
checkSpans(MultiSpansWrapper.wrap(reader.getContext(), snq), 1,new int[]{2});
reader.close();
directory.close();
}

View File

@ -77,9 +77,9 @@ public class TestLockFactory extends LuceneTestCase {
// exceptions raised:
// Verify: NoLockFactory allows two IndexWriters
public void testRAMDirectoryNoLocking() throws IOException {
Directory dir = new MockDirectoryWrapper(random(), new RAMDirectory());
MockDirectoryWrapper dir = new MockDirectoryWrapper(random(), new RAMDirectory());
dir.setLockFactory(NoLockFactory.getNoLockFactory());
dir.setWrapLockFactory(false); // we are gonna explicitly test we get this back
assertTrue("RAMDirectory.setLockFactory did not take",
NoLockFactory.class.isInstance(dir.getLockFactory()));

View File

@ -0,0 +1,54 @@
package org.apache.lucene.store;
/*
* 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.io.IOException;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.util.LuceneTestCase;
public class TestMockDirectoryWrapper extends LuceneTestCase {
public void testFailIfIndexWriterNotClosed() throws IOException {
MockDirectoryWrapper dir = newMockDirectory();
IndexWriter iw = new IndexWriter(dir, new IndexWriterConfig(TEST_VERSION_CURRENT, null));
try {
dir.close();
fail();
} catch (Exception expected) {
assertTrue(expected.getMessage().contains("there are still open locks"));
}
iw.close();
dir.close();
}
public void testFailIfIndexWriterNotClosedChangeLockFactory() throws IOException {
MockDirectoryWrapper dir = newMockDirectory();
dir.setLockFactory(new SingleInstanceLockFactory());
IndexWriter iw = new IndexWriter(dir, new IndexWriterConfig(TEST_VERSION_CURRENT, null));
try {
dir.close();
fail();
} catch (Exception expected) {
assertTrue(expected.getMessage().contains("there are still open locks"));
}
iw.close();
dir.close();
}
}

View File

@ -1,17 +1,20 @@
package org.apache.lucene.facet.example.merge;
import java.io.IOException;
import java.util.List;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.PayloadProcessorProvider;
import org.apache.lucene.index.MultiReader;
import org.apache.lucene.store.Directory;
import org.apache.lucene.facet.example.ExampleUtils;
import org.apache.lucene.facet.index.FacetsPayloadProcessorProvider;
import org.apache.lucene.facet.index.OrdinalMappingAtomicReader;
import org.apache.lucene.facet.index.params.DefaultFacetIndexingParams;
import org.apache.lucene.facet.index.params.FacetIndexingParams;
import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter;
import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter.DiskOrdinalMap;
import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter.MemoryOrdinalMap;
@ -84,13 +87,17 @@ public class TaxonomyMergeUtils {
// merge the taxonomies
destTaxWriter.addTaxonomy(srcTaxDir, map);
PayloadProcessorProvider payloadProcessor = new FacetsPayloadProcessorProvider(
srcIndexDir, map.getMap(), new DefaultFacetIndexingParams());
destIndexWriter.setPayloadProcessorProvider(payloadProcessor);
int ordinalMap[] = map.getMap();
FacetIndexingParams params = new DefaultFacetIndexingParams();
IndexReader reader = DirectoryReader.open(srcIndexDir);
DirectoryReader reader = DirectoryReader.open(srcIndexDir, -1);
List<AtomicReaderContext> leaves = reader.leaves();
AtomicReader wrappedLeaves[] = new AtomicReader[leaves.size()];
for (int i = 0; i < leaves.size(); i++) {
wrappedLeaves[i] = new OrdinalMappingAtomicReader(leaves.get(i).reader(), ordinalMap, params);
}
try {
destIndexWriter.addIndexes(reader);
destIndexWriter.addIndexes(new MultiReader(wrappedLeaves));
// commit changes to taxonomy and index respectively.
destTaxWriter.commit();

View File

@ -1,4 +1,4 @@
<!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Transitional//EN" "http://www.w3.org/TR/xhtml1/DTD/xhtml1-transitional.dtd">
<!DOCTYPE HTML PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
<!--
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with

View File

@ -1,197 +0,0 @@
package org.apache.lucene.facet.index;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.util.HashMap;
import java.util.Map;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.PayloadProcessorProvider;
import org.apache.lucene.index.PayloadProcessorProvider.ReaderPayloadProcessor; // javadocs
import org.apache.lucene.index.SegmentReader;
import org.apache.lucene.index.Term;
import org.apache.lucene.store.Directory;
import org.apache.lucene.facet.index.params.CategoryListParams;
import org.apache.lucene.facet.index.params.FacetIndexingParams;
import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter.OrdinalMap;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.encoding.IntDecoder;
import org.apache.lucene.util.encoding.IntEncoder;
/*
* 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.
*/
/**
* A {@link PayloadProcessorProvider} for updating facets ordinal references,
* based on an ordinal map. You should use this code in conjunction with merging
* taxonomies - after you merge taxonomies, you receive an {@link OrdinalMap}
* which maps the 'old' payloads to the 'new' ones. You can use that map to
* re-map the payloads which contain the facets information (ordinals) either
* before or while merging the indexes.
* <p>
* For re-mapping the ordinals before you merge the indexes, do the following:
*
* <pre>
* // merge the old taxonomy with the new one.
* OrdinalMap map = LuceneTaxonomyWriter.addTaxonomies();
* int[] ordmap = map.getMap();
*
* // re-map the ordinals on the old directory.
* Directory oldDir;
* FacetsPayloadProcessorProvider fppp = new FacetsPayloadProcessorProvider(
* oldDir, ordmap);
* IndexWriterConfig conf = new IndexWriterConfig(VER, ANALYZER);
* conf.setMergePolicy(new ForceOptimizeMergePolicy());
* IndexWriter writer = new IndexWriter(oldDir, conf);
* writer.setPayloadProcessorProvider(fppp);
* writer.forceMerge(1);
* writer.close();
*
* // merge that directory with the new index.
* IndexWriter newWriter; // opened on the 'new' Directory
* newWriter.addIndexes(oldDir);
* newWriter.commit();
* </pre>
*
* For re-mapping the ordinals during index merge, do the following:
*
* <pre>
* // merge the old taxonomy with the new one.
* OrdinalMap map = LuceneTaxonomyWriter.addTaxonomies();
* int[] ordmap = map.getMap();
*
* // Add the index and re-map ordinals on the go
* IndexReader r = IndexReader.open(oldDir);
* IndexWriterConfig conf = new IndexWriterConfig(VER, ANALYZER);
* IndexWriter writer = new IndexWriter(newDir, conf);
* writer.setPayloadProcessorProvider(fppp);
* writer.addIndexes(r);
* writer.commit();
* </pre>
* <p>
* <b>NOTE:</b> while the second example looks simpler, IndexWriter may trigger
* a long merge due to addIndexes. The first example avoids this perhaps
* unneeded merge, as well as can be done separately (e.g. on another node)
* before the index is merged.
*
* @lucene.experimental
*/
public class FacetsPayloadProcessorProvider extends PayloadProcessorProvider {
private final Directory workDir;
private final ReaderPayloadProcessor dirProcessor;
/**
* Construct FacetsPayloadProcessorProvider with FacetIndexingParams
*
* @param dir the {@link Directory} containing the segments to update
* @param ordinalMap an array mapping previous facets ordinals to new ones
* @param indexingParams the facets indexing parameters
*/
public FacetsPayloadProcessorProvider(Directory dir, int[] ordinalMap,
FacetIndexingParams indexingParams) {
workDir = dir;
dirProcessor = new FacetsDirPayloadProcessor(indexingParams, ordinalMap);
}
@Override
public ReaderPayloadProcessor getReaderProcessor(AtomicReader reader) throws IOException {
if (reader instanceof SegmentReader) {
if (workDir == ((SegmentReader) reader).directory()) {
return dirProcessor;
}
}
return null;
}
/**
* {@link ReaderPayloadProcessor} that processes
* facet ordinals according to the passed in {@link FacetIndexingParams}.
*/
public static class FacetsDirPayloadProcessor extends ReaderPayloadProcessor {
private final Map<Term, CategoryListParams> termMap = new HashMap<Term, CategoryListParams>(1);
private final int[] ordinalMap;
/**
* Construct FacetsDirPayloadProcessor with custom FacetIndexingParams
* @param ordinalMap an array mapping previous facets ordinals to new ones
* @param indexingParams the facets indexing parameters
*/
protected FacetsDirPayloadProcessor(FacetIndexingParams indexingParams, int[] ordinalMap) {
this.ordinalMap = ordinalMap;
for (CategoryListParams params: indexingParams.getAllCategoryListParams()) {
termMap.put(params.getTerm(), params);
}
}
@Override
public PayloadProcessor getProcessor(String field, BytesRef bytes) throws IOException {
// TODO (Facet): don't create terms
CategoryListParams params = termMap.get(new Term(field, bytes));
if (params == null) {
return null;
}
return new FacetsPayloadProcessor(params, ordinalMap);
}
}
/** A PayloadProcessor for updating facets ordinal references, based on an ordinal map */
public static class FacetsPayloadProcessor extends PayloadProcessor {
private final IntEncoder encoder;
private final IntDecoder decoder;
private final int[] ordinalMap;
private final ByteArrayOutputStream os = new ByteArrayOutputStream();
/**
* @param params defines the encoding of facet ordinals as payload
* @param ordinalMap an array mapping previous facets ordinals to new ones
*/
protected FacetsPayloadProcessor(CategoryListParams params, int[] ordinalMap) {
encoder = params.createEncoder();
decoder = encoder.createMatchingDecoder();
this.ordinalMap = ordinalMap;
}
@Override
public void processPayload(BytesRef payload) throws IOException {
InputStream is = new ByteArrayInputStream(payload.bytes, payload.offset, payload.length);
decoder.reInit(is);
os.reset();
encoder.reInit(os);
long ordinal;
while ((ordinal = decoder.decode()) != IntDecoder.EOS) {
int newOrdinal = ordinalMap[(int)ordinal];
encoder.encode(newOrdinal);
}
encoder.close();
// TODO (Facet): avoid copy?
byte out[] = os.toByteArray();
payload.bytes = out;
payload.offset = 0;
payload.length = out.length;
}
}
}

View File

@ -0,0 +1,224 @@
package org.apache.lucene.facet.index;
/*
* 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.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.util.HashMap;
import java.util.Map;
import org.apache.lucene.facet.index.params.CategoryListParams;
import org.apache.lucene.facet.index.params.DefaultFacetIndexingParams;
import org.apache.lucene.facet.index.params.FacetIndexingParams;
import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter.OrdinalMap;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.FilterAtomicReader;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.encoding.IntDecoder;
import org.apache.lucene.util.encoding.IntEncoder;
/**
* A {@link FilterAtomicReader} for updating facets ordinal references,
* based on an ordinal map. You should use this code in conjunction with merging
* taxonomies - after you merge taxonomies, you receive an {@link OrdinalMap}
* which maps the 'old' payloads to the 'new' ones. You can use that map to
* re-map the payloads which contain the facets information (ordinals) either
* before or while merging the indexes.
* <p>
* For re-mapping the ordinals during index merge, do the following:
*
* <pre class="prettyprint">
* // merge the old taxonomy with the new one.
* OrdinalMap map = DirectoryTaxonomyWriter.addTaxonomies();
* int[] ordmap = map.getMap();
*
* // Add the index and re-map ordinals on the go
* DirectoryReader reader = DirectoryReader.open(oldDir);
* IndexWriterConfig conf = new IndexWriterConfig(VER, ANALYZER);
* IndexWriter writer = new IndexWriter(newDir, conf);
* List<AtomicReaderContext> leaves = reader.leaves();
* AtomicReader wrappedLeaves[] = new AtomicReader[leaves.size()];
* for (int i = 0; i < leaves.size(); i++) {
* wrappedLeaves[i] = new OrdinalMappingAtomicReader(leaves.get(i).reader(), ordmap);
* }
* writer.addIndexes(new MultiReader(wrappedLeaves));
* writer.commit();
* </pre>
*
* @lucene.experimental
*/
public class OrdinalMappingAtomicReader extends FilterAtomicReader {
private final int[] ordinalMap;
// a little obtuse: but we dont need to create Term objects this way
private final Map<String,Map<BytesRef,CategoryListParams>> termMap =
new HashMap<String,Map<BytesRef,CategoryListParams>>(1);
/**
* Wraps an AtomicReader, mapping ordinals according to the ordinalMap.
* Calls {@link #OrdinalMappingAtomicReader(AtomicReader, int[], FacetIndexingParams)
* OrdinalMappingAtomicReader(in, ordinalMap, new DefaultFacetIndexingParams())}
*/
public OrdinalMappingAtomicReader(AtomicReader in, int[] ordinalMap) {
this(in, ordinalMap, new DefaultFacetIndexingParams());
}
/**
* Wraps an AtomicReader, mapping ordinals according to the ordinalMap,
* using the provided indexingParams.
*/
public OrdinalMappingAtomicReader(AtomicReader in, int[] ordinalMap, FacetIndexingParams indexingParams) {
super(in);
this.ordinalMap = ordinalMap;
for (CategoryListParams params: indexingParams.getAllCategoryListParams()) {
Term term = params.getTerm();
Map<BytesRef,CategoryListParams> fieldMap = termMap.get(term.field());
if (fieldMap == null) {
fieldMap = new HashMap<BytesRef,CategoryListParams>(1);
termMap.put(term.field(), fieldMap);
}
fieldMap.put(term.bytes(), params);
}
}
@Override
public Fields getTermVectors(int docID) throws IOException {
Fields fields = super.getTermVectors(docID);
if (fields == null) {
return null;
} else {
return new OrdinalMappingFields(fields);
}
}
@Override
public Fields fields() throws IOException {
Fields fields = super.fields();
if (fields == null) {
return null;
} else {
return new OrdinalMappingFields(fields);
}
}
private class OrdinalMappingFields extends FilterFields {
public OrdinalMappingFields(Fields in) {
super(in);
}
@Override
public Terms terms(String field) throws IOException {
Terms terms = super.terms(field);
if (terms == null) {
return terms;
}
Map<BytesRef,CategoryListParams> termsMap = termMap.get(field);
if (termsMap == null) {
return terms;
} else {
return new OrdinalMappingTerms(terms, termsMap);
}
}
}
private class OrdinalMappingTerms extends FilterTerms {
private final Map<BytesRef,CategoryListParams> termsMap;
public OrdinalMappingTerms(Terms in, Map<BytesRef,CategoryListParams> termsMap) {
super(in);
this.termsMap = termsMap;
}
@Override
public TermsEnum iterator(TermsEnum reuse) throws IOException {
// TODO: should we reuse the inner termsenum?
return new OrdinalMappingTermsEnum(super.iterator(reuse), termsMap);
}
}
private class OrdinalMappingTermsEnum extends FilterTermsEnum {
private final Map<BytesRef,CategoryListParams> termsMap;
public OrdinalMappingTermsEnum(TermsEnum in, Map<BytesRef,CategoryListParams> termsMap) {
super(in);
this.termsMap = termsMap;
}
@Override
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
// TODO: we could reuse our D&P enum if we need
DocsAndPositionsEnum inner = super.docsAndPositions(liveDocs, reuse, flags);
if (inner == null) {
return inner;
}
CategoryListParams params = termsMap.get(term());
if (params == null) {
return inner;
}
return new OrdinalMappingDocsAndPositionsEnum(inner, params);
}
}
private class OrdinalMappingDocsAndPositionsEnum extends FilterDocsAndPositionsEnum {
private final IntEncoder encoder;
private final IntDecoder decoder;
private final ByteArrayOutputStream os = new ByteArrayOutputStream();
private final BytesRef payloadOut = new BytesRef();
public OrdinalMappingDocsAndPositionsEnum(DocsAndPositionsEnum in, CategoryListParams params) {
super(in);
encoder = params.createEncoder();
decoder = encoder.createMatchingDecoder();
}
@Override
public BytesRef getPayload() throws IOException {
BytesRef payload = super.getPayload();
if (payload == null) {
return payload;
} else {
InputStream is = new ByteArrayInputStream(payload.bytes, payload.offset, payload.length);
decoder.reInit(is);
os.reset();
encoder.reInit(os);
long ordinal;
while ((ordinal = decoder.decode()) != IntDecoder.EOS) {
int newOrdinal = ordinalMap[(int)ordinal];
encoder.encode(newOrdinal);
}
encoder.close();
// TODO (Facet): avoid copy?
byte out[] = os.toByteArray();
payloadOut.bytes = out;
payloadOut.offset = 0;
payloadOut.length = out.length;
return payloadOut;
}
}
}
}

View File

@ -29,6 +29,7 @@ import org.apache.lucene.facet.taxonomy.writercache.TaxonomyWriterCache;
import org.apache.lucene.facet.taxonomy.writercache.cl2o.Cl2oTaxonomyWriterCache;
import org.apache.lucene.facet.taxonomy.writercache.lru.LruTaxonomyWriterCache;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.DocsEnum;
@ -409,14 +410,12 @@ public class DirectoryTaxonomyWriter implements TaxonomyWriter {
DirectoryReader reader = readerManager.acquire();
try {
final BytesRef catTerm = new BytesRef(categoryPath.toString(delimiter));
int base = 0;
for (AtomicReader r : reader.getSequentialSubReaders()) {
DocsEnum docs = r.termDocsEnum(null, Consts.FULL, catTerm, 0);
for (AtomicReaderContext ctx : reader.leaves()) {
DocsEnum docs = ctx.reader().termDocsEnum(null, Consts.FULL, catTerm, 0);
if (docs != null) {
doc = docs.nextDoc() + base;
doc = docs.nextDoc() + ctx.docBase;
break;
}
base += r.maxDoc(); // we don't have deletions, so it's ok to call maxDoc
}
} finally {
readerManager.release(reader);
@ -452,14 +451,12 @@ public class DirectoryTaxonomyWriter implements TaxonomyWriter {
DirectoryReader reader = readerManager.acquire();
try {
final BytesRef catTerm = new BytesRef(categoryPath.toString(delimiter, prefixLen));
int base = 0;
for (AtomicReader r : reader.getSequentialSubReaders()) {
DocsEnum docs = r.termDocsEnum(null, Consts.FULL, catTerm, 0);
for (AtomicReaderContext ctx : reader.leaves()) {
DocsEnum docs = ctx.reader().termDocsEnum(null, Consts.FULL, catTerm, 0);
if (docs != null) {
doc = docs.nextDoc() + base;
doc = docs.nextDoc() + ctx.docBase;
break;
}
base += r.maxDoc(); // we don't have deletions, so it's ok to call maxDoc
}
} finally {
readerManager.release(reader);
@ -752,9 +749,8 @@ public class DirectoryTaxonomyWriter implements TaxonomyWriter {
CategoryPath cp = new CategoryPath();
TermsEnum termsEnum = null;
DocsEnum docsEnum = null;
int base = 0;
for (AtomicReader r : reader.getSequentialSubReaders()) {
Terms terms = r.terms(Consts.FULL);
for (AtomicReaderContext ctx : reader.leaves()) {
Terms terms = ctx.reader().terms(Consts.FULL);
if (terms != null) { // cannot really happen, but be on the safe side
termsEnum = terms.iterator(termsEnum);
while (termsEnum.next() != null) {
@ -768,7 +764,7 @@ public class DirectoryTaxonomyWriter implements TaxonomyWriter {
cp.clear();
cp.add(t.utf8ToString(), delimiter);
docsEnum = termsEnum.docs(null, docsEnum, 0);
boolean res = cache.put(cp, docsEnum.nextDoc() + base);
boolean res = cache.put(cp, docsEnum.nextDoc() + ctx.docBase);
assert !res : "entries should not have been evicted from the cache";
} else {
// the cache is full and the next put() will evict entries from it, therefore abort the iteration.
@ -780,7 +776,6 @@ public class DirectoryTaxonomyWriter implements TaxonomyWriter {
if (aborted) {
break;
}
base += r.maxDoc(); // we don't have any deletions, so we're ok
}
} finally {
readerManager.release(reader);
@ -845,8 +840,9 @@ public class DirectoryTaxonomyWriter implements TaxonomyWriter {
int base = 0;
TermsEnum te = null;
DocsEnum docs = null;
for (AtomicReader ar : r.getSequentialSubReaders()) {
Terms terms = ar.terms(Consts.FULL);
for (final AtomicReaderContext ctx : r.leaves()) {
final AtomicReader ar = ctx.reader();
final Terms terms = ar.terms(Consts.FULL);
te = terms.iterator(te);
while (te.next() != null) {
String value = te.term().utf8ToString();

View File

@ -248,7 +248,7 @@ public class TestBlockJoin extends LuceneTestCase {
}
private Document getParentDoc(IndexReader reader, Filter parents, int childDocID) throws IOException {
final List<AtomicReaderContext> leaves = reader.getTopReaderContext().leaves();
final List<AtomicReaderContext> leaves = reader.leaves();
final int subIndex = ReaderUtil.subIndex(childDocID, leaves);
final AtomicReaderContext leaf = leaves.get(subIndex);
final FixedBitSet bits = (FixedBitSet) parents.getDocIdSet(leaf, null);
@ -962,7 +962,7 @@ public class TestBlockJoin extends LuceneTestCase {
ToParentBlockJoinQuery q = new ToParentBlockJoinQuery(tq, parentFilter, ScoreMode.Avg);
Weight weight = s.createNormalizedWeight(q);
DocIdSetIterator disi = weight.scorer(s.getIndexReader().getTopReaderContext().leaves().get(0), true, true, null);
DocIdSetIterator disi = weight.scorer(s.getIndexReader().leaves().get(0), true, true, null);
assertEquals(1, disi.advance(1));
r.close();
dir.close();
@ -996,7 +996,7 @@ public class TestBlockJoin extends LuceneTestCase {
ToParentBlockJoinQuery q = new ToParentBlockJoinQuery(tq, parentFilter, ScoreMode.Avg);
Weight weight = s.createNormalizedWeight(q);
DocIdSetIterator disi = weight.scorer(s.getIndexReader().getTopReaderContext().leaves().get(0), true, true, null);
DocIdSetIterator disi = weight.scorer(s.getIndexReader().leaves().get(0), true, true, null);
assertEquals(2, disi.advance(0));
r.close();
dir.close();

View File

@ -185,7 +185,7 @@ public class MultiPassIndexSplitter {
}
private static FakeDeleteAtomicIndexReader[] initSubReaders(IndexReader reader) {
final List<AtomicReaderContext> leaves = reader.getTopReaderContext().leaves();
final List<AtomicReaderContext> leaves = reader.leaves();
final FakeDeleteAtomicIndexReader[] subs = new FakeDeleteAtomicIndexReader[leaves.size()];
int i = 0;
for (final AtomicReaderContext ctx : leaves) {

View File

@ -102,7 +102,7 @@ public class PKIndexSplitter {
boolean success = false;
final IndexWriter w = new IndexWriter(target, config);
try {
final List<AtomicReaderContext> leaves = reader.getTopReaderContext().leaves();
final List<AtomicReaderContext> leaves = reader.leaves();
final IndexReader[] subReaders = new IndexReader[leaves.size()];
int i = 0;
for (final AtomicReaderContext ctx : leaves) {

View File

@ -179,7 +179,7 @@ public class HighFreqTerms {
public static long getTotalTermFreq(IndexReader reader, final String field, final BytesRef termText) throws Exception {
long totalTF = 0L;
for (final AtomicReaderContext ctx : reader.getTopReaderContext().leaves()) {
for (final AtomicReaderContext ctx : reader.leaves()) {
AtomicReader r = ctx.reader();
Bits liveDocs = r.getLiveDocs();
if (liveDocs == null) {

View File

@ -59,7 +59,7 @@ public class TestIndexSplitter extends LuceneTestCase {
}
iw.commit();
DirectoryReader iwReader = iw.getReader();
assertEquals(3, iwReader.getSequentialSubReaders().size());
assertEquals(3, iwReader.leaves().size());
iwReader.close();
iw.close();
// we should have 2 segments now
@ -87,7 +87,7 @@ public class TestIndexSplitter extends LuceneTestCase {
// now remove the copied segment from src
IndexSplitter.main(new String[] {dir.getAbsolutePath(), "-d", splitSegName});
r = DirectoryReader.open(fsDir);
assertEquals(2, r.getSequentialSubReaders().size());
assertEquals(2, r.leaves().size());
r.close();
fsDir.close();
}

View File

@ -132,7 +132,7 @@ public class BooleanFilterTest extends LuceneTestCase {
private void tstFilterCard(String mes, int expected, Filter filt)
throws Exception {
// BooleanFilter never returns null DIS or null DISI!
DocIdSetIterator disi = filt.getDocIdSet(reader.getTopReaderContext(), reader.getLiveDocs()).iterator();
DocIdSetIterator disi = filt.getDocIdSet(reader.getContext(), reader.getLiveDocs()).iterator();
int actual = 0;
while (disi.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
actual++;

View File

@ -62,8 +62,8 @@ public class TermsFilterTest extends LuceneTestCase {
w.addDocument(doc);
}
IndexReader reader = new SlowCompositeReaderWrapper(w.getReader());
assertTrue(reader.getTopReaderContext() instanceof AtomicReaderContext);
AtomicReaderContext context = (AtomicReaderContext) reader.getTopReaderContext();
assertTrue(reader.getContext() instanceof AtomicReaderContext);
AtomicReaderContext context = (AtomicReaderContext) reader.getContext();
w.close();
TermsFilter tf = new TermsFilter();
@ -110,7 +110,7 @@ public class TermsFilterTest extends LuceneTestCase {
tf.addTerm(new Term(fieldName, "content1"));
MultiReader multi = new MultiReader(reader1, reader2);
for (AtomicReaderContext context : multi.getTopReaderContext().leaves()) {
for (AtomicReaderContext context : multi.leaves()) {
FixedBitSet bits = (FixedBitSet) tf.getDocIdSet(context, context.reader().getLiveDocs());
assertTrue("Must be >= 0", bits.cardinality() >= 0);
}

View File

@ -66,7 +66,7 @@ public class TestNumericRangeFilterBuilder extends LuceneTestCase {
try {
AtomicReader reader = new SlowCompositeReaderWrapper(DirectoryReader.open(ramDir));
try {
assertNull(filter.getDocIdSet(reader.getTopReaderContext(), reader.getLiveDocs()));
assertNull(filter.getDocIdSet(reader.getContext(), reader.getLiveDocs()));
}
finally {
reader.close();

View File

@ -498,7 +498,7 @@ public class SpellChecker implements java.io.Closeable {
final IndexReader reader = searcher.getIndexReader();
if (reader.maxDoc() > 0) {
for (final AtomicReaderContext ctx : reader.getTopReaderContext().leaves()) {
for (final AtomicReaderContext ctx : reader.leaves()) {
Terms terms = ctx.reader().terms(F_WORD);
if (terms != null)
termsEnums.add(terms.iterator(null));

View File

@ -334,15 +334,15 @@ public abstract class ThreadedIndexingAndSearchingTestCase extends LuceneTestCas
// Verify 1) IW is correctly setting
// diagnostics, and 2) segment warming for
// merged segments is actually happening:
for(AtomicReader sub : ((DirectoryReader) s.getIndexReader()).getSequentialSubReaders()) {
SegmentReader segReader = (SegmentReader) sub;
for(final AtomicReaderContext sub : s.getIndexReader().leaves()) {
SegmentReader segReader = (SegmentReader) sub.reader();
Map<String,String> diagnostics = segReader.getSegmentInfo().info.getDiagnostics();
assertNotNull(diagnostics);
String source = diagnostics.get("source");
assertNotNull(source);
if (source.equals("merge")) {
assertTrue("sub reader " + sub + " wasn't warmed: warmed=" + warmed + " diagnostics=" + diagnostics + " si=" + segReader.getSegmentInfo(),
!assertMergedSegmentsWarmed || warmed.containsKey(((SegmentReader) sub).core));
!assertMergedSegmentsWarmed || warmed.containsKey(segReader.core));
}
}
if (s.getIndexReader().numDocs() > 0) {

View File

@ -353,7 +353,7 @@ public class QueryUtils {
IndexSearcher indexSearcher = LuceneTestCase.newSearcher(previousReader, false);
indexSearcher.setSimilarity(s.getSimilarity());
Weight w = indexSearcher.createNormalizedWeight(q);
AtomicReaderContext ctx = previousReader.getTopReaderContext();
AtomicReaderContext ctx = previousReader.getContext();
Scorer scorer = w.scorer(ctx, true, false, ctx.reader().getLiveDocs());
if (scorer != null) {
boolean more = scorer.advance(lastDoc[0] + 1) != DocIdSetIterator.NO_MORE_DOCS;

View File

@ -179,7 +179,7 @@ public abstract class ShardSearchingTestBase extends LuceneTestCase {
}
try {
for(Term term : terms) {
final TermContext termContext = TermContext.build(s.getIndexReader().getTopReaderContext(), term, false);
final TermContext termContext = TermContext.build(s.getIndexReader().getContext(), term, false);
stats.put(term, s.termStatistics(term, termContext));
}
} finally {

View File

@ -67,6 +67,7 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
boolean noDeleteOpenFile = true;
boolean preventDoubleWrite = true;
boolean trackDiskUsage = false;
boolean wrapLockFactory = true;
private Set<String> unSyncedFiles;
private Set<String> createdFiles;
private Set<String> openFilesForWrite = new HashSet<String>();
@ -114,11 +115,7 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
this.throttledOutput = new ThrottledIndexOutput(ThrottledIndexOutput
.mBitsToBytes(40 + randomState.nextInt(10)), 5 + randomState.nextInt(5), null);
// force wrapping of lockfactory
try {
setLockFactory(new MockLockFactoryWrapper(this, delegate.getLockFactory()));
} catch (IOException e) {
throw new RuntimeException(e);
}
this.lockFactory = new MockLockFactoryWrapper(this, delegate.getLockFactory());
// 2% of the time use rate limiter
if (randomState.nextInt(50) == 17) {
@ -530,6 +527,19 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
public void setAssertNoUnrefencedFilesOnClose(boolean v) {
assertNoUnreferencedFilesOnClose = v;
}
/**
* Set to false if you want to return the pure lockfactory
* and not wrap it with MockLockFactoryWrapper.
* <p>
* Be careful if you turn this off: MockDirectoryWrapper might
* no longer be able to detect if you forget to close an IndexWriter,
* and spit out horribly scary confusing exceptions instead of
* simply telling you that.
*/
public void setWrapLockFactory(boolean v) {
this.wrapLockFactory = v;
}
@Override
public synchronized void close() throws IOException {
@ -699,25 +709,33 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
@Override
public synchronized Lock makeLock(String name) {
maybeYield();
return delegate.makeLock(name);
return getLockFactory().makeLock(name);
}
@Override
public synchronized void clearLock(String name) throws IOException {
maybeYield();
delegate.clearLock(name);
getLockFactory().clearLock(name);
}
@Override
public synchronized void setLockFactory(LockFactory lockFactory) throws IOException {
maybeYield();
// sneaky: we must pass the original this way to the dir, because
// some impls (e.g. FSDir) do instanceof here.
delegate.setLockFactory(lockFactory);
// now set our wrapped factory here
this.lockFactory = new MockLockFactoryWrapper(this, lockFactory);
}
@Override
public synchronized LockFactory getLockFactory() {
maybeYield();
return delegate.getLockFactory();
if (wrapLockFactory) {
return lockFactory;
} else {
return delegate.getLockFactory();
}
}
@Override

View File

@ -504,10 +504,10 @@ public abstract class LuceneTestCase extends Assert {
* do tests on that segment's reader. This is an utility method to help them.
*/
public static SegmentReader getOnlySegmentReader(DirectoryReader reader) {
List<? extends IndexReader> subReaders = reader.getSequentialSubReaders();
List<AtomicReaderContext> subReaders = reader.leaves();
if (subReaders.size() != 1)
throw new IllegalArgumentException(reader + " has " + subReaders.size() + " segments instead of exactly one");
final IndexReader r = subReaders.get(0);
final AtomicReader r = subReaders.get(0).reader();
assertTrue(r instanceof SegmentReader);
return (SegmentReader) r;
}
@ -1166,7 +1166,7 @@ public abstract class LuceneTestCase extends Assert {
if (maybeWrap) {
r = maybeWrapReader(r);
}
IndexSearcher ret = random.nextBoolean() ? new AssertingIndexSearcher(random, r) : new AssertingIndexSearcher(random, r.getTopReaderContext());
IndexSearcher ret = random.nextBoolean() ? new AssertingIndexSearcher(random, r) : new AssertingIndexSearcher(random, r.getContext());
ret.setSimilarity(classEnvRule.similarity);
return ret;
} else {
@ -1195,7 +1195,7 @@ public abstract class LuceneTestCase extends Assert {
}
IndexSearcher ret = random.nextBoolean()
? new AssertingIndexSearcher(random, r, ex)
: new AssertingIndexSearcher(random, r.getTopReaderContext(), ex);
: new AssertingIndexSearcher(random, r.getContext(), ex);
ret.setSimilarity(classEnvRule.similarity);
return ret;
}

View File

@ -35,6 +35,11 @@ public class QuickPatchThreadsFilter implements ThreadFilter {
if (t.getName().equals("MemoryPoolMXBean notification dispatcher")) {
return true;
}
// forked process reaper on Unixish systems
if (t.getName().equals("process reaper")) {
return true;
}
return false;
}

View File

@ -31,6 +31,15 @@ Upgrading from Solr 4.0.0-BETA
In order to better support distributed search mode, the TermVectorComponent's response format has been changed so that if the schema defines a uniqueKeyField, then that field value is used as the "key" for each document in it's response section, instead of the internal lucene doc id. Users w/o a uniqueKeyField will continue to see the same response format. See SOLR-3229 for more details.
Detailed Change List
----------------------
New Features
----------------------
* SOLR-3670: New CountFieldValuesUpdateProcessorFactory makes it easy to index
the number of values in another field for later use at query time. (hossman)
Optimizations
----------------------

View File

@ -19,6 +19,9 @@ package org.apache.solr.client.solrj.embedded;
import java.io.IOException;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.Iterator;
import java.util.Map;
import java.util.Random;
import javax.servlet.DispatcherType;
@ -27,7 +30,8 @@ import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
import org.apache.solr.servlet.SolrDispatchFilter;
import org.eclipse.jetty.server.*;
import org.eclipse.jetty.server.Connector;
import org.eclipse.jetty.server.Server;
import org.eclipse.jetty.server.bio.SocketConnector;
import org.eclipse.jetty.server.handler.GzipHandler;
import org.eclipse.jetty.server.session.HashSessionIdManager;
@ -43,6 +47,8 @@ import org.eclipse.jetty.util.thread.QueuedThreadPool;
* @since solr 1.3
*/
public class JettySolrRunner {
static Map<JettySolrRunner,Exception> RUNNING_JETTIES = new HashMap<JettySolrRunner,Exception>();
Server server;
FilterHolder dispatchFilter;
@ -202,6 +208,7 @@ public class JettySolrRunner {
if (!server.isRunning()) {
server.start();
RUNNING_JETTIES.put(this, new RuntimeException());
}
synchronized (JettySolrRunner.this) {
int cnt = 0;
@ -220,9 +227,19 @@ public class JettySolrRunner {
public void stop() throws Exception {
if (!server.isStopped() && !server.isStopping()) {
server.stop();
RUNNING_JETTIES.remove(this);
}
server.join();
}
public static void assertStoppedJetties() {
if (RUNNING_JETTIES.size() > 0) {
Iterator<Exception> stacktraces = RUNNING_JETTIES.values().iterator();
Exception cause = null;
cause = stacktraces.next();
throw new RuntimeException("Found a bad one!", cause);
}
}
/**
* Returns the Local Port of the jetty Server.

View File

@ -25,6 +25,7 @@ import java.util.Map.Entry;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.ClosableThread;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkCoreNodeProps;
@ -47,7 +48,7 @@ public class Overseer {
private static Logger log = LoggerFactory.getLogger(Overseer.class);
private class ClusterStateUpdater implements Runnable {
private class ClusterStateUpdater implements Runnable, ClosableThread {
private static final String DELETECORE = "deletecore";
private final ZkStateReader reader;
@ -58,6 +59,7 @@ public class Overseer {
//Internal queue where overseer stores events that have not yet been published into cloudstate
//If Overseer dies while extracting the main queue a new overseer will start from this queue
private final DistributedQueue workQueue;
private volatile boolean isClosed;
public ClusterStateUpdater(final ZkStateReader reader, final String myId) {
this.zkClient = reader.getZkClient();
@ -70,7 +72,7 @@ public class Overseer {
@Override
public void run() {
if(amILeader() && !Overseer.this.isClosed) {
if(!this.isClosed && amILeader()) {
// see if there's something left from the previous Overseer and re
// process all events that were not persisted into cloud state
synchronized (reader.getUpdateLock()) { //XXX this only protects against edits inside single node
@ -110,7 +112,7 @@ public class Overseer {
}
log.info("Starting to work on the main queue");
while (amILeader() && !isClosed) {
while (!this.isClosed && amILeader()) {
synchronized (reader.getUpdateLock()) {
try {
byte[] head = stateUpdateQueue.peek();
@ -399,12 +401,48 @@ public class Overseer {
ClusterState newState = new ClusterState(clusterState.getLiveNodes(), newStates);
return newState;
}
@Override
public void close() {
this.isClosed = true;
}
@Override
public boolean isClosed() {
return this.isClosed;
}
}
private Thread ccThread;
class OverseerThread extends Thread implements ClosableThread {
private Thread updaterThread;
private volatile boolean isClosed;
public OverseerThread(ThreadGroup tg,
ClusterStateUpdater clusterStateUpdater) {
super(tg, clusterStateUpdater);
}
public OverseerThread(ThreadGroup ccTg,
OverseerCollectionProcessor overseerCollectionProcessor, String string) {
super(ccTg, overseerCollectionProcessor, string);
}
@Override
public void close() {
this.isClosed = true;
}
@Override
public boolean isClosed() {
return this.isClosed;
}
}
private OverseerThread ccThread;
private OverseerThread updaterThread;
private volatile boolean isClosed;
@ -425,11 +463,11 @@ public class Overseer {
createOverseerNode(reader.getZkClient());
//launch cluster state updater thread
ThreadGroup tg = new ThreadGroup("Overseer state updater.");
updaterThread = new Thread(tg, new ClusterStateUpdater(reader, id));
updaterThread = new OverseerThread(tg, new ClusterStateUpdater(reader, id));
updaterThread.setDaemon(true);
ThreadGroup ccTg = new ThreadGroup("Overseer collection creation process.");
ccThread = new Thread(ccTg, new OverseerCollectionProcessor(reader, id, shardHandler, adminPath),
ccThread = new OverseerThread(ccTg, new OverseerCollectionProcessor(reader, id, shardHandler, adminPath),
"Overseer-" + id);
ccThread.setDaemon(true);
@ -439,6 +477,14 @@ public class Overseer {
public void close() {
isClosed = true;
if (updaterThread != null) {
updaterThread.close();
updaterThread.interrupt();
}
if (ccThread != null) {
ccThread.close();
ccThread.interrupt();
}
}
/**

View File

@ -31,7 +31,7 @@ import org.apache.solr.client.solrj.request.CoreAdminRequest.WaitForState;
import org.apache.solr.client.solrj.request.UpdateRequest;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.cloud.SafeStopThread;
import org.apache.solr.common.cloud.ClosableThread;
import org.apache.solr.common.cloud.ZkCoreNodeProps;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
@ -56,7 +56,7 @@ import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class RecoveryStrategy extends Thread implements SafeStopThread {
public class RecoveryStrategy extends Thread implements ClosableThread {
private static final int MAX_RETRIES = 500;
private static final int INTERRUPTED = MAX_RETRIES + 1;
private static final int STARTING_RECOVERY_DELAY = 1000;
@ -139,7 +139,7 @@ public class RecoveryStrategy extends Thread implements SafeStopThread {
solrParams.set(ReplicationHandler.MASTER_URL, leaderUrl);
if (isClosed()) retries = INTERRUPTED;
boolean success = replicationHandler.doFetch(solrParams, true); // TODO: look into making sure force=true does not download files we already have
boolean success = replicationHandler.doFetch(solrParams, true); // TODO: look into making force=true not download files we already have?
if (!success) {
throw new SolrException(ErrorCode.SERVER_ERROR, "Replication for recovery failed.");
@ -292,9 +292,6 @@ public class RecoveryStrategy extends Thread implements SafeStopThread {
while (!successfulRecovery && !isClosed() && !isInterrupted()) { // don't use interruption or it will close channels though
try {
// first thing we just try to sync
zkController.publish(core.getCoreDescriptor(), ZkStateReader.RECOVERING);
CloudDescriptor cloudDesc = core.getCoreDescriptor()
.getCloudDescriptor();
ZkNodeProps leaderprops = zkStateReader.getLeaderProps(
@ -305,6 +302,19 @@ public class RecoveryStrategy extends Thread implements SafeStopThread {
String leaderUrl = ZkCoreNodeProps.getCoreUrl(leaderBaseUrl, leaderCoreName);
String ourUrl = ZkCoreNodeProps.getCoreUrl(baseUrl, coreName);
boolean isLeader = leaderUrl.equals(ourUrl);
if (isLeader) {
// we are now the leader - no one else must have been suitable
log.warn("We have not yet recovered - but we are now the leader! core=" + coreName);
log.info("Finished recovery process. core=" + coreName);
zkController.publish(core.getCoreDescriptor(), ZkStateReader.ACTIVE);
return;
}
zkController.publish(core.getCoreDescriptor(), ZkStateReader.RECOVERING);
sendPrepRecoveryCmd(leaderBaseUrl, leaderCoreName);
@ -358,7 +368,19 @@ public class RecoveryStrategy extends Thread implements SafeStopThread {
log.info("Begin buffering updates. core=" + coreName);
ulog.bufferUpdates();
replayed = false;
// // open a new IndexWriter - we don't want any background merges ongoing
// // also ensures something like NRTCachingDirectory is flushed
// boolean forceNewIndexDir = false;
// try {
// core.getUpdateHandler().newIndexWriter(false);
// } catch (Throwable t) {
// SolrException.log(log, "Could not read the current index - replicating to a new directory", t);
// // something is wrong with the index
// // we need to force using a new index directory
// forceNewIndexDir = true;
// }
//
try {
replicate(zkController.getNodeName(), core,
@ -377,7 +399,7 @@ public class RecoveryStrategy extends Thread implements SafeStopThread {
log.warn("Recovery was interrupted", e);
retries = INTERRUPTED;
} catch (Throwable t) {
log.error("Error while trying to recover", t);
SolrException.log(log, "Error while trying to recover", t);
} finally {
if (!replayed) {
try {
@ -390,7 +412,7 @@ public class RecoveryStrategy extends Thread implements SafeStopThread {
}
} catch (Throwable t) {
log.error("Error while trying to recover. core=" + coreName, t);
SolrException.log(log, "Error while trying to recover. core=" + coreName, t);
}
if (!successfulRecovery) {
@ -405,7 +427,7 @@ public class RecoveryStrategy extends Thread implements SafeStopThread {
if (retries == INTERRUPTED) {
} else {
log.error("Recovery failed - max retries exceeded. core=" + coreName);
SolrException.log(log, "Recovery failed - max retries exceeded. core=" + coreName);
recoveryFailed(core, zkController, baseUrl, coreZkNodeName,
core.getCoreDescriptor());
}
@ -413,7 +435,7 @@ public class RecoveryStrategy extends Thread implements SafeStopThread {
}
} catch (Exception e) {
log.error("core=" + coreName, e);
SolrException.log(log, "core=" + coreName, e);
}
try {

View File

@ -247,15 +247,7 @@ public final class ZkController {
} catch(Throwable t) {
log.error("Error closing overseer", t);
}
try {
zkClient.close();
} catch (InterruptedException e) {
// Restore the interrupted status
Thread.currentThread().interrupt();
log.warn("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
}
zkClient.close();
}
/**

View File

@ -105,7 +105,11 @@ public abstract class CachingDirectoryFactory extends DirectoryFactory {
public void close() throws IOException {
synchronized (this) {
for (CacheValue val : byDirectoryCache.values()) {
val.directory.close();
try {
val.directory.close();
} catch (Throwable t) {
SolrException.log(log, "Error closing directory", t);
}
}
byDirectoryCache.clear();
byPathCache.clear();

View File

@ -283,7 +283,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
private volatile SnapPuller tempSnapPuller;
public boolean doFetch(SolrParams solrParams, boolean force) {
public boolean doFetch(SolrParams solrParams, boolean forceReplication) {
String masterUrl = solrParams == null ? null : solrParams.get(MASTER_URL);
if (!snapPullLock.tryLock())
return false;
@ -294,7 +294,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
nl.remove(SnapPuller.POLL_INTERVAL);
tempSnapPuller = new SnapPuller(nl, this, core);
}
return tempSnapPuller.fetchLatestIndex(core, force);
return tempSnapPuller.fetchLatestIndex(core, forceReplication);
} catch (Exception e) {
SolrException.log(LOG, "SnapPull failed ", e);
} finally {

View File

@ -243,12 +243,11 @@ public class SnapPuller {
* downloaded. It also downloads the conf files (if they are modified).
*
* @param core the SolrCore
* @param force force a replication in all cases
* @param forceReplication force a replication in all cases
* @return true on success, false if slave is already in sync
* @throws IOException if an exception occurs
*/
@SuppressWarnings("unchecked")
boolean fetchLatestIndex(SolrCore core, boolean force) throws IOException, InterruptedException {
boolean fetchLatestIndex(SolrCore core, boolean forceReplication) throws IOException, InterruptedException {
successfulInstall = false;
replicationStartTime = System.currentTimeMillis();
try {
@ -278,7 +277,7 @@ public class SnapPuller {
}
if (latestVersion == 0L) {
if (force && commit.getGeneration() != 0) {
if (forceReplication && commit.getGeneration() != 0) {
// since we won't get the files for an empty index,
// we just clear ours and commit
RefCounted<IndexWriter> iw = core.getUpdateHandler().getSolrCoreState().getIndexWriter(core);
@ -297,7 +296,7 @@ public class SnapPuller {
return true;
}
if (!force && IndexDeletionPolicyWrapper.getCommitTimestamp(commit) == latestVersion) {
if (!forceReplication && IndexDeletionPolicyWrapper.getCommitTimestamp(commit) == latestVersion) {
//master and slave are already in sync just return
LOG.info("Slave in sync with master.");
successfulInstall = true;
@ -318,10 +317,11 @@ public class SnapPuller {
filesDownloaded = Collections.synchronizedList(new ArrayList<Map<String, Object>>());
// if the generateion of master is older than that of the slave , it means they are not compatible to be copied
// then a new index direcory to be created and all the files need to be copied
boolean isFullCopyNeeded = IndexDeletionPolicyWrapper.getCommitTimestamp(commit) >= latestVersion || force;
boolean isFullCopyNeeded = IndexDeletionPolicyWrapper.getCommitTimestamp(commit) >= latestVersion || forceReplication;
File tmpIndexDir = createTempindexDir(core);
if (isIndexStale())
if (isIndexStale()) {
isFullCopyNeeded = true;
}
LOG.info("Starting download to " + tmpIndexDir + " fullCopy=" + isFullCopyNeeded);
successfulInstall = false;
boolean deleteTmpIdxDir = true;

View File

@ -31,6 +31,7 @@ import org.apache.lucene.document.Field;
import org.apache.lucene.index.*;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.similarities.Similarity;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CharsRef;
@ -425,6 +426,7 @@ public class LukeRequestHandler extends RequestHandlerBase
field.add("className", ft.getClass().getName());
field.add("indexAnalyzer", getAnalyzerInfo(ft.getAnalyzer()));
field.add("queryAnalyzer", getAnalyzerInfo(ft.getQueryAnalyzer()));
field.add("similarity", getSimilarityInfo(ft.getSimilarity()));
types.add( ft.getTypeName(), field );
}
@ -450,6 +452,14 @@ public class LukeRequestHandler extends RequestHandlerBase
return finfo;
}
private static SimpleOrderedMap<Object> getSimilarityInfo(Similarity similarity) {
SimpleOrderedMap<Object> toReturn = new SimpleOrderedMap<Object>();
if (similarity != null) {
toReturn.add("className", similarity.getClass().getName());
toReturn.add("details", similarity.toString());
}
return toReturn;
}
private static SimpleOrderedMap<Object> getAnalyzerInfo(Analyzer analyzer) {
SimpleOrderedMap<Object> aninfo = new SimpleOrderedMap<Object>();
@ -544,7 +554,7 @@ public class LukeRequestHandler extends RequestHandlerBase
indexInfo.add("maxDoc", reader.maxDoc());
indexInfo.add("version", reader.getVersion()); // TODO? Is this different then: IndexReader.getCurrentVersion( dir )?
indexInfo.add("segmentCount", reader.getTopReaderContext().leaves().size());
indexInfo.add("segmentCount", reader.leaves().size());
indexInfo.add("current", reader.isCurrent() );
indexInfo.add("hasDeletions", reader.hasDeletions() );
indexInfo.add("directory", dir );

View File

@ -679,7 +679,7 @@ public class SpellCheckComponent extends SearchComponent implements SolrCoreAwar
if (buildOnCommit) {
buildSpellIndex(newSearcher);
} else if (buildOnOptimize) {
if (newSearcher.getIndexReader().getSequentialSubReaders().size() == 1) {
if (newSearcher.getIndexReader().leaves().size() == 1) {
buildSpellIndex(newSearcher);
} else {
LOG.info("Index is not optimized therefore skipping building spell check index for: " + checker.getDictionaryName());

View File

@ -254,7 +254,7 @@ class ExtendedBufferedReader extends BufferedReader {
* Skips all chars in the input until (but excluding) the given char
*
* @param c
* @return
* @return counter
* @throws IllegalArgumentException
* @throws IOException
*/

View File

@ -63,7 +63,7 @@ public class ValueSourceAugmenter extends DocTransformer
public void setContext( TransformContext context ) {
try {
IndexReader reader = qparser.getReq().getSearcher().getIndexReader();
readerContexts = reader.getTopReaderContext().leaves();
readerContexts = reader.leaves();
docValuesArr = new FunctionValues[readerContexts.size()];
searcher = qparser.getReq().getSearcher();

View File

@ -958,7 +958,7 @@ class ExtendedDismaxQParser extends QParser {
* Returns the aliases found for a field.
* Returns null if there are no aliases for the field
* @param field
* @return
* @return Alias
*/
public Alias getAlias(String field) {
return aliases.get(field);

View File

@ -17,6 +17,16 @@
package org.apache.solr.servlet;
import java.io.IOException;
import java.io.PrintWriter;
import java.net.URLEncoder;
import java.util.Date;
import java.util.List;
import javax.servlet.http.HttpServlet;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
import org.apache.lucene.util.BytesRef;
import org.apache.noggit.CharArr;
import org.apache.noggit.JSONWriter;
@ -28,16 +38,6 @@ import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.servlet.http.HttpServlet;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
import java.io.IOException;
import java.io.PrintWriter;
import java.net.URLEncoder;
import java.util.Date;
import java.util.List;
import java.util.concurrent.TimeoutException;
/**
* Zookeeper Info
@ -148,13 +148,7 @@ public final class ZookeeperInfoServlet extends HttpServlet {
try {
zkClient = new SolrZkClient(addr, 10000);
doClose = true;
} catch (TimeoutException e) {
writeError(503, "Could not connect to zookeeper at '" + addr + "'\"");
zkClient = null;
return;
} catch (InterruptedException e) {
// Restore the interrupted status
Thread.currentThread().interrupt();
} catch (Exception e) {
writeError(503, "Could not connect to zookeeper at '" + addr + "'\"");
zkClient = null;
return;
@ -163,12 +157,8 @@ public final class ZookeeperInfoServlet extends HttpServlet {
}
public void close() {
try {
if (doClose) {
zkClient.close();
}
} catch (InterruptedException e) {
// ignore exception on close
if (doClose) {
zkClient.close();
}
}

View File

@ -97,12 +97,14 @@ public final class DefaultSolrCoreState extends SolrCoreState {
@Override
public synchronized void newIndexWriter(SolrCore core, boolean rollback) throws IOException {
log.info("Creating new IndexWriter...");
String coreName = core.getName();
synchronized (writerPauseLock) {
// we need to wait for the Writer to fall out of use
// first lets stop it from being lent out
pauseWriter = true;
// then lets wait until its out of use
log.info("Waiting until IndexWriter is unused... core=" + coreName);
while (!writerFree) {
try {
writerPauseLock.wait();
@ -112,14 +114,15 @@ public final class DefaultSolrCoreState extends SolrCoreState {
try {
if (indexWriter != null) {
try {
log.info("Closing old IndexWriter... core=" + coreName);
indexWriter.close();
} catch (Throwable t) {
SolrException.log(log, "Error closing old IndexWriter", t);
SolrException.log(log, "Error closing old IndexWriter. core=" + coreName, t);
}
}
indexWriter = createMainIndexWriter(core, "DirectUpdateHandler2",
false, true);
log.info("New IndexWriter is ready to be used.");
// we need to null this so it picks up the new writer next get call
refCntWriter = null;
} finally {
@ -136,6 +139,7 @@ public final class DefaultSolrCoreState extends SolrCoreState {
refCnt--;
if (refCnt == 0) {
try {
log.info("SolrCoreState ref count has reached 0 - closing IndexWriter");
if (closer != null) {
closer.closeWriter(indexWriter);
} else if (indexWriter != null) {

View File

@ -0,0 +1,95 @@
/*
* 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.solr.update.processor;
import org.apache.solr.core.SolrCore;
import org.apache.solr.schema.IndexSchema;
import org.apache.solr.schema.FieldType;
import org.apache.solr.schema.SchemaField;
import org.apache.solr.schema.TextField;
import org.apache.solr.schema.StrField;
import org.apache.solr.common.SolrInputField;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.response.SolrQueryResponse;
import org.apache.commons.lang.StringUtils;
/**
* <p>
* Replaces any list of values for a field matching the specified
* conditions with the the count of the number of values for that field.
* </p>
* <p>
* By default, this processor matches no fields.
* </p>
* <p>
* The typical use case for this processor would be in combination with the
* {@link CloneFieldUpdateProcessorFactory} so that it's possible to query by
* the quantity of values in the source field.
* <p>
* For example, in the configuration below, the end result will be that the
* <code>category_count</code> field can be used to search for documents based
* on how many values they contain in the <code>category</code> field.
* </p>
*
* <pre class="prettyprint">
* &lt;updateRequestProcessorChain&gt;
* &lt;processor class="solr.CloneFieldUpdateProcessorFactory"&gt;
* &lt;str name="source"&gt;category&lt;/str&gt;
* &lt;str name="dest"&gt;category_count&lt;/str&gt;
* &lt;/processor&gt;
* &lt;processor class="solr.CountFieldValuesUpdateProcessorFactory"&gt;
* &lt;str name="fieldName"&gt;category_count&lt;/str&gt;
* &lt;/processor&gt;
* &lt;processor class="solr.DefaultValueUpdateProcessorFactory"&gt;
* &lt;str name="fieldName"&gt;category_count&lt;/str&gt;
* &lt;int name="value"&gt;0&lt;/int&gt;
* &lt;/processor&gt;
* &lt;/updateRequestProcessorChain&gt;
* </pre>
*
* <p>
* <b>NOTE:</b> The use of {@link DefaultValueUpdateProcessorFactory} is
* important in this example to ensure that all documents have a value for the
* <code>category_count</code> field, because
* <code>CountFieldValuesUpdateProcessorFactory</code> only <i>replaces</i> the
* list of values with the size of that list. If
* <code>DefaultValueUpdateProcessorFactory</code> was not used, then any
* document that had no values for the <code>category</code> field, would also
* have no value in the <code>category_count</code> field.
* </p>
*/
public final class CountFieldValuesUpdateProcessorFactory extends FieldMutatingUpdateProcessorFactory {
@Override
public UpdateRequestProcessor getInstance(SolrQueryRequest req,
SolrQueryResponse rsp,
UpdateRequestProcessor next) {
return new FieldMutatingUpdateProcessor(getSelector(), next) {
protected SolrInputField mutate(final SolrInputField src) {
SolrInputField result = new SolrInputField(src.getName());
result.setValue(src.getValueCount(),
src.getBoost());
return result;
}
};
}
}

Some files were not shown because too many files have changed in this diff Show More