Index: src/test/org/apache/lucene/TestExternalCodecs.java =================================================================== --- src/test/org/apache/lucene/TestExternalCodecs.java (revision 904750) +++ src/test/org/apache/lucene/TestExternalCodecs.java (working copy) @@ -846,7 +846,7 @@ // Verify sort order matches what my comparator said: BytesRef lastBytesRef = null; - TermsEnum terms = r.fields().terms("id").iterator(); + TermsEnum terms = MultiFields.getFields(r).terms("id").iterator(); //System.out.println("id terms:"); while(true) { BytesRef t = terms.next(); Index: src/test/org/apache/lucene/index/TestIndexWriter.java =================================================================== --- src/test/org/apache/lucene/index/TestIndexWriter.java (revision 904750) +++ src/test/org/apache/lucene/index/TestIndexWriter.java (working copy) @@ -4685,7 +4685,7 @@ } private void checkTermsOrder(IndexReader r, Set allTerms, boolean isTop) throws IOException { - TermsEnum terms = r.fields().terms("f").iterator(); + TermsEnum terms = MultiFields.getFields(r).terms("f").iterator(); char[] last = new char[2]; int lastLength = 0; Index: src/test/org/apache/lucene/index/TestBackwardsCompatibility.java =================================================================== --- src/test/org/apache/lucene/index/TestBackwardsCompatibility.java (revision 904750) +++ src/test/org/apache/lucene/index/TestBackwardsCompatibility.java (working copy) @@ -670,7 +670,7 @@ String fullPath = fullDir(oldNames[i]); Directory dir = FSDirectory.open(new File(fullPath)); IndexReader r = IndexReader.open(dir); - TermsEnum terms = r.fields().terms("content").iterator(); + TermsEnum terms = MultiFields.getFields(r).terms("content").iterator(); BytesRef t = terms.next(); assertNotNull(t); Index: src/java/org/apache/lucene/search/FilteredTermsEnum.java =================================================================== --- src/java/org/apache/lucene/search/FilteredTermsEnum.java (revision 904750) +++ src/java/org/apache/lucene/search/FilteredTermsEnum.java (working copy) @@ -22,6 +22,7 @@ import org.apache.lucene.index.Terms; import org.apache.lucene.util.BytesRef; import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.index.MultiFields; import org.apache.lucene.index.DocsEnum; import org.apache.lucene.index.DocsAndPositionsEnum; import org.apache.lucene.util.AttributeSource; @@ -62,8 +63,12 @@ * Creates a filtered {@link TermsEnum} for the given field name and reader. */ public FilteredTermsEnum(final IndexReader reader, final String field) throws IOException { - final Terms terms = reader.fields().terms(field); - tenum = (terms != null) ? terms.iterator() : null; + final Terms terms = MultiFields.getTerms(reader, field); + if (terms != null) { + tenum = terms.iterator(); + } else { + tenum = null; + } } /** Index: src/java/org/apache/lucene/search/spans/SpanTermQuery.java =================================================================== --- src/java/org/apache/lucene/search/spans/SpanTermQuery.java (revision 904750) +++ src/java/org/apache/lucene/search/spans/SpanTermQuery.java (working copy) @@ -19,6 +19,8 @@ import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.Term; +import org.apache.lucene.index.DocsAndPositionsEnum; +import org.apache.lucene.index.MultiFields; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.ToStringUtils; @@ -40,7 +42,7 @@ @Override public void extractTerms(Set terms) { - terms.add(term); + terms.add(term); } @Override @@ -81,10 +83,17 @@ @Override public Spans getSpans(final IndexReader reader) throws IOException { - return new TermSpans(reader.termPositionsEnum(reader.getDeletedDocs(), - term.field(), - new BytesRef(term.text())), - term); + // NOTE: debateably, the caller should never pass in a + // multi reader... + final DocsAndPositionsEnum postings = MultiFields.getTermPositionsEnum(reader, + reader.getDeletedDocs(), + term.field(), + new BytesRef(term.text())); + + if (postings != null) { + return new TermSpans(postings, term); + } else { + return TermSpans.EMPTY_TERM_SPANS; + } } - } Index: src/java/org/apache/lucene/search/spans/Spans.java =================================================================== --- src/java/org/apache/lucene/search/spans/Spans.java (revision 904746) +++ src/java/org/apache/lucene/search/spans/Spans.java (working copy) @@ -86,5 +86,4 @@ * @return true if there is a payload available at this position that can be loaded */ public abstract boolean isPayloadAvailable(); - } Index: src/java/org/apache/lucene/search/spans/TermSpans.java =================================================================== --- src/java/org/apache/lucene/search/spans/TermSpans.java (revision 904750) +++ src/java/org/apache/lucene/search/spans/TermSpans.java (working copy) @@ -18,6 +18,7 @@ import org.apache.lucene.index.Term; import org.apache.lucene.index.DocsAndPositionsEnum; +import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.util.BytesRef; import java.io.IOException; @@ -42,6 +43,12 @@ doc = -1; } + // only for EmptyTermSpans (below) + TermSpans() { + term = null; + postings = null; + } + @Override public boolean next() throws IOException { if (count == freq) { @@ -119,4 +126,44 @@ public DocsAndPositionsEnum getPostings() { return postings; } + + private static final class EmptyTermSpans extends TermSpans { + + @Override + public boolean next() { + return false; + } + + @Override + public boolean skipTo(int target) { + return false; + } + + @Override + public int doc() { + return DocIdSetIterator.NO_MORE_DOCS; + } + + @Override + public int start() { + return -1; + } + + @Override + public int end() { + return -1; + } + + @Override + public Collection getPayload() { + return null; + } + + @Override + public boolean isPayloadAvailable() { + return false; + } + } + + public static final TermSpans EMPTY_TERM_SPANS = new EmptyTermSpans(); } Index: src/java/org/apache/lucene/search/PrefixQuery.java =================================================================== --- src/java/org/apache/lucene/search/PrefixQuery.java (revision 904746) +++ src/java/org/apache/lucene/search/PrefixQuery.java (working copy) @@ -23,6 +23,7 @@ import org.apache.lucene.index.Terms; import org.apache.lucene.index.TermsEnum; import org.apache.lucene.index.Term; +import org.apache.lucene.index.MultiFields; import org.apache.lucene.util.ToStringUtils; /** A Query that matches documents containing terms with a specified prefix. A PrefixQuery @@ -51,7 +52,7 @@ @Override protected TermsEnum getTermsEnum(IndexReader reader) throws IOException { if (prefix.text().length() == 0) { - final Terms terms = reader.fields().terms(getField()); + final Terms terms = MultiFields.getTerms(reader, getField()); return (terms != null) ? terms.iterator() : new EmptyTermsEnum(); } return new PrefixTermsEnum(reader, prefix); Index: src/java/org/apache/lucene/search/PhraseQuery.java =================================================================== --- src/java/org/apache/lucene/search/PhraseQuery.java (revision 904750) +++ src/java/org/apache/lucene/search/PhraseQuery.java (working copy) @@ -25,6 +25,7 @@ import org.apache.lucene.util.BytesRef; import org.apache.lucene.index.DocsAndPositionsEnum; import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.MultiFields; import org.apache.lucene.search.Explanation.IDFExplanation; import org.apache.lucene.util.ToStringUtils; import org.apache.lucene.util.Bits; @@ -157,11 +158,17 @@ for (int i = 0; i < terms.size(); i++) { final Term t = terms.get(i); final BytesRef text = new BytesRef(t.text()); - DocsAndPositionsEnum postingsEnum = reader.termPositionsEnum(delDocs, - t.field(), - text); + // NOTE: debateably, the caller should never pass in a + // multi reader... + DocsAndPositionsEnum postingsEnum = MultiFields.getTermPositionsEnum(reader, + delDocs, + t.field(), + text); if (postingsEnum == null) { - if (reader.termDocsEnum(delDocs, t.field(), text) != null) { + if (MultiFields.getTermDocsEnum(reader, + delDocs, + t.field(), + text) != null) { // term does exist, but has no positions throw new IllegalStateException("field \"" + t.field() + "\" was indexed with Field.omitTermFreqAndPositions=true; cannot run PhraseQuery (term=" + text + ")"); } else { Index: src/java/org/apache/lucene/search/TermRangeQuery.java =================================================================== --- src/java/org/apache/lucene/search/TermRangeQuery.java (revision 904746) +++ src/java/org/apache/lucene/search/TermRangeQuery.java (working copy) @@ -23,6 +23,7 @@ import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.TermsEnum; import org.apache.lucene.index.Terms; +import org.apache.lucene.index.MultiFields; import org.apache.lucene.util.ToStringUtils; /** @@ -140,7 +141,9 @@ return new EmptyTermsEnum(); } if ((lowerTerm == null || (collator == null && includeLower && "".equals(lowerTerm))) && upperTerm == null) { - final Terms terms = reader.fields().terms(field); + // NOTE: debateably, the caller should never pass in a + // multi reader... + final Terms terms = MultiFields.getTerms(reader, field); return (terms != null) ? terms.iterator() : new EmptyTermsEnum(); } return new TermRangeTermsEnum(reader, field, Index: src/java/org/apache/lucene/search/TermQuery.java =================================================================== --- src/java/org/apache/lucene/search/TermQuery.java (revision 904746) +++ src/java/org/apache/lucene/search/TermQuery.java (working copy) @@ -22,6 +22,7 @@ import org.apache.lucene.index.DocsEnum; import org.apache.lucene.index.Term; +import org.apache.lucene.index.MultiFields; import org.apache.lucene.util.BytesRef; import org.apache.lucene.index.IndexReader; import org.apache.lucene.search.Explanation.IDFExplanation; @@ -72,7 +73,9 @@ @Override public Scorer scorer(IndexReader reader, boolean scoreDocsInOrder, boolean topScorer) throws IOException { - DocsEnum docs = reader.termDocsEnum(reader.getDeletedDocs(), term.field(), new BytesRef(term.text())); + // NOTE: debateably, the caller should never pass in a + // multi reader... + DocsEnum docs = MultiFields.getTermDocsEnum(reader, reader.getDeletedDocs(), term.field(), new BytesRef(term.text())); if (docs == null) { return null; } Index: src/java/org/apache/lucene/search/FieldCacheImpl.java =================================================================== --- src/java/org/apache/lucene/search/FieldCacheImpl.java (revision 904750) +++ src/java/org/apache/lucene/search/FieldCacheImpl.java (working copy) @@ -28,6 +28,7 @@ import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.DocsEnum; import org.apache.lucene.index.Terms; +import org.apache.lucene.index.MultiFields; import org.apache.lucene.index.TermsEnum; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.Bits; @@ -280,7 +281,7 @@ return wrapper.getBytes(reader, field, FieldCache.DEFAULT_BYTE_PARSER); } final byte[] retArray = new byte[reader.maxDoc()]; - Terms terms = reader.fields().terms(field); + Terms terms = MultiFields.getTerms(reader, field); if (terms != null) { final TermsEnum termsEnum = terms.iterator(); final Bits delDocs = reader.getDeletedDocs(); @@ -334,7 +335,7 @@ return wrapper.getShorts(reader, field, FieldCache.DEFAULT_SHORT_PARSER); } final short[] retArray = new short[reader.maxDoc()]; - Terms terms = reader.fields().terms(field); + Terms terms = MultiFields.getTerms(reader, field); if (terms != null) { final TermsEnum termsEnum = terms.iterator(); final Bits delDocs = reader.getDeletedDocs(); @@ -393,7 +394,7 @@ } int[] retArray = null; - Terms terms = reader.fields().terms(field); + Terms terms = MultiFields.getTerms(reader, field); if (terms != null) { final TermsEnum termsEnum = terms.iterator(); final Bits delDocs = reader.getDeletedDocs(); @@ -465,7 +466,7 @@ } float[] retArray = null; - Terms terms = reader.fields().terms(field); + Terms terms = MultiFields.getTerms(reader, field); if (terms != null) { final TermsEnum termsEnum = terms.iterator(); final Bits delDocs = reader.getDeletedDocs(); @@ -532,7 +533,8 @@ } } long[] retArray = null; - Terms terms = reader.fields().terms(field); + + Terms terms = MultiFields.getTerms(reader, field); if (terms != null) { final TermsEnum termsEnum = terms.iterator(); final Bits delDocs = reader.getDeletedDocs(); @@ -601,7 +603,8 @@ } } double[] retArray = null; - Terms terms = reader.fields().terms(field); + + Terms terms = MultiFields.getTerms(reader, field); if (terms != null) { final TermsEnum termsEnum = terms.iterator(); final Bits delDocs = reader.getDeletedDocs(); @@ -653,7 +656,7 @@ String field = StringHelper.intern(entryKey.field); final String[] retArray = new String[reader.maxDoc()]; - Terms terms = reader.fields().terms(field); + Terms terms = MultiFields.getTerms(reader, field); if (terms != null) { final TermsEnum termsEnum = terms.iterator(); final Bits delDocs = reader.getDeletedDocs(); @@ -697,7 +700,7 @@ String[] mterms = new String[reader.maxDoc()+1]; //System.out.println("FC: getStringIndex field=" + field); - Terms terms = reader.fields().terms(field); + Terms terms = MultiFields.getTerms(reader, field); int t = 0; // current term number Index: src/java/org/apache/lucene/search/AutomatonQuery.java =================================================================== --- src/java/org/apache/lucene/search/AutomatonQuery.java (revision 904746) +++ src/java/org/apache/lucene/search/AutomatonQuery.java (working copy) @@ -23,6 +23,7 @@ import org.apache.lucene.index.Term; import org.apache.lucene.index.Terms; import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.index.MultiFields; import org.apache.lucene.util.ToStringUtils; import org.apache.lucene.util.automaton.Automaton; import org.apache.lucene.util.automaton.BasicAutomata; @@ -72,12 +73,14 @@ protected TermsEnum getTermsEnum(IndexReader reader) throws IOException { // matches nothing if (BasicOperations.isEmpty(automaton)) { + // nocommit -- should we just return null? singleton? return new EmptyTermsEnum(); } // matches all possible strings if (BasicOperations.isTotal(automaton)) { - final Terms terms = reader.fields().terms(getField()); + final Terms terms = MultiFields.getTerms(reader, getField()); + // nocommit -- should we just return null? singleton? return (terms != null) ? terms.iterator() : new EmptyTermsEnum(); } Index: src/java/org/apache/lucene/index/DocInverterPerField.java =================================================================== --- src/java/org/apache/lucene/index/DocInverterPerField.java (revision 904746) +++ src/java/org/apache/lucene/index/DocInverterPerField.java (working copy) @@ -116,8 +116,9 @@ reader = readerValue; else { String stringValue = field.stringValue(); - if (stringValue == null) + if (stringValue == null) { throw new IllegalArgumentException("field must have either TokenStream, String or Reader value"); + } perThread.stringReader.init(stringValue); reader = perThread.stringReader; } @@ -132,8 +133,6 @@ final int startLength = fieldState.length; try { - int offsetEnd = fieldState.offset-1; - boolean hasMoreTokens = stream.incrementToken(); fieldState.attributeSource = stream; @@ -178,7 +177,6 @@ docState.docWriter.setAborting(); } fieldState.position++; - offsetEnd = fieldState.offset + offsetAttribute.endOffset(); if (++fieldState.length >= maxFieldLength) { if (docState.infoStream != null) docState.infoStream.println("maxFieldLength " +maxFieldLength+ " reached for field " + fieldInfo.name + ", ignoring following tokens"); Index: src/java/org/apache/lucene/index/FieldsEnum.java =================================================================== --- src/java/org/apache/lucene/index/FieldsEnum.java (revision 904750) +++ src/java/org/apache/lucene/index/FieldsEnum.java (working copy) @@ -28,6 +28,8 @@ public abstract class FieldsEnum { + public final static FieldsEnum[] EMPTY_ARRAY = new FieldsEnum[0]; + private AttributeSource atts = null; /** Index: src/java/org/apache/lucene/index/MultiTerms.java =================================================================== --- src/java/org/apache/lucene/index/MultiTerms.java (revision 0) +++ src/java/org/apache/lucene/index/MultiTerms.java (revision 0) @@ -0,0 +1,84 @@ +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 org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.ReaderUtil; + +import java.io.IOException; +import java.util.List; +import java.util.ArrayList; + +/** + * Exposes flex API, merged from flex API of + * sub-segments. + * + * @lucene.experimental + */ + +public final class MultiTerms extends Terms { + private final Terms[] subs; + private final ReaderUtil.Slice[] subSlices; + private final BytesRef.Comparator termComp; + + public MultiTerms(Terms[] subs, ReaderUtil.Slice[] subSlices) throws IOException { + this.subs = subs; + this.subSlices = subSlices; + + BytesRef.Comparator _termComp = null; + for(int i=0;i termsEnums = new ArrayList(); + for(int i=0;i 0) { + return new MultiTermsEnum(subSlices).reset(termsEnums.toArray(MultiTermsEnum.TermsEnumIndex.EMPTY_ARRAY)); + } else { + return null; + } + } + + @Override + public BytesRef.Comparator getComparator() { + return termComp; + } +} + Property changes on: src/java/org/apache/lucene/index/MultiTerms.java ___________________________________________________________________ Added: svn:eol-style + native Index: src/java/org/apache/lucene/index/ParallelReader.java =================================================================== --- src/java/org/apache/lucene/index/ParallelReader.java (revision 904746) +++ src/java/org/apache/lucene/index/ParallelReader.java (working copy) @@ -147,7 +147,6 @@ String currentField; IndexReader currentReader; Iterator keys; - private final HashMap readerFields = new HashMap(); ParallelFieldsEnum() { keys = fieldToReader.keySet().iterator(); @@ -168,7 +167,7 @@ @Override public TermsEnum terms() throws IOException { assert currentReader != null; - Terms terms = currentReader.fields().terms(currentField); + Terms terms = MultiFields.getTerms(currentReader, currentField); if (terms != null) { return terms.iterator(); } else { @@ -182,7 +181,7 @@ final HashMap fields = new HashMap(); public void addField(String field, IndexReader r) throws IOException { - fields.put(field, r.fields().terms(field)); + fields.put(field, MultiFields.getFields(r).terms(field)); } @Override Index: src/java/org/apache/lucene/index/MultiDocsEnum.java =================================================================== --- src/java/org/apache/lucene/index/MultiDocsEnum.java (revision 0) +++ src/java/org/apache/lucene/index/MultiDocsEnum.java (revision 0) @@ -0,0 +1,113 @@ +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 org.apache.lucene.util.Bits; +import org.apache.lucene.util.ReaderUtil; +import java.io.IOException; + +/** + * Exposes flex API, merged from flex API of sub-segments. + * + * @lucene.experimental + */ + +public final class MultiDocsEnum extends DocsEnum { + private EnumWithSlice[] subs; + int numSubs; + int upto; + DocsEnum current; + int currentBase; + Bits skipDocs; + int doc = -1; + + MultiDocsEnum reset(final EnumWithSlice[] subs, final int numSubs) throws IOException { + this.numSubs = numSubs; + this.subs = subs; + upto = -1; + current = null; + return this; + } + + public int getNumSubs() { + return numSubs; + } + + public EnumWithSlice[] getSubs() { + return subs; + } + + @Override + public int freq() { + return current.freq(); + } + + @Override + public int docID() { + return doc; + } + + @Override + public int advance(int target) throws IOException { + while(true) { + if (current != null) { + final int doc = current.advance(target-currentBase); + if (doc == NO_MORE_DOCS) { + current = null; + } else { + return this.doc = doc + currentBase; + } + } else if (upto == numSubs-1) { + return this.doc = NO_MORE_DOCS; + } else { + upto++; + current = subs[upto].docsEnum; + currentBase = subs[upto].slice.start; + } + } + } + + @Override + public int nextDoc() throws IOException { + while(true) { + if (current == null) { + if (upto == numSubs-1) { + return this.doc = NO_MORE_DOCS; + } else { + upto++; + current = subs[upto].docsEnum; + currentBase = subs[upto].slice.start; + } + } + + final int doc = current.nextDoc(); + if (doc != NO_MORE_DOCS) { + return this.doc = currentBase + doc; + } else { + current = null; + } + } + } + + // TODO: implement bulk read more efficiently than super + public final static class EnumWithSlice { + public DocsEnum docsEnum; + public ReaderUtil.Slice slice; + } +} + Property changes on: src/java/org/apache/lucene/index/MultiDocsEnum.java ___________________________________________________________________ Added: svn:eol-style + native Index: src/java/org/apache/lucene/index/Terms.java =================================================================== --- src/java/org/apache/lucene/index/Terms.java (revision 904750) +++ src/java/org/apache/lucene/index/Terms.java (working copy) @@ -29,6 +29,8 @@ public abstract class Terms { + public final static Terms[] EMPTY_ARRAY = new Terms[0]; + // Privately cache a TermsEnum per-thread for looking up // docFreq and getting a private DocsEnum private final CloseableThreadLocal threadEnums = new CloseableThreadLocal(); Index: src/java/org/apache/lucene/index/MultiReader.java =================================================================== --- src/java/org/apache/lucene/index/MultiReader.java (revision 904750) +++ src/java/org/apache/lucene/index/MultiReader.java (working copy) @@ -25,29 +25,29 @@ import org.apache.lucene.document.Document; import org.apache.lucene.document.FieldSelector; -import org.apache.lucene.index.DirectoryReader.MultiBits; -import org.apache.lucene.index.DirectoryReader.MultiFields; -import org.apache.lucene.index.DirectoryReader.MultiTermDocs; -import org.apache.lucene.index.DirectoryReader.MultiTermEnum; -import org.apache.lucene.index.DirectoryReader.MultiTermPositions; +import org.apache.lucene.index.DirectoryReader.MultiTermDocs; // deprecated +import org.apache.lucene.index.DirectoryReader.MultiTermEnum; // deprecated +import org.apache.lucene.index.DirectoryReader.MultiTermPositions; // deprecated import org.apache.lucene.search.Similarity; -import org.apache.lucene.util.Bits; import org.apache.lucene.search.FieldCache; // not great (circular); used only to purge FieldCache entry on close +import org.apache.lucene.util.MultiBits; +import org.apache.lucene.util.Bits; import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.ReaderUtil; /** An IndexReader which reads multiple indexes, appending - * their content. */ + * their content. */ public class MultiReader extends IndexReader implements Cloneable { protected IndexReader[] subReaders; private int[] starts; // 1st docno for each segment - private final Map subReaderToDocBase = new HashMap(); + private final Map subReaderToSlice = new HashMap(); private boolean[] decrefOnClose; // remember which subreaders to decRef on close private Map normsCache = new HashMap(); private int maxDoc = 0; private int numDocs = -1; private boolean hasDeletions = false; private MultiBits deletedDocs; - private MultiFields fields; + //private Fields fields; /** *

Construct a MultiReader aggregating the named set of (sub)readers. @@ -79,6 +79,7 @@ starts = new int[subReaders.length + 1]; // build starts array decrefOnClose = new boolean[subReaders.length]; Bits[] subs = new Bits[subReaders.length]; + for (int i = 0; i < subReaders.length; i++) { starts[i] = maxDoc; maxDoc += subReaders[i].maxDoc(); // compute maxDocs @@ -94,7 +95,11 @@ hasDeletions = true; } subs[i] = subReaders[i].getDeletedDocs(); - subReaderToDocBase.put(subReaders[i], Integer.valueOf(starts[i])); + + final ReaderUtil.Slice slice = new ReaderUtil.Slice(starts[i], + subReaders[i].maxDoc(), + i); + subReaderToSlice.put(subReaders[i], slice); } starts[subReaders.length] = maxDoc; @@ -103,22 +108,16 @@ } else { deletedDocs = null; } - fields = new MultiFields(subReaders, starts); } @Override public int getSubReaderDocBase(IndexReader subReader) { - return subReaderToDocBase.get(subReader).intValue(); + return subReaderToSlice.get(subReader).start; } @Override public Fields fields() throws IOException { - if (subReaders.length == 1) { - // Optimize the single reader case - return subReaders[0].fields(); - } else { - return fields; - } + throw new UnsupportedOperationException("please use MultiFields.getFields if you really need a top level Fields for this reader"); } /** Index: src/java/org/apache/lucene/index/DirectoryReader.java =================================================================== --- src/java/org/apache/lucene/index/DirectoryReader.java (revision 904750) +++ src/java/org/apache/lucene/index/DirectoryReader.java (working copy) @@ -36,10 +36,10 @@ import org.apache.lucene.store.Lock; import org.apache.lucene.store.LockObtainFailedException; import org.apache.lucene.index.codecs.Codecs; -import org.apache.lucene.util.PriorityQueue; import org.apache.lucene.util.Bits; import org.apache.lucene.util.ReaderUtil; import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.MultiBits; import org.apache.lucene.search.FieldCache; // not great (circular); used only to purge FieldCache entry on close @@ -66,14 +66,12 @@ private SegmentReader[] subReaders; private int[] starts; // 1st docno for each segment - private final Map subReaderToDocBase = new HashMap(); + private final Map subReaderToSlice = new HashMap(); private Map normsCache = new HashMap(); private int maxDoc = 0; private int numDocs = -1; private boolean hasDeletions = false; - private MultiFields fields; - // static IndexReader open(final Directory directory, final IndexDeletionPolicy deletionPolicy, final IndexCommit commit, final boolean readOnly, // final int termInfosIndexDivisor) throws CorruptIndexException, IOException { // return open(directory, deletionPolicy, commit, readOnly, termInfosIndexDivisor, null); @@ -359,19 +357,31 @@ return buffer.toString(); } - private void initialize(SegmentReader[] subReaders) { + private void initialize(SegmentReader[] subReaders) throws IOException { this.subReaders = subReaders; starts = new int[subReaders.length + 1]; // build starts array Bits[] subs = new Bits[subReaders.length]; + + final List subFields = new ArrayList(); + final List fieldSlices = new ArrayList(); + for (int i = 0; i < subReaders.length; i++) { starts[i] = maxDoc; maxDoc += subReaders[i].maxDoc(); // compute maxDocs if (subReaders[i].hasDeletions()) { hasDeletions = true; + subs[i] = subReaders[i].getDeletedDocs(); } - subs[i] = subReaders[i].getDeletedDocs(); - subReaderToDocBase.put(subReaders[i], Integer.valueOf(starts[i])); + + final ReaderUtil.Slice slice = new ReaderUtil.Slice(starts[i], subReaders[i].maxDoc(), i); + subReaderToSlice.put(subReaders[i], slice); + + final Fields f = subReaders[i].fields(); + if (f != null) { + subFields.add(f); + fieldSlices.add(slice); + } } starts[subReaders.length] = maxDoc; @@ -380,66 +390,13 @@ } else { deletedDocs = null; } - - fields = new MultiFields(subReaders, starts); } - private MultiBits deletedDocs; + private Bits deletedDocs; - // Exposes a slice of an existing Bits as a new Bits. - // Only used when one provides an external skipDocs (ie, - // not the del docs from this DirectoryReader), to pull - // the DocsEnum of the sub readers - private final static class SubBits implements Bits { - private final Bits parent; - private final int start; - private final int length; - - // start is inclusive; end is exclusive (length = end-start) - public SubBits(Bits parent, int start, int length) { - this.parent = parent; - this.start = start; - this.length = length; - assert length >= 0: "length=" + length; - } - - public boolean get(int doc) { - if (doc >= length) { - throw new RuntimeException("doc " + doc + " is out of bounds 0 .. " + (length-1)); - } - assert doc < length: "doc=" + doc + " length=" + length; - return parent.get(doc+start); - } - } - - // Concatenates multiple Bits together - // nocommit -- if none of the subs have deletions we - // should return null from getDeletedDocs: - static final class MultiBits implements Bits { - private final Bits[] subs; - // this is 1+subs.length, ie the last entry has the maxDoc - final int[] starts; - - public MultiBits(Bits[] subs, int[] starts) { - this.subs = subs; - this.starts = starts; - } - - public boolean get(int doc) { - final int reader = ReaderUtil.subIndex(doc, starts); - final Bits bits = subs[reader]; - if (bits == null) { - return false; - } else { - final int length = starts[1+reader]-starts[reader]; - assert doc - starts[reader] < length: "doc=" + doc + " reader=" + reader + " starts[reader]=" + starts[reader] + " length=" + length; - return bits.get(doc-starts[reader]); - } - } - } - @Override public Bits getDeletedDocs() { + // nocommit -- maybe not supported? return deletedDocs; } @@ -828,11 +785,10 @@ @Override public Fields fields() throws IOException { - if (subReaders.length == 1) { - // Optimize the single reader case - return subReaders[0].fields(); + if (subReaders.length == 0) { + return null; } else { - return fields; + throw new UnsupportedOperationException("please use MultiFields.getFields if you really need a top level Fields for this reader"); } } @@ -901,6 +857,11 @@ */ @Override protected void doCommit(Map commitUserData) throws IOException { + // poll subreaders for changes + for (int i = 0; !hasChanges && i < subReaders.length; i++) { + hasChanges |= subReaders[i].hasChanges; + } + if (hasChanges) { segmentInfos.setUserData(commitUserData); // Default deleter (for backwards compatibility) is @@ -1051,7 +1012,7 @@ @Override public int getSubReaderDocBase(IndexReader subReader) { - return subReaderToDocBase.get(subReader).intValue(); + return subReaderToSlice.get(subReader).start; } /** Returns the directory this index resides in. */ @@ -1187,572 +1148,6 @@ } } - private final static class TermsWithBase { - Terms terms; - int base; - int length; - Bits skipDocs; - - public TermsWithBase(IndexReader reader, int base, String field) throws IOException { - this.base = base; - length = reader.maxDoc(); - assert length >= 0: "length=" + length; - skipDocs = reader.getDeletedDocs(); - terms = reader.fields().terms(field); - } - } - - private final static class FieldsEnumWithBase { - FieldsEnum fields; - String current; - int base; - int length; - Bits skipDocs; - - public FieldsEnumWithBase(IndexReader reader, int base) throws IOException { - this.base = base; - length = reader.maxDoc(); - assert length >= 0: "length=" + length; - skipDocs = reader.getDeletedDocs(); - fields = reader.fields().iterator(); - } - } - - private final static class TermsEnumWithBase { - final TermsEnum terms; - final int base; - final int length; - BytesRef current; - final Bits skipDocs; - - public TermsEnumWithBase(FieldsEnumWithBase start, TermsEnum terms, BytesRef term) { - this.terms = terms; - current = term; - skipDocs = start.skipDocs; - base = start.base; - length = start.length; - assert length >= 0: "length=" + length; - } - - public TermsEnumWithBase(TermsWithBase start, TermsEnum terms, BytesRef term) { - this.terms = terms; - current = term; - skipDocs = start.skipDocs; - base = start.base; - length = start.length; - assert length >= 0: "length=" + length; - } - } - - private final static class PostingsEnumWithBase { - DocsAndPositionsEnum postings; - int base; - } - - private final static class FieldMergeQueue extends PriorityQueue { - FieldMergeQueue(int size) { - initialize(size); - } - - @Override - protected final boolean lessThan(FieldsEnumWithBase fieldsA, FieldsEnumWithBase fieldsB) { - return fieldsA.current.compareTo(fieldsB.current) < 0; - } - } - - private final static class TermMergeQueue extends PriorityQueue { - BytesRef.Comparator termComp; - TermMergeQueue(int size) { - initialize(size); - } - - @Override - protected final boolean lessThan(TermsEnumWithBase termsA, TermsEnumWithBase termsB) { - final int cmp = termComp.compare(termsA.current, termsB.current); - if (cmp != 0) { - return cmp < 0; - } else { - return termsA.base < termsB.base; - } - } - } - - // Exposes flex API, merged from flex API of - // sub-segments. - final static class MultiFields extends Fields { - private final IndexReader[] readers; - private final int[] starts; - private final HashMap terms = new HashMap(); - - public MultiFields(IndexReader[] readers, int[] starts) { - this.readers = readers; - this.starts = starts; - } - - @Override - public FieldsEnum iterator() throws IOException { - FieldsEnumWithBase[] subs = new FieldsEnumWithBase[readers.length]; - for(int i=0;i subs = new ArrayList(); - - // Gather all sub-readers that share this field - for(int i=0;i 0; - return currentField; - } - - @Override - public String next() throws IOException { - - // restore queue - for(int i=0;i 0) { - while(true) { - top[numTop++] = queue.pop(); - if (queue.size() == 0 || (queue.top()).current != top[0].current) { - break; - } - } - currentField = top[0].current; - } else { - currentField = null; - } - - return currentField; - } - - @Override - public TermsEnum terms() throws IOException { - return terms.reset(top, numTop); - } - } - - // Exposes flex API, merged from flex API of - // sub-segments. This does a merge sort, by term text, of - // the sub-readers. - private static final class MultiTermsEnum extends TermsEnum { - - private final TermMergeQueue queue; - private final TermsEnumWithBase[] subs; - private final TermsEnumWithBase[] top; - int numTop; - int numSubs; - private BytesRef current; - private BytesRef.Comparator termComp; - - MultiTermsEnum(int size) { - queue = new TermMergeQueue(size); - top = new TermsEnumWithBase[size]; - subs = new TermsEnumWithBase[size]; - } - - @Override - public BytesRef term() { - return current; - } - - @Override - public BytesRef.Comparator getComparator() { - return termComp; - } - - MultiTermsEnum reset(TermsWithBase[] terms) throws IOException { - assert terms.length <= top.length; - numSubs = 0; - numTop = 0; - termComp = null; - queue.clear(); - for(int i=0;i 0) { - current = term; - return SeekStatus.FOUND; - } else if (queue.size() > 0) { - pullTop(); - return SeekStatus.NOT_FOUND; - } else { - return SeekStatus.END; - } - } - - @Override - public SeekStatus seek(long ord) throws IOException { - throw new UnsupportedOperationException(); - } - - @Override - public long ord() throws IOException { - throw new UnsupportedOperationException(); - } - - private final void pullTop() { - // extract all subs from the queue that have the same - // top term - assert numTop == 0; - while(true) { - top[numTop++] = queue.pop(); - if (queue.size() == 0 || !(queue.top()).current.bytesEquals(top[0].current)) { - break; - } - } - current = top[0].current; - } - - private final void pushTop() throws IOException { - // call next() on each top, and put back into queue - for(int i=0;i 0) { - pullTop(); - } else { - current = null; - } - - return current; - } - - @Override - public int docFreq() { - int sum = 0; - for(int i=0;i= 0: "subs[" + i + " of " + numSubs + "].length=" + subs[i].length; - - // Optimize for common case: requested skip docs is - // simply our (DiretoryReader's) deleted docs. In - // this case, we just pull the skipDocs from the sub - // reader, rather than making the inefficient - // Sub(Multi(sub-readers)): - if (skipDocs instanceof MultiBits) { - MultiBits multiBits = (MultiBits) skipDocs; - int reader = ReaderUtil.subIndex(subs[i].base, multiBits.starts); - assert reader < multiBits.starts.length-1: " reader=" + reader + " multiBits.starts.length=" + multiBits.starts.length; - final int length = multiBits.starts[reader+1] - multiBits.starts[reader]; - if (multiBits.starts[reader] == subs[i].base && - length == subs[i].length) { - bits = multiBits.subs[reader]; - handled = true; - } - } - - if (!handled && skipDocs != null) { - // custom case: requested skip docs is foreign - bits = new SubBits(skipDocs, subs[i].base, subs[i].length); - } - - final DocsAndPositionsEnum postings = subs[i].terms.docsAndPositions(bits, null); - if (postings != null) { - this.subs[this.numSubs] = new PostingsEnumWithBase(); - this.subs[this.numSubs].postings = postings; - this.subs[this.numSubs].base = subs[i].base; - this.numSubs++; - } - } - upto = -1; - currentDocs = null; - return this; - } - - @Override - public int freq() { - return currentDocs.freq(); - } - - @Override - public int docID() { - return doc; - } - - @Override - public int advance(int target) throws IOException { - while(true) { - if (currentDocs != null) { - final int doc = currentDocs.advance(target-currentBase); - if (doc == NO_MORE_DOCS) { - currentDocs = null; - } else { - return this.doc = doc + currentBase; - } - } else if (upto == numSubs-1) { - return this.doc = NO_MORE_DOCS; - } else { - upto++; - currentDocs = subs[upto].postings; - currentBase = subs[upto].base; - } - } - } - - @Override - public int nextDoc() throws IOException { - while(true) { - if (currentDocs == null) { - if (upto == numSubs-1) { - return this.doc = NO_MORE_DOCS; - } else { - upto++; - currentDocs = subs[upto].postings; - currentBase = subs[upto].base; - } - } - - final int doc = currentDocs.nextDoc(); - if (doc != NO_MORE_DOCS) { - return this.doc = currentBase + doc; - } else { - currentDocs = null; - } - } - } - - @Override - public int nextPosition() throws IOException { - return currentDocs.nextPosition(); - } - - @Override - public int getPayloadLength() { - return currentDocs.getPayloadLength(); - } - - @Override - public BytesRef getPayload() throws IOException { - return currentDocs.getPayload(); - } - - @Override - public boolean hasPayload() { - return currentDocs.hasPayload(); - } - } - // @deprecated This is pre-flex API // Exposes pre-flex API by doing on-the-fly merging // pre-flex API to each segment Index: src/java/org/apache/lucene/index/SegmentMerger.java =================================================================== --- src/java/org/apache/lucene/index/SegmentMerger.java (revision 904746) +++ src/java/org/apache/lucene/index/SegmentMerger.java (working copy) @@ -34,6 +34,8 @@ import org.apache.lucene.store.IndexInput; import org.apache.lucene.store.IndexOutput; import org.apache.lucene.util.Bits; +import org.apache.lucene.util.ReaderUtil; +import org.apache.lucene.util.MultiBits; /** * The SegmentMerger class combines two or more Segments, represented by an IndexReader ({@link #add}, @@ -586,10 +588,22 @@ // the new segment: codec = codecs.getWriter(state); + int docBase = 0; + + final List fields = new ArrayList(); + final List subReaders = new ArrayList(); + final List slices = new ArrayList(); + + final int numReaders = readers.size(); + for(int i=0;i 0) { + while(true) { + top[numTop++] = queue.pop(); + if (queue.size() == 0 || (queue.top()).current != top[0].current) { + break; + } + } + currentField = top[0].current; + } else { + currentField = null; + } + + return currentField; + } + + @Override + public TermsEnum terms() throws IOException { + final List termsEnums = new ArrayList(); + for(int i=0;i= 0: "length=" + slice.length; + this.fields = fields; + } + } + + private final static class FieldMergeQueue extends PriorityQueue { + FieldMergeQueue(int size) { + initialize(size); + } + + @Override + protected final boolean lessThan(FieldsEnumWithSlice fieldsA, FieldsEnumWithSlice fieldsB) { + // No need to break ties by field name: TermsEnum handles that + return fieldsA.current.compareTo(fieldsB.current) < 0; + } + } +} + Property changes on: src/java/org/apache/lucene/index/MultiFieldsEnum.java ___________________________________________________________________ Added: svn:eol-style + native Index: src/java/org/apache/lucene/index/TermsHashPerField.java =================================================================== --- src/java/org/apache/lucene/index/TermsHashPerField.java (revision 904746) +++ src/java/org/apache/lucene/index/TermsHashPerField.java (working copy) @@ -412,7 +412,6 @@ // can be inserted into the analyzer chain if // other behavior is wanted (pruning the term // to a prefix, throwing an exception, etc). - if (docState.maxTermPrefix == null) { docState.maxTermPrefix = new String(tokenText, 0, 30); } Index: src/java/org/apache/lucene/index/TermsEnum.java =================================================================== --- src/java/org/apache/lucene/index/TermsEnum.java (revision 904750) +++ src/java/org/apache/lucene/index/TermsEnum.java (working copy) @@ -57,6 +57,7 @@ * indicate whether exact term was found, a different * term was found, or EOF was hit. The target term may * be befor or after the current term. */ + // nocommit -- add boolean doCache? public abstract SeekStatus seek(BytesRef text) throws IOException; /** Seeks to the specified term by ordinal (position) as Index: src/java/org/apache/lucene/index/SegmentMergeInfo.java =================================================================== --- src/java/org/apache/lucene/index/SegmentMergeInfo.java (revision 904746) +++ src/java/org/apache/lucene/index/SegmentMergeInfo.java (working copy) @@ -1,101 +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.index.codecs.Codec; -import org.apache.lucene.util.BytesRef; - -final class SegmentMergeInfo { - int base; - int ord; // the position of the segment in a MultiReader - final FieldsEnum fields; - TermsEnum terms; - String field; - BytesRef term; - - IndexReader reader; - int delCount; - //private TermPositions postings; // use getPositions() - private int[] docMap; // use getDocMap() - - // nocommit - private String segment; - - SegmentMergeInfo(int b, IndexReader r) - throws IOException { - base = b; - reader = r; - fields = r.fields().iterator(); - // nocommit - if (Codec.DEBUG) { - if (r instanceof SegmentReader) { - segment = ((SegmentReader) r).core.segment; - } - System.out.println("smi create seg=" + segment); - } - } - - // maps around deleted docs - int[] getDocMap() { - if (docMap == null) { - delCount = 0; - // build array which maps document numbers around deletions - if (reader.hasDeletions()) { - int maxDoc = reader.maxDoc(); - docMap = new int[maxDoc]; - int j = 0; - for (int i = 0; i < maxDoc; i++) { - if (reader.isDeleted(i)) { - delCount++; - docMap[i] = -1; - } else - docMap[i] = j++; - } - } - } - return docMap; - } - - final boolean nextField() throws IOException { - field = fields.next(); - if (field != null) { - terms = fields.terms(); - return true; - } else { - return false; - } - } - - final boolean nextTerm() throws IOException { - term = terms.next(); - if (term != null) { - if (Codec.DEBUG) { - System.out.println(" smi.next: term=" + term + " seg=" + segment); - } - return true; - } else { - if (Codec.DEBUG) { - System.out.println(" smi.next: term=null seg=" + segment); - } - return false; - } - } -} - Index: src/java/org/apache/lucene/index/MultiDocsAndPositionsEnum.java =================================================================== --- src/java/org/apache/lucene/index/MultiDocsAndPositionsEnum.java (revision 0) +++ src/java/org/apache/lucene/index/MultiDocsAndPositionsEnum.java (revision 0) @@ -0,0 +1,135 @@ +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 org.apache.lucene.util.Bits; +import org.apache.lucene.util.ReaderUtil; +import org.apache.lucene.util.BytesRef; + +import java.io.IOException; + +/** + * Exposes flex API, merged from flex API of sub-segments. + * + * @lucene.experimental + */ + +public final class MultiDocsAndPositionsEnum extends DocsAndPositionsEnum { + private EnumWithSlice[] subs; + int numSubs; + int upto; + DocsAndPositionsEnum current; + int currentBase; + Bits skipDocs; + int doc = -1; + + MultiDocsAndPositionsEnum reset(final EnumWithSlice[] subs, final int numSubs) throws IOException { + this.numSubs = numSubs; + this.subs = subs; + upto = -1; + current = null; + return this; + } + + public int getNumSubs() { + return numSubs; + } + + public EnumWithSlice[] getSubs() { + return subs; + } + + @Override + public int freq() { + return current.freq(); + } + + @Override + public int docID() { + return doc; + } + + @Override + public int advance(int target) throws IOException { + while(true) { + if (current != null) { + final int doc = current.advance(target-currentBase); + if (doc == NO_MORE_DOCS) { + current = null; + } else { + return this.doc = doc + currentBase; + } + } else if (upto == numSubs-1) { + return this.doc = NO_MORE_DOCS; + } else { + upto++; + current = subs[upto].docsAndPositionsEnum; + currentBase = subs[upto].slice.start; + } + } + } + + @Override + public int nextDoc() throws IOException { + while(true) { + if (current == null) { + if (upto == numSubs-1) { + return this.doc = NO_MORE_DOCS; + } else { + upto++; + current = subs[upto].docsAndPositionsEnum; + currentBase = subs[upto].slice.start; + } + } + + final int doc = current.nextDoc(); + if (doc != NO_MORE_DOCS) { + return this.doc = currentBase + doc; + } else { + current = null; + } + } + } + + @Override + public int nextPosition() throws IOException { + return current.nextPosition(); + } + + @Override + public int getPayloadLength() { + return current.getPayloadLength(); + } + + @Override + public boolean hasPayload() { + return current.hasPayload(); + } + + @Override + public BytesRef getPayload() throws IOException { + return current.getPayload(); + } + + // TODO: implement bulk read more efficiently than super + public final static class EnumWithSlice { + public DocsAndPositionsEnum docsAndPositionsEnum; + public ReaderUtil.Slice slice; + } +} + Property changes on: src/java/org/apache/lucene/index/MultiDocsAndPositionsEnum.java ___________________________________________________________________ Added: svn:eol-style + native Index: src/java/org/apache/lucene/index/IndexReader.java =================================================================== --- src/java/org/apache/lucene/index/IndexReader.java (revision 904750) +++ src/java/org/apache/lucene/index/IndexReader.java (working copy) @@ -882,7 +882,17 @@ @Deprecated public abstract TermEnum terms() throws IOException; - // Default impl emulates new API using old one + /** Flex API: returns {@link Fields} for this reader. + * This may return null if there are no fields. + * + *

NOTE: if this is a multi reader ({@link + * #getSequentialSubReaders} is not null) then this + * method will throw UnsupportedOperationException. If + * you really need a {@link Fields} for such a reader, + * use {@link MultiFields#getFields}. However, for + * performance reasons, it's best to get all sub-readers + * using {@link ReaderUtil#gatherSubReaders} and iterate + * through them yourself. */ public Fields fields() throws IOException { return new LegacyFields(this); } @@ -940,6 +950,15 @@ return termDocs; } + public Terms terms(String field) throws IOException { + final Fields fields = fields(); + if (fields != null) { + return fields.terms(field); + } else { + return null; + } + } + /** Returns {@link DocsEnum} for the specified field & * term. This may return null, for example if either the * field or term does not exist. */ @@ -947,24 +966,22 @@ assert field != null; assert term != null; - - final Terms terms = fields().terms(field); - if (terms != null) { - if (Codec.DEBUG) { - System.out.println("ir.termDocsEnum field=" + field + " term=" + term + " terms=" + terms + " this=" + this); - } - final DocsEnum docs = terms.docs(skipDocs, term, null); - if (Codec.DEBUG) { - System.out.println("ir.termDocsEnum field=" + field + " docs=" +docs); - } - if (docs != null) { + final Fields fields = fields(); + if (fields != null) { + final Terms terms = fields.terms(field); + if (terms != null) { + if (Codec.DEBUG) { + System.out.println("ir.termDocsEnum field=" + field + " term=" + term + " terms=" + terms + " this=" + this); + } + final DocsEnum docs = terms.docs(skipDocs, term, null); + if (Codec.DEBUG) { + System.out.println("ir.termDocsEnum field=" + field + " docs=" +docs); + } return docs; - } else { - return null; } - } else { - return null; } + + return null; } /** Returns {@link DocsAndPositionsEnum} for the specified @@ -975,19 +992,22 @@ assert field != null; assert term != null; - final Terms terms = fields().terms(field); - if (terms != null) { - if (Codec.DEBUG) { - System.out.println("ir.termPositionsEnum field=" + field + " term=" + term + " terms=" + terms + " this=" + this); + final Fields fields = fields(); + if (fields != null) { + final Terms terms = fields.terms(field); + if (terms != null) { + if (Codec.DEBUG) { + System.out.println("ir.termPositionsEnum field=" + field + " term=" + term + " terms=" + terms + " this=" + this); + } + final DocsAndPositionsEnum postings = terms.docsAndPositions(skipDocs, term, null); + if (Codec.DEBUG) { + System.out.println("ir.termPositionsEnum field=" + field + " postings=" +postings); + } + return postings; } - final DocsAndPositionsEnum postings = terms.docsAndPositions(skipDocs, term, null); - if (Codec.DEBUG) { - System.out.println("ir.termPositionsEnum field=" + field + " postings=" +postings); - } - return postings; - } else { - return null; } + + return null; } /** Returns an unpositioned {@link TermDocs} enumerator. @@ -1166,9 +1186,7 @@ * @throws IOException if there is a low-level IO error */ public final synchronized void commit(Map commitUserData) throws IOException { - if (hasChanges) { - doCommit(commitUserData); - } + doCommit(commitUserData); hasChanges = false; } @@ -1209,6 +1227,9 @@ public boolean get(int docID) { return isDeleted(docID); } + public int length() { + return maxDoc(); + } } /** @@ -1409,4 +1430,17 @@ public int getTermInfosIndexDivisor() { throw new UnsupportedOperationException("This reader does not support this method."); } + + + private Fields fields; + + /** lucene.experimental */ + public void storeFields(Fields fields) { + this.fields = fields; + } + + /** lucene.experimental */ + public Fields retrieveFields() { + return fields; + } } Index: src/java/org/apache/lucene/index/MultiTermsEnum.java =================================================================== --- src/java/org/apache/lucene/index/MultiTermsEnum.java (revision 0) +++ src/java/org/apache/lucene/index/MultiTermsEnum.java (revision 0) @@ -0,0 +1,384 @@ +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 org.apache.lucene.util.PriorityQueue; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.BitsSlice; +import org.apache.lucene.util.MultiBits; +import org.apache.lucene.util.ReaderUtil; + +import java.io.IOException; + +/** + * Exposes flex API, merged from flex API of sub-segments. + * This does a merge sort, by term text, of the sub-readers. + * + * @lucene.experimental + */ +public final class MultiTermsEnum extends TermsEnum { + + private final TermMergeQueue queue; + private final TermsEnumWithSlice[] subs; + private final TermsEnumWithSlice[] currentSubs; + private final TermsEnumWithSlice[] top; + private final MultiDocsEnum.EnumWithSlice[] subDocs; + private final MultiDocsAndPositionsEnum.EnumWithSlice[] subDocsAndPositions; + + private int numTop; + private int numSubs; + private BytesRef current; + private BytesRef.Comparator termComp; + + public static class TermsEnumIndex { + public final static TermsEnumIndex[] EMPTY_ARRAY = new TermsEnumIndex[0]; + final int subIndex; + final TermsEnum termsEnum; + + public TermsEnumIndex(TermsEnum termsEnum, int subIndex) { + this.termsEnum = termsEnum; + this.subIndex = subIndex; + } + } + + public int getMatchCount() { + return numTop; + } + + public TermsEnumWithSlice[] getMatchArray() { + return top; + } + + public MultiTermsEnum(ReaderUtil.Slice[] slices) { + queue = new TermMergeQueue(slices.length); + top = new TermsEnumWithSlice[slices.length]; + subs = new TermsEnumWithSlice[slices.length]; + subDocs = new MultiDocsEnum.EnumWithSlice[slices.length]; + subDocsAndPositions = new MultiDocsAndPositionsEnum.EnumWithSlice[slices.length]; + for(int i=0;i 0) { + // at least one sub had exact match to the requested term + current = term; + return SeekStatus.FOUND; + } else if (queue.size() > 0) { + // no sub had exact match, but at least one sub found + // a term after the requested term -- advance to that + // next term: + pullTop(); + return SeekStatus.NOT_FOUND; + } else { + return SeekStatus.END; + } + } + + @Override + public SeekStatus seek(long ord) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public long ord() throws IOException { + throw new UnsupportedOperationException(); + } + + private final void pullTop() { + // extract all subs from the queue that have the same + // top term + assert numTop == 0; + while(true) { + top[numTop++] = queue.pop(); + if (queue.size() == 0 || !(queue.top()).current.bytesEquals(top[0].current)) { + break; + } + } + current = top[0].current; + } + + private final void pushTop() throws IOException { + // call next() on each top, and put back into queue + for(int i=0;i 0) { + pullTop(); + } else { + current = null; + } + + return current; + } + + @Override + public int docFreq() { + int sum = 0; + for(int i=0;i= 0: "length=" + subSlice.length; + } + + public void reset(TermsEnum terms, BytesRef term) { + this.terms = terms; + current = term; + reuseDocs = null; + reusePostings = null; + } + } + + private final static class TermMergeQueue extends PriorityQueue { + BytesRef.Comparator termComp; + TermMergeQueue(int size) { + initialize(size); + } + + @Override + protected final boolean lessThan(TermsEnumWithSlice termsA, TermsEnumWithSlice termsB) { + final int cmp = termComp.compare(termsA.current, termsB.current); + if (cmp != 0) { + return cmp < 0; + } else { + return termsA.subSlice.start < termsB.subSlice.start; + } + } + } +} Property changes on: src/java/org/apache/lucene/index/MultiTermsEnum.java ___________________________________________________________________ Added: svn:eol-style + native Index: src/java/org/apache/lucene/index/Fields.java =================================================================== --- src/java/org/apache/lucene/index/Fields.java (revision 904750) +++ src/java/org/apache/lucene/index/Fields.java (working copy) @@ -24,6 +24,8 @@ public abstract class Fields { + public final static Fields[] EMPTY_ARRAY = new Fields[0]; + /** Returns an iterator that will step through all fields * names */ public abstract FieldsEnum iterator() throws IOException; Index: src/java/org/apache/lucene/index/codecs/MappingMultiDocsAndPositionsEnum.java =================================================================== --- src/java/org/apache/lucene/index/codecs/MappingMultiDocsAndPositionsEnum.java (revision 0) +++ src/java/org/apache/lucene/index/codecs/MappingMultiDocsAndPositionsEnum.java (revision 0) @@ -0,0 +1,121 @@ +package org.apache.lucene.index.codecs; + +/** + * 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 org.apache.lucene.util.BytesRef; +import org.apache.lucene.index.DocsAndPositionsEnum; +import org.apache.lucene.index.MultiDocsAndPositionsEnum; +import java.io.IOException; + +/** + * Exposes flex API, merged from flex API of sub-segments, + * remapping docIDs (this is used for segment merging). + * + * @lucene.experimental + */ + +public final class MappingMultiDocsAndPositionsEnum extends DocsAndPositionsEnum { + private MultiDocsAndPositionsEnum.EnumWithSlice[] subs; + int numSubs; + int upto; + int[] currentMap; + DocsAndPositionsEnum current; + int currentBase; + int doc = -1; + private MergeState mergeState; + + MappingMultiDocsAndPositionsEnum reset(MultiDocsAndPositionsEnum postingsEnum) throws IOException { + this.numSubs = postingsEnum.getNumSubs(); + this.subs = postingsEnum.getSubs(); + upto = -1; + current = null; + return this; + } + + public void setMergeState(MergeState mergeState) { + this.mergeState = mergeState; + } + + @Override + public int freq() { + return current.freq(); + } + + @Override + public int docID() { + return doc; + } + + @Override + public int advance(int target) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public int nextDoc() throws IOException { + while(true) { + if (current == null) { + if (upto == numSubs-1) { + return this.doc = NO_MORE_DOCS; + } else { + upto++; + final int reader = subs[upto].slice.readerIndex; + current = subs[upto].docsAndPositionsEnum; + currentBase = mergeState.docBase[reader]; + currentMap = mergeState.docMaps[reader]; + } + } + + int doc = current.nextDoc(); + if (doc != NO_MORE_DOCS) { + if (currentMap != null) { + // compact deletions + doc = currentMap[doc]; + if (doc == -1) { + continue; + } + } + return this.doc = currentBase + doc; + } else { + current = null; + } + } + } + + @Override + public int nextPosition() throws IOException { + return current.nextPosition(); + } + + + @Override + public int getPayloadLength() { + return current.getPayloadLength(); + } + + @Override + public BytesRef getPayload() throws IOException { + return current.getPayload(); + } + + @Override + public boolean hasPayload() { + return current.hasPayload(); + } +} + Property changes on: src/java/org/apache/lucene/index/codecs/MappingMultiDocsAndPositionsEnum.java ___________________________________________________________________ Added: svn:eol-style + native Index: src/java/org/apache/lucene/index/codecs/MergeState.java =================================================================== --- src/java/org/apache/lucene/index/codecs/MergeState.java (revision 904746) +++ src/java/org/apache/lucene/index/codecs/MergeState.java (working copy) @@ -20,6 +20,7 @@ import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.FieldInfos; import org.apache.lucene.index.IndexReader; +import org.apache.lucene.util.Bits; import java.util.List; /** Holds common state used during segment merging @@ -34,9 +35,9 @@ public int[] delCounts; // Deletion count per reader public int[] docBase; // New docID base per reader public int mergedDocCount; // Total # merged docs + public Bits multiDeletedDocs; // Updated per field; public FieldInfo fieldInfo; - public boolean omitTermFreqAndPositions; } Index: src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsReaderImpl.java =================================================================== --- src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsReaderImpl.java (revision 904750) +++ src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsReaderImpl.java (working copy) @@ -22,13 +22,11 @@ import org.apache.lucene.index.DocsEnum; import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.DocsAndPositionsEnum; -import org.apache.lucene.index.SegmentInfo; import org.apache.lucene.index.codecs.Codec; import org.apache.lucene.index.codecs.standard.TermState; import org.apache.lucene.index.codecs.standard.StandardPostingsReader; import org.apache.lucene.index.codecs.pulsing.PulsingPostingsWriterImpl.Document; import org.apache.lucene.index.codecs.pulsing.PulsingPostingsWriterImpl.Position; -import org.apache.lucene.store.Directory; import org.apache.lucene.store.IndexInput; import org.apache.lucene.util.Bits; import org.apache.lucene.util.BytesRef; @@ -197,7 +195,8 @@ } } - // nocommit -- not great that we can't always reuse + // TODO: we could actually reuse, by having TL that + // holds the last wrapped reuse, and vice-versa @Override public DocsEnum docs(FieldInfo field, TermState _termState, Bits skipDocs, DocsEnum reuse) throws IOException { PulsingTermState termState = (PulsingTermState) _termState; Index: src/java/org/apache/lucene/index/codecs/MappingMultiDocsEnum.java =================================================================== --- src/java/org/apache/lucene/index/codecs/MappingMultiDocsEnum.java (revision 0) +++ src/java/org/apache/lucene/index/codecs/MappingMultiDocsEnum.java (revision 0) @@ -0,0 +1,99 @@ +package org.apache.lucene.index.codecs; + +/** + * 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 org.apache.lucene.index.DocsEnum; +import org.apache.lucene.index.MultiDocsEnum; +import java.io.IOException; + +/** + * Exposes flex API, merged from flex API of sub-segments, + * remapping docIDs (this is used for segment merging). + * + * @lucene.experimental + */ + +public final class MappingMultiDocsEnum extends DocsEnum { + private MultiDocsEnum.EnumWithSlice[] subs; + int numSubs; + int upto; + int[] currentMap; + DocsEnum current; + int currentBase; + int doc = -1; + private MergeState mergeState; + + MappingMultiDocsEnum reset(MultiDocsEnum docsEnum) throws IOException { + this.numSubs = docsEnum.getNumSubs(); + this.subs = docsEnum.getSubs(); + upto = -1; + current = null; + return this; + } + + public void setMergeState(MergeState mergeState) { + this.mergeState = mergeState; + } + + @Override + public int freq() { + return current.freq(); + } + + @Override + public int docID() { + return doc; + } + + @Override + public int advance(int target) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public int nextDoc() throws IOException { + while(true) { + if (current == null) { + if (upto == numSubs-1) { + return this.doc = NO_MORE_DOCS; + } else { + upto++; + final int reader = subs[upto].slice.readerIndex; + current = subs[upto].docsEnum; + currentBase = mergeState.docBase[reader]; + currentMap = mergeState.docMaps[reader]; + } + } + + int doc = current.nextDoc(); + if (doc != NO_MORE_DOCS) { + if (currentMap != null) { + // compact deletions + doc = currentMap[doc]; + if (doc == -1) { + continue; + } + } + return this.doc = currentBase + doc; + } else { + current = null; + } + } + } +} + Property changes on: src/java/org/apache/lucene/index/codecs/MappingMultiDocsEnum.java ___________________________________________________________________ Added: svn:eol-style + native Index: src/java/org/apache/lucene/index/codecs/TermsConsumer.java =================================================================== --- src/java/org/apache/lucene/index/codecs/TermsConsumer.java (revision 904750) +++ src/java/org/apache/lucene/index/codecs/TermsConsumer.java (working copy) @@ -20,8 +20,9 @@ import java.io.IOException; import org.apache.lucene.index.TermsEnum; -import org.apache.lucene.index.DocsEnum; -import org.apache.lucene.util.PriorityQueue; +import org.apache.lucene.index.MultiDocsEnum; +import org.apache.lucene.index.MultiDocsAndPositionsEnum; + import org.apache.lucene.util.BytesRef; /** @@ -43,109 +44,46 @@ * before feeding to this API. */ public abstract BytesRef.Comparator getComparator() throws IOException; - // For default merge impl - public static class TermMergeState { - BytesRef current; - TermsEnum termsEnum; - int readerIndex; - } - - private final static class MergeQueue extends PriorityQueue { - - final BytesRef.Comparator termComp; - - public MergeQueue(int size, BytesRef.Comparator termComp) { - initialize(size); - this.termComp = termComp; - } - - @Override - protected final boolean lessThan(TermMergeState a, TermMergeState b) { - final int cmp = termComp.compare(a.current, b.current); - if (cmp != 0) { - return cmp < 0; - } else { - return a.readerIndex < b.readerIndex; - } - } - } - - private MergeQueue queue; - private PostingsConsumer.PostingsMergeState[] match; - private TermMergeState[] pending; - /** Default merge impl */ - public void merge(MergeState mergeState, TermMergeState[] termsStates, int count) throws IOException { + private MappingMultiDocsEnum docsEnum = null; + private MappingMultiDocsAndPositionsEnum postingsEnum = null; - final BytesRef.Comparator termComp = getComparator(); + public void merge(MergeState mergeState, TermsEnum termsEnum) throws IOException { - //System.out.println("merge terms field=" + mergeState.fieldInfo.name + " comp=" + termComp); + BytesRef term; - if (queue == null) { - queue = new MergeQueue(mergeState.readerCount, termComp); - match = new PostingsConsumer.PostingsMergeState[mergeState.readerCount]; - for(int i=0;i 0) { - // Merge one term - final BytesRef term = pending[0].current; - //System.out.println(" merge term=" + term); - final PostingsConsumer postingsConsumer = startTerm(term); - final int numDocs = postingsConsumer.merge(mergeState, match, matchCount); - finishTerm(term, numDocs); + } else { + if (postingsEnum == null) { + postingsEnum = new MappingMultiDocsAndPositionsEnum(); } - - // Put terms back into queue - for(int i=0;i fields = new HashMap(); @@ -128,7 +131,6 @@ } } else { this.in = in; - // nocommit -- we should close if index gets read on demand? } } } @@ -139,17 +141,10 @@ * usage of SegmentReader searching a tiny segment. */ private final void trimByteBlock() { if (blockOffset == 0) { - // nocommit -- should not happen? fields w/ no terms - // are not written by STDW. hmmm it does - // happen... must explain why -- oh, could be only - // on exception; I added only calling this on - // success above - //assert false; - // nocommit -- hit AIOOBE here (blocks is length 0): + // There were no fields in this segment: if (blocks != null) { blocks[blockUpto] = null; } - //System.out.println("Simple terms index consumed no bytes! blockCount=" + blocks.length); } else { byte[] last = new byte[blockOffset]; System.arraycopy(blocks[blockUpto], 0, last, 0, blockOffset); @@ -157,7 +152,7 @@ } } - // nocommit -- we can record precisely how many bytes are + // TODO: we can record precisely how many bytes are // required during indexing, save that into file, and be // precise when we allocate the blocks; we even don't need // to use blocks anymore (though my still want to, to @@ -170,14 +165,18 @@ int blockUpto; int blockOffset; - // nocommit -- is this big enough, given max allowed term - // size (measured in chars!!) ? - // nocommit -- or, we could allocate one block way to big, - // to accommodate such ridiculous terms private static final int BYTE_BLOCK_SHIFT = 15; private static final int BYTE_BLOCK_SIZE = 1 << BYTE_BLOCK_SHIFT; private static final int BYTE_BLOCK_MASK = BYTE_BLOCK_SIZE - 1; + static { + // Make sure DW can't ever write a term whose length + // cannot be encoded with short (because we use short[] + // to hold the length of each term). + assert IndexWriter.MAX_TERM_LENGTH < Short.MAX_VALUE; + assert BYTE_BLOCK_SIZE >= IndexWriter.MAX_TERM_LENGTH; + } + private final class FieldIndexReader extends FieldReader { final private FieldInfo fieldInfo; @@ -270,14 +269,13 @@ // precise # bits final long[] blockPointer; - // Length of each term - // nocommit -- this is length in bytes; is short - // sufficient? have to use negative space? // TODO: used packed ints here: we know max term // length; often its small // TODO: can we inline this w/ the bytes? like // DW. vast majority of terms only need 1 byte, not 2 + + // Length of each term final short[] termLength; final int numIndexTerms; @@ -292,9 +290,6 @@ // IndexWriter because a SegmentReader that at // first was opened for merging, is now being // opened to perform deletes or for an NRT reader - - // nocommit -- how to allow apps to indexDivisor - // in this case? this.numIndexTerms = numIndexTerms; } else { this.numIndexTerms = 1+(numIndexTerms-1) / indexDivisor; @@ -331,8 +326,6 @@ final int suffix = clone.readVInt(); final int thisTermLength = start + suffix; - // nocommit -- verify this is in fact guaranteed by - // DW -- we are talking bytes not chars here assert thisTermLength <= BYTE_BLOCK_SIZE; if (i%indexDivisor == 0) { @@ -351,7 +344,6 @@ blockOffset = 0; } - //System.out.println("blockUpto=" + blockUpto + " blocks.length=" + blocks.length); final byte[] block = blocks[blockUpto]; // Copy old prefix @@ -403,7 +395,6 @@ } } - // nocommit: put in finally clause clone.close(); assert upto == this.numIndexTerms; @@ -501,8 +492,10 @@ while(it.hasNext()) { it.next().loadTermsIndex(); } - indexLoaded = true; trimByteBlock(); + + indexLoaded = true; + in.close(); } } @@ -526,7 +519,7 @@ @Override public void close() throws IOException { - if (in != null) { + if (in != null && !indexLoaded) { in.close(); } } Index: src/java/org/apache/lucene/index/codecs/standard/StandardTermsDictReader.java =================================================================== --- src/java/org/apache/lucene/index/codecs/standard/StandardTermsDictReader.java (revision 904750) +++ src/java/org/apache/lucene/index/codecs/standard/StandardTermsDictReader.java (working copy) @@ -359,6 +359,9 @@ } } + // Useed only for assert: + final int startOrd; + if (doSeek) { // As index to find biggest index term that's <= @@ -379,15 +382,20 @@ // scanning over an index term we'd have to // special case it: bytesReader.reset(indexResult.term); - + state.ord = (int) indexResult.position-1; assert state.ord >= -1: "ord=" + state.ord; + startOrd = (int) indexResult.position; + if (Codec.DEBUG) { Codec.debug(" set ord=" + state.ord); } - } else if (Codec.DEBUG) { - Codec.debug(": use scanning only (no seek)"); + } else { + startOrd = -1; + if (Codec.DEBUG) { + Codec.debug(": use scanning only (no seek)"); + } } // Now scan: @@ -421,8 +429,12 @@ return SeekStatus.NOT_FOUND; } - // nocommit -- put back assert that we don't cross - // another index term while scanning, here + // The purpose of the terms dict index is to seek + // the enum to the closest index term before the + // term we are looking for. So, we should never + // cross another index term (besides the first + // one) while we are scanning: + assert state.ord == startOrd || !indexReader.isIndexTerm(state.ord, state.docFreq); } if (Codec.DEBUG) { @@ -509,6 +521,12 @@ // a "how many terms until next index entry" in each // index entry, but that'd require some tricky // lookahead work when writing the index + + // nocommit -- this call to isIndexTerm is not + // right, when indexDivisor > 1? ie, this will + // return false for entries that actually are index + // terms, and then the postings impl will read the + // wrong offset. make a test... postingsReader.readTerm(in, fieldInfo, state, indexReader.isIndexTerm(1+state.ord, state.docFreq)); Index: src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java =================================================================== --- src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java (revision 904750) +++ src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java (working copy) @@ -27,11 +27,13 @@ import org.apache.lucene.util.Bits; /** StandardTermsDictReader interacts with a single instance - * of this to manage creation of docs enum instances. It - * provides an IndexInput (termsIn) where this class may - * read any previously stored data that it had written in - * its corresponding StandardDocsConsumer at indexing + * of this to manage creation of {@link DocsEnum} and + * {@link DocsAndPositionsEnum} instances. It provides an + * IndexInput (termsIn) where this class may read any + * previously stored data that it had written in its + * corresponding {@link StandardPostingsWriter} at indexing * time. */ + public abstract class StandardPostingsReader implements Closeable { public abstract void init(IndexInput termsIn) throws IOException; Index: src/java/org/apache/lucene/index/codecs/standard/SimpleStandardTermsIndexWriter.java =================================================================== --- src/java/org/apache/lucene/index/codecs/standard/SimpleStandardTermsIndexWriter.java (revision 904750) +++ src/java/org/apache/lucene/index/codecs/standard/SimpleStandardTermsIndexWriter.java (working copy) @@ -42,7 +42,6 @@ private final FieldInfos fieldInfos; // unread private IndexOutput termsOut; - // nocommit final private String segment; public SimpleStandardTermsIndexWriter(SegmentWriteState state) throws IOException { @@ -125,9 +124,6 @@ out.writeLong(field.indexStart); } out.seek(Codec.headerSize(CODEC_NAME)); - // nocommit -- why not simply write last 8 bytes of - // file? hmm would require accurate filelength() in - // reader out.writeLong(dirStart); if (Codec.DEBUG) { System.out.println(" writeDirStart " + dirStart + " @ " + Codec.headerSize(CODEC_NAME)); Index: src/java/org/apache/lucene/index/codecs/standard/StandardTermsDictWriter.java =================================================================== --- src/java/org/apache/lucene/index/codecs/standard/StandardTermsDictWriter.java (revision 904750) +++ src/java/org/apache/lucene/index/codecs/standard/StandardTermsDictWriter.java (working copy) @@ -61,7 +61,6 @@ private final List fields = new ArrayList(); private final BytesRef.Comparator termComp; - // nocommit private String segment; public StandardTermsDictWriter(StandardTermsIndexWriter indexWriter, SegmentWriteState state, StandardPostingsWriter postingsWriter, BytesRef.Comparator termComp) throws IOException { @@ -107,8 +106,9 @@ @Override public void close() throws IOException { - if (Codec.DEBUG) + if (Codec.DEBUG) { System.out.println("stdw.close seg=" + segment); + } try { final int fieldCount = fields.size(); @@ -142,14 +142,12 @@ } } - long lastIndexPointer; - class TermsWriter extends TermsConsumer { - final FieldInfo fieldInfo; - final StandardPostingsWriter postingsWriter; - final long termsStartPointer; - int numTerms; - final StandardTermsIndexWriter.FieldWriter fieldIndexWriter; + private final FieldInfo fieldInfo; + private final StandardPostingsWriter postingsWriter; + private final long termsStartPointer; + private int numTerms; + private final StandardTermsIndexWriter.FieldWriter fieldIndexWriter; TermsWriter(StandardTermsIndexWriter.FieldWriter fieldIndexWriter, FieldInfo fieldInfo, StandardPostingsWriter postingsWriter) { this.fieldInfo = fieldInfo; @@ -158,7 +156,6 @@ termWriter.reset(); termsStartPointer = out.getFilePointer(); postingsWriter.setField(fieldInfo); - lastIndexPointer = termsStartPointer; this.postingsWriter = postingsWriter; if (Codec.DEBUG) { @@ -175,8 +172,8 @@ public PostingsConsumer startTerm(BytesRef text) throws IOException { postingsWriter.startTerm(); if (Codec.DEBUG) { - postingsWriter.desc = fieldInfo.name + ":" + text.toBytesString(); - System.out.println("stdw.startTerm term=" + fieldInfo.name + ":" + text.toBytesString() + " seg=" + segment); + postingsWriter.desc = fieldInfo.name + ":" + text.toString(); + System.out.println("stdw.startTerm term=" + fieldInfo.name + ":" + text.toString() + " seg=" + segment); } return postingsWriter; } @@ -184,19 +181,16 @@ @Override public void finishTerm(BytesRef text, int numDocs) throws IOException { - // mxx if (Codec.DEBUG) { - // nocommit - Codec.debug("finishTerm seg=" + segment + " text=" + fieldInfo.name + ":" + text.toBytesString() + " numDocs=" + numDocs + " numTerms=" + numTerms); + Codec.debug("finishTerm seg=" + segment + " text=" + fieldInfo.name + ":" + text.toString() + " numDocs=" + numDocs + " numTerms=" + numTerms); } if (numDocs > 0) { final boolean isIndexTerm = fieldIndexWriter.checkIndexTerm(text, numDocs); - // mxx if (Codec.DEBUG) { Codec.debug(" tis.fp=" + out.getFilePointer() + " isIndexTerm?=" + isIndexTerm); - System.out.println(" term bytes=" + text.toBytesString()); + System.out.println(" term bytes=" + text.toString()); } termWriter.write(text); out.writeVInt(numDocs); Index: src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java =================================================================== --- src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java (revision 904750) +++ src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java (working copy) @@ -40,6 +40,8 @@ public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { StandardPostingsWriter docs = new StandardPostingsWriterImpl(state); + // nocommit -- how to gracefully upgrade to a new terms + // index impl? could just make a new named codec... StandardTermsIndexWriter indexWriter; boolean success = false; try { @@ -74,8 +76,6 @@ StandardPostingsReader postings = new StandardPostingsReaderImpl(dir, si, readBufferSize); StandardTermsIndexReader indexReader; - // nocommit -- not clean that every codec must deal w/ - // this... dup'd code boolean success = false; try { indexReader = new SimpleStandardTermsIndexReader(dir, Index: src/java/org/apache/lucene/index/codecs/FieldsConsumer.java =================================================================== --- src/java/org/apache/lucene/index/codecs/FieldsConsumer.java (revision 904746) +++ src/java/org/apache/lucene/index/codecs/FieldsConsumer.java (working copy) @@ -21,7 +21,6 @@ import org.apache.lucene.index.Fields; import org.apache.lucene.index.FieldsEnum; import org.apache.lucene.index.TermsEnum; -import org.apache.lucene.util.PriorityQueue; import java.io.IOException; @@ -40,98 +39,16 @@ /** Called when we are done adding everything. */ public abstract void close() throws IOException; - private final static class FieldMergeState { - String current; - FieldsEnum fieldsEnum; - int readerIndex; - } - - // Used for merge-sorting by field - private final static class MergeQueue extends PriorityQueue { - public MergeQueue(int size) { - initialize(size); - } - - @Override - protected final boolean lessThan(FieldMergeState a, FieldMergeState b) { - final int cmp = a.current.compareTo(b.current); - if (cmp != 0) { - return cmp < 0; - } else { - // nocommit -- technically not required to break - // ties, since the terms merging will do so? - return a.readerIndex < b.readerIndex; + public void merge(MergeState mergeState, Fields fields) throws IOException { + FieldsEnum fieldsEnum = fields.iterator(); + String field; + while((field = fieldsEnum.next()) != null) { + mergeState.fieldInfo = mergeState.fieldInfos.fieldInfo(field); + final TermsConsumer termsConsumer = addField(mergeState.fieldInfo); + final TermsEnum termsEnum = fieldsEnum.terms(); + if (termsEnum != null) { + termsConsumer.merge(mergeState, termsEnum); } } } - - public void merge(MergeState mergeState, Fields[] fields) throws IOException { - - MergeQueue queue = new MergeQueue(fields.length); - - for(int i=0;i 0) { - // Merge one field - final String field = pending[0].current; - mergeState.fieldInfo = mergeState.fieldInfos.fieldInfo(field); - mergeState.omitTermFreqAndPositions = mergeState.fieldInfo.omitTermFreqAndPositions; - final TermsConsumer termsConsumer = addField(mergeState.fieldInfo); - termsConsumer.merge(mergeState, match, matchCount); - } - - // Put fields back into queue - for(int i=0;i 0) { - payload = postingsEnum.getPayload(); - } else { - payload = null; - } - addPosition(position, payload); + for(int i=0;i 0) { + payload = postingsEnum.getPayload(); + } else { + payload = null; } - finishDoc(); + addPosition(position, payload); } + df++; } } - return df; } } Index: src/java/org/apache/lucene/index/MultiFields.java =================================================================== --- src/java/org/apache/lucene/index/MultiFields.java (revision 0) +++ src/java/org/apache/lucene/index/MultiFields.java (revision 0) @@ -0,0 +1,200 @@ +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.Map; +import java.util.HashMap; +import java.util.List; +import java.util.ArrayList; +import org.apache.lucene.util.ReaderUtil; +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.index.codecs.Codec; + +/** + * Exposes flex API, merged from flex API of sub-segments. + * This is useful when you're interacting with an {@link + * IndexReader} implementation that consists of sequential + * sub-readers (eg {@link DirectoryReade} or {@link + * MultiReader}). + * + *

NOTE: for multi readers, you'll get better + * performance by gathering the sub readers using {@link + * ReaderUtil#gatherSubReaders} and then operate per-reader, + * instead of using this class. + * + * @lucene.experimental + */ + +public final class MultiFields extends Fields { + private final Fields[] subs; + private final ReaderUtil.Slice[] subSlices; + private final Map terms = new HashMap(); + + public static Fields getFields(IndexReader r) throws IOException { + final IndexReader[] subs = r.getSequentialSubReaders(); + if (subs == null) { + // already an atomic reader + return r.fields(); + } else if (subs.length == 1) { + return getFields(subs[0]); + } else { + + Fields currentFields = r.retrieveFields(); + if (currentFields == null) { + + final List fields = new ArrayList(); + final List slices = new ArrayList(); + ReaderUtil.gatherSubFields(null, fields, slices, r, 0); + + if (fields.size() == 0) { + return null; + } else if (fields.size() == 1) { + currentFields = fields.get(0); + } else { + currentFields = new MultiFields(fields.toArray(Fields.EMPTY_ARRAY), + slices.toArray(ReaderUtil.Slice.EMPTY_ARRAY)); + } + r.storeFields(currentFields); + } + return currentFields; + } + } + + public static Terms getTerms(IndexReader r, String field) throws IOException { + final Fields fields = getFields(r); + if (fields != null) { + return fields.terms(field); + } else { + return null; + } + } + + /** Returns {@link DocsEnum} for the specified field & + * term. This may return null, for example if either the + * field or term does not exist. */ + public static DocsEnum getTermDocsEnum(IndexReader r, Bits skipDocs, String field, BytesRef term) throws IOException { + + assert field != null; + assert term != null; + final Fields fields = getFields(r); + if (fields != null) { + final Terms terms = fields.terms(field); + if (terms != null) { + if (Codec.DEBUG) { + System.out.println("mf.termDocsEnum field=" + field + " term=" + term + " terms=" + terms); + } + final DocsEnum docs = terms.docs(skipDocs, term, null); + if (Codec.DEBUG) { + System.out.println("mf.termDocsEnum field=" + field + " docs=" +docs); + } + return docs; + } + } + + return null; + } + + /** Returns {@link DocsAndPositionsEnum} for the specified + * field & term. This may return null, for example if + * either the field or term does not exist. */ + public static DocsAndPositionsEnum getTermPositionsEnum(IndexReader r, Bits skipDocs, String field, BytesRef term) throws IOException { + assert field != null; + assert term != null; + + final Fields fields = getFields(r); + if (fields != null) { + final Terms terms = fields.terms(field); + if (terms != null) { + if (Codec.DEBUG) { + System.out.println("mf.termPositionsEnum field=" + field + " term=" + term + " terms=" + terms); + } + final DocsAndPositionsEnum postings = terms.docsAndPositions(skipDocs, term, null); + if (Codec.DEBUG) { + System.out.println("mf.termPositionsEnum field=" + field + " postings=" +postings); + } + return postings; + } + } + + return null; + } + + + public MultiFields(Fields[] subs, ReaderUtil.Slice[] subSlices) { + this.subs = subs; + this.subSlices = subSlices; + } + + @Override + public FieldsEnum iterator() throws IOException { + + final List fieldsEnums = new ArrayList(); + final List fieldsSlices = new ArrayList(); + for(int i=0;i subs2 = new ArrayList(); + final List slices2 = new ArrayList(); + + // Gather all sub-readers that share this field + for(int i=0;iNOTE: This is very costly, as every lookup must + * do a binary search to locate the right sub-reader. + */ +public final class MultiBits implements Bits { + private final Bits[] subs; + + // length is 1+subs.length (the last entry has the maxDoc): + private final int[] starts; + + public MultiBits(Bits[] subs, int[] starts) { + this.subs = subs; + this.starts = starts; + } + + private boolean checkLength(int reader, int doc) { + final int length = starts[1+reader]-starts[reader]; + assert doc - starts[reader] < length: "doc=" + doc + " reader=" + reader + " starts[reader]=" + starts[reader] + " length=" + length; + return true; + } + + public boolean get(int doc) { + final int reader = ReaderUtil.subIndex(doc, starts); + final Bits bits = subs[reader]; + if (bits == null) { + return false; + } else { + assert checkLength(reader, doc); + return bits.get(doc-starts[reader]); + } + } + + public Bits getMatchingSub(ReaderUtil.Slice slice) { + int reader = ReaderUtil.subIndex(slice.start, starts); + if (starts[reader] == slice.start && starts[1+reader] == slice.start+slice.length) { + return subs[reader]; + } else { + return null; + } + } + + public int length() { + return starts[starts.length-1]; + } +} Property changes on: src/java/org/apache/lucene/util/MultiBits.java ___________________________________________________________________ Added: svn:eol-style + native Index: src/java/org/apache/lucene/util/BitsSlice.java =================================================================== --- src/java/org/apache/lucene/util/BitsSlice.java (revision 0) +++ src/java/org/apache/lucene/util/BitsSlice.java (revision 0) @@ -0,0 +1,46 @@ +package org.apache.lucene.util; + +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** Exposes a slice of an existing Bits as a new Bits. */ + +public final class BitsSlice implements Bits { + private final Bits parent; + private final int start; + private final int length; + + // start is inclusive; end is exclusive (length = end-start) + public BitsSlice(Bits parent, ReaderUtil.Slice slice) { + this.parent = parent; + this.start = slice.start; + this.length = slice.length; + assert length >= 0: "length=" + length; + } + + public boolean get(int doc) { + if (doc >= length) { + throw new RuntimeException("doc " + doc + " is out of bounds 0 .. " + (length-1)); + } + assert doc < length: "doc=" + doc + " length=" + length; + return parent.get(doc+start); + } + + public int length() { + return length; + } +} Property changes on: src/java/org/apache/lucene/util/BitsSlice.java ___________________________________________________________________ Added: svn:eol-style + native Index: contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SrndTruncQuery.java =================================================================== --- contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SrndTruncQuery.java (revision 904746) +++ contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SrndTruncQuery.java (working copy) @@ -21,6 +21,7 @@ import org.apache.lucene.index.Terms; import org.apache.lucene.util.BytesRef; import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.MultiFields; import java.io.IOException; @@ -89,7 +90,7 @@ { boolean expanded = false; int prefixLength = prefix.length(); - Terms terms = reader.fields().terms(fieldName); + Terms terms = MultiFields.getTerms(reader, fieldName); if (terms != null) { Matcher matcher = pattern.matcher(""); try { Index: contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SrndPrefixQuery.java =================================================================== --- contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SrndPrefixQuery.java (revision 904746) +++ contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SrndPrefixQuery.java (working copy) @@ -21,6 +21,7 @@ import org.apache.lucene.util.BytesRef; import org.apache.lucene.index.TermsEnum; import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.MultiFields; import java.io.IOException; @@ -57,7 +58,7 @@ MatchingTermVisitor mtv) throws IOException { /* inspired by PrefixQuery.rewrite(): */ - Terms terms = reader.fields().terms(fieldName); + Terms terms = MultiFields.getTerms(reader, fieldName); boolean expanded = false; if (terms != null) { TermsEnum termsEnum = terms.iterator(); Index: contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SrndTermQuery.java =================================================================== --- contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SrndTermQuery.java (revision 904746) +++ contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SrndTermQuery.java (working copy) @@ -22,6 +22,7 @@ import org.apache.lucene.index.Term; import org.apache.lucene.index.TermsEnum; import org.apache.lucene.index.Terms; +import org.apache.lucene.index.MultiFields; import org.apache.lucene.util.BytesRef; @@ -48,7 +49,7 @@ MatchingTermVisitor mtv) throws IOException { /* check term presence in index here for symmetry with other SimpleTerm's */ - Terms terms = reader.fields().terms(fieldName); + Terms terms = MultiFields.getTerms(reader, fieldName); if (terms != null) { TermsEnum termsEnum = terms.iterator(); Index: contrib/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java =================================================================== --- contrib/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java (revision 904750) +++ contrib/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java (working copy) @@ -37,6 +37,7 @@ import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.index.MultiFields; import org.apache.lucene.index.FieldsEnum; import org.apache.lucene.index.DocsEnum; import org.apache.lucene.index.SerialMergeScheduler; @@ -469,7 +470,7 @@ int totalTokenCount2 = 0; - FieldsEnum fields = reader.fields().iterator(); + FieldsEnum fields = MultiFields.getFields(reader).iterator(); String fieldName = null; while((fieldName = fields.next()) != null) { if (fieldName == DocMaker.ID_FIELD) Index: contrib/misc/src/test/org/apache/lucene/index/TestFieldNormModifier.java =================================================================== --- contrib/misc/src/test/org/apache/lucene/index/TestFieldNormModifier.java (revision 904746) +++ contrib/misc/src/test/org/apache/lucene/index/TestFieldNormModifier.java (working copy) @@ -76,13 +76,9 @@ writer.close(); } - public void testMissingField() { + public void testMissingField() throws Exception { FieldNormModifier fnm = new FieldNormModifier(store, s); - try { - fnm.reSetNorms("nobodyherebutuschickens"); - } catch (Exception e) { - assertNull("caught something", e); - } + fnm.reSetNorms("nobodyherebutuschickens"); } public void testFieldWithNoNorm() throws Exception { @@ -97,11 +93,7 @@ r.close(); FieldNormModifier fnm = new FieldNormModifier(store, s); - try { - fnm.reSetNorms("nonorm"); - } catch (Exception e) { - assertNull("caught something", e); - } + fnm.reSetNorms("nonorm"); // nothing should have changed r = IndexReader.open(store, false); Index: contrib/misc/src/java/org/apache/lucene/index/MultiPassIndexSplitter.java =================================================================== --- contrib/misc/src/java/org/apache/lucene/index/MultiPassIndexSplitter.java (revision 904746) +++ contrib/misc/src/java/org/apache/lucene/index/MultiPassIndexSplitter.java (working copy) @@ -170,6 +170,8 @@ * list of deletions. */ public static class FakeDeleteIndexReader extends FilterIndexReader { + // nocommit -- implement flex api here + OpenBitSet dels; OpenBitSet oldDels = null; @@ -213,6 +215,11 @@ } @Override + public IndexReader[] getSequentialSubReaders() { + return null; + } + + @Override public Bits getDeletedDocs() { return dels; } Index: contrib/misc/src/java/org/apache/lucene/index/TermVectorAccessor.java =================================================================== --- contrib/misc/src/java/org/apache/lucene/index/TermVectorAccessor.java (revision 904750) +++ contrib/misc/src/java/org/apache/lucene/index/TermVectorAccessor.java (working copy) @@ -102,7 +102,7 @@ final Bits delDocs = indexReader.getDeletedDocs(); - Terms terms = indexReader.fields().terms(field); + Terms terms = MultiFields.getTerms(indexReader, field); boolean anyTerms = false; if (terms != null) { TermsEnum termsEnum = terms.iterator(); Index: contrib/misc/src/java/org/apache/lucene/index/FieldNormModifier.java =================================================================== --- contrib/misc/src/java/org/apache/lucene/index/FieldNormModifier.java (revision 904750) +++ contrib/misc/src/java/org/apache/lucene/index/FieldNormModifier.java (working copy) @@ -19,12 +19,15 @@ import java.io.IOException; import java.io.File; import java.util.Date; +import java.util.List; +import java.util.ArrayList; import org.apache.lucene.search.Similarity; import org.apache.lucene.store.Directory; import org.apache.lucene.store.FSDirectory; import org.apache.lucene.util.StringHelper; import org.apache.lucene.util.Bits; +import org.apache.lucene.util.ReaderUtil; /** * Given a directory and a list of fields, updates the fieldNorms in place for every document. @@ -105,43 +108,46 @@ */ public void reSetNorms(String field) throws IOException { String fieldName = StringHelper.intern(field); - int[] termCounts = new int[0]; - byte[] fakeNorms = new byte[0]; IndexReader reader = null; try { - reader = IndexReader.open(dir, true); - final Bits delDocs = reader.getDeletedDocs(); + reader = IndexReader.open(dir, false); - termCounts = new int[reader.maxDoc()]; - Terms terms = reader.fields().terms(field); - if (terms != null) { - TermsEnum termsEnum = terms.iterator(); - DocsEnum docs = null; - while(termsEnum.next() != null) { - docs = termsEnum.docs(delDocs, docs); - while(true) { - int docID = docs.nextDoc(); - if (docID != docs.NO_MORE_DOCS) { - termCounts[docID] += docs.freq(); - } else { - break; + final List subReaders = new ArrayList(); + ReaderUtil.gatherSubReaders(subReaders, reader); + + for(IndexReader subReader : subReaders) { + final Bits delDocs = subReader.getDeletedDocs(); + + int[] termCounts = new int[subReader.maxDoc()]; + Fields fields = subReader.fields(); + if (fields != null) { + Terms terms = fields.terms(field); + if (terms != null) { + TermsEnum termsEnum = terms.iterator(); + DocsEnum docs = null; + while(termsEnum.next() != null) { + docs = termsEnum.docs(delDocs, docs); + while(true) { + int docID = docs.nextDoc(); + if (docID != docs.NO_MORE_DOCS) { + termCounts[docID] += docs.freq(); + } else { + break; + } } } } } - } finally { - if (null != reader) reader.close(); - } - - try { - reader = IndexReader.open(dir, false); - for (int d = 0; d < termCounts.length; d++) { - if (! reader.isDeleted(d)) { - if (sim == null) - reader.setNorm(d, fieldName, fakeNorms[0]); - else - reader.setNorm(d, fieldName, sim.encodeNormValue(sim.lengthNorm(fieldName, termCounts[d]))); + + for (int d = 0; d < termCounts.length; d++) { + if (delDocs == null || !delDocs.get(d)) { + if (sim == null) { + subReader.setNorm(d, fieldName, Similarity.encodeNorm(1.0f)); + } else { + subReader.setNorm(d, fieldName, sim.encodeNormValue(sim.lengthNorm(fieldName, termCounts[d]))); + } + } } } Index: contrib/spellchecker/src/java/org/apache/lucene/search/spell/LuceneDictionary.java =================================================================== --- contrib/spellchecker/src/java/org/apache/lucene/search/spell/LuceneDictionary.java (revision 904746) +++ contrib/spellchecker/src/java/org/apache/lucene/search/spell/LuceneDictionary.java (working copy) @@ -24,6 +24,7 @@ import org.apache.lucene.index.TermsEnum; import org.apache.lucene.util.BytesRef; import org.apache.lucene.index.Terms; +import org.apache.lucene.index.MultiFields; import org.apache.lucene.util.StringHelper; import java.io.*; @@ -58,7 +59,7 @@ LuceneIterator() { try { - Terms terms = reader.fields().terms(field); + final Terms terms = MultiFields.getTerms(reader, field); if (terms != null) { termsEnum = terms.iterator(); pendingTerm = termsEnum.next();