Index: lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java =================================================================== --- lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java (revision 1522598) +++ lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java (working copy) @@ -804,11 +804,6 @@ } @Override - public Comparator getComparator() { - return BytesRef.getUTF8SortedAsUnicodeComparator(); - } - - @Override public long size() { return info.terms.size(); } @@ -966,11 +961,6 @@ } @Override - public Comparator getComparator() { - return BytesRef.getUTF8SortedAsUnicodeComparator(); - } - - @Override public void seekExact(BytesRef term, TermState state) throws IOException { assert state != null; this.seekExact(((OrdTermState)state).ord); Index: lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java =================================================================== --- lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java (revision 1522598) +++ lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java (working copy) @@ -1477,7 +1477,6 @@ * checks collection-level statistics on Terms */ public void assertTermsStatisticsEquals(String info, Terms leftTerms, Terms rightTerms) throws IOException { - assert leftTerms.getComparator() == rightTerms.getComparator(); if (leftTerms.getDocCount() != -1 && rightTerms.getDocCount() != -1) { assertEquals(info, leftTerms.getDocCount(), rightTerms.getDocCount()); } Index: lucene/test-framework/src/java/org/apache/lucene/codecs/ramonly/RAMOnlyPostingsFormat.java =================================================================== --- lucene/test-framework/src/java/org/apache/lucene/codecs/ramonly/RAMOnlyPostingsFormat.java (revision 1522598) +++ lucene/test-framework/src/java/org/apache/lucene/codecs/ramonly/RAMOnlyPostingsFormat.java (working copy) @@ -34,6 +34,7 @@ import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.PostingsConsumer; import org.apache.lucene.codecs.PostingsFormat; +import org.apache.lucene.codecs.PushFieldsConsumer; import org.apache.lucene.codecs.TermStats; import org.apache.lucene.codecs.TermsConsumer; import org.apache.lucene.index.DocsAndPositionsEnum; @@ -60,41 +61,6 @@ public final class RAMOnlyPostingsFormat extends PostingsFormat { - // For fun, test that we can override how terms are - // sorted, and basic things still work -- this comparator - // sorts in reversed unicode code point order: - private static final Comparator reverseUnicodeComparator = new Comparator() { - @Override - public int compare(BytesRef t1, BytesRef t2) { - byte[] b1 = t1.bytes; - byte[] b2 = t2.bytes; - int b1Stop; - int b1Upto = t1.offset; - int b2Upto = t2.offset; - if (t1.length < t2.length) { - b1Stop = t1.offset + t1.length; - } else { - b1Stop = t1.offset + t2.length; - } - while(b1Upto < b1Stop) { - final int bb1 = b1[b1Upto++] & 0xff; - final int bb2 = b2[b2Upto++] & 0xff; - if (bb1 != bb2) { - //System.out.println("cmp 1=" + t1 + " 2=" + t2 + " return " + (bb2-bb1)); - return bb2 - bb1; - } - } - - // One is prefix of another, or they are equal - return t2.length-t1.length; - } - - @Override - public boolean equals(Object other) { - return this == other; - } - }; - public RAMOnlyPostingsFormat() { super("RAMOnly"); } @@ -180,11 +146,6 @@ } @Override - public Comparator getComparator() { - return reverseUnicodeComparator; - } - - @Override public boolean hasOffsets() { return info.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0; } @@ -243,12 +204,13 @@ } // Classes for writing to the postings state - private static class RAMFieldsConsumer extends FieldsConsumer { + private static class RAMFieldsConsumer extends PushFieldsConsumer { private final RAMPostings postings; private final RAMTermsConsumer termsConsumer = new RAMTermsConsumer(); - public RAMFieldsConsumer(RAMPostings postings) { + public RAMFieldsConsumer(SegmentWriteState writeState, RAMPostings postings) { + super(writeState); this.postings = postings; } @@ -286,13 +248,7 @@ return postingsWriter; } - @Override - public Comparator getComparator() { - return BytesRef.getUTF8SortedAsUnicodeComparator(); - } - - @Override public void finishTerm(BytesRef text, TermStats stats) { assert stats.docFreq > 0; assert stats.docFreq == current.docs.size(); @@ -355,11 +311,6 @@ } @Override - public Comparator getComparator() { - return BytesRef.getUTF8SortedAsUnicodeComparator(); - } - - @Override public BytesRef next() { if (it == null) { if (current == null) { @@ -586,7 +537,7 @@ } final RAMPostings postings = new RAMPostings(); - final RAMFieldsConsumer consumer = new RAMFieldsConsumer(postings); + final RAMFieldsConsumer consumer = new RAMFieldsConsumer(writeState, postings); synchronized(state) { state.put(id, postings); Index: lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPostingsFormat.java =================================================================== --- lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPostingsFormat.java (revision 1522598) +++ lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPostingsFormat.java (working copy) @@ -25,15 +25,20 @@ import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.PostingsConsumer; import org.apache.lucene.codecs.PostingsFormat; +import org.apache.lucene.codecs.PushFieldsConsumer; import org.apache.lucene.codecs.TermStats; import org.apache.lucene.codecs.TermsConsumer; import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat; import org.apache.lucene.index.AssertingAtomicReader; +import org.apache.lucene.index.DocsAndPositionsEnum; +import org.apache.lucene.index.DocsEnum; +import org.apache.lucene.index.FieldInfo.IndexOptions; import org.apache.lucene.index.FieldInfo; -import org.apache.lucene.index.FieldInfo.IndexOptions; +import org.apache.lucene.index.Fields; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.Terms; +import org.apache.lucene.index.TermsEnum; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.OpenBitSet; @@ -49,7 +54,12 @@ @Override public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { - return new AssertingFieldsConsumer(in.fieldsConsumer(state)); + FieldsConsumer fieldsConsumer = in.fieldsConsumer(state); + if (fieldsConsumer instanceof PushFieldsConsumer) { + return new AssertingPushFieldsConsumer(state, (PushFieldsConsumer) fieldsConsumer); + } else { + return new AssertingFieldsConsumer(state, fieldsConsumer); + } } @Override @@ -92,11 +102,12 @@ return in.ramBytesUsed(); } } - - static class AssertingFieldsConsumer extends FieldsConsumer { - private final FieldsConsumer in; + + static class AssertingPushFieldsConsumer extends PushFieldsConsumer { + private final PushFieldsConsumer in; - AssertingFieldsConsumer(FieldsConsumer in) { + AssertingPushFieldsConsumer(SegmentWriteState writeState, PushFieldsConsumer in) { + super(writeState); this.in = in; } @@ -112,6 +123,110 @@ in.close(); } } + + static class AssertingFieldsConsumer extends FieldsConsumer { + private final FieldsConsumer in; + private final SegmentWriteState writeState; + + AssertingFieldsConsumer(SegmentWriteState writeState, FieldsConsumer in) { + this.writeState = writeState; + this.in = in; + } + + @Override + public void write(Fields fields) throws IOException { + in.write(fields); + + // nocommit more asserts? can we somehow run a + // "limited" CheckIndex here??? + String lastField = null; + TermsEnum termsEnum = null; + + for(String field : fields) { + + FieldInfo fieldInfo = writeState.fieldInfos.fieldInfo(field); + assert fieldInfo != null; + assert lastField == null || lastField.compareTo(field) < 0; + lastField = field; + + Terms terms = fields.terms(field); + assert terms != null; + + termsEnum = terms.iterator(termsEnum); + BytesRef lastTerm = null; + DocsEnum docsEnum = null; + DocsAndPositionsEnum posEnum = null; + + boolean hasFreqs = fieldInfo.getIndexOptions().compareTo(FieldInfo.IndexOptions.DOCS_AND_FREQS) >= 0; + boolean hasPositions = fieldInfo.getIndexOptions().compareTo(FieldInfo.IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; + boolean hasOffsets = fieldInfo.getIndexOptions().compareTo(FieldInfo.IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0; + + assert hasPositions == terms.hasPositions(); + assert hasOffsets == terms.hasOffsets(); + + while(true) { + BytesRef term = termsEnum.next(); + if (term == null) { + break; + } + assert lastTerm == null || lastTerm.compareTo(term) < 0; + if (lastTerm == null) { + lastTerm = BytesRef.deepCopyOf(term); + } else { + lastTerm.copyBytes(term); + } + + if (hasPositions == false) { + int flags = 0; + if (hasFreqs) { + flags = flags | DocsEnum.FLAG_FREQS; + } + docsEnum = termsEnum.docs(null, docsEnum, flags); + } else { + int flags = DocsAndPositionsEnum.FLAG_PAYLOADS; + if (hasOffsets) { + flags = flags | DocsAndPositionsEnum.FLAG_OFFSETS; + } + posEnum = termsEnum.docsAndPositions(null, posEnum, flags); + docsEnum = posEnum; + } + + int lastDocID = -1; + + while(true) { + int docID = docsEnum.nextDoc(); + if (docID == DocsEnum.NO_MORE_DOCS) { + break; + } + assert docID > lastDocID; + lastDocID = docID; + if (hasFreqs) { + int freq = docsEnum.freq(); + assert freq > 0; + + if (hasPositions) { + int lastPos = -1; + int lastStartOffset = -1; + for(int i=0;i lastPos; + lastPos = pos; + + if (hasOffsets) { + int startOffset = posEnum.startOffset(); + int endOffset = posEnum.endOffset(); + assert endOffset > startOffset; + assert startOffset >= lastStartOffset; + lastStartOffset = startOffset; + } + } + } + } + } + } + } + } + } static enum TermsConsumerState { INITIAL, START, FINISHED }; static class AssertingTermsConsumer extends TermsConsumer { @@ -123,6 +238,7 @@ private long sumTotalTermFreq = 0; private long sumDocFreq = 0; private OpenBitSet visitedDocs = new OpenBitSet(); + private static final Comparator termComp = BytesRef.getUTF8SortedAsUnicodeComparator(); AssertingTermsConsumer(TermsConsumer in, FieldInfo fieldInfo) { this.in = in; @@ -133,7 +249,7 @@ public PostingsConsumer startTerm(BytesRef text) throws IOException { assert state == TermsConsumerState.INITIAL || state == TermsConsumerState.START && lastPostingsConsumer.docFreq == 0; state = TermsConsumerState.START; - assert lastTerm == null || in.getComparator().compare(text, lastTerm) > 0; + assert lastTerm == null || termComp.compare(text, lastTerm) > 0; lastTerm = BytesRef.deepCopyOf(text); return lastPostingsConsumer = new AssertingPostingsConsumer(in.startTerm(text), fieldInfo, visitedDocs); } @@ -171,11 +287,6 @@ } in.finish(sumTotalTermFreq, sumDocFreq, docCount); } - - @Override - public Comparator getComparator() throws IOException { - return in.getComparator(); - } } static enum PostingsConsumerState { INITIAL, START }; Index: lucene/test-framework/src/java/org/apache/lucene/analysis/LookaheadTokenFilter.java =================================================================== --- lucene/test-framework/src/java/org/apache/lucene/analysis/LookaheadTokenFilter.java (revision 1522598) +++ lucene/test-framework/src/java/org/apache/lucene/analysis/LookaheadTokenFilter.java (working copy) @@ -37,7 +37,7 @@ public abstract class LookaheadTokenFilter extends TokenFilter { - private final static boolean DEBUG = false; + private final static boolean DEBUG = true; protected final PositionIncrementAttribute posIncAtt = addAttribute(PositionIncrementAttribute.class); protected final PositionLengthAttribute posLenAtt = addAttribute(PositionLengthAttribute.class); Index: lucene/test-framework/src/java/org/apache/lucene/index/BaseTermVectorsFormatTestCase.java =================================================================== --- lucene/test-framework/src/java/org/apache/lucene/index/BaseTermVectorsFormatTestCase.java (revision 1522598) +++ lucene/test-framework/src/java/org/apache/lucene/index/BaseTermVectorsFormatTestCase.java (working copy) @@ -406,7 +406,7 @@ uniqueTerms.add(new BytesRef(term)); } final BytesRef[] sortedTerms = uniqueTerms.toArray(new BytesRef[0]); - Arrays.sort(sortedTerms, terms.getComparator()); + Arrays.sort(sortedTerms, BytesRef.getUTF8SortedAsUnicodeComparator()); final TermsEnum termsEnum = terms.iterator(random().nextBoolean() ? null : this.termsEnum.get()); this.termsEnum.set(termsEnum); for (int i = 0; i < sortedTerms.length; ++i) { Index: lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java =================================================================== --- lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java (revision 1522598) +++ lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java (working copy) @@ -22,7 +22,9 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.Comparator; import java.util.EnumSet; +import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.List; @@ -30,14 +32,17 @@ import java.util.NoSuchElementException; import java.util.Random; import java.util.Set; +import java.util.SortedMap; import java.util.TreeMap; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.lucene.analysis.MockAnalyzer; import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.FieldsConsumer; import org.apache.lucene.codecs.FieldsProducer; -import org.apache.lucene.codecs.PostingsConsumer; -import org.apache.lucene.codecs.TermStats; -import org.apache.lucene.codecs.TermsConsumer; +import org.apache.lucene.codecs.PostingsFormat; +import org.apache.lucene.codecs.lucene45.Lucene45Codec; +import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; import org.apache.lucene.index.FieldInfo.DocValuesType; @@ -49,7 +54,9 @@ import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.Constants; import org.apache.lucene.util.FixedBitSet; +import org.apache.lucene.util.LineFileDocs; import org.apache.lucene.util.LuceneTestCase; +import org.apache.lucene.util.RamUsageEstimator; import org.apache.lucene.util._TestUtil; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -126,6 +133,7 @@ private final BytesRef payload; private final IndexOptions options; private final boolean doPositions; + private final boolean allowPayloads; private int docID; private int freq; @@ -138,11 +146,12 @@ private int posSpacing; private int posUpto; - public SeedPostings(long seed, int minDocFreq, int maxDocFreq, Bits liveDocs, IndexOptions options) { + public SeedPostings(long seed, int minDocFreq, int maxDocFreq, Bits liveDocs, IndexOptions options, boolean allowPayloads) { random = new Random(seed); docRandom = new Random(random.nextLong()); docFreq = _TestUtil.nextInt(random, minDocFreq, maxDocFreq); this.liveDocs = liveDocs; + this.allowPayloads = allowPayloads; // TODO: more realistic to inversely tie this to numDocs: maxDocSpacing = _TestUtil.nextInt(random, 1, 100); @@ -249,6 +258,9 @@ } else { payload.length = 0; } + if (!allowPayloads) { + payload.length = 0; + } startOffset = offset + random.nextInt(5); endOffset = startOffset + random.nextInt(10); @@ -295,7 +307,7 @@ } // Holds all postings: - private static Map> fields; + private static Map> fields; private static FieldInfos fieldInfos; @@ -307,7 +319,7 @@ private static long totalPostings; private static long totalPayloadBytes; - private static SeedPostings getSeedPostings(String term, long seed, boolean withLiveDocs, IndexOptions options) { + private static SeedPostings getSeedPostings(String term, long seed, boolean withLiveDocs, IndexOptions options, boolean allowPayloads) { int minDocFreq, maxDocFreq; if (term.startsWith("big_")) { minDocFreq = RANDOM_MULTIPLIER * 50000; @@ -323,14 +335,14 @@ maxDocFreq = 3; } - return new SeedPostings(seed, minDocFreq, maxDocFreq, withLiveDocs ? globalLiveDocs : null, options); + return new SeedPostings(seed, minDocFreq, maxDocFreq, withLiveDocs ? globalLiveDocs : null, options, allowPayloads); } @BeforeClass public static void createPostings() throws IOException { totalPostings = 0; totalPayloadBytes = 0; - fields = new TreeMap>(); + fields = new TreeMap>(); final int numFields = _TestUtil.nextInt(random(), 1, 5); if (VERBOSE) { @@ -351,7 +363,7 @@ null, DocValuesType.NUMERIC, null); fieldUpto++; - Map postings = new TreeMap(); + SortedMap postings = new TreeMap(); fields.put(field, postings); Set seenTerms = new HashSet(); @@ -388,7 +400,7 @@ // NOTE: sort of silly: we enum all the docs just to // get the maxDoc - DocsEnum docsEnum = getSeedPostings(term, termSeed, false, IndexOptions.DOCS_ONLY); + DocsEnum docsEnum = getSeedPostings(term, termSeed, false, IndexOptions.DOCS_ONLY, true); int doc; int lastDoc = 0; while((doc = docsEnum.nextDoc()) != DocsEnum.NO_MORE_DOCS) { @@ -412,7 +424,7 @@ } allTerms = new ArrayList(); - for(Map.Entry> fieldEnt : fields.entrySet()) { + for(Map.Entry> fieldEnt : fields.entrySet()) { String field = fieldEnt.getKey(); for(Map.Entry termEnt : fieldEnt.getValue().entrySet()) { allTerms.add(new FieldAndTerm(field, termEnt.getKey())); @@ -432,6 +444,206 @@ globalLiveDocs = null; } + private static class SeedFields extends Fields { + final Map> fields; + final FieldInfos fieldInfos; + final IndexOptions maxAllowed; + final boolean allowPayloads; + + public SeedFields(Map> fields, FieldInfos fieldInfos, IndexOptions maxAllowed, boolean allowPayloads) { + this.fields = fields; + this.fieldInfos = fieldInfos; + this.maxAllowed = maxAllowed; + this.allowPayloads = allowPayloads; + } + + @Override + public Iterator iterator() { + return fields.keySet().iterator(); + } + + @Override + public Terms terms(String field) { + SortedMap terms = fields.get(field); + if (terms == null) { + return null; + } else { + return new SeedTerms(terms, fieldInfos.fieldInfo(field), maxAllowed, allowPayloads); + } + } + + @Override + public int size() { + return fields.size(); + } + } + + private static class SeedTerms extends Terms { + final SortedMap terms; + final FieldInfo fieldInfo; + final IndexOptions maxAllowed; + final boolean allowPayloads; + + public SeedTerms(SortedMap terms, FieldInfo fieldInfo, IndexOptions maxAllowed, boolean allowPayloads) { + this.terms = terms; + this.fieldInfo = fieldInfo; + this.maxAllowed = maxAllowed; + this.allowPayloads = allowPayloads; + } + + @Override + public TermsEnum iterator(TermsEnum reuse) { + SeedTermsEnum termsEnum; + if (reuse != null && reuse instanceof SeedTermsEnum) { + termsEnum = (SeedTermsEnum) reuse; + if (termsEnum.terms != terms) { + termsEnum = new SeedTermsEnum(terms, maxAllowed, allowPayloads); + } + } else { + termsEnum = new SeedTermsEnum(terms, maxAllowed, allowPayloads); + } + termsEnum.reset(); + + return termsEnum; + } + + @Override + public long size() { + return terms.size(); + } + + @Override + public long getSumTotalTermFreq() { + throw new UnsupportedOperationException(); + } + + @Override + public long getSumDocFreq() { + throw new UnsupportedOperationException(); + } + + @Override + public int getDocCount() { + throw new UnsupportedOperationException(); + } + + @Override + public boolean hasOffsets() { + return fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0; + } + + @Override + public boolean hasPositions() { + return fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; + } + + @Override + public boolean hasPayloads() { + return fieldInfo.hasPayloads(); + } + } + + private static class SeedTermsEnum extends TermsEnum { + final SortedMap terms; + final IndexOptions maxAllowed; + final boolean allowPayloads; + + private Iterator> iterator; + + private Map.Entry current; + + public SeedTermsEnum(SortedMap terms, IndexOptions maxAllowed, boolean allowPayloads) { + this.terms = terms; + this.maxAllowed = maxAllowed; + this.allowPayloads = allowPayloads; + } + + void reset() { + iterator = terms.entrySet().iterator(); + } + + @Override + public SeekStatus seekCeil(BytesRef text) { + SortedMap tailMap = terms.tailMap(text); + if (tailMap.isEmpty()) { + return SeekStatus.END; + } else { + iterator = tailMap.entrySet().iterator(); + if (tailMap.firstKey().equals(text)) { + return SeekStatus.FOUND; + } else { + return SeekStatus.NOT_FOUND; + } + } + } + + @Override + public BytesRef next() { + if (iterator.hasNext()) { + current = iterator.next(); + return term(); + } else { + return null; + } + } + + @Override + public void seekExact(long ord) { + throw new UnsupportedOperationException(); + } + + @Override + public BytesRef term() { + return current.getKey(); + } + + @Override + public long ord() { + throw new UnsupportedOperationException(); + } + + @Override + public int docFreq() { + throw new UnsupportedOperationException(); + } + + @Override + public long totalTermFreq() { + throw new UnsupportedOperationException(); + } + + @Override + public final DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException { + if (liveDocs != null) { + throw new IllegalArgumentException("liveDocs must be null"); + } + if ((flags & DocsEnum.FLAG_FREQS) != 0 && maxAllowed.compareTo(IndexOptions.DOCS_AND_FREQS) < 0) { + return null; + } + return getSeedPostings(current.getKey().utf8ToString(), current.getValue(), false, maxAllowed, allowPayloads); + } + + @Override + public final DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException { + if (liveDocs != null) { + throw new IllegalArgumentException("liveDocs must be null"); + } + if (maxAllowed.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) { + System.out.println("no: max"); + return null; + } + if ((flags & DocsAndPositionsEnum.FLAG_OFFSETS) != 0 && maxAllowed.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) < 0) { + System.out.println("no: offsets"); + return null; + } + if ((flags & DocsAndPositionsEnum.FLAG_PAYLOADS) != 0 && allowPayloads == false) { + System.out.println("no: payloads"); + return null; + } + return getSeedPostings(current.getKey().utf8ToString(), current.getValue(), false, maxAllowed, allowPayloads); + } + } + // TODO maybe instead of @BeforeClass just make a single test run: build postings & index & test it? private FieldInfos currentFieldInfos; @@ -489,80 +701,11 @@ SegmentWriteState writeState = new SegmentWriteState(null, dir, segmentInfo, newFieldInfos, null, new IOContext(new FlushInfo(maxDoc, bytes))); - FieldsConsumer fieldsConsumer = codec.postingsFormat().fieldsConsumer(writeState); - for(Map.Entry> fieldEnt : fields.entrySet()) { - String field = fieldEnt.getKey(); - Map terms = fieldEnt.getValue(); + Fields seedFields = new SeedFields(fields, newFieldInfos, maxAllowed, allowPayloads); - FieldInfo fieldInfo = newFieldInfos.fieldInfo(field); + codec.postingsFormat().fieldsConsumer(writeState).write(seedFields); - IndexOptions indexOptions = fieldInfo.getIndexOptions(); - - if (VERBOSE) { - System.out.println("field=" + field + " indexOtions=" + indexOptions); - } - - boolean doFreq = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS) >= 0; - boolean doPos = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; - boolean doPayloads = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0 && allowPayloads; - boolean doOffsets = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0; - - TermsConsumer termsConsumer = fieldsConsumer.addField(fieldInfo); - long sumTotalTF = 0; - long sumDF = 0; - FixedBitSet seenDocs = new FixedBitSet(maxDoc); - for(Map.Entry termEnt : terms.entrySet()) { - BytesRef term = termEnt.getKey(); - SeedPostings postings = getSeedPostings(term.utf8ToString(), termEnt.getValue(), false, maxAllowed); - if (VERBOSE) { - System.out.println(" term=" + field + ":" + term.utf8ToString() + " docFreq=" + postings.docFreq + " seed=" + termEnt.getValue()); - } - - PostingsConsumer postingsConsumer = termsConsumer.startTerm(term); - long totalTF = 0; - int docID = 0; - while((docID = postings.nextDoc()) != DocsEnum.NO_MORE_DOCS) { - final int freq = postings.freq(); - if (VERBOSE) { - System.out.println(" " + postings.upto + ": docID=" + docID + " freq=" + postings.freq); - } - postingsConsumer.startDoc(docID, doFreq ? postings.freq : -1); - seenDocs.set(docID); - if (doPos) { - totalTF += postings.freq; - for(int posUpto=0;posUpto termComp = BytesRef.getUTF8SortedAsUnicodeComparator(); + main: while (thisTerm != null) {//terminates for other reasons too! //Advance curVNode pointer @@ -176,7 +179,7 @@ //Seek to curVNode's cell (or skip if termsEnum has moved beyond) curVNodeTerm.bytes = curVNode.cell.getTokenBytes(); curVNodeTerm.length = curVNodeTerm.bytes.length; - int compare = termsEnum.getComparator().compare(thisTerm, curVNodeTerm); + int compare = termComp.compare(thisTerm, curVNodeTerm); if (compare > 0) { // leap frog (termsEnum is beyond where we would otherwise seek) assert ! context.reader().terms(fieldName).iterator(null).seekExact(curVNodeTerm) : "should be absent"; Index: lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPostingsFormat.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPostingsFormat.java (revision 1522598) +++ lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPostingsFormat.java (working copy) @@ -22,9 +22,9 @@ import org.apache.lucene.codecs.FieldsConsumer; import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.PostingsFormat; +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentWriteState; -import org.apache.lucene.index.SegmentReadState; -import org.apache.lucene.index.IndexFileNames; /** For debugging, curiosity, transparency only!! Do not * use this codec in production. @@ -40,7 +40,7 @@ super("SimpleText"); } - @Override + /** Writes a new segment */ public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { return new SimpleTextFieldsWriter(state); } Index: lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java (revision 1522598) +++ lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java (working copy) @@ -218,11 +218,6 @@ } return docsAndPositionsEnum.reset(docsStart, liveDocs, indexOptions, docFreq); } - - @Override - public Comparator getComparator() { - return BytesRef.getUTF8SortedAsUnicodeComparator(); - } } private class SimpleTextDocsEnum extends DocsEnum { @@ -590,11 +585,6 @@ } @Override - public Comparator getComparator() { - return BytesRef.getUTF8SortedAsUnicodeComparator(); - } - - @Override public long size() { return (long) termCount; } Index: lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java (revision 1522598) +++ lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java (working copy) @@ -17,23 +17,28 @@ * limitations under the License. */ -import org.apache.lucene.util.BytesRef; +import java.io.Closeable; +import java.io.IOException; + import org.apache.lucene.codecs.FieldsConsumer; -import org.apache.lucene.codecs.PostingsConsumer; -import org.apache.lucene.codecs.TermStats; -import org.apache.lucene.codecs.TermsConsumer; +import org.apache.lucene.index.DocsAndPositionsEnum; +import org.apache.lucene.index.DocsEnum; +import org.apache.lucene.index.FieldInfo.IndexOptions; import org.apache.lucene.index.FieldInfo; -import org.apache.lucene.index.FieldInfo.IndexOptions; +import org.apache.lucene.index.FieldInfos; +import org.apache.lucene.index.Fields; import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.index.Terms; +import org.apache.lucene.index.TermsEnum; import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.IOUtils; -import java.io.IOException; -import java.util.Comparator; - -class SimpleTextFieldsWriter extends FieldsConsumer { +class SimpleTextFieldsWriter extends FieldsConsumer implements Closeable { private final IndexOutput out; private final BytesRef scratch = new BytesRef(10); + private final SegmentWriteState writeState; final static BytesRef END = new BytesRef("END"); final static BytesRef FIELD = new BytesRef("field "); @@ -45,136 +50,169 @@ final static BytesRef END_OFFSET = new BytesRef(" endOffset "); final static BytesRef PAYLOAD = new BytesRef(" payload "); - public SimpleTextFieldsWriter(SegmentWriteState state) throws IOException { - final String fileName = SimpleTextPostingsFormat.getPostingsFileName(state.segmentInfo.name, state.segmentSuffix); - out = state.directory.createOutput(fileName, state.context); + public SimpleTextFieldsWriter(SegmentWriteState writeState) throws IOException { + final String fileName = SimpleTextPostingsFormat.getPostingsFileName(writeState.segmentInfo.name, writeState.segmentSuffix); + out = writeState.directory.createOutput(fileName, writeState.context); + this.writeState = writeState; } - private void write(String s) throws IOException { - SimpleTextUtil.write(out, s, scratch); + @Override + public void write(Fields fields) throws IOException { + boolean success = false; + try { + write(writeState.fieldInfos, fields); + success = true; + } finally { + if (success) { + IOUtils.close(this); + } else { + IOUtils.closeWhileHandlingException(this); + } + } } - private void write(BytesRef b) throws IOException { - SimpleTextUtil.write(out, b); - } + public void write(FieldInfos fieldInfos, Fields fields) throws IOException { - private void newline() throws IOException { - SimpleTextUtil.writeNewline(out); - } + // for each field + for(String field : fields) { + Terms terms = fields.terms(field); + FieldInfo fieldInfo = fieldInfos.fieldInfo(field); - @Override - public TermsConsumer addField(FieldInfo field) throws IOException { - write(FIELD); - write(field.name); - newline(); - return new SimpleTextTermsWriter(field); - } + boolean wroteField = false; - private class SimpleTextTermsWriter extends TermsConsumer { - private final SimpleTextPostingsWriter postingsWriter; - - public SimpleTextTermsWriter(FieldInfo field) { - postingsWriter = new SimpleTextPostingsWriter(field); - } + boolean hasPositions = terms.hasPositions(); - @Override - public PostingsConsumer startTerm(BytesRef term) throws IOException { - return postingsWriter.reset(term); - } + // TODO: shouldn't we add hasFreqs to Terms? + // then we don't need FieldInfos here? + boolean hasFreqs = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_ONLY) > 0; + boolean hasPayloads = fieldInfo.hasPayloads(); + boolean hasOffsets = terms.hasOffsets(); - @Override - public void finishTerm(BytesRef term, TermStats stats) throws IOException { - } + int flags = 0; + if (hasPositions) { + + if (hasPayloads) { + flags = flags | DocsAndPositionsEnum.FLAG_PAYLOADS; + } + if (hasOffsets) { + flags = flags | DocsAndPositionsEnum.FLAG_OFFSETS; + } + } else { + if (hasFreqs) { + flags = flags | DocsEnum.FLAG_FREQS; + } + } - @Override - public void finish(long sumTotalTermFreq, long sumDocFreq, int docCount) throws IOException { - } + TermsEnum termsEnum = terms.iterator(null); + DocsAndPositionsEnum posEnum = null; + DocsEnum docsEnum = null; - @Override - public Comparator getComparator() { - return BytesRef.getUTF8SortedAsUnicodeComparator(); - } - } + // for each term in field + while(true) { + BytesRef term = termsEnum.next(); + if (term == null) { + break; + } - private class SimpleTextPostingsWriter extends PostingsConsumer { - private BytesRef term; - private boolean wroteTerm; - private final IndexOptions indexOptions; - private final boolean writePositions; - private final boolean writeOffsets; + if (hasPositions) { + posEnum = termsEnum.docsAndPositions(null, posEnum, flags); + docsEnum = posEnum; + } else { + docsEnum = termsEnum.docs(null, docsEnum, flags); + } + assert docsEnum != null: "termsEnum=" + termsEnum + " hasPos=" + hasPositions + " flags=" + flags; - // for assert: - private int lastStartOffset = 0; + boolean wroteTerm = false; - public SimpleTextPostingsWriter(FieldInfo field) { - this.indexOptions = field.getIndexOptions(); - writePositions = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; - writeOffsets = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0; - //System.out.println("writeOffsets=" + writeOffsets); - //System.out.println("writePos=" + writePositions); - } + // for each doc in field+term + while(true) { + int doc = docsEnum.nextDoc(); + if (doc == DocsEnum.NO_MORE_DOCS) { + break; + } - @Override - public void startDoc(int docID, int termDocFreq) throws IOException { - if (!wroteTerm) { - // we lazily do this, in case the term had zero docs - write(TERM); - write(term); - newline(); - wroteTerm = true; - } + if (!wroteTerm) { - write(DOC); - write(Integer.toString(docID)); - newline(); - if (indexOptions != IndexOptions.DOCS_ONLY) { - write(FREQ); - write(Integer.toString(termDocFreq)); - newline(); - } + if (!wroteField) { + // we lazily do this, in case the field had + // no terms + write(FIELD); + write(field); + newline(); + wroteField = true; + } - lastStartOffset = 0; - } - - public PostingsConsumer reset(BytesRef term) { - this.term = term; - wroteTerm = false; - return this; - } + // we lazily do this, in case the term had + // zero docs + write(TERM); + write(term); + newline(); + wroteTerm = true; + } - @Override - public void addPosition(int position, BytesRef payload, int startOffset, int endOffset) throws IOException { - if (writePositions) { - write(POS); - write(Integer.toString(position)); - newline(); - } + write(DOC); + write(Integer.toString(doc)); + newline(); + if (hasFreqs) { + int freq = docsEnum.freq(); + write(FREQ); + write(Integer.toString(freq)); + newline(); - if (writeOffsets) { - assert endOffset >= startOffset; - assert startOffset >= lastStartOffset: "startOffset=" + startOffset + " lastStartOffset=" + lastStartOffset; - lastStartOffset = startOffset; - write(START_OFFSET); - write(Integer.toString(startOffset)); - newline(); - write(END_OFFSET); - write(Integer.toString(endOffset)); - newline(); - } + if (hasPositions) { + // for assert: + int lastStartOffset = 0; - if (payload != null && payload.length > 0) { - assert payload.length != 0; - write(PAYLOAD); - write(payload); - newline(); + // for each pos in field+term+doc + for(int i=0;i= startOffset; + assert startOffset >= lastStartOffset: "startOffset=" + startOffset + " lastStartOffset=" + lastStartOffset; + lastStartOffset = startOffset; + write(START_OFFSET); + write(Integer.toString(startOffset)); + newline(); + write(END_OFFSET); + write(Integer.toString(endOffset)); + newline(); + } + + BytesRef payload = posEnum.getPayload(); + + if (payload != null && payload.length > 0) { + assert payload.length != 0; + write(PAYLOAD); + write(payload); + newline(); + } + } + } + } + } } } + } - @Override - public void finishDoc() { - } + private void write(String s) throws IOException { + SimpleTextUtil.write(out, s, scratch); } + private void write(BytesRef b) throws IOException { + SimpleTextUtil.write(out, b); + } + + private void newline() throws IOException { + SimpleTextUtil.writeNewline(out); + } + @Override public void close() throws IOException { try { Index: lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java (revision 1522598) +++ lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java (working copy) @@ -272,11 +272,6 @@ } @Override - public Comparator getComparator() { - return BytesRef.getUTF8SortedAsUnicodeComparator(); - } - - @Override public long size() throws IOException { return terms.size(); } @@ -394,11 +389,6 @@ e.reset(liveDocs, postings.positions, postings.startOffsets, postings.endOffsets, postings.payloads); return e; } - - @Override - public Comparator getComparator() { - return BytesRef.getUTF8SortedAsUnicodeComparator(); - } } // note: these two enum classes are exactly like the Default impl... Index: lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsReader.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsReader.java (revision 1522598) +++ lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsReader.java (working copy) @@ -245,11 +245,6 @@ } @Override - public Comparator getComparator() { - return BytesRef.getUTF8SortedAsUnicodeComparator(); - } - - @Override public TermsEnum iterator(TermsEnum reuse) throws IOException { return new SegmentTermsEnum(); } @@ -349,11 +344,6 @@ longs = new long[longsSize]; } - @Override - public Comparator getComparator() { - return BytesRef.getUTF8SortedAsUnicodeComparator(); - } - // TODO: we may want an alternate mode here which is // "if you are about to return NOT_FOUND I won't use // the terms data from that"; eg FuzzyTermsEnum will Index: lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsWriter.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsWriter.java (revision 1522598) +++ lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsWriter.java (working copy) @@ -19,18 +19,17 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Comparator; import java.util.List; import org.apache.lucene.codecs.CodecUtil; -import org.apache.lucene.codecs.FieldsConsumer; import org.apache.lucene.codecs.PostingsConsumer; import org.apache.lucene.codecs.PostingsWriterBase; +import org.apache.lucene.codecs.PushFieldsConsumer; import org.apache.lucene.codecs.TermStats; import org.apache.lucene.codecs.BlockTermState; import org.apache.lucene.codecs.TermsConsumer; +import org.apache.lucene.index.FieldInfo.IndexOptions; import org.apache.lucene.index.FieldInfo; -import org.apache.lucene.index.FieldInfo.IndexOptions; import org.apache.lucene.index.FieldInfos; import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.SegmentWriteState; @@ -53,7 +52,7 @@ * @lucene.experimental */ -public class BlockTermsWriter extends FieldsConsumer { +public class BlockTermsWriter extends PushFieldsConsumer { final static String CODEC_NAME = "BLOCK_TERMS_DICT"; @@ -100,6 +99,7 @@ public BlockTermsWriter(TermsIndexWriterBase termsIndexWriter, SegmentWriteState state, PostingsWriterBase postingsWriter) throws IOException { + super(state); final String termsFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TERMS_EXTENSION); this.termsIndexWriter = termsIndexWriter; out = state.directory.createOutput(termsFileName, state.context); @@ -201,11 +201,6 @@ } @Override - public Comparator getComparator() { - return BytesRef.getUTF8SortedAsUnicodeComparator(); - } - - @Override public PostingsConsumer startTerm(BytesRef text) throws IOException { //System.out.println("BTW: startTerm term=" + fieldInfo.name + ":" + text.utf8ToString() + " " + text + " seg=" + segment); postingsWriter.startTerm(); Index: lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsWriter.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsWriter.java (revision 1522598) +++ lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsWriter.java (working copy) @@ -18,10 +18,16 @@ */ import java.io.IOException; +import java.util.ArrayList; import java.util.List; -import java.util.ArrayList; -import java.util.Comparator; +import org.apache.lucene.codecs.BlockTermState; +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.PostingsConsumer; +import org.apache.lucene.codecs.PostingsWriterBase; +import org.apache.lucene.codecs.PushFieldsConsumer; +import org.apache.lucene.codecs.TermStats; +import org.apache.lucene.codecs.TermsConsumer; import org.apache.lucene.index.FieldInfo.IndexOptions; import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.FieldInfos; @@ -30,20 +36,12 @@ import org.apache.lucene.store.DataOutput; import org.apache.lucene.store.IndexOutput; import org.apache.lucene.store.RAMOutputStream; -import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IntsRef; import org.apache.lucene.util.fst.Builder; import org.apache.lucene.util.fst.FST; import org.apache.lucene.util.fst.Util; -import org.apache.lucene.codecs.BlockTermState; -import org.apache.lucene.codecs.PostingsWriterBase; -import org.apache.lucene.codecs.PostingsConsumer; -import org.apache.lucene.codecs.FieldsConsumer; -import org.apache.lucene.codecs.TermsConsumer; -import org.apache.lucene.codecs.TermStats; -import org.apache.lucene.codecs.CodecUtil; /** * FST-based term dict, using metadata as FST output. @@ -121,7 +119,7 @@ * @lucene.experimental */ -public class FSTTermsWriter extends FieldsConsumer { +public class FSTTermsWriter extends PushFieldsConsumer { static final String TERMS_EXTENSION = "tmp"; static final String TERMS_CODEC_NAME = "FST_TERMS_DICT"; public static final int TERMS_VERSION_START = 0; @@ -133,6 +131,7 @@ final List fields = new ArrayList(); public FSTTermsWriter(SegmentWriteState state, PostingsWriterBase postingsWriter) throws IOException { + super(state); final String termsFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TERMS_EXTENSION); this.postingsWriter = postingsWriter; @@ -217,7 +216,6 @@ private long numTerms; private final IntsRef scratchTerm = new IntsRef(); - private final RAMOutputStream statsWriter = new RAMOutputStream(); private final RAMOutputStream metaWriter = new RAMOutputStream(); TermsWriter(FieldInfo fieldInfo) { @@ -229,11 +227,6 @@ } @Override - public Comparator getComparator() { - return BytesRef.getUTF8SortedAsUnicodeComparator(); - } - - @Override public PostingsConsumer startTerm(BytesRef text) throws IOException { postingsWriter.startTerm(); return postingsWriter; Index: lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java (revision 1522598) +++ lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java (working copy) @@ -29,6 +29,7 @@ import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.PostingsConsumer; import org.apache.lucene.codecs.PostingsFormat; +import org.apache.lucene.codecs.PushFieldsConsumer; import org.apache.lucene.codecs.TermStats; import org.apache.lucene.codecs.TermsConsumer; import org.apache.lucene.index.DocsAndPositionsEnum; @@ -278,11 +279,6 @@ //System.out.println("finish field=" + field.name + " fp=" + out.getFilePointer()); } } - - @Override - public Comparator getComparator() { - return BytesRef.getUTF8SortedAsUnicodeComparator(); - } } private static String EXTENSION = "ram"; @@ -293,7 +289,7 @@ final String fileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, EXTENSION); final IndexOutput out = state.directory.createOutput(fileName, state.context); - return new FieldsConsumer() { + return new PushFieldsConsumer(state) { @Override public TermsConsumer addField(FieldInfo field) { //System.out.println("\naddField field=" + field.name); @@ -759,11 +755,6 @@ } @Override - public Comparator getComparator() { - return BytesRef.getUTF8SortedAsUnicodeComparator(); - } - - @Override public void seekExact(long ord) { // NOTE: we could add this... throw new UnsupportedOperationException(); @@ -827,11 +818,6 @@ } @Override - public Comparator getComparator() { - return BytesRef.getUTF8SortedAsUnicodeComparator(); - } - - @Override public boolean hasOffsets() { return field.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0; } Index: lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsWriter.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsWriter.java (revision 1522598) +++ lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsWriter.java (working copy) @@ -18,10 +18,16 @@ */ import java.io.IOException; +import java.util.ArrayList; import java.util.List; -import java.util.ArrayList; -import java.util.Comparator; +import org.apache.lucene.codecs.BlockTermState; +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.PostingsConsumer; +import org.apache.lucene.codecs.PostingsWriterBase; +import org.apache.lucene.codecs.PushFieldsConsumer; +import org.apache.lucene.codecs.TermStats; +import org.apache.lucene.codecs.TermsConsumer; import org.apache.lucene.index.FieldInfo.IndexOptions; import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.FieldInfos; @@ -30,7 +36,6 @@ import org.apache.lucene.store.DataOutput; import org.apache.lucene.store.IndexOutput; import org.apache.lucene.store.RAMOutputStream; -import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IntsRef; @@ -38,13 +43,6 @@ import org.apache.lucene.util.fst.FST; import org.apache.lucene.util.fst.PositiveIntOutputs; import org.apache.lucene.util.fst.Util; -import org.apache.lucene.codecs.BlockTermState; -import org.apache.lucene.codecs.PostingsWriterBase; -import org.apache.lucene.codecs.PostingsConsumer; -import org.apache.lucene.codecs.FieldsConsumer; -import org.apache.lucene.codecs.TermsConsumer; -import org.apache.lucene.codecs.TermStats; -import org.apache.lucene.codecs.CodecUtil; /** * FST-based term dict, using ord as FST output. @@ -144,7 +142,7 @@ * @lucene.experimental */ -public class FSTOrdTermsWriter extends FieldsConsumer { +public class FSTOrdTermsWriter extends PushFieldsConsumer { static final String TERMS_INDEX_EXTENSION = "tix"; static final String TERMS_BLOCK_EXTENSION = "tbk"; static final String TERMS_CODEC_NAME = "FST_ORD_TERMS_DICT"; @@ -159,6 +157,7 @@ IndexOutput indexOut = null; public FSTOrdTermsWriter(SegmentWriteState state, PostingsWriterBase postingsWriter) throws IOException { + super(state); final String termsIndexFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TERMS_INDEX_EXTENSION); final String termsBlockFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TERMS_BLOCK_EXTENSION); @@ -189,7 +188,6 @@ public void close() throws IOException { IOException ioe = null; try { - final long indexDirStart = indexOut.getFilePointer(); final long blockDirStart = blockOut.getFilePointer(); // write field summary @@ -287,11 +285,6 @@ } @Override - public Comparator getComparator() { - return BytesRef.getUTF8SortedAsUnicodeComparator(); - } - - @Override public PostingsConsumer startTerm(BytesRef text) throws IOException { postingsWriter.startTerm(); return postingsWriter; Index: lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java (revision 1522598) +++ lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java (working copy) @@ -661,11 +661,6 @@ } @Override - public Comparator getComparator() { - return BytesRef.getUTF8SortedAsUnicodeComparator(); - } - - @Override public boolean hasOffsets() { return hasOffsets; } @@ -701,11 +696,6 @@ } @Override - public Comparator getComparator() { - return BytesRef.getUTF8SortedAsUnicodeComparator(); - } - - @Override public BytesRef next() { termOrd++; if (termOrd < terms.length) { @@ -1096,11 +1086,6 @@ } } - @Override - public Comparator getComparator() { - return BytesRef.getUTF8SortedAsUnicodeComparator(); - } - private void grow() { if (states.length == 1+stateUpto) { final State[] newStates = new State[states.length+1]; Index: lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesProducer.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesProducer.java (revision 1522598) +++ lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesProducer.java (working copy) @@ -18,7 +18,6 @@ */ import java.io.IOException; -import java.util.Comparator; import java.util.HashMap; import java.util.Map; @@ -566,11 +565,6 @@ } @Override - public Comparator getComparator() { - return BytesRef.getUTF8SortedAsUnicodeComparator(); - } - - @Override public SeekStatus seekCeil(BytesRef text) throws IOException { if (in.seekCeil(text) == null) { return SeekStatus.END; Index: lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsReader.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsReader.java (revision 1522598) +++ lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsReader.java (working copy) @@ -18,8 +18,6 @@ */ import java.io.IOException; -import java.io.PrintWriter; -import java.io.File; import java.util.ArrayList; import java.util.BitSet; import java.util.Collections; @@ -40,7 +38,6 @@ import org.apache.lucene.index.Terms; import org.apache.lucene.index.TermsEnum; import org.apache.lucene.store.ByteArrayDataInput; -import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.automaton.ByteRunAutomaton; @@ -180,11 +177,6 @@ } @Override - public Comparator getComparator() { - return BytesRef.getUTF8SortedAsUnicodeComparator(); - } - - @Override public boolean hasOffsets() { return fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0; } @@ -252,11 +244,6 @@ } @Override - public Comparator getComparator() { - return BytesRef.getUTF8SortedAsUnicodeComparator(); - } - - @Override public TermState termState() throws IOException { decodeMetaData(); return state.clone(); Index: lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsReader.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsReader.java (revision 1522598) +++ lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsReader.java (working copy) @@ -18,8 +18,6 @@ */ import java.io.IOException; -import java.io.PrintWriter; -import java.io.File; import java.util.Arrays; import java.util.ArrayList; import java.util.BitSet; @@ -41,7 +39,6 @@ import org.apache.lucene.index.Terms; import org.apache.lucene.index.TermsEnum; import org.apache.lucene.store.ByteArrayDataInput; -import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.automaton.ByteRunAutomaton; @@ -210,11 +207,6 @@ blockIn.readBytes(metaBytesBlock, 0, metaBytesBlock.length); } - @Override - public Comparator getComparator() { - return BytesRef.getUTF8SortedAsUnicodeComparator(); - } - public boolean hasFreqs() { return fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0; } @@ -377,11 +369,6 @@ } @Override - public Comparator getComparator() { - return BytesRef.getUTF8SortedAsUnicodeComparator(); - } - - @Override public TermState termState() throws IOException { decodeMetaData(); return state.clone(); Index: lucene/codecs/src/java/org/apache/lucene/codecs/bloom/BloomFilteringPostingsFormat.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/bloom/BloomFilteringPostingsFormat.java (revision 1522598) +++ lucene/codecs/src/java/org/apache/lucene/codecs/bloom/BloomFilteringPostingsFormat.java (working copy) @@ -23,14 +23,15 @@ import java.util.HashMap; import java.util.Iterator; import java.util.List; +import java.util.Map.Entry; import java.util.Map; -import java.util.Map.Entry; import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.FieldsConsumer; import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.PostingsConsumer; import org.apache.lucene.codecs.PostingsFormat; +import org.apache.lucene.codecs.PushFieldsConsumer; import org.apache.lucene.codecs.TermStats; import org.apache.lucene.codecs.TermsConsumer; import org.apache.lucene.codecs.bloom.FuzzySet.ContainsResult; @@ -111,14 +112,16 @@ this.delegatePostingsFormat = delegatePostingsFormat; this.bloomFilterFactory = bloomFilterFactory; } - + /** * Creates Bloom filters for a selection of fields created in the index. This * is recorded as a set of Bitsets held as a segment summary in an additional * "blm" file. This PostingsFormat delegates to a choice of delegate * PostingsFormat for encoding all other postings data. This choice of * constructor defaults to the {@link DefaultBloomFilterFactory} for - * configuring per-field BloomFilters. + * configuring per-field BloomFilters. Note that the + * wrapped PostingsFormat must use a {@link PushFieldsConsumer} + * for writing. * * @param delegatePostingsFormat * The PostingsFormat that records all the non-bloom filter data i.e. @@ -141,8 +144,12 @@ throw new UnsupportedOperationException("Error - " + getClass().getName() + " has been constructed without a choice of PostingsFormat"); } + FieldsConsumer fieldsConsumer = delegatePostingsFormat.fieldsConsumer(state); + if (!(fieldsConsumer instanceof PushFieldsConsumer)) { + throw new UnsupportedOperationException("Wrapped PostingsFormat must return a PushFieldsConsumer"); + } return new BloomFilteredFieldsConsumer( - delegatePostingsFormat.fieldsConsumer(state), state, + (PushFieldsConsumer) fieldsConsumer, state, delegatePostingsFormat); } @@ -252,11 +259,6 @@ } @Override - public Comparator getComparator() { - return delegateTerms.getComparator(); - } - - @Override public long size() throws IOException { return delegateTerms.size(); } @@ -327,11 +329,6 @@ } @Override - public final Comparator getComparator() { - return delegateTerms.getComparator(); - } - - @Override public final boolean seekExact(BytesRef text) throws IOException { // The magical fail-fast speed up that is the entire point of all of @@ -388,8 +385,6 @@ throws IOException { return delegate().docs(liveDocs, reuse, flags); } - - } @Override @@ -401,17 +396,16 @@ } return sizeInBytes; } - } - class BloomFilteredFieldsConsumer extends FieldsConsumer { - private FieldsConsumer delegateFieldsConsumer; + class BloomFilteredFieldsConsumer extends PushFieldsConsumer { + private PushFieldsConsumer delegateFieldsConsumer; private Map bloomFilters = new HashMap(); private SegmentWriteState state; - - public BloomFilteredFieldsConsumer(FieldsConsumer fieldsConsumer, + public BloomFilteredFieldsConsumer(PushFieldsConsumer fieldsConsumer, SegmentWriteState state, PostingsFormat delegatePostingsFormat) { + super(state); this.delegateFieldsConsumer = fieldsConsumer; this.state = state; } @@ -422,7 +416,7 @@ if (bloomFilter != null) { assert bloomFilters.containsKey(field) == false; bloomFilters.put(field, bloomFilter); - return new WrappedTermsConsumer(delegateFieldsConsumer.addField(field),bloomFilter); + return new WrappedTermsConsumer(delegateFieldsConsumer.addField(field), bloomFilter); } else { // No, use the unfiltered fieldsConsumer - we are not interested in // recording any term Bitsets. @@ -510,12 +504,5 @@ throws IOException { delegateTermsConsumer.finish(sumTotalTermFreq, sumDocFreq, docCount); } - - @Override - public Comparator getComparator() throws IOException { - return delegateTermsConsumer.getComparator(); - } - } - } Index: lucene/suggest/src/test/org/apache/lucene/search/suggest/TermFreqArrayIterator.java =================================================================== --- lucene/suggest/src/test/org/apache/lucene/search/suggest/TermFreqArrayIterator.java (revision 1522598) +++ lucene/suggest/src/test/org/apache/lucene/search/suggest/TermFreqArrayIterator.java (working copy) @@ -18,7 +18,6 @@ */ import java.util.Arrays; -import java.util.Comparator; import java.util.Iterator; import org.apache.lucene.search.spell.TermFreqIterator; @@ -58,9 +57,4 @@ } return null; } - - @Override - public Comparator getComparator() { - return null; - } } \ No newline at end of file Index: lucene/suggest/src/test/org/apache/lucene/search/suggest/TermFreqPayloadArrayIterator.java =================================================================== --- lucene/suggest/src/test/org/apache/lucene/search/suggest/TermFreqPayloadArrayIterator.java (revision 1522598) +++ lucene/suggest/src/test/org/apache/lucene/search/suggest/TermFreqPayloadArrayIterator.java (working copy) @@ -18,7 +18,6 @@ */ import java.util.Arrays; -import java.util.Comparator; import java.util.Iterator; import org.apache.lucene.search.spell.TermFreqIterator; @@ -64,9 +63,4 @@ public BytesRef payload() { return current.payload; } - - @Override - public Comparator getComparator() { - return null; - } } \ No newline at end of file Index: lucene/suggest/src/test/org/apache/lucene/search/suggest/TestHighFrequencyDictionary.java =================================================================== --- lucene/suggest/src/test/org/apache/lucene/search/suggest/TestHighFrequencyDictionary.java (revision 1522598) +++ lucene/suggest/src/test/org/apache/lucene/search/suggest/TestHighFrequencyDictionary.java (working copy) @@ -36,7 +36,6 @@ IndexReader ir = DirectoryReader.open(dir); Dictionary dictionary = new HighFrequencyDictionary(ir, "bogus", 0.1f); BytesRefIterator tf = dictionary.getWordsIterator(); - assertNull(tf.getComparator()); assertNull(tf.next()); dir.close(); } Index: lucene/suggest/src/java/org/apache/lucene/search/spell/HighFrequencyDictionary.java =================================================================== --- lucene/suggest/src/java/org/apache/lucene/search/spell/HighFrequencyDictionary.java (revision 1522598) +++ lucene/suggest/src/java/org/apache/lucene/search/spell/HighFrequencyDictionary.java (working copy) @@ -18,7 +18,6 @@ package org.apache.lucene.search.spell; import java.io.IOException; -import java.util.Comparator; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.TermsEnum; @@ -99,14 +98,5 @@ } return null; } - - @Override - public Comparator getComparator() { - if (termsEnum == null) { - return null; - } else { - return termsEnum.getComparator(); - } - } } } Index: lucene/suggest/src/java/org/apache/lucene/search/spell/PlainTextDictionary.java =================================================================== --- lucene/suggest/src/java/org/apache/lucene/search/spell/PlainTextDictionary.java (revision 1522598) +++ lucene/suggest/src/java/org/apache/lucene/search/spell/PlainTextDictionary.java (working copy) @@ -17,10 +17,12 @@ * limitations under the License. */ +import java.io.BufferedReader; +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.io.Reader; -import java.util.Comparator; -import java.io.*; - import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRefIterator; import org.apache.lucene.util.IOUtils; @@ -96,11 +98,5 @@ } return result; } - - @Override - public Comparator getComparator() { - return null; - } } - } Index: lucene/suggest/src/java/org/apache/lucene/search/spell/TermFreqIterator.java =================================================================== --- lucene/suggest/src/java/org/apache/lucene/search/spell/TermFreqIterator.java (revision 1522598) +++ lucene/suggest/src/java/org/apache/lucene/search/spell/TermFreqIterator.java (working copy) @@ -18,7 +18,6 @@ */ import java.io.IOException; -import java.util.Comparator; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRefIterator; @@ -55,10 +54,5 @@ public BytesRef next() throws IOException { return wrapped.next(); } - - @Override - public Comparator getComparator() { - return wrapped.getComparator(); - } } } Index: lucene/suggest/src/java/org/apache/lucene/search/suggest/BytesRefArray.java =================================================================== --- lucene/suggest/src/java/org/apache/lucene/search/suggest/BytesRefArray.java (revision 1522598) +++ lucene/suggest/src/java/org/apache/lucene/search/suggest/BytesRefArray.java (working copy) @@ -187,11 +187,6 @@ } return null; } - - @Override - public Comparator getComparator() { - return comp; - } }; } } Index: lucene/suggest/src/java/org/apache/lucene/search/suggest/BufferingTermFreqIteratorWrapper.java =================================================================== --- lucene/suggest/src/java/org/apache/lucene/search/suggest/BufferingTermFreqIteratorWrapper.java (revision 1522598) +++ lucene/suggest/src/java/org/apache/lucene/search/suggest/BufferingTermFreqIteratorWrapper.java (working copy) @@ -18,7 +18,6 @@ */ import java.io.IOException; -import java.util.Comparator; import org.apache.lucene.search.spell.TermFreqIterator; import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.BytesRef; @@ -37,11 +36,9 @@ /** buffered weights, parallel with {@link #entries} */ protected long[] freqs = new long[1]; private final BytesRef spare = new BytesRef(); - private final Comparator comp; /** Creates a new iterator, buffering entries from the specified iterator */ public BufferingTermFreqIteratorWrapper(TermFreqIterator source) throws IOException { - this.comp = source.getComparator(); BytesRef spare; int freqIndex = 0; while((spare = source.next()) != null) { @@ -67,11 +64,4 @@ } return null; } - - @Override - public Comparator getComparator() { - return comp; - } - - } Index: lucene/suggest/src/java/org/apache/lucene/search/suggest/tst/TSTLookup.java =================================================================== --- lucene/suggest/src/java/org/apache/lucene/search/suggest/tst/TSTLookup.java (revision 1522598) +++ lucene/suggest/src/java/org/apache/lucene/search/suggest/tst/TSTLookup.java (working copy) @@ -56,12 +56,10 @@ throw new IllegalArgumentException("this suggester doesn't support payloads"); } root = new TernaryTreeNode(); - // buffer first - if (tfit.getComparator() != BytesRef.getUTF8SortedAsUTF16Comparator()) { - // make sure it's sorted and the comparator uses UTF16 sort order - tfit = new SortedTermFreqIteratorWrapper(tfit, BytesRef.getUTF8SortedAsUTF16Comparator()); - } + // make sure it's sorted and the comparator uses UTF16 sort order + tfit = new SortedTermFreqIteratorWrapper(tfit, BytesRef.getUTF8SortedAsUTF16Comparator()); + ArrayList tokens = new ArrayList(); ArrayList vals = new ArrayList(); BytesRef spare; Index: lucene/suggest/src/java/org/apache/lucene/search/suggest/FileDictionary.java =================================================================== --- lucene/suggest/src/java/org/apache/lucene/search/suggest/FileDictionary.java (revision 1522598) +++ lucene/suggest/src/java/org/apache/lucene/search/suggest/FileDictionary.java (working copy) @@ -19,7 +19,6 @@ import java.io.*; -import java.util.Comparator; import org.apache.lucene.search.spell.Dictionary; import org.apache.lucene.search.spell.TermFreqIterator; @@ -99,11 +98,5 @@ return null; } } - - @Override - public Comparator getComparator() { - return null; - } } - } Index: lucene/suggest/src/java/org/apache/lucene/search/suggest/jaspell/JaspellLookup.java =================================================================== --- lucene/suggest/src/java/org/apache/lucene/search/suggest/jaspell/JaspellLookup.java (revision 1522598) +++ lucene/suggest/src/java/org/apache/lucene/search/suggest/jaspell/JaspellLookup.java (working copy) @@ -57,11 +57,9 @@ if (tfit instanceof TermFreqPayloadIterator) { throw new IllegalArgumentException("this suggester doesn't support payloads"); } - if (tfit.getComparator() != null) { - // make sure it's unsorted - // WTF - this could result in yet another sorted iteration.... - tfit = new UnsortedTermFreqIteratorWrapper(tfit); - } + // make sure it's unsorted + // WTF - this could result in yet another sorted iteration.... + tfit = new UnsortedTermFreqIteratorWrapper(tfit); trie = new JaspellTernarySearchTrie(); trie.setMatchAlmostDiff(editDistance); BytesRef spare; Index: lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/ExternalRefSorter.java =================================================================== --- lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/ExternalRefSorter.java (revision 1522598) +++ lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/ExternalRefSorter.java (working copy) @@ -66,8 +66,7 @@ input = null; } - return new ByteSequenceIterator(new Sort.ByteSequencesReader(sorted), - sort.getComparator()); + return new ByteSequenceIterator(new Sort.ByteSequencesReader(sorted)); } private void closeWriter() throws IOException { @@ -96,12 +95,9 @@ class ByteSequenceIterator implements BytesRefIterator { private final ByteSequencesReader reader; private BytesRef scratch = new BytesRef(); - private final Comparator comparator; - public ByteSequenceIterator(ByteSequencesReader reader, - Comparator comparator) { + public ByteSequenceIterator(ByteSequencesReader reader) { this.reader = reader; - this.comparator = comparator; } @Override @@ -128,11 +124,6 @@ } } } - - @Override - public Comparator getComparator() { - return comparator; - } } @Override Index: lucene/suggest/src/java/org/apache/lucene/search/suggest/SortedTermFreqIteratorWrapper.java =================================================================== --- lucene/suggest/src/java/org/apache/lucene/search/suggest/SortedTermFreqIteratorWrapper.java (revision 1522598) +++ lucene/suggest/src/java/org/apache/lucene/search/suggest/SortedTermFreqIteratorWrapper.java (working copy) @@ -65,11 +65,6 @@ } @Override - public Comparator getComparator() { - return comparator; - } - - @Override public BytesRef next() throws IOException { boolean success = false; if (done) { Index: lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestBlockPostingsFormat3.java =================================================================== --- lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestBlockPostingsFormat3.java (revision 1522598) +++ lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestBlockPostingsFormat3.java (working copy) @@ -261,7 +261,6 @@ * checks collection-level statistics on Terms */ public void assertTermsStatistics(Terms leftTerms, Terms rightTerms) throws Exception { - assert leftTerms.getComparator() == rightTerms.getComparator(); if (leftTerms.getDocCount() != -1 && rightTerms.getDocCount() != -1) { assertEquals(leftTerms.getDocCount(), rightTerms.getDocCount()); } Index: lucene/core/src/test/org/apache/lucene/index/TestCodecs.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestCodecs.java (revision 1522598) +++ lucene/core/src/test/org/apache/lucene/index/TestCodecs.java (working copy) @@ -19,6 +19,7 @@ import java.io.IOException; import java.util.Arrays; +import java.util.Comparator; import java.util.HashSet; import java.util.Iterator; import java.util.Random; @@ -48,6 +49,7 @@ import org.apache.lucene.search.Query; import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.store.Directory; +import org.apache.lucene.util.Bits; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.Constants; import org.apache.lucene.util.InfoStream; @@ -145,22 +147,6 @@ public int compareTo(final FieldData other) { return fieldInfo.name.compareTo(other.fieldInfo.name); } - - public void write(final FieldsConsumer consumer) throws Throwable { - Arrays.sort(terms); - final TermsConsumer termsConsumer = consumer.addField(fieldInfo); - long sumTotalTermCount = 0; - long sumDF = 0; - OpenBitSet visitedDocs = new OpenBitSet(); - for (final TermData term : terms) { - for (int i = 0; i < term.docs.length; i++) { - visitedDocs.set(term.docs[i]); - } - sumDF += term.docs.length; - sumTotalTermCount += term.write(termsConsumer); - } - termsConsumer.finish(omitTF ? -1 : sumTotalTermCount, sumDF, (int) visitedDocs.cardinality()); - } } class PositionData { @@ -191,30 +177,6 @@ public int compareTo(final TermData o) { return text.compareTo(o.text); } - - public long write(final TermsConsumer termsConsumer) throws Throwable { - final PostingsConsumer postingsConsumer = termsConsumer.startTerm(text); - long totTF = 0; - for(int i=0;i iterator() { + return new Iterator() { + int upto = -1; + + @Override + public boolean hasNext() { + return upto+1 < fields.length; + } + + @Override + public String next() { + upto++; + return fields[upto].fieldInfo.name; + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + }; + } + + @Override + public Terms terms(String field) { + // Slow linear search: + for(FieldData fieldData : fields) { + if (fieldData.fieldInfo.name.equals(field)) { + return new DataTerms(fieldData); + } + } + return null; + } + + @Override + public int size() { + return fields.length; + } + } + + private static class DataTerms extends Terms { + final FieldData fieldData; + + public DataTerms(FieldData fieldData) { + this.fieldData = fieldData; + } + + @Override + public TermsEnum iterator(TermsEnum reuse) { + return new DataTermsEnum(fieldData); + } + + @Override + public long size() { + throw new UnsupportedOperationException(); + } + + @Override + public long getSumTotalTermFreq() { + throw new UnsupportedOperationException(); + } + + @Override + public long getSumDocFreq() { + throw new UnsupportedOperationException(); + } + + @Override + public int getDocCount() { + throw new UnsupportedOperationException(); + } + + @Override + public boolean hasOffsets() { + return fieldData.fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0; + } + + @Override + public boolean hasPositions() { + return fieldData.fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; + } + + @Override + public boolean hasPayloads() { + return fieldData.fieldInfo.hasPayloads(); + } + } + + private static class DataTermsEnum extends TermsEnum { + final FieldData fieldData; + private int upto = -1; + + public DataTermsEnum(FieldData fieldData) { + this.fieldData = fieldData; + } + + @Override + public BytesRef next() { + upto++; + if (upto == fieldData.terms.length) { + return null; + } + + return term(); + } + + @Override + public BytesRef term() { + return fieldData.terms[upto].text; + } + + @Override + public SeekStatus seekCeil(BytesRef text) { + // Stupid linear impl: + for(int i=0;i 0) { + upto = i; + return SeekStatus.NOT_FOUND; + } + } + + return SeekStatus.END; + } + + @Override + public void seekExact(long ord) { + throw new UnsupportedOperationException(); + } + + @Override + public long ord() { + throw new UnsupportedOperationException(); + } + + @Override + public int docFreq() { + throw new UnsupportedOperationException(); + } + + @Override + public long totalTermFreq() { + throw new UnsupportedOperationException(); + } + + @Override + public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) { + assert liveDocs == null; + return new DataDocsAndPositionsEnum(fieldData.terms[upto]); + } + + @Override + public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) { + assert liveDocs == null; + return new DataDocsAndPositionsEnum(fieldData.terms[upto]); + } + } + + private static class DataDocsAndPositionsEnum extends DocsAndPositionsEnum { + final TermData termData; + int docUpto = -1; + int posUpto; + + public DataDocsAndPositionsEnum(TermData termData) { + this.termData = termData; + } + + @Override + public long cost() { + throw new UnsupportedOperationException(); + } + + @Override + public int nextDoc() { + docUpto++; + if (docUpto == termData.docs.length) { + return NO_MORE_DOCS; + } + posUpto = -1; + return docID(); + } + + @Override + public int docID() { + return termData.docs[docUpto]; + } + + @Override + public int advance(int target) { + // Slow linear impl: + nextDoc(); + while (docID() < target) { + nextDoc(); + } + + return docID(); + } + + @Override + public int freq() { + return termData.positions[docUpto].length; + } + + @Override + public int nextPosition() { + posUpto++; + return termData.positions[docUpto][posUpto].pos; + } + + @Override + public BytesRef getPayload() { + return termData.positions[docUpto][posUpto].payload; + } + + @Override + public int startOffset() { + throw new UnsupportedOperationException(); + } + + @Override + public int endOffset() { + throw new UnsupportedOperationException(); + } + } + private void write(final FieldInfos fieldInfos, final Directory dir, final FieldData[] fields) throws Throwable { final Codec codec = Codec.getDefault(); final SegmentInfo si = new SegmentInfo(dir, Constants.LUCENE_MAIN_VERSION, SEGMENT, 10000, false, codec, null, null); final SegmentWriteState state = new SegmentWriteState(InfoStream.getDefault(), dir, si, fieldInfos, null, newIOContext(random())); - final FieldsConsumer consumer = codec.postingsFormat().fieldsConsumer(state); Arrays.sort(fields); - for (final FieldData field : fields) { - field.write(consumer); - } - consumer.close(); + codec.postingsFormat().fieldsConsumer(state).write(new DataFields(fields)); } public void testDocsOnlyFreq() throws Exception { Index: lucene/core/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java (revision 1522598) +++ lucene/core/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java (working copy) @@ -249,7 +249,8 @@ directory, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())). setOpenMode(OpenMode.APPEND). - setMergePolicy(newLogMergePolicy(100)) + setMergePolicy(newLogMergePolicy(100)). + setMaxBufferedDocs(2) ); } writer.close(); Index: lucene/core/src/test/org/apache/lucene/index/TestLongPostings.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestLongPostings.java (revision 1522598) +++ lucene/core/src/test/org/apache/lucene/index/TestLongPostings.java (working copy) @@ -199,6 +199,9 @@ } if (random().nextInt(6) == 3) { + if (VERBOSE) { + System.out.println(" check positions"); + } final int freq = postings.freq(); assertTrue(freq >=1 && freq <= 4); for(int pos=0;pos= 30) { + if (sawFlush && count++ >= 30) { doFail = false; throw new IOException("now failing during flush"); } Index: lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java (revision 1522598) +++ lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java (working copy) @@ -17,26 +17,29 @@ * limitations under the License. */ -import java.io.Closeable; import java.io.IOException; import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.Map; import java.util.ServiceLoader; // javadocs +import java.util.Set; import java.util.TreeMap; +import java.util.TreeSet; import org.apache.lucene.codecs.FieldsConsumer; import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.PostingsFormat; -import org.apache.lucene.codecs.TermsConsumer; import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.Fields; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.Terms; import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.RamUsageEstimator; +import static org.apache.lucene.index.FilterAtomicReader.FilterFields; + /** * Enables per field postings support. *

@@ -65,96 +68,22 @@ * segment suffix name for each field. */ public static final String PER_FIELD_SUFFIX_KEY = PerFieldPostingsFormat.class.getSimpleName() + ".suffix"; - /** Sole constructor. */ public PerFieldPostingsFormat() { super(PER_FIELD_NAME); } - @Override - public final FieldsConsumer fieldsConsumer(SegmentWriteState state) - throws IOException { - return new FieldsWriter(state); - } - - static class FieldsConsumerAndSuffix implements Closeable { - FieldsConsumer consumer; + /** Group of fields written by one PostingsFormat */ + static class FieldsGroup { + final Set fields = new TreeSet(); int suffix; - - @Override - public void close() throws IOException { - consumer.close(); - } - } - - private class FieldsWriter extends FieldsConsumer { - private final Map formats = new HashMap(); - private final Map suffixes = new HashMap(); - - private final SegmentWriteState segmentWriteState; + /** Custom SegmentWriteState for this group of fields, + * with the segmentSuffix uniqueified for this + * PostingsFormat */ + SegmentWriteState state; + }; - public FieldsWriter(SegmentWriteState state) { - segmentWriteState = state; - } - - @Override - public TermsConsumer addField(FieldInfo field) throws IOException { - final PostingsFormat format = getPostingsFormatForField(field.name); - if (format == null) { - throw new IllegalStateException("invalid null PostingsFormat for field=\"" + field.name + "\""); - } - final String formatName = format.getName(); - - String previousValue = field.putAttribute(PER_FIELD_FORMAT_KEY, formatName); - assert previousValue == null; - - Integer suffix; - - FieldsConsumerAndSuffix consumer = formats.get(format); - if (consumer == null) { - // First time we are seeing this format; create a new instance - - // bump the suffix - suffix = suffixes.get(formatName); - if (suffix == null) { - suffix = 0; - } else { - suffix = suffix + 1; - } - suffixes.put(formatName, suffix); - - final String segmentSuffix = getFullSegmentSuffix(field.name, - segmentWriteState.segmentSuffix, - getSuffix(formatName, Integer.toString(suffix))); - consumer = new FieldsConsumerAndSuffix(); - consumer.consumer = format.fieldsConsumer(new SegmentWriteState(segmentWriteState, segmentSuffix)); - consumer.suffix = suffix; - formats.put(format, consumer); - } else { - // we've already seen this format, so just grab its suffix - assert suffixes.containsKey(formatName); - suffix = consumer.suffix; - } - - previousValue = field.putAttribute(PER_FIELD_SUFFIX_KEY, Integer.toString(suffix)); - assert previousValue == null; - - // TODO: we should only provide the "slice" of FIS - // that this PF actually sees ... then stuff like - // .hasProx could work correctly? - // NOTE: .hasProx is already broken in the same way for the non-perfield case, - // if there is a fieldinfo with prox that has no postings, you get a 0 byte file. - return consumer.consumer.addField(field); - } - - @Override - public void close() throws IOException { - // Close all subs - IOUtils.close(formats.values()); - } - } - static String getSuffix(String formatName, String suffix) { return formatName + "_" + suffix; } @@ -169,7 +98,88 @@ throw new IllegalStateException("cannot embed PerFieldPostingsFormat inside itself (field \"" + fieldName + "\" returned PerFieldPostingsFormat)"); } } + + private class FieldsWriter extends FieldsConsumer { + final SegmentWriteState writeState; + public FieldsWriter(SegmentWriteState writeState) { + this.writeState = writeState; + } + + @Override + public void write(Fields fields) throws IOException { + + // Maps a PostingsFormat instance to the suffix it + // should use + Map formatToGroups = new HashMap(); + + // Holds last suffix of each PostingFormat name + Map suffixes = new HashMap(); + + // First pass: assign field -> PostingsFormat + for(String field : fields) { + FieldInfo fieldInfo = writeState.fieldInfos.fieldInfo(field); + + final PostingsFormat format = getPostingsFormatForField(field); + + if (format == null) { + throw new IllegalStateException("invalid null PostingsFormat for field=\"" + field + "\""); + } + String formatName = format.getName(); + + FieldsGroup group = formatToGroups.get(format); + if (group == null) { + // First time we are seeing this format; create a + // new instance + + // bump the suffix + Integer suffix = suffixes.get(formatName); + if (suffix == null) { + suffix = 0; + } else { + suffix = suffix + 1; + } + suffixes.put(formatName, suffix); + + String segmentSuffix = getFullSegmentSuffix(field, + writeState.segmentSuffix, + getSuffix(formatName, Integer.toString(suffix))); + group = new FieldsGroup(); + group.state = new SegmentWriteState(writeState, segmentSuffix); + group.suffix = suffix; + formatToGroups.put(format, group); + } else { + // we've already seen this format, so just grab its suffix + assert suffixes.containsKey(formatName); + } + + group.fields.add(field); + + String previousValue = fieldInfo.putAttribute(PER_FIELD_FORMAT_KEY, formatName); + assert previousValue == null; + + previousValue = fieldInfo.putAttribute(PER_FIELD_SUFFIX_KEY, Integer.toString(group.suffix)); + assert previousValue == null; + } + + // Second pass: write postings + for(Map.Entry ent : formatToGroups.entrySet()) { + PostingsFormat format = ent.getKey(); + final FieldsGroup group = ent.getValue(); + + // Exposes only the fields from this group: + Fields maskedFields = new FilterFields(fields) { + @Override + public Iterator iterator() { + return group.fields.iterator(); + } + }; + + format.fieldsConsumer(group.state).write(maskedFields); + } + } + } + private class FieldsReader extends FieldsProducer { private final Map fields = new TreeMap(); @@ -239,6 +249,12 @@ } @Override + public final FieldsConsumer fieldsConsumer(SegmentWriteState state) + throws IOException { + return new FieldsWriter(state); + } + + @Override public final FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { return new FieldsReader(state); Index: lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java (revision 1522598) +++ lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java (working copy) @@ -361,13 +361,6 @@ } @Override - public Comparator getComparator() { - // TODO: really indexer hardwires - // this...? I guess codec could buffer and re-sort... - return BytesRef.getUTF8SortedAsUnicodeComparator(); - } - - @Override public boolean hasOffsets() { return storeOffsets; } @@ -569,11 +562,6 @@ docsAndPositionsEnum.reset(liveDocs, positions, startOffsets, endOffsets, payloadOffsets, payloadData); return docsAndPositionsEnum; } - - @Override - public Comparator getComparator() { - return BytesRef.getUTF8SortedAsUnicodeComparator(); - } } // NOTE: sort of a silly class, since you can get the Index: lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java (revision 1522598) +++ lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java (working copy) @@ -18,7 +18,6 @@ */ import java.io.IOException; -import java.util.Comparator; import java.util.HashMap; import java.util.Map; @@ -511,11 +510,6 @@ } @Override - public Comparator getComparator() { - return BytesRef.getUTF8SortedAsUnicodeComparator(); - } - - @Override public SeekStatus seekCeil(BytesRef text) throws IOException { if (in.seekCeil(text) == null) { return SeekStatus.END; Index: lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesProducer.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesProducer.java (revision 1522598) +++ lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesProducer.java (working copy) @@ -26,7 +26,6 @@ import java.io.Closeable; // javadocs import java.io.IOException; -import java.util.Comparator; import java.util.HashMap; import java.util.Map; @@ -811,11 +810,6 @@ public long ord() throws IOException { return currentOrd; } - - @Override - public Comparator getComparator() { - return BytesRef.getUTF8SortedAsUnicodeComparator(); - } @Override public int docFreq() throws IOException { Index: lucene/core/src/java/org/apache/lucene/codecs/MappedMultiFields.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/MappedMultiFields.java (revision 0) +++ lucene/core/src/java/org/apache/lucene/codecs/MappedMultiFields.java (working copy) @@ -0,0 +1,151 @@ +package org.apache.lucene.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. + */ + +// nocommit move this and the other Mapped* into oal.index? +// then keep them all package private... + +import java.io.IOException; + +import org.apache.lucene.index.DocsAndPositionsEnum; +import org.apache.lucene.index.DocsEnum; +import org.apache.lucene.index.MergeState; +import org.apache.lucene.index.MultiDocsAndPositionsEnum; +import org.apache.lucene.index.MultiDocsEnum; +import org.apache.lucene.index.MultiFields; +import org.apache.lucene.index.MultiTerms; +import org.apache.lucene.index.MultiTermsEnum; +import org.apache.lucene.index.Terms; +import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.util.Bits; + +import static org.apache.lucene.index.FilterAtomicReader.FilterFields; +import static org.apache.lucene.index.FilterAtomicReader.FilterTerms; +import static org.apache.lucene.index.FilterAtomicReader.FilterTermsEnum; + +/** A {@link Fields} implementation that merges multiple + * Fields into one, and maps around deleted documents. + * This is used for merging. */ + +// nocommit how to get checkAbort in here... + +public class MappedMultiFields extends FilterFields { + final MergeState mergeState; + + public MappedMultiFields(MergeState mergeState, MultiFields multiFields) { + super(multiFields); + this.mergeState = mergeState; + } + + @Override + public Terms terms(String field) throws IOException { + MultiTerms terms = (MultiTerms) in.terms(field); + if (terms == null) { + return null; + } else { + return new MappedMultiTerms(mergeState, terms); + } + } + + private static class MappedMultiTerms extends FilterTerms { + final MergeState mergeState; + + public MappedMultiTerms(MergeState mergeState, MultiTerms multiTerms) { + super(multiTerms); + this.mergeState = mergeState; + } + + @Override + public TermsEnum iterator(TermsEnum reuse) throws IOException { + return new MappedMultiTermsEnum(mergeState, (MultiTermsEnum) in.iterator(reuse)); + } + + @Override + public long size() throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public long getSumTotalTermFreq() throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public long getSumDocFreq() throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public int getDocCount() throws IOException { + throw new UnsupportedOperationException(); + } + } + + private static class MappedMultiTermsEnum extends FilterTermsEnum { + final MergeState mergeState; + + public MappedMultiTermsEnum(MergeState mergeState, MultiTermsEnum multiTermsEnum) { + super(multiTermsEnum); + this.mergeState = mergeState; + } + + @Override + public int docFreq() throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public long totalTermFreq() throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException { + if (liveDocs != null) { + throw new IllegalArgumentException("liveDocs must be null"); + } + MappingMultiDocsEnum mappingDocsEnum; + if (reuse instanceof MappingMultiDocsEnum) { + mappingDocsEnum = (MappingMultiDocsEnum) reuse; + } else { + mappingDocsEnum = new MappingMultiDocsEnum(mergeState); + } + + MultiDocsEnum docsEnum = (MultiDocsEnum) in.docs(liveDocs, mappingDocsEnum.multiDocsEnum, flags); + mappingDocsEnum.reset(docsEnum); + return mappingDocsEnum; + } + + @Override + public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException { + if (liveDocs != null) { + throw new IllegalArgumentException("liveDocs must be null"); + } + MappingMultiDocsAndPositionsEnum mappingDocsAndPositionsEnum; + if (reuse instanceof MappingMultiDocsAndPositionsEnum) { + mappingDocsAndPositionsEnum = (MappingMultiDocsAndPositionsEnum) reuse; + } else { + mappingDocsAndPositionsEnum = new MappingMultiDocsAndPositionsEnum(mergeState); + } + + MultiDocsAndPositionsEnum docsAndPositionsEnum = (MultiDocsAndPositionsEnum) in.docsAndPositions(liveDocs, mappingDocsAndPositionsEnum.multiDocsAndPositionsEnum, flags); + mappingDocsAndPositionsEnum.reset(docsAndPositionsEnum); + return mappingDocsAndPositionsEnum; + } + } +} Property changes on: lucene/core/src/java/org/apache/lucene/codecs/MappedMultiFields.java ___________________________________________________________________ Added: svn:eol-style ## -0,0 +1 ## +native \ No newline at end of property Index: lucene/core/src/java/org/apache/lucene/codecs/PostingsFormat.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/PostingsFormat.java (revision 1522598) +++ lucene/core/src/java/org/apache/lucene/codecs/PostingsFormat.java (working copy) @@ -22,8 +22,8 @@ import java.util.Set; import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat; // javadocs +import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentWriteState; -import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.util.NamedSPILoader; /** @@ -71,6 +71,7 @@ public final String getName() { return name; } + /** Writes a new segment */ public abstract FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException; Index: lucene/core/src/java/org/apache/lucene/codecs/PushFieldsConsumer.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/PushFieldsConsumer.java (revision 0) +++ lucene/core/src/java/org/apache/lucene/codecs/PushFieldsConsumer.java (working copy) @@ -0,0 +1,167 @@ +package org.apache.lucene.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 java.io.Closeable; +import java.io.IOException; + +import org.apache.lucene.index.DocsAndPositionsEnum; +import org.apache.lucene.index.DocsEnum; +import org.apache.lucene.index.FieldInfo.IndexOptions; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.Fields; +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.index.Terms; +import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.FixedBitSet; +import org.apache.lucene.util.IOUtils; + +/** Translates the "pull" API from {@link FieldsConsumer} + * into a "push" API that pushes fields, terms, postings to + * the consumer */ +public abstract class PushFieldsConsumer extends FieldsConsumer implements Closeable { + + final SegmentWriteState writeState; + + protected PushFieldsConsumer(SegmentWriteState writeState) { + this.writeState = writeState; + } + + /** Add a new field */ + public abstract TermsConsumer addField(FieldInfo field) throws IOException; + + /** Called when we are done adding everything. */ + @Override + public abstract void close() throws IOException; + + @Override + public final void write(Fields fields) throws IOException { + + boolean success = false; + try { + for(String field : fields) { // for all fields + FieldInfo fieldInfo = writeState.fieldInfos.fieldInfo(field); + IndexOptions indexOptions = fieldInfo.getIndexOptions(); + TermsConsumer termsConsumer = addField(fieldInfo); + + Terms terms = fields.terms(field); + if (terms != null) { + + // Holds all docs that have this field: + FixedBitSet visitedDocs = new FixedBitSet(writeState.segmentInfo.getDocCount()); + + boolean hasFreq = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS) >= 0; + boolean hasPositions = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; + assert hasPositions == terms.hasPositions(); + boolean hasOffsets = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0; + assert hasOffsets == terms.hasOffsets(); + boolean hasPayloads = fieldInfo.hasPayloads(); + + long sumTotalTermFreq = 0; + long sumDocFreq = 0; + + int flags = 0; + if (hasPositions == false) { + if (hasFreq) { + flags = flags | DocsEnum.FLAG_FREQS; + } + } else { + if (hasPayloads) { + flags = flags | DocsAndPositionsEnum.FLAG_PAYLOADS; + } + if (hasOffsets) { + flags = flags | DocsAndPositionsEnum.FLAG_OFFSETS; + } + } + + DocsEnum docsEnum = null; + DocsAndPositionsEnum docsAndPositionsEnum = null; + TermsEnum termsEnum = terms.iterator(null); + + while (true) { // for all terms in this field + BytesRef term = termsEnum.next(); + if (term == null) { + break; + } + if (hasPositions) { + docsAndPositionsEnum = termsEnum.docsAndPositions(null, docsAndPositionsEnum, flags); + docsEnum = docsAndPositionsEnum; + } else { + docsEnum = termsEnum.docs(null, docsEnum, flags); + docsAndPositionsEnum = null; + } + assert docsEnum != null; + + PostingsConsumer postingsConsumer = termsConsumer.startTerm(term); + + // How many documents have this term: + int docFreq = 0; + + // How many times this term occurs: + long totalTermFreq = 0; + + while(true) { // for all docs in this field+term + int doc = docsEnum.nextDoc(); + if (doc == DocsEnum.NO_MORE_DOCS) { + break; + } + docFreq++; + visitedDocs.set(doc); + if (hasFreq) { + int freq = docsEnum.freq(); + postingsConsumer.startDoc(doc, freq); + totalTermFreq += freq; + + if (hasPositions) { + for(int i=0;i 0) { + termsConsumer.finishTerm(term, new TermStats(docFreq, hasFreq ? totalTermFreq : -1)); + sumTotalTermFreq += totalTermFreq; + sumDocFreq += docFreq; + } + } + + termsConsumer.finish(hasFreq ? sumTotalTermFreq : -1, sumDocFreq, visitedDocs.cardinality()); + } + } + success = true; + } finally { + if (success) { + IOUtils.close(this); + } else { + IOUtils.closeWhileHandlingException(this); + } + } + } +} Property changes on: lucene/core/src/java/org/apache/lucene/codecs/PushFieldsConsumer.java ___________________________________________________________________ Added: svn:eol-style ## -0,0 +1 ## +native \ No newline at end of property Index: lucene/core/src/java/org/apache/lucene/codecs/PostingsConsumer.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/PostingsConsumer.java (revision 1522598) +++ lucene/core/src/java/org/apache/lucene/codecs/PostingsConsumer.java (working copy) @@ -19,13 +19,7 @@ import java.io.IOException; -import org.apache.lucene.index.DocsAndPositionsEnum; -import org.apache.lucene.index.DocsEnum; -import org.apache.lucene.index.MergeState; -import org.apache.lucene.index.FieldInfo.IndexOptions; -import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.util.BytesRef; -import org.apache.lucene.util.FixedBitSet; /** * Abstract API that consumes postings for an individual term. @@ -70,79 +64,4 @@ /** Called when we are done adding positions & payloads * for each doc. */ public abstract void finishDoc() throws IOException; - - /** Default merge impl: append documents, mapping around - * deletes */ - public TermStats merge(final MergeState mergeState, IndexOptions indexOptions, final DocsEnum postings, final FixedBitSet visitedDocs) throws IOException { - - int df = 0; - long totTF = 0; - - if (indexOptions == IndexOptions.DOCS_ONLY) { - while(true) { - final int doc = postings.nextDoc(); - if (doc == DocIdSetIterator.NO_MORE_DOCS) { - break; - } - visitedDocs.set(doc); - this.startDoc(doc, -1); - this.finishDoc(); - df++; - } - totTF = -1; - } else if (indexOptions == IndexOptions.DOCS_AND_FREQS) { - while(true) { - final int doc = postings.nextDoc(); - if (doc == DocIdSetIterator.NO_MORE_DOCS) { - break; - } - visitedDocs.set(doc); - final int freq = postings.freq(); - this.startDoc(doc, freq); - this.finishDoc(); - df++; - totTF += freq; - } - } else if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) { - final DocsAndPositionsEnum postingsEnum = (DocsAndPositionsEnum) postings; - while(true) { - final int doc = postingsEnum.nextDoc(); - if (doc == DocIdSetIterator.NO_MORE_DOCS) { - break; - } - visitedDocs.set(doc); - final int freq = postingsEnum.freq(); - this.startDoc(doc, freq); - totTF += freq; - for(int i=0;isumTotalTermFreq will be -1 when term * frequencies are omitted for the field. */ public abstract void finish(long sumTotalTermFreq, long sumDocFreq, int docCount) throws IOException; - - /** Return the BytesRef Comparator used to sort terms - * before feeding to this API. */ - public abstract Comparator getComparator() throws IOException; - - private MappingMultiDocsEnum docsEnum; - private MappingMultiDocsEnum docsAndFreqsEnum; - private MappingMultiDocsAndPositionsEnum postingsEnum; - - /** Default merge impl */ - public void merge(MergeState mergeState, IndexOptions indexOptions, TermsEnum termsEnum) throws IOException { - - BytesRef term; - assert termsEnum != null; - long sumTotalTermFreq = 0; - long sumDocFreq = 0; - long sumDFsinceLastAbortCheck = 0; - FixedBitSet visitedDocs = new FixedBitSet(mergeState.segmentInfo.getDocCount()); - - if (indexOptions == IndexOptions.DOCS_ONLY) { - if (docsEnum == null) { - docsEnum = new MappingMultiDocsEnum(); - } - docsEnum.setMergeState(mergeState); - - MultiDocsEnum docsEnumIn = null; - - while((term = termsEnum.next()) != null) { - // We can pass null for liveDocs, because the - // mapping enum will skip the non-live docs: - docsEnumIn = (MultiDocsEnum) termsEnum.docs(null, docsEnumIn, DocsEnum.FLAG_NONE); - if (docsEnumIn != null) { - docsEnum.reset(docsEnumIn); - final PostingsConsumer postingsConsumer = startTerm(term); - final TermStats stats = postingsConsumer.merge(mergeState, indexOptions, docsEnum, visitedDocs); - if (stats.docFreq > 0) { - finishTerm(term, stats); - sumTotalTermFreq += stats.docFreq; - sumDFsinceLastAbortCheck += stats.docFreq; - sumDocFreq += stats.docFreq; - if (sumDFsinceLastAbortCheck > 60000) { - mergeState.checkAbort.work(sumDFsinceLastAbortCheck/5.0); - sumDFsinceLastAbortCheck = 0; - } - } - } - } - } else if (indexOptions == IndexOptions.DOCS_AND_FREQS) { - if (docsAndFreqsEnum == null) { - docsAndFreqsEnum = new MappingMultiDocsEnum(); - } - docsAndFreqsEnum.setMergeState(mergeState); - - MultiDocsEnum docsAndFreqsEnumIn = null; - - while((term = termsEnum.next()) != null) { - // We can pass null for liveDocs, because the - // mapping enum will skip the non-live docs: - docsAndFreqsEnumIn = (MultiDocsEnum) termsEnum.docs(null, docsAndFreqsEnumIn); - assert docsAndFreqsEnumIn != null; - docsAndFreqsEnum.reset(docsAndFreqsEnumIn); - final PostingsConsumer postingsConsumer = startTerm(term); - final TermStats stats = postingsConsumer.merge(mergeState, indexOptions, docsAndFreqsEnum, visitedDocs); - if (stats.docFreq > 0) { - finishTerm(term, stats); - sumTotalTermFreq += stats.totalTermFreq; - sumDFsinceLastAbortCheck += stats.docFreq; - sumDocFreq += stats.docFreq; - if (sumDFsinceLastAbortCheck > 60000) { - mergeState.checkAbort.work(sumDFsinceLastAbortCheck/5.0); - sumDFsinceLastAbortCheck = 0; - } - } - } - } else if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) { - if (postingsEnum == null) { - postingsEnum = new MappingMultiDocsAndPositionsEnum(); - } - postingsEnum.setMergeState(mergeState); - MultiDocsAndPositionsEnum postingsEnumIn = null; - while((term = termsEnum.next()) != null) { - // We can pass null for liveDocs, because the - // mapping enum will skip the non-live docs: - postingsEnumIn = (MultiDocsAndPositionsEnum) termsEnum.docsAndPositions(null, postingsEnumIn, DocsAndPositionsEnum.FLAG_PAYLOADS); - assert postingsEnumIn != null; - postingsEnum.reset(postingsEnumIn); - - final PostingsConsumer postingsConsumer = startTerm(term); - final TermStats stats = postingsConsumer.merge(mergeState, indexOptions, postingsEnum, visitedDocs); - if (stats.docFreq > 0) { - finishTerm(term, stats); - sumTotalTermFreq += stats.totalTermFreq; - sumDFsinceLastAbortCheck += stats.docFreq; - sumDocFreq += stats.docFreq; - if (sumDFsinceLastAbortCheck > 60000) { - mergeState.checkAbort.work(sumDFsinceLastAbortCheck/5.0); - sumDFsinceLastAbortCheck = 0; - } - } - } - } else { - assert indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS; - if (postingsEnum == null) { - postingsEnum = new MappingMultiDocsAndPositionsEnum(); - } - postingsEnum.setMergeState(mergeState); - MultiDocsAndPositionsEnum postingsEnumIn = null; - while((term = termsEnum.next()) != null) { - // We can pass null for liveDocs, because the - // mapping enum will skip the non-live docs: - postingsEnumIn = (MultiDocsAndPositionsEnum) termsEnum.docsAndPositions(null, postingsEnumIn); - assert postingsEnumIn != null; - postingsEnum.reset(postingsEnumIn); - - final PostingsConsumer postingsConsumer = startTerm(term); - final TermStats stats = postingsConsumer.merge(mergeState, indexOptions, postingsEnum, visitedDocs); - if (stats.docFreq > 0) { - finishTerm(term, stats); - sumTotalTermFreq += stats.totalTermFreq; - sumDFsinceLastAbortCheck += stats.docFreq; - sumDocFreq += stats.docFreq; - if (sumDFsinceLastAbortCheck > 60000) { - mergeState.checkAbort.work(sumDFsinceLastAbortCheck/5.0); - sumDFsinceLastAbortCheck = 0; - } - } - } - } - finish(indexOptions == IndexOptions.DOCS_ONLY ? -1 : sumTotalTermFreq, sumDocFreq, visitedDocs.cardinality()); - } } Index: lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java (revision 1522598) +++ lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java (working copy) @@ -498,11 +498,6 @@ } @Override - public Comparator getComparator() { - return BytesRef.getUTF8SortedAsUnicodeComparator(); - } - - @Override public boolean hasOffsets() { return fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0; } @@ -1239,11 +1234,6 @@ } @Override - public Comparator getComparator() { - return BytesRef.getUTF8SortedAsUnicodeComparator(); - } - - @Override public boolean seekExact(BytesRef text) { throw new UnsupportedOperationException(); } @@ -1454,11 +1444,6 @@ return arcs[ord]; } - @Override - public Comparator getComparator() { - return BytesRef.getUTF8SortedAsUnicodeComparator(); - } - // Pushes a frame we seek'd to Frame pushFrame(FST.Arc arc, BytesRef frameData, int length) throws IOException { scratchReader.reset(frameData.bytes, frameData.offset, frameData.length); Index: lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java (revision 1522598) +++ lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java (working copy) @@ -19,7 +19,6 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Comparator; import java.util.List; import org.apache.lucene.index.FieldInfo.IndexOptions; @@ -175,7 +174,7 @@ * @lucene.experimental */ -public class BlockTreeTermsWriter extends FieldsConsumer { +public class BlockTreeTermsWriter extends PushFieldsConsumer { /** Suggested default value for the {@code * minItemsInBlock} parameter to {@link @@ -274,6 +273,7 @@ int maxItemsInBlock) throws IOException { + super(state); if (minItemsInBlock <= 1) { throw new IllegalArgumentException("minItemsInBlock must be >= 2; got " + minItemsInBlock); } @@ -1018,11 +1018,6 @@ } @Override - public Comparator getComparator() { - return BytesRef.getUTF8SortedAsUnicodeComparator(); - } - - @Override public PostingsConsumer startTerm(BytesRef text) throws IOException { //if (DEBUG) System.out.println("\nBTTW.startTerm term=" + fieldInfo.name + ":" + toString(text) + " seg=" + segment); postingsWriter.startTerm(); Index: lucene/core/src/java/org/apache/lucene/codecs/MappingMultiDocsAndPositionsEnum.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/MappingMultiDocsAndPositionsEnum.java (revision 1522598) +++ lucene/core/src/java/org/apache/lucene/codecs/MappingMultiDocsAndPositionsEnum.java (working copy) @@ -41,25 +41,27 @@ int currentBase; int doc = -1; private MergeState mergeState; + MultiDocsAndPositionsEnum multiDocsAndPositionsEnum; - /** Sole constructor. */ + /** Constructor for non-merge case. */ public MappingMultiDocsAndPositionsEnum() { + this.mergeState = null; } + /** Constructor for merge case. */ + public MappingMultiDocsAndPositionsEnum(MergeState mergeState) { + this.mergeState = mergeState; + } + MappingMultiDocsAndPositionsEnum reset(MultiDocsAndPositionsEnum postingsEnum) { this.numSubs = postingsEnum.getNumSubs(); this.subs = postingsEnum.getSubs(); upto = -1; current = null; + this.multiDocsAndPositionsEnum = postingsEnum; return this; } - /** Sets the {@link MergeState}, which is used to re-map - * document IDs. */ - public void setMergeState(MergeState mergeState) { - this.mergeState = mergeState; - } - /** How many sub-readers we are merging. * @see #getSubs */ public int getNumSubs() { Index: lucene/core/src/java/org/apache/lucene/codecs/MappingMultiDocsEnum.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/MappingMultiDocsEnum.java (revision 1522598) +++ lucene/core/src/java/org/apache/lucene/codecs/MappingMultiDocsEnum.java (working copy) @@ -39,26 +39,28 @@ DocsEnum current; int currentBase; int doc = -1; - private MergeState mergeState; + private final MergeState mergeState; + MultiDocsEnum multiDocsEnum; - /** Sole constructor. */ + /** Constructor for non-merge case. */ public MappingMultiDocsEnum() { + this.mergeState = null; } + /** Constructor for merge case. */ + public MappingMultiDocsEnum(MergeState mergeState) { + this.mergeState = mergeState; + } + MappingMultiDocsEnum reset(MultiDocsEnum docsEnum) { this.numSubs = docsEnum.getNumSubs(); this.subs = docsEnum.getSubs(); + this.multiDocsEnum = docsEnum; upto = -1; current = null; return this; } - /** Sets the {@link MergeState}, which is used to re-map - * document IDs. */ - public void setMergeState(MergeState mergeState) { - this.mergeState = mergeState; - } - /** How many sub-readers we are merging. * @see #getSubs */ public int getNumSubs() { Index: lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java (revision 1522598) +++ lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java (working copy) @@ -723,11 +723,6 @@ } @Override - public Comparator getComparator() { - return BytesRef.getUTF8SortedAsUnicodeComparator(); - } - - @Override public long size() throws IOException { return numTerms; } @@ -820,11 +815,6 @@ } @Override - public Comparator getComparator() { - return BytesRef.getUTF8SortedAsUnicodeComparator(); - } - - @Override public SeekStatus seekCeil(BytesRef text) throws IOException { if (ord < numTerms && ord >= 0) { Index: lucene/core/src/java/org/apache/lucene/codecs/FieldsConsumer.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/FieldsConsumer.java (revision 1522598) +++ lucene/core/src/java/org/apache/lucene/codecs/FieldsConsumer.java (working copy) @@ -17,14 +17,11 @@ * limitations under the License. */ -import java.io.Closeable; import java.io.IOException; -import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FieldInfo; // javadocs import org.apache.lucene.index.Fields; -import org.apache.lucene.index.MergeState; import org.apache.lucene.index.SegmentWriteState; // javadocs -import org.apache.lucene.index.Terms; /** * Abstract API that consumes terms, doc, freq, prox, offset and @@ -43,34 +40,49 @@ * * @lucene.experimental */ -public abstract class FieldsConsumer implements Closeable { +public abstract class FieldsConsumer { + /** Sole constructor. (For invocation by subclass * constructors, typically implicit.) */ protected FieldsConsumer() { } - /** Add a new field */ - public abstract TermsConsumer addField(FieldInfo field) throws IOException; - - /** Called when we are done adding everything. */ - @Override - public abstract void close() throws IOException; + // TODO: can we somehow compute stats for you...? - /** Called during merging to merge all {@link Fields} from - * sub-readers. This must recurse to merge all postings - * (terms, docs, positions, etc.). A {@link - * PostingsFormat} can override this default - * implementation to do its own merging. */ - public void merge(MergeState mergeState, Fields fields) throws IOException { - for (String field : fields) { - FieldInfo info = mergeState.fieldInfos.fieldInfo(field); - assert info != null : "FieldInfo for field is null: "+ field; - Terms terms = fields.terms(field); - if (terms != null) { - final TermsConsumer termsConsumer = addField(info); - termsConsumer.merge(mergeState, info.getIndexOptions(), terms.iterator(null)); - } - } - } + // TODO: maybe we should factor out "limited" (only + // iterables, no counts/stats) base classes from + // Fields/Terms/Docs/AndPositions? + + /** Write all fields, terms and postings. This the "pull" + * API, allowing you to iterate more than once over the + * postings, somewhat analogous to using a DOM API to + * traverse an XML tree. Alternatively, if you subclass + * {@link PushPostingsConsumer}, then all postings are + * pushed in a single pass, somewhat analogous to using a + * SAX API to traverse an XML tree. + * + *

This API is has certain restrictions vs {@link + * PushPostingsConsumer}: + * + *

    + *
  • You must compute index statistics yourself, + * including each Term's docFreq and totalTermFreq, + * as well as the summary sumTotalTermFreq, + * sumTotalDocFreq and docCount. + * + *
  • You must skip terms that have no docs and + * fields that have no terms, even though the provided + * Fields API will expose them; this typically + * requires lazily writing the field or term until + * you've actually seen the first term or + * document. + * + *
  • The provided Fields instance is limited: you + * cannot call any methods that return + * statistics/counts; you cannot pass a non-null + * live docs when pulling docs/positions enums. + *
+ */ + public abstract void write(Fields fields) throws IOException; } Index: lucene/core/src/java/org/apache/lucene/search/TermRangeTermsEnum.java =================================================================== --- lucene/core/src/java/org/apache/lucene/search/TermRangeTermsEnum.java (revision 1522598) +++ lucene/core/src/java/org/apache/lucene/search/TermRangeTermsEnum.java (working copy) @@ -17,7 +17,6 @@ * limitations under the License. */ -import java.io.IOException; import java.util.Comparator; import org.apache.lucene.index.FilteredTermsEnum; @@ -82,7 +81,7 @@ } setInitialSeekTerm(lowerBytesRef); - termComp = getComparator(); + termComp = BytesRef.getUTF8SortedAsUnicodeComparator(); } @Override Index: lucene/core/src/java/org/apache/lucene/search/NumericRangeQuery.java =================================================================== --- lucene/core/src/java/org/apache/lucene/search/NumericRangeQuery.java (revision 1522598) +++ lucene/core/src/java/org/apache/lucene/search/NumericRangeQuery.java (working copy) @@ -482,7 +482,7 @@ throw new IllegalArgumentException("Invalid NumericType"); } - termComp = getComparator(); + termComp = BytesRef.getUTF8SortedAsUnicodeComparator(); } private void nextRange() { Index: lucene/core/src/java/org/apache/lucene/search/TopTermsRewrite.java =================================================================== --- lucene/core/src/java/org/apache/lucene/search/TopTermsRewrite.java (revision 1522598) +++ lucene/core/src/java/org/apache/lucene/search/TopTermsRewrite.java (working copy) @@ -77,7 +77,7 @@ @Override public void setNextEnum(TermsEnum termsEnum) { this.termsEnum = termsEnum; - this.termComp = termsEnum.getComparator(); + this.termComp = BytesRef.getUTF8SortedAsUnicodeComparator(); assert compareToLastTerm(null); @@ -95,7 +95,7 @@ } else if (t == null) { lastTerm = null; } else { - assert termsEnum.getComparator().compare(lastTerm, t) < 0: "lastTerm=" + lastTerm + " t=" + t; + assert termComp.compare(lastTerm, t) < 0: "lastTerm=" + lastTerm + " t=" + t; lastTerm.copyBytes(t); } return true; Index: lucene/core/src/java/org/apache/lucene/search/ScoringRewrite.java =================================================================== --- lucene/core/src/java/org/apache/lucene/search/ScoringRewrite.java (revision 1522598) +++ lucene/core/src/java/org/apache/lucene/search/ScoringRewrite.java (working copy) @@ -109,7 +109,7 @@ final int size = col.terms.size(); if (size > 0) { - final int sort[] = col.terms.sort(col.termsEnum.getComparator()); + final int sort[] = col.terms.sort(BytesRef.getUTF8SortedAsUnicodeComparator()); final float[] boost = col.array.boost; final TermContext[] termStates = col.array.termState; for (int i = 0; i < size; i++) { Index: lucene/core/src/java/org/apache/lucene/search/DocTermOrdsRewriteMethod.java =================================================================== --- lucene/core/src/java/org/apache/lucene/search/DocTermOrdsRewriteMethod.java (revision 1522598) +++ lucene/core/src/java/org/apache/lucene/search/DocTermOrdsRewriteMethod.java (working copy) @@ -91,11 +91,6 @@ TermsEnum termsEnum = query.getTermsEnum(new Terms() { @Override - public Comparator getComparator() { - return BytesRef.getUTF8SortedAsUnicodeComparator(); - } - - @Override public TermsEnum iterator(TermsEnum reuse) { return docTermOrds.termsEnum(); } Index: lucene/core/src/java/org/apache/lucene/search/FieldCacheRewriteMethod.java =================================================================== --- lucene/core/src/java/org/apache/lucene/search/FieldCacheRewriteMethod.java (revision 1522598) +++ lucene/core/src/java/org/apache/lucene/search/FieldCacheRewriteMethod.java (working copy) @@ -91,11 +91,6 @@ TermsEnum termsEnum = query.getTermsEnum(new Terms() { @Override - public Comparator getComparator() { - return BytesRef.getUTF8SortedAsUnicodeComparator(); - } - - @Override public TermsEnum iterator(TermsEnum reuse) { return fcsi.termsEnum(); } Index: lucene/core/src/java/org/apache/lucene/search/ConstantScoreAutoRewrite.java =================================================================== --- lucene/core/src/java/org/apache/lucene/search/ConstantScoreAutoRewrite.java (revision 1522598) +++ lucene/core/src/java/org/apache/lucene/search/ConstantScoreAutoRewrite.java (working copy) @@ -101,7 +101,7 @@ } else { final BooleanQuery bq = getTopLevelQuery(); final BytesRefHash pendingTerms = col.pendingTerms; - final int sort[] = pendingTerms.sort(col.termsEnum.getComparator()); + final int sort[] = pendingTerms.sort(BytesRef.getUTF8SortedAsUnicodeComparator()); for(int i = 0; i < size; i++) { final int pos = sort[i]; // docFreq is not used for constant score here, we pass 1 Index: lucene/core/src/java/org/apache/lucene/search/TermCollectingRewrite.java =================================================================== --- lucene/core/src/java/org/apache/lucene/search/TermCollectingRewrite.java (revision 1522598) +++ lucene/core/src/java/org/apache/lucene/search/TermCollectingRewrite.java (working copy) @@ -18,7 +18,6 @@ */ import java.io.IOException; -import java.util.Comparator; import org.apache.lucene.index.AtomicReaderContext; import org.apache.lucene.index.Fields; @@ -47,7 +46,6 @@ final void collectTerms(IndexReader reader, MultiTermQuery query, TermCollector collector) throws IOException { IndexReaderContext topReaderContext = reader.getContext(); - Comparator lastTermComp = null; for (AtomicReaderContext context : topReaderContext.leaves()) { final Fields fields = context.reader().fields(); if (fields == null) { @@ -67,11 +65,6 @@ if (termsEnum == TermsEnum.EMPTY) continue; - // Check comparator compatibility: - final Comparator newTermComp = termsEnum.getComparator(); - if (lastTermComp != null && newTermComp != null && newTermComp != lastTermComp) - throw new RuntimeException("term comparator should not change between segments: "+lastTermComp+" != "+newTermComp); - lastTermComp = newTermComp; collector.setReaderContext(topReaderContext, context); collector.setNextEnum(termsEnum); BytesRef bytes; Index: lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java =================================================================== --- lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java (revision 1522598) +++ lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java (working copy) @@ -293,11 +293,6 @@ } @Override - public Comparator getComparator() { - return actualEnum.getComparator(); - } - - @Override public long ord() throws IOException { return actualEnum.ord(); } Index: lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java (revision 1522598) +++ lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java (working copy) @@ -77,13 +77,15 @@ nextPerField = null; } + /* void shrinkHash(int targetSize) { // Fully free the bytesHash on each flush but keep the pool untouched // bytesHash.clear will clear the ByteStartArray and in turn the ParallelPostingsArray too bytesHash.clear(false); } + */ - public void reset() { + void reset() { bytesHash.clear(false); if (nextPerField != null) nextPerField.reset(); @@ -108,7 +110,7 @@ /** Collapse the hash table & sort in-place. */ public int[] sortPostings(Comparator termComp) { - return bytesHash.sort(termComp); + return bytesHash.sort(termComp); } private boolean doCall; @@ -136,7 +138,8 @@ // Secondary entry point (for 2nd & subsequent TermsHash), // because token text has already been "interned" into - // textStart, so we hash by textStart + // textStart, so we hash by textStart. term vectors use + // this API. public void add(int textStart) throws IOException { int termID = bytesHash.addByPoolOffset(textStart); if (termID >= 0) { // New posting @@ -173,7 +176,8 @@ } } - // Primary entry point (for first TermsHash) + // Primary entry point (for first TermsHash); postings use + // this API. @Override void add() throws IOException { Index: lucene/core/src/java/org/apache/lucene/index/TermsEnum.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/TermsEnum.java (revision 1522598) +++ lucene/core/src/java/org/apache/lucene/index/TermsEnum.java (working copy) @@ -18,7 +18,6 @@ */ import java.io.IOException; -import java.util.Comparator; import org.apache.lucene.util.AttributeSource; import org.apache.lucene.util.Bits; @@ -32,9 +31,9 @@ * DocsAndPositionsEnum} for the current term ({@link * #docs}. * - *

Term enumerations are always ordered by - * {@link #getComparator}. Each term in the enumeration is - * greater than the one before it.

+ *

Term enumerations are always ordered by Unicode sort + * order. Each term in the enumeration is greater than the + * one before it.

* *

The TermsEnum is unpositioned when you first obtain it * and you must first successfully call {@link #next} or one @@ -230,11 +229,6 @@ } @Override - public Comparator getComparator() { - return null; - } - - @Override public int docFreq() { throw new IllegalStateException("this method should never be called"); } Index: lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesTermsEnum.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesTermsEnum.java (revision 1522598) +++ lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesTermsEnum.java (working copy) @@ -18,7 +18,6 @@ */ import java.io.IOException; -import java.util.Comparator; import org.apache.lucene.util.Bits; import org.apache.lucene.util.BytesRef; @@ -125,11 +124,6 @@ } @Override - public Comparator getComparator() { - return BytesRef.getUTF8SortedAsUnicodeComparator(); - } - - @Override public void seekExact(BytesRef term, TermState state) throws IOException { assert state != null && state instanceof OrdTermState; this.seekExact(((OrdTermState)state).ord); Index: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/CheckIndex.java (revision 1522598) +++ lucene/core/src/java/org/apache/lucene/index/CheckIndex.java (working copy) @@ -761,8 +761,6 @@ BytesRef lastTerm = null; - Comparator termComp = terms.getComparator(); - long sumTotalTermFreq = 0; long sumDocFreq = 0; FixedBitSet visitedDocs = new FixedBitSet(maxDoc); @@ -780,7 +778,7 @@ if (lastTerm == null) { lastTerm = BytesRef.deepCopyOf(term); } else { - if (termComp.compare(lastTerm, term) >= 0) { + if (lastTerm.compareTo(term) >= 0) { throw new RuntimeException("terms out of order: lastTerm=" + lastTerm + " term=" + term); } lastTerm.copyBytes(term); Index: lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java (revision 1522598) +++ lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java (working copy) @@ -19,19 +19,63 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; import java.util.List; import java.util.Map; -import org.apache.lucene.codecs.FieldsConsumer; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.CollectionUtil; -import org.apache.lucene.util.IOUtils; final class FreqProxTermsWriter extends TermsHashConsumer { @Override void abort() {} + private void applyDeletes(SegmentWriteState state, Fields fields) throws IOException { + // Process any pending Term deletes for this newly + // flushed segment: + if (state.segDeletes != null && state.segDeletes.terms.size() > 0) { + Map segDeletes = state.segDeletes.terms; + List deleteTerms = new ArrayList(segDeletes.keySet()); + Collections.sort(deleteTerms); + String lastField = null; + TermsEnum termsEnum = null; + DocsEnum docsEnum = null; + for(Term deleteTerm : deleteTerms) { + if (deleteTerm.field().equals(lastField) == false) { + lastField = deleteTerm.field(); + Terms terms = fields.terms(lastField); + if (terms != null) { + termsEnum = terms.iterator(termsEnum); + } + } + + if (termsEnum != null && termsEnum.seekExact(deleteTerm.bytes())) { + docsEnum = termsEnum.docs(null, docsEnum, 0); + int delDocLimit = segDeletes.get(deleteTerm); + while (true) { + int doc = docsEnum.nextDoc(); + if (doc == DocsEnum.NO_MORE_DOCS) { + break; + } + if (doc < delDocLimit) { + if (state.liveDocs == null) { + state.liveDocs = state.segmentInfo.getCodec().liveDocsFormat().newLiveDocs(state.segmentInfo.getDocCount()); + } + if (state.liveDocs.get(doc)) { + state.delCountOnFlush++; + state.liveDocs.clear(doc); + } + } else { + break; + } + } + } + } + } + } + // TODO: would be nice to factor out more of this, eg the // FreqProxFieldMergeState, and code to visit all Fields // under the same FieldInfo together, up into TermsHash*. @@ -44,66 +88,25 @@ // ThreadStates List allFields = new ArrayList(); + Comparator termComp = BytesRef.getUTF8SortedAsUnicodeComparator(); + for (TermsHashConsumerPerField f : fieldsToFlush.values()) { final FreqProxTermsWriterPerField perField = (FreqProxTermsWriterPerField) f; if (perField.termsHashPerField.bytesHash.size() > 0) { + perField.sortPostings(termComp); + assert perField.fieldInfo.isIndexed(); allFields.add(perField); } } - final int numAllFields = allFields.size(); - // Sort by field name CollectionUtil.introSort(allFields); - final FieldsConsumer consumer = state.segmentInfo.getCodec().postingsFormat().fieldsConsumer(state); + Fields fields = new FreqProxFields(allFields); - boolean success = false; + applyDeletes(state, fields); - try { - TermsHash termsHash = null; - - /* - Current writer chain: - FieldsConsumer - -> IMPL: FormatPostingsTermsDictWriter - -> TermsConsumer - -> IMPL: FormatPostingsTermsDictWriter.TermsWriter - -> DocsConsumer - -> IMPL: FormatPostingsDocsWriter - -> PositionsConsumer - -> IMPL: FormatPostingsPositionsWriter - */ - - for (int fieldNumber = 0; fieldNumber < numAllFields; fieldNumber++) { - final FieldInfo fieldInfo = allFields.get(fieldNumber).fieldInfo; - - final FreqProxTermsWriterPerField fieldWriter = allFields.get(fieldNumber); - - // If this field has postings then add them to the - // segment - fieldWriter.flush(fieldInfo.name, consumer, state); - - TermsHashPerField perField = fieldWriter.termsHashPerField; - assert termsHash == null || termsHash == perField.termsHash; - termsHash = perField.termsHash; - int numPostings = perField.bytesHash.size(); - perField.reset(); - perField.shrinkHash(numPostings); - fieldWriter.reset(); - } - - if (termsHash != null) { - termsHash.reset(); - } - success = true; - } finally { - if (success) { - IOUtils.close(consumer); - } else { - IOUtils.closeWhileHandlingException(consumer); - } - } + state.segmentInfo.getCodec().postingsFormat().fieldsConsumer(state).write(fields); } BytesRef payload; Index: lucene/core/src/java/org/apache/lucene/index/Terms.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/Terms.java (revision 1522598) +++ lucene/core/src/java/org/apache/lucene/index/Terms.java (working copy) @@ -75,13 +75,6 @@ } } - /** Return the BytesRef Comparator used to sort terms - * provided by the iterator. This method may return null - * if there are no terms. This method may be invoked - * many times; it's best to cache a single instance & - * reuse it. */ - public abstract Comparator getComparator(); - /** Returns the number of terms for this field, or -1 if this * measure isn't stored by the codec. Note that, just like * other term measures, this measure does not take deleted Index: lucene/core/src/java/org/apache/lucene/index/DocTermOrds.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/DocTermOrds.java (revision 1522598) +++ lucene/core/src/java/org/apache/lucene/index/DocTermOrds.java (working copy) @@ -20,7 +20,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; -import java.util.Comparator; import java.util.List; import org.apache.lucene.codecs.PostingsFormat; // javadocs @@ -611,11 +610,6 @@ termsEnum = reader.fields().terms(field).iterator(null); } - @Override - public Comparator getComparator() { - return termsEnum.getComparator(); - } - @Override public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException { return termsEnum.docs(liveDocs, reuse, flags); Index: lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumer.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumer.java (revision 1522598) +++ lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumer.java (working copy) @@ -66,12 +66,6 @@ hasVectors = false; } } - - for (final TermsHashConsumerPerField field : fieldsToFlush.values() ) { - TermVectorsConsumerPerField perField = (TermVectorsConsumerPerField) field; - perField.termsHashPerField.reset(); - perField.shrinkHash(); - } } /** Fills in no-term-vectors for all docs we haven't seen Index: lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java (revision 1522598) +++ lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java (working copy) @@ -191,11 +191,6 @@ fieldInfo.setStoreTermVectors(); } - void shrinkHash() { - termsHashPerField.shrinkHash(maxNumPostings); - maxNumPostings = 0; - } - @Override void start(IndexableField f) { if (doVectorOffsets) { Index: lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java (revision 1522598) +++ lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java (working copy) @@ -22,9 +22,9 @@ import java.util.List; import org.apache.lucene.codecs.Codec; +import org.apache.lucene.codecs.DocValuesConsumer; import org.apache.lucene.codecs.FieldInfosWriter; -import org.apache.lucene.codecs.FieldsConsumer; -import org.apache.lucene.codecs.DocValuesConsumer; +import org.apache.lucene.codecs.MappedMultiFields; import org.apache.lucene.codecs.StoredFieldsWriter; import org.apache.lucene.codecs.TermVectorsWriter; import org.apache.lucene.index.FieldInfo.DocValuesType; @@ -375,7 +375,15 @@ docBase += maxDoc; } - final FieldsConsumer consumer = codec.postingsFormat().fieldsConsumer(segmentWriteState); + //final FieldsConsumer consumer = codec.postingsFormat().fieldsConsumer(segmentWriteState); + + Fields mergedFields = new MappedMultiFields(mergeState, + new MultiFields(fields.toArray(Fields.EMPTY_ARRAY), + slices.toArray(ReaderSlice.EMPTY_ARRAY))); + + codec.postingsFormat().fieldsConsumer(segmentWriteState).write(mergedFields); + + /* boolean success = false; try { consumer.merge(mergeState, @@ -389,5 +397,6 @@ IOUtils.closeWhileHandlingException(consumer); } } + */ } } Index: lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java (revision 1522598) +++ lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java (working copy) @@ -98,11 +98,6 @@ public TermsEnum iterator(TermsEnum reuse) throws IOException { return in.iterator(reuse); } - - @Override - public Comparator getComparator() { - return in.getComparator(); - } @Override public long size() throws IOException { @@ -200,11 +195,6 @@ public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException { return in.docsAndPositions(liveDocs, reuse, flags); } - - @Override - public Comparator getComparator() { - return in.getComparator(); - } } /** Base class for filtering {@link DocsEnum} implementations. */ Index: lucene/core/src/java/org/apache/lucene/index/AutomatonTermsEnum.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/AutomatonTermsEnum.java (revision 1522598) +++ lucene/core/src/java/org/apache/lucene/index/AutomatonTermsEnum.java (working copy) @@ -86,7 +86,7 @@ // used for path tracking, where each bit is a numbered state. visited = new long[runAutomaton.getSize()]; - termComp = getComparator(); + termComp = BytesRef.getUTF8SortedAsUnicodeComparator(); } /** Index: lucene/core/src/java/org/apache/lucene/index/MultiTerms.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/MultiTerms.java (revision 1522598) +++ lucene/core/src/java/org/apache/lucene/index/MultiTerms.java (working copy) @@ -36,7 +36,6 @@ public final class MultiTerms extends Terms { private final Terms[] subs; private final ReaderSlice[] subSlices; - private final Comparator termComp; private final boolean hasOffsets; private final boolean hasPositions; private final boolean hasPayloads; @@ -51,28 +50,16 @@ this.subs = subs; this.subSlices = subSlices; - Comparator _termComp = null; assert subs.length > 0 : "inefficient: don't use MultiTerms over one sub"; boolean _hasOffsets = true; boolean _hasPositions = true; boolean _hasPayloads = false; for(int i=0;i subTermComp = subs[i].getComparator(); - if (subTermComp != null && !subTermComp.equals(_termComp)) { - throw new IllegalStateException("sub-readers have different BytesRef.Comparators; cannot merge"); - } - } _hasOffsets &= subs[i].hasOffsets(); _hasPositions &= subs[i].hasPositions(); _hasPayloads |= subs[i].hasPayloads(); } - termComp = _termComp; hasOffsets = _hasOffsets; hasPositions = _hasPositions; hasPayloads = hasPositions && _hasPayloads; // if all subs have pos, and at least one has payloads. @@ -158,11 +145,6 @@ } @Override - public Comparator getComparator() { - return termComp; - } - - @Override public boolean hasOffsets() { return hasOffsets; } Index: lucene/core/src/java/org/apache/lucene/index/SortedDocValuesTermsEnum.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/SortedDocValuesTermsEnum.java (revision 1522598) +++ lucene/core/src/java/org/apache/lucene/index/SortedDocValuesTermsEnum.java (working copy) @@ -18,7 +18,6 @@ */ import java.io.IOException; -import java.util.Comparator; import org.apache.lucene.util.Bits; import org.apache.lucene.util.BytesRef; @@ -125,11 +124,6 @@ } @Override - public Comparator getComparator() { - return BytesRef.getUTF8SortedAsUnicodeComparator(); - } - - @Override public void seekExact(BytesRef term, TermState state) throws IOException { assert state != null && state instanceof OrdTermState; this.seekExact(((OrdTermState)state).ord); Index: lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java (revision 1522598) +++ lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java (working copy) @@ -17,19 +17,12 @@ * limitations under the License. */ -import java.io.IOException; import java.util.Comparator; -import java.util.Map; import org.apache.lucene.analysis.tokenattributes.OffsetAttribute; import org.apache.lucene.analysis.tokenattributes.PayloadAttribute; -import org.apache.lucene.codecs.FieldsConsumer; -import org.apache.lucene.codecs.PostingsConsumer; -import org.apache.lucene.codecs.TermStats; -import org.apache.lucene.codecs.TermsConsumer; import org.apache.lucene.index.FieldInfo.IndexOptions; import org.apache.lucene.util.BytesRef; -import org.apache.lucene.util.FixedBitSet; import org.apache.lucene.util.RamUsageEstimator; // TODO: break into separate freq and prox writers as @@ -42,12 +35,17 @@ final FieldInfo fieldInfo; final DocumentsWriterPerThread.DocState docState; final FieldInvertState fieldState; - private boolean hasFreq; - private boolean hasProx; - private boolean hasOffsets; + boolean hasFreq; + boolean hasProx; + boolean hasOffsets; PayloadAttribute payloadAttribute; OffsetAttribute offsetAttribute; + long sumTotalTermFreq; + long sumDocFreq; + // How many docs have this field: + int docCount; + public FreqProxTermsWriterPerField(TermsHashPerField termsHashPerField, FreqProxTermsWriter parent, FieldInfo fieldInfo) { this.termsHashPerField = termsHashPerField; this.parent = parent; @@ -68,6 +66,12 @@ @Override void finish() { + sumDocFreq += fieldState.uniqueTermCount; + sumTotalTermFreq += fieldState.length; + if (fieldState.length > 0) { + docCount++; + } + if (hasPayloads) { fieldInfo.setStorePayloads(); } @@ -83,14 +87,6 @@ return fieldInfo.name.compareTo(other.fieldInfo.name); } - // Called after flush - void reset() { - // Record, up front, whether our in-RAM format will be - // with or without term freqs: - setIndexOptions(fieldInfo.getIndexOptions()); - payloadAttribute = null; - } - private void setIndexOptions(IndexOptions indexOptions) { if (indexOptions == null) { // field could later be updated with indexed=true, so set everything on @@ -318,233 +314,10 @@ BytesRef payload; - /* Walk through all unique text tokens (Posting - * instances) found in this field and serialize them - * into a single RAM segment. */ - void flush(String fieldName, FieldsConsumer consumer, final SegmentWriteState state) - throws IOException { + int[] sortedTermIDs; - if (!fieldInfo.isIndexed()) { - return; // nothing to flush, don't bother the codec with the unindexed field - } - - final TermsConsumer termsConsumer = consumer.addField(fieldInfo); - final Comparator termComp = termsConsumer.getComparator(); - - // CONFUSING: this.indexOptions holds the index options - // that were current when we first saw this field. But - // it's possible this has changed, eg when other - // documents are indexed that cause a "downgrade" of the - // IndexOptions. So we must decode the in-RAM buffer - // according to this.indexOptions, but then write the - // new segment to the directory according to - // currentFieldIndexOptions: - final IndexOptions currentFieldIndexOptions = fieldInfo.getIndexOptions(); - assert currentFieldIndexOptions != null; - - final boolean writeTermFreq = currentFieldIndexOptions.compareTo(IndexOptions.DOCS_AND_FREQS) >= 0; - final boolean writePositions = currentFieldIndexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; - final boolean writeOffsets = currentFieldIndexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0; - - final boolean readTermFreq = this.hasFreq; - final boolean readPositions = this.hasProx; - final boolean readOffsets = this.hasOffsets; - - //System.out.println("flush readTF=" + readTermFreq + " readPos=" + readPositions + " readOffs=" + readOffsets); - - // Make sure FieldInfo.update is working correctly!: - assert !writeTermFreq || readTermFreq; - assert !writePositions || readPositions; - assert !writeOffsets || readOffsets; - - assert !writeOffsets || writePositions; - - final Map segDeletes; - if (state.segDeletes != null && state.segDeletes.terms.size() > 0) { - segDeletes = state.segDeletes.terms; - } else { - segDeletes = null; - } - - final int[] termIDs = termsHashPerField.sortPostings(termComp); - final int numTerms = termsHashPerField.bytesHash.size(); - final BytesRef text = new BytesRef(); - final FreqProxPostingsArray postings = (FreqProxPostingsArray) termsHashPerField.postingsArray; - final ByteSliceReader freq = new ByteSliceReader(); - final ByteSliceReader prox = new ByteSliceReader(); - - FixedBitSet visitedDocs = new FixedBitSet(state.segmentInfo.getDocCount()); - long sumTotalTermFreq = 0; - long sumDocFreq = 0; - - Term protoTerm = new Term(fieldName); - for (int i = 0; i < numTerms; i++) { - final int termID = termIDs[i]; - //System.out.println("term=" + termID); - // Get BytesRef - final int textStart = postings.textStarts[termID]; - termsHashPerField.bytePool.setBytesRef(text, textStart); - - termsHashPerField.initReader(freq, termID, 0); - if (readPositions || readOffsets) { - termsHashPerField.initReader(prox, termID, 1); - } - - // TODO: really TermsHashPerField should take over most - // of this loop, including merge sort of terms from - // multiple threads and interacting with the - // TermsConsumer, only calling out to us (passing us the - // DocsConsumer) to handle delivery of docs/positions - - final PostingsConsumer postingsConsumer = termsConsumer.startTerm(text); - - final int delDocLimit; - if (segDeletes != null) { - protoTerm.bytes = text; - final Integer docIDUpto = segDeletes.get(protoTerm); - if (docIDUpto != null) { - delDocLimit = docIDUpto; - } else { - delDocLimit = 0; - } - } else { - delDocLimit = 0; - } - - // Now termStates has numToMerge FieldMergeStates - // which all share the same term. Now we must - // interleave the docID streams. - int docFreq = 0; - long totalTermFreq = 0; - int docID = 0; - - while(true) { - //System.out.println(" cycle"); - final int termFreq; - if (freq.eof()) { - if (postings.lastDocCodes[termID] != -1) { - // Return last doc - docID = postings.lastDocIDs[termID]; - if (readTermFreq) { - termFreq = postings.termFreqs[termID]; - } else { - termFreq = -1; - } - postings.lastDocCodes[termID] = -1; - } else { - // EOF - break; - } - } else { - final int code = freq.readVInt(); - if (!readTermFreq) { - docID += code; - termFreq = -1; - } else { - docID += code >>> 1; - if ((code & 1) != 0) { - termFreq = 1; - } else { - termFreq = freq.readVInt(); - } - } - - assert docID != postings.lastDocIDs[termID]; - } - - docFreq++; - assert docID < state.segmentInfo.getDocCount(): "doc=" + docID + " maxDoc=" + state.segmentInfo.getDocCount(); - - // NOTE: we could check here if the docID was - // deleted, and skip it. However, this is somewhat - // dangerous because it can yield non-deterministic - // behavior since we may see the docID before we see - // the term that caused it to be deleted. This - // would mean some (but not all) of its postings may - // make it into the index, which'd alter the docFreq - // for those terms. We could fix this by doing two - // passes, ie first sweep marks all del docs, and - // 2nd sweep does the real flush, but I suspect - // that'd add too much time to flush. - visitedDocs.set(docID); - postingsConsumer.startDoc(docID, writeTermFreq ? termFreq : -1); - if (docID < delDocLimit) { - // Mark it deleted. TODO: we could also skip - // writing its postings; this would be - // deterministic (just for this Term's docs). - - // TODO: can we do this reach-around in a cleaner way???? - if (state.liveDocs == null) { - state.liveDocs = docState.docWriter.codec.liveDocsFormat().newLiveDocs(state.segmentInfo.getDocCount()); - } - if (state.liveDocs.get(docID)) { - state.delCountOnFlush++; - state.liveDocs.clear(docID); - } - } - - totalTermFreq += termFreq; - - // Carefully copy over the prox + payload info, - // changing the format to match Lucene's segment - // format. - - if (readPositions || readOffsets) { - // we did record positions (& maybe payload) and/or offsets - int position = 0; - int offset = 0; - for(int j=0;j>> 1; - - if ((code & 1) != 0) { - - // This position has a payload - final int payloadLength = prox.readVInt(); - - if (payload == null) { - payload = new BytesRef(); - payload.bytes = new byte[payloadLength]; - } else if (payload.bytes.length < payloadLength) { - payload.grow(payloadLength); - } - - prox.readBytes(payload.bytes, 0, payloadLength); - payload.length = payloadLength; - thisPayload = payload; - - } else { - thisPayload = null; - } - - if (readOffsets) { - final int startOffset = offset + prox.readVInt(); - final int endOffset = startOffset + prox.readVInt(); - if (writePositions) { - if (writeOffsets) { - assert startOffset >=0 && endOffset >= startOffset : "startOffset=" + startOffset + ",endOffset=" + endOffset + ",offset=" + offset; - postingsConsumer.addPosition(position, thisPayload, startOffset, endOffset); - } else { - postingsConsumer.addPosition(position, thisPayload, -1, -1); - } - } - offset = startOffset; - } else if (writePositions) { - postingsConsumer.addPosition(position, thisPayload, -1, -1); - } - } - } - } - postingsConsumer.finishDoc(); - } - termsConsumer.finishTerm(text, new TermStats(docFreq, writeTermFreq ? totalTermFreq : -1)); - sumTotalTermFreq += totalTermFreq; - sumDocFreq += docFreq; - } - - termsConsumer.finish(writeTermFreq ? sumTotalTermFreq : -1, sumDocFreq, visitedDocs.cardinality()); + void sortPostings(Comparator termComp) { + assert sortedTermIDs == null; + sortedTermIDs = termsHashPerField.sortPostings(termComp); } } Index: lucene/core/src/java/org/apache/lucene/index/FreqProxFields.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/FreqProxFields.java (revision 0) +++ lucene/core/src/java/org/apache/lucene/index/FreqProxFields.java (working copy) @@ -0,0 +1,522 @@ +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.Comparator; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; + +import org.apache.lucene.index.FieldInfo.IndexOptions; +import org.apache.lucene.index.FreqProxTermsWriterPerField.FreqProxPostingsArray; +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.BytesRef; + +/** Implements {@link Fields} interface over the in-RAM + * buffered fields/terms/postings, to flush postings + * through the PostingsFormat. */ + +class FreqProxFields extends Fields { + final Map fields = new LinkedHashMap(); + + public FreqProxFields(List fieldList) { + // NOTE: fields are already sorted by field name + for(FreqProxTermsWriterPerField field : fieldList) { + fields.put(field.fieldInfo.name, field); + } + } + + public Iterator iterator() { + return fields.keySet().iterator(); + } + + @Override + public Terms terms(String field) throws IOException { + FreqProxTermsWriterPerField perField = fields.get(field); + return perField == null ? null : new FreqProxTerms(perField); + } + + @Override + public int size() { + //return fields.size(); + throw new UnsupportedOperationException(); + } + + private static class FreqProxTerms extends Terms { + final FreqProxTermsWriterPerField terms; + + public FreqProxTerms(FreqProxTermsWriterPerField terms) { + this.terms = terms; + } + + @Override + public TermsEnum iterator(TermsEnum reuse) { + FreqProxTermsEnum termsEnum; + if (reuse instanceof FreqProxTermsEnum && ((FreqProxTermsEnum) reuse).terms == this.terms) { + termsEnum = (FreqProxTermsEnum) reuse; + } else { + termsEnum = new FreqProxTermsEnum(terms); + } + termsEnum.reset(); + return termsEnum; + } + + @Override + public long size() { + //return terms.termsHashPerField.bytesHash.size(); + throw new UnsupportedOperationException(); + } + + @Override + public long getSumTotalTermFreq() { + //return terms.sumTotalTermFreq; + throw new UnsupportedOperationException(); + } + + @Override + public long getSumDocFreq() { + //return terms.sumDocFreq; + throw new UnsupportedOperationException(); + } + + @Override + public int getDocCount() { + //return terms.docCount; + throw new UnsupportedOperationException(); + } + + @Override + public boolean hasOffsets() { + // NOTE: the in-memory buffer may have indexed offsets + // because that's what FieldInfo said when we started, + // but during indexing this may have been downgraded: + return terms.fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0; + } + + @Override + public boolean hasPositions() { + // NOTE: the in-memory buffer may have indexed positions + // because that's what FieldInfo said when we started, + // but during indexing this may have been downgraded: + return terms.fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; + } + + @Override + public boolean hasPayloads() { + return terms.hasPayloads; + } + } + + private static class FreqProxTermsEnum extends TermsEnum { + final FreqProxTermsWriterPerField terms; + final int[] sortedTermIDs; + final FreqProxPostingsArray postingsArray; + final BytesRef scratch = new BytesRef(); + final int numTerms; + int ord; + + public FreqProxTermsEnum(FreqProxTermsWriterPerField terms) { + this.terms = terms; + this.numTerms = terms.termsHashPerField.bytesHash.size(); + sortedTermIDs = terms.sortedTermIDs; + assert sortedTermIDs != null; + postingsArray = (FreqProxPostingsArray) terms.termsHashPerField.postingsArray; + } + + public void reset() { + ord = -1; + } + + public SeekStatus seekCeil(BytesRef text) { + + // TODO: we could instead keep the BytesRefHash + // intact so this is a hash lookup + + // binary search: + int lo = 0; + int hi = numTerms - 1; + while (hi >= lo) { + int mid = (lo + hi) >>> 1; + int textStart = postingsArray.textStarts[sortedTermIDs[mid]]; + terms.termsHashPerField.bytePool.setBytesRef(scratch, textStart); + int cmp = scratch.compareTo(text); + if (cmp < 0) { + lo = mid + 1; + } else if (cmp > 0) { + hi = mid - 1; + } else { + // found: + ord = mid; + return SeekStatus.FOUND; + } + } + + // not found: + ord = lo + 1; + if (ord == numTerms) { + return SeekStatus.END; + } else { + return SeekStatus.NOT_FOUND; + } + } + + public void seekExact(long ord) { + this.ord = (int) ord; + int textStart = postingsArray.textStarts[sortedTermIDs[this.ord]]; + terms.termsHashPerField.bytePool.setBytesRef(scratch, textStart); + } + + @Override + public BytesRef next() { + ord++; + if (ord >= numTerms) { + return null; + } else { + int textStart = postingsArray.textStarts[sortedTermIDs[ord]]; + terms.termsHashPerField.bytePool.setBytesRef(scratch, textStart); + return scratch; + } + } + + @Override + public BytesRef term() { + return scratch; + } + + @Override + public long ord() { + return ord; + } + + @Override + public int docFreq() { + // We do not store this per-term, and we cannot + // implement this at merge time w/o an added pass + // through the postings: + throw new UnsupportedOperationException(); + } + + @Override + public long totalTermFreq() { + // We do not store this per-term, and we cannot + // implement this at merge time w/o an added pass + // through the postings: + throw new UnsupportedOperationException(); + } + + @Override + public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) { + if (liveDocs != null) { + throw new IllegalArgumentException("liveDocs must be null"); + } + + FreqProxDocsEnum docsEnum; + + if (!terms.hasFreq && (flags & DocsEnum.FLAG_FREQS) != 0) { + // Caller wants freqs but we didn't index them; + // don't lie: + throw new IllegalArgumentException("did not index freq"); + } + + if (reuse instanceof FreqProxDocsEnum) { + docsEnum = (FreqProxDocsEnum) reuse; + if (docsEnum.postingsArray != postingsArray) { + docsEnum = new FreqProxDocsEnum(terms, postingsArray); + } + } else { + docsEnum = new FreqProxDocsEnum(terms, postingsArray); + } + docsEnum.reset(sortedTermIDs[ord]); + return docsEnum; + } + + @Override + public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) { + if (liveDocs != null) { + throw new IllegalArgumentException("liveDocs must be null"); + } + FreqProxDocsAndPositionsEnum posEnum; + + if (!terms.hasProx) { + // Caller wants positions but we didn't index them; + // don't lie: + throw new IllegalArgumentException("did not index positions"); + } + + if (!terms.hasOffsets && (flags & DocsAndPositionsEnum.FLAG_OFFSETS) != 0) { + // Caller wants offsets but we didn't index them; + // don't lie: + throw new IllegalArgumentException("did not index offsets"); + } + + if (reuse instanceof FreqProxDocsAndPositionsEnum) { + posEnum = (FreqProxDocsAndPositionsEnum) reuse; + if (posEnum.postingsArray != postingsArray) { + posEnum = new FreqProxDocsAndPositionsEnum(terms, postingsArray); + } + } else { + posEnum = new FreqProxDocsAndPositionsEnum(terms, postingsArray); + } + posEnum.reset(sortedTermIDs[ord]); + return posEnum; + } + + /** + * Expert: Returns the TermsEnums internal state to position the TermsEnum + * without re-seeking the term dictionary. + *

+ * NOTE: A seek by {@link TermState} might not capture the + * {@link AttributeSource}'s state. Callers must maintain the + * {@link AttributeSource} states separately + * + * @see TermState + * @see #seekExact(BytesRef, TermState) + */ + public TermState termState() throws IOException { + return new TermState() { + @Override + public void copyFrom(TermState other) { + throw new UnsupportedOperationException(); + } + }; + } + } + + private static class FreqProxDocsEnum extends DocsEnum { + + final FreqProxTermsWriterPerField terms; + final FreqProxPostingsArray postingsArray; + final ByteSliceReader reader = new ByteSliceReader(); + final boolean readTermFreq; + int docID; + int freq; + boolean ended; + int termID; + + public FreqProxDocsEnum(FreqProxTermsWriterPerField terms, FreqProxPostingsArray postingsArray) { + this.terms = terms; + this.postingsArray = postingsArray; + this.readTermFreq = terms.hasFreq; + } + + public void reset(int termID) { + this.termID = termID; + terms.termsHashPerField.initReader(reader, termID, 0); + ended = false; + docID = 0; + } + + @Override + public int docID() { + return docID; + } + + @Override + public int freq() { + // Don't lie here ... don't want codecs writings lots + // of wasted 1s into the index: + if (!readTermFreq) { + throw new IllegalStateException("freq was not indexed"); + } else { + return freq; + } + } + + @Override + public int nextDoc() throws IOException { + if (reader.eof()) { + if (ended) { + return NO_MORE_DOCS; + } else { + ended = true; + docID = postingsArray.lastDocIDs[termID]; + if (readTermFreq) { + freq = postingsArray.termFreqs[termID]; + } + } + } else { + int code = reader.readVInt(); + if (!readTermFreq) { + docID += code; + } else { + docID += code >>> 1; + if ((code & 1) != 0) { + freq = 1; + } else { + freq = reader.readVInt(); + } + } + + assert docID != postingsArray.lastDocIDs[termID]; + } + + return docID; + } + + @Override + public int advance(int target) { + throw new UnsupportedOperationException(); + } + + @Override + public long cost() { + throw new UnsupportedOperationException(); + } + } + + private static class FreqProxDocsAndPositionsEnum extends DocsAndPositionsEnum { + + final FreqProxTermsWriterPerField terms; + final FreqProxPostingsArray postingsArray; + final ByteSliceReader reader = new ByteSliceReader(); + final ByteSliceReader posReader = new ByteSliceReader(); + final boolean readOffsets; + int docID; + int freq; + int pos; + int startOffset; + int endOffset; + int posLeft; + int termID; + boolean ended; + boolean hasPayload; + BytesRef payload = new BytesRef(); + + public FreqProxDocsAndPositionsEnum(FreqProxTermsWriterPerField terms, FreqProxPostingsArray postingsArray) { + this.terms = terms; + this.postingsArray = postingsArray; + this.readOffsets = terms.hasOffsets; + assert terms.hasProx; + assert terms.hasFreq; + } + + public void reset(int termID) { + this.termID = termID; + terms.termsHashPerField.initReader(reader, termID, 0); + terms.termsHashPerField.initReader(posReader, termID, 1); + ended = false; + docID = 0; + posLeft = 0; + } + + @Override + public int docID() { + return docID; + } + + @Override + public int freq() { + return freq; + } + + @Override + public int nextDoc() throws IOException { + while (posLeft != 0) { + nextPosition(); + } + + if (reader.eof()) { + if (ended) { + return NO_MORE_DOCS; + } else { + ended = true; + docID = postingsArray.lastDocIDs[termID]; + freq = postingsArray.termFreqs[termID]; + } + } else { + int code = reader.readVInt(); + docID += code >>> 1; + if ((code & 1) != 0) { + freq = 1; + } else { + freq = reader.readVInt(); + } + + assert docID != postingsArray.lastDocIDs[termID]; + } + + posLeft = freq; + pos = 0; + startOffset = 0; + return docID; + } + + @Override + public int advance(int target) { + throw new UnsupportedOperationException(); + } + + @Override + public long cost() { + throw new UnsupportedOperationException(); + } + + @Override + public int nextPosition() throws IOException { + assert posLeft > 0; + posLeft--; + int code = posReader.readVInt(); + pos += code >>> 1; + if ((code & 1) != 0) { + hasPayload = true; + // has a payload + payload.length = posReader.readVInt(); + if (payload.bytes.length < payload.length) { + payload.grow(payload.length); + } + posReader.readBytes(payload.bytes, 0, payload.length); + } else { + hasPayload = false; + } + + if (readOffsets) { + startOffset += posReader.readVInt(); + endOffset = startOffset + posReader.readVInt(); + } + + return pos; + } + + @Override + public int startOffset() { + if (!readOffsets) { + throw new IllegalStateException("offsets were not indexed"); + } + return startOffset; + } + + @Override + public int endOffset() { + if (!readOffsets) { + throw new IllegalStateException("offsets were not indexed"); + } + return endOffset; + } + + @Override + public BytesRef getPayload() { + if (hasPayload) { + return payload; + } else { + return null; + } + } + } +} Property changes on: lucene/core/src/java/org/apache/lucene/index/FreqProxFields.java ___________________________________________________________________ Added: svn:eol-style ## -0,0 +1 ## +native \ No newline at end of property Index: lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java (revision 1522598) +++ lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java (working copy) @@ -39,6 +39,7 @@ private final TermsEnumWithSlice[] top; private final MultiDocsEnum.EnumWithSlice[] subDocs; private final MultiDocsAndPositionsEnum.EnumWithSlice[] subDocsAndPositions; + private static final Comparator termComp = BytesRef.getUTF8SortedAsUnicodeComparator(); private BytesRef lastSeek; private boolean lastSeekExact; @@ -47,7 +48,6 @@ private int numTop; private int numSubs; private BytesRef current; - private Comparator termComp; static class TermsEnumIndex { public final static TermsEnumIndex[] EMPTY_ARRAY = new TermsEnumIndex[0]; @@ -95,36 +95,18 @@ return current; } - @Override - public Comparator getComparator() { - return termComp; - } - /** The terms array must be newly created TermsEnum, ie * {@link TermsEnum#next} has not yet been called. */ public TermsEnum reset(TermsEnumIndex[] termsEnumsIndex) throws IOException { assert termsEnumsIndex.length <= top.length; numSubs = 0; numTop = 0; - termComp = null; queue.clear(); for(int i=0;i subTermComp = termsEnumIndex.termsEnum.getComparator(); - if (subTermComp != null && !subTermComp.equals(termComp)) { - throw new IllegalStateException("sub-readers have different BytesRef.Comparators: " + subTermComp + " vs " + termComp + "; cannot merge"); - } - } - final BytesRef term = termsEnumIndex.termsEnum.next(); if (term != null) { final TermsEnumWithSlice entry = subs[termsEnumIndex.subIndex]; @@ -519,14 +501,13 @@ } private final static class TermMergeQueue extends PriorityQueue { - Comparator termComp; TermMergeQueue(int size) { super(size); } @Override protected boolean lessThan(TermsEnumWithSlice termsA, TermsEnumWithSlice termsB) { - final int cmp = termComp.compare(termsA.current, termsB.current); + final int cmp = termsA.current.compareTo(termsB.current); if (cmp != 0) { return cmp < 0; } else { Index: lucene/core/src/java/org/apache/lucene/index/FilteredTermsEnum.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/FilteredTermsEnum.java (revision 1522598) +++ lucene/core/src/java/org/apache/lucene/index/FilteredTermsEnum.java (working copy) @@ -135,11 +135,6 @@ } @Override - public Comparator getComparator() { - return tenum.getComparator(); - } - - @Override public int docFreq() throws IOException { return tenum.docFreq(); } @@ -209,6 +204,9 @@ return tenum.termState(); } + // for asserts + private final static Comparator termComparator = BytesRef.getUTF8SortedAsUnicodeComparator(); + @SuppressWarnings("fallthrough") @Override public BytesRef next() throws IOException { @@ -221,7 +219,7 @@ final BytesRef t = nextSeekTerm(actualTerm); //System.out.println(" seek to t=" + (t == null ? "null" : t.utf8ToString()) + " tenum=" + tenum); // Make sure we always seek forward: - assert actualTerm == null || t == null || getComparator().compare(t, actualTerm) > 0: "curTerm=" + actualTerm + " seekTerm=" + t; + assert actualTerm == null || t == null || termComparator.compare(t, actualTerm) > 0: "curTerm=" + actualTerm + " seekTerm=" + t; if (t == null || tenum.seekCeil(t) == SeekStatus.END) { // no more terms to seek to or enum exhausted //System.out.println(" return null"); Index: lucene/core/src/java/org/apache/lucene/util/BytesRefIterator.java =================================================================== --- lucene/core/src/java/org/apache/lucene/util/BytesRefIterator.java (revision 1522598) +++ lucene/core/src/java/org/apache/lucene/util/BytesRefIterator.java (working copy) @@ -18,7 +18,6 @@ */ import java.io.IOException; -import java.util.Comparator; /** * A simple iterator interface for {@link BytesRef} iteration. @@ -38,14 +37,6 @@ */ public BytesRef next() throws IOException; - /** - * Return the {@link BytesRef} Comparator used to sort terms provided by the - * iterator. This may return null if there are no items or the iterator is not - * sorted. Callers may invoke this method many times, so it's best to cache a - * single instance & reuse it. - */ - public Comparator getComparator(); - /** Singleton BytesRefIterator that iterates over 0 BytesRefs. */ public static final BytesRefIterator EMPTY = new BytesRefIterator() { @@ -53,10 +44,5 @@ public BytesRef next() { return null; } - - @Override - public Comparator getComparator() { - return null; - } }; }