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/perfield/PerFieldPostingsFormat.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java (revision 1522947)
+++ 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/search/FieldCacheRewriteMethod.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/search/FieldCacheRewriteMethod.java (revision 1522947)
+++ lucene/core/src/java/org/apache/lucene/search/FieldCacheRewriteMethod.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.IndexReader;
@@ -26,7 +25,6 @@
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.OpenBitSet;
/**
@@ -91,11 +89,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/DocTermOrdsRewriteMethod.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/search/DocTermOrdsRewriteMethod.java (revision 1522947)
+++ lucene/core/src/java/org/apache/lucene/search/DocTermOrdsRewriteMethod.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.IndexReader;
@@ -26,7 +25,6 @@
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.OpenBitSet;
/**
@@ -91,11 +89,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/TermRangeTermsEnum.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/search/TermRangeTermsEnum.java (revision 1522947)
+++ lucene/core/src/java/org/apache/lucene/search/TermRangeTermsEnum.java (working copy)
@@ -17,18 +17,13 @@
* limitations under the License.
*/
-import java.io.IOException;
-import java.util.Comparator;
-
import org.apache.lucene.index.FilteredTermsEnum;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.util.BytesRef;
/**
* Subclass of FilteredTermEnum for enumerating all terms that match the
- * specified range parameters.
- * Term enumerations are always ordered by
- * {@link #getComparator}. Each term in the enumeration is
+ * specified range parameters. Each term in the enumeration is
* greater than all that precede it.
*/
public class TermRangeTermsEnum extends FilteredTermsEnum {
@@ -37,7 +32,6 @@
final private boolean includeUpper;
final private BytesRef lowerBytesRef;
final private BytesRef upperBytesRef;
- private final Comparator termComp;
/**
* Enumerates all terms greater/equal than lowerTerm
@@ -82,7 +76,6 @@
}
setInitialSeekTerm(lowerBytesRef);
- termComp = getComparator();
}
@Override
@@ -92,7 +85,7 @@
// Use this field's default sort ordering
if (upperBytesRef != null) {
- final int cmp = termComp.compare(upperBytesRef, term);
+ final int cmp = upperBytesRef.compareTo(term);
/*
* if beyond the upper term, or is exclusive and this is equal to
* the upper term, break out
Index: lucene/core/src/java/org/apache/lucene/search/ConstantScoreAutoRewrite.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/search/ConstantScoreAutoRewrite.java (revision 1522947)
+++ 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/PrefixTermsEnum.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/search/PrefixTermsEnum.java (revision 1522947)
+++ lucene/core/src/java/org/apache/lucene/search/PrefixTermsEnum.java (working copy)
@@ -26,7 +26,7 @@
* Subclass of FilteredTermEnum for enumerating all terms that match the
* specified prefix filter term.
* Term enumerations are always ordered by
- * {@link #getComparator}. Each term in the enumeration is
+ * {@link BytesRef#compareTo}. Each term in the enumeration is
* greater than all that precede it.
*/
public class PrefixTermsEnum extends FilteredTermsEnum {
Index: lucene/core/src/java/org/apache/lucene/search/package.html
===================================================================
--- lucene/core/src/java/org/apache/lucene/search/package.html (revision 1522947)
+++ lucene/core/src/java/org/apache/lucene/search/package.html (working copy)
@@ -173,7 +173,7 @@
{@link org.apache.lucene.index.Term Term}
and an upper
{@link org.apache.lucene.index.Term Term}
- according to {@link org.apache.lucene.index.TermsEnum#getComparator TermsEnum.getComparator()}. It is not intended
+ according to {@link org.apache.lucene.util.BytesRef#compareTo BytesRef.compareTo()}. It is not intended
for numerical ranges; use {@link org.apache.lucene.search.NumericRangeQuery NumericRangeQuery} instead.
For example, one could find all documents
Index: lucene/core/src/java/org/apache/lucene/search/ScoringRewrite.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/search/ScoringRewrite.java (revision 1522947)
+++ 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/TopTermsRewrite.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/search/TopTermsRewrite.java (revision 1522947)
+++ lucene/core/src/java/org/apache/lucene/search/TopTermsRewrite.java (working copy)
@@ -70,20 +70,18 @@
private final Map visitedTerms = new HashMap();
private TermsEnum termsEnum;
- private Comparator termComp;
private BoostAttribute boostAtt;
private ScoreTerm st;
@Override
public void setNextEnum(TermsEnum termsEnum) {
this.termsEnum = termsEnum;
- this.termComp = termsEnum.getComparator();
assert compareToLastTerm(null);
// lazy init the initial ScoreTerm because comparator is not known on ctor:
if (st == null)
- st = new ScoreTerm(this.termComp, new TermContext(topReaderContext));
+ st = new ScoreTerm(new TermContext(topReaderContext));
boostAtt = termsEnum.attributes().addAttribute(BoostAttribute.class);
}
@@ -95,7 +93,7 @@
} else if (t == null) {
lastTerm = null;
} else {
- assert termsEnum.getComparator().compare(lastTerm, t) < 0: "lastTerm=" + lastTerm + " t=" + t;
+ assert lastTerm.compareTo(t) < 0: "lastTerm=" + lastTerm + " t=" + t;
lastTerm.copyBytes(t);
}
return true;
@@ -115,7 +113,7 @@
final ScoreTerm t = stQueue.peek();
if (boost < t.boost)
return true;
- if (boost == t.boost && termComp.compare(bytes, t.bytes) > 0)
+ if (boost == t.boost && bytes.compareTo(t.bytes) > 0)
return true;
}
ScoreTerm t = visitedTerms.get(bytes);
@@ -139,7 +137,7 @@
visitedTerms.remove(st.bytes);
st.termState.clear(); // reset the termstate!
} else {
- st = new ScoreTerm(termComp, new TermContext(topReaderContext));
+ st = new ScoreTerm(new TermContext(topReaderContext));
}
assert stQueue.size() <= maxSize : "the PQ size must be limited to maxSize";
// set maxBoostAtt with values to help FuzzyTermsEnum to optimize
@@ -185,26 +183,22 @@
new Comparator() {
@Override
public int compare(ScoreTerm st1, ScoreTerm st2) {
- assert st1.termComp == st2.termComp :
- "term comparator should not change between segments";
- return st1.termComp.compare(st1.bytes, st2.bytes);
+ return st1.bytes.compareTo(st2.bytes);
}
};
static final class ScoreTerm implements Comparable {
- public final Comparator termComp;
public final BytesRef bytes = new BytesRef();
public float boost;
public final TermContext termState;
- public ScoreTerm(Comparator termComp, TermContext termState) {
- this.termComp = termComp;
+ public ScoreTerm(TermContext termState) {
this.termState = termState;
}
@Override
public int compareTo(ScoreTerm other) {
if (this.boost == other.boost)
- return termComp.compare(other.bytes, this.bytes);
+ return other.bytes.compareTo(this.bytes);
else
return Float.compare(this.boost, other.boost);
}
Index: lucene/core/src/java/org/apache/lucene/search/TermCollectingRewrite.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/search/TermCollectingRewrite.java (revision 1522947)
+++ 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 1522947)
+++ lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java (working copy)
@@ -46,7 +46,7 @@
* to the specified filter term.
*
* Term enumerations are always ordered by
- * {@link #getComparator}. Each term in the enumeration is
+ * {@link BytesRef#compareTo}. Each term in the enumeration is
* greater than all that precede it.
*/
public class FuzzyTermsEnum extends TermsEnum {
@@ -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/search/NumericRangeQuery.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/search/NumericRangeQuery.java (revision 1522947)
+++ lucene/core/src/java/org/apache/lucene/search/NumericRangeQuery.java (working copy)
@@ -18,7 +18,6 @@
*/
import java.io.IOException;
-import java.util.Comparator;
import java.util.LinkedList;
import org.apache.lucene.analysis.NumericTokenStream; // for javadocs
@@ -392,7 +391,6 @@
private BytesRef currentLowerBound, currentUpperBound;
private final LinkedList rangeBounds = new LinkedList();
- private final Comparator termComp;
NumericRangeTermsEnum(final TermsEnum tenum) {
super(tenum);
@@ -481,15 +479,13 @@
// should never happen
throw new IllegalArgumentException("Invalid NumericType");
}
-
- termComp = getComparator();
}
private void nextRange() {
assert rangeBounds.size() % 2 == 0;
currentLowerBound = rangeBounds.removeFirst();
- assert currentUpperBound == null || termComp.compare(currentUpperBound, currentLowerBound) <= 0 :
+ assert currentUpperBound == null || currentUpperBound.compareTo(currentLowerBound) <= 0 :
"The current upper bound must be <= the new lower bound";
currentUpperBound = rangeBounds.removeFirst();
@@ -501,10 +497,10 @@
nextRange();
// if the new upper bound is before the term parameter, the sub-range is never a hit
- if (term != null && termComp.compare(term, currentUpperBound) > 0)
+ if (term != null && term.compareTo(currentUpperBound) > 0)
continue;
// never seek backwards, so use current term if lower bound is smaller
- return (term != null && termComp.compare(term, currentLowerBound) > 0) ?
+ return (term != null && term.compareTo(currentLowerBound) > 0) ?
term : currentLowerBound;
}
@@ -516,11 +512,11 @@
@Override
protected final AcceptStatus accept(BytesRef term) {
- while (currentUpperBound == null || termComp.compare(term, currentUpperBound) > 0) {
+ while (currentUpperBound == null || term.compareTo(currentUpperBound) > 0) {
if (rangeBounds.isEmpty())
return AcceptStatus.END;
// peek next sub-range, only seek if the current term is smaller than next lower bound
- if (termComp.compare(term, rangeBounds.getFirst()) < 0)
+ if (term.compareTo(rangeBounds.getFirst()) < 0)
return AcceptStatus.NO_AND_SEEK;
// step forward to next range without seeking, as next lower range bound is less or equal current term
nextRange();
Index: lucene/core/src/java/org/apache/lucene/index/MergeState.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/index/MergeState.java (revision 1522947)
+++ lucene/core/src/java/org/apache/lucene/index/MergeState.java (working copy)
@@ -151,6 +151,10 @@
/** InfoStream for debugging messages. */
public final InfoStream infoStream;
+ /** Counter used for periodic calls to checkAbort
+ * @lucene.internal */
+ public int checkAbortCount;
+
// TODO: get rid of this? it tells you which segments are 'aligned' (e.g. for bulk merging)
// but is this really so expensive to compute again in different components, versus once in SM?
Index: lucene/core/src/java/org/apache/lucene/index/MappedMultiFields.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/index/MappedMultiFields.java (revision 0)
+++ lucene/core/src/java/org/apache/lucene/index/MappedMultiFields.java (working copy)
@@ -0,0 +1,136 @@
+package org.apache.lucene.index;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.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. */
+
+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/index/MappedMultiFields.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/core/src/java/org/apache/lucene/index/FilteredTermsEnum.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/index/FilteredTermsEnum.java (revision 1522947)
+++ lucene/core/src/java/org/apache/lucene/index/FilteredTermsEnum.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.AttributeSource;
@@ -28,7 +27,7 @@
* Abstract class for enumerating a subset of all terms.
*
* Term enumerations are always ordered by
- * {@link #getComparator}. Each term in the enumeration is
+ * {@link BytesRef#compareTo}. Each term in the enumeration is
* greater than all that precede it.
* Please note: Consumers of this enum cannot
* call {@code seek()}, it is forward only; it throws
@@ -135,11 +134,6 @@
}
@Override
- public Comparator getComparator() {
- return tenum.getComparator();
- }
-
- @Override
public int docFreq() throws IOException {
return tenum.docFreq();
}
@@ -221,7 +215,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 || t.compareTo(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/index/TermsHashPerField.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java (revision 1522947)
+++ lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java (working copy)
@@ -18,7 +18,6 @@
*/
import java.io.IOException;
-import java.util.Comparator;
import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
import org.apache.lucene.util.ByteBlockPool;
@@ -77,14 +76,8 @@
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
+ void reset() {
bytesHash.clear(false);
- }
-
- public void reset() {
- bytesHash.clear(false);
if (nextPerField != null)
nextPerField.reset();
}
@@ -107,8 +100,8 @@
}
/** Collapse the hash table & sort in-place. */
- public int[] sortPostings(Comparator termComp) {
- return bytesHash.sort(termComp);
+ public int[] sortPostings() {
+ return bytesHash.sort(BytesRef.getUTF8SortedAsUnicodeComparator());
}
private boolean doCall;
@@ -136,7 +129,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 +167,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/DocTermOrds.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/index/DocTermOrds.java (revision 1522947)
+++ 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/TermVectorsConsumerPerField.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java (revision 1522947)
+++ lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java (working copy)
@@ -156,7 +156,7 @@
TermVectorsPostingsArray postings = (TermVectorsPostingsArray) termsHashPerField.postingsArray;
final TermVectorsWriter tv = termsWriter.writer;
- final int[] termIDs = termsHashPerField.sortPostings(tv.getComparator());
+ final int[] termIDs = termsHashPerField.sortPostings();
tv.startField(fieldInfo, numPostings, doVectorPositions, doVectorOffsets, hasPayloads);
@@ -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/SortedDocValuesTermsEnum.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/index/SortedDocValuesTermsEnum.java (revision 1522947)
+++ 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/MultiTerms.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/index/MultiTerms.java (revision 1522947)
+++ lucene/core/src/java/org/apache/lucene/index/MultiTerms.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.util.BytesRef;
@@ -36,7 +35,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 +49,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 +144,6 @@
}
@Override
- public Comparator getComparator() {
- return termComp;
- }
-
- @Override
public boolean hasOffsets() {
return hasOffsets;
}
Index: lucene/core/src/java/org/apache/lucene/index/MappingMultiDocsEnum.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/index/MappingMultiDocsEnum.java (working copy)
+++ lucene/core/src/java/org/apache/lucene/index/MappingMultiDocsEnum.java (working copy)
@@ -1,4 +1,4 @@
-package org.apache.lucene.codecs;
+package org.apache.lucene.index;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@@ -17,9 +17,6 @@
* limitations under the License.
*/
-import org.apache.lucene.index.DocsEnum;
-import org.apache.lucene.index.MergeState;
-import org.apache.lucene.index.MultiDocsEnum;
import org.apache.lucene.index.MultiDocsEnum.EnumWithSlice;
import java.io.IOException;
@@ -31,7 +28,7 @@
* @lucene.experimental
*/
-public final class MappingMultiDocsEnum extends DocsEnum {
+final class MappingMultiDocsEnum extends DocsEnum {
private MultiDocsEnum.EnumWithSlice[] subs;
int numSubs;
int upto;
@@ -39,26 +36,23 @@
DocsEnum current;
int currentBase;
int doc = -1;
- private MergeState mergeState;
+ private final MergeState mergeState;
+ MultiDocsEnum multiDocsEnum;
/** Sole constructor. */
- public MappingMultiDocsEnum() {
+ 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() {
@@ -103,6 +97,13 @@
int doc = current.nextDoc();
if (doc != NO_MORE_DOCS) {
+
+ mergeState.checkAbortCount++;
+ if (mergeState.checkAbortCount > 60000) {
+ mergeState.checkAbort.work(mergeState.checkAbortCount/5.0);
+ mergeState.checkAbortCount = 0;
+ }
+
// compact deletions
doc = currentMap.get(doc);
if (doc == -1) {
Index: lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java (revision 1522947)
+++ lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java (working copy)
@@ -22,9 +22,8 @@
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.StoredFieldsWriter;
import org.apache.lucene.codecs.TermVectorsWriter;
import org.apache.lucene.index.FieldInfo.DocValuesType;
@@ -375,19 +374,10 @@
docBase += maxDoc;
}
- final FieldsConsumer consumer = codec.postingsFormat().fieldsConsumer(segmentWriteState);
- boolean success = false;
- try {
- consumer.merge(mergeState,
- new MultiFields(fields.toArray(Fields.EMPTY_ARRAY),
- slices.toArray(ReaderSlice.EMPTY_ARRAY)));
- success = true;
- } finally {
- if (success) {
- IOUtils.close(consumer);
- } else {
- IOUtils.closeWhileHandlingException(consumer);
- }
- }
+ Fields mergedFields = new MappedMultiFields(mergeState,
+ new MultiFields(fields.toArray(Fields.EMPTY_ARRAY),
+ slices.toArray(ReaderSlice.EMPTY_ARRAY)));
+
+ codec.postingsFormat().fieldsConsumer(segmentWriteState).write(mergedFields);
}
}
Index: lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java (revision 1522947)
+++ lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java (working copy)
@@ -19,19 +19,62 @@
import java.io.IOException;
import java.util.ArrayList;
+import java.util.Collections;
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*.
@@ -47,63 +90,20 @@
for (TermsHashConsumerPerField f : fieldsToFlush.values()) {
final FreqProxTermsWriterPerField perField = (FreqProxTermsWriterPerField) f;
if (perField.termsHashPerField.bytesHash.size() > 0) {
+ perField.sortPostings();
+ 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/FreqProxTermsWriterPerField.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java (revision 1522947)
+++ lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java (working copy)
@@ -17,19 +17,10 @@
* 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 +33,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 +64,12 @@
@Override
void finish() {
+ sumDocFreq += fieldState.uniqueTermCount;
+ sumTotalTermFreq += fieldState.length;
+ if (fieldState.length > 0) {
+ docCount++;
+ }
+
if (hasPayloads) {
fieldInfo.setStorePayloads();
}
@@ -83,14 +85,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 +312,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() {
+ assert sortedTermIDs == null;
+ sortedTermIDs = termsHashPerField.sortPostings();
}
}
Index: lucene/core/src/java/org/apache/lucene/index/TermsEnum.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/index/TermsEnum.java (revision 1522947)
+++ 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;
@@ -33,8 +32,9 @@
* #docs}.
*
* Term enumerations are always ordered by
- * {@link #getComparator}. Each term in the enumeration is
- * greater than the one before it.
+ * BytesRef.compareTo, which is Unicode sort
+ * order if the terms are UTF-8 bytes. 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 +230,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/FilterAtomicReader.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java (revision 1522947)
+++ lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java (working copy)
@@ -18,7 +18,6 @@
*/
import java.io.IOException;
-import java.util.Comparator;
import java.util.Iterator;
import org.apache.lucene.search.CachingWrapperFilter;
@@ -98,11 +97,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 +194,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 1522947)
+++ lucene/core/src/java/org/apache/lucene/index/AutomatonTermsEnum.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.IntsRef;
@@ -65,7 +64,6 @@
// of terms where we should simply do sequential reads instead.
private boolean linear = false;
private final BytesRef linearUpperBound = new BytesRef(10);
- private final Comparator termComp;
/**
* Construct an enumerator based upon an automaton, enumerating the specified
@@ -85,8 +83,6 @@
// used for path tracking, where each bit is a numbered state.
visited = new long[runAutomaton.getSize()];
-
- termComp = getComparator();
}
/**
@@ -99,10 +95,10 @@
if (runAutomaton.run(term.bytes, term.offset, term.length))
return linear ? AcceptStatus.YES : AcceptStatus.YES_AND_SEEK;
else
- return (linear && termComp.compare(term, linearUpperBound) < 0) ?
+ return (linear && term.compareTo(linearUpperBound) < 0) ?
AcceptStatus.NO : AcceptStatus.NO_AND_SEEK;
} else {
- return (linear && termComp.compare(term, linearUpperBound) < 0) ?
+ return (linear && term.compareTo(linearUpperBound) < 0) ?
AcceptStatus.NO : AcceptStatus.NO_AND_SEEK;
}
}
Index: lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumer.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumer.java (revision 1522947)
+++ 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/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,523 @@
+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.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.AttributeSource; // javadocs
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
+
+/** Implements limited (iterators only, no stats) {@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/SortedSetDocValuesTermsEnum.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesTermsEnum.java (revision 1522947)
+++ 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/Terms.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/index/Terms.java (revision 1522947)
+++ lucene/core/src/java/org/apache/lucene/index/Terms.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.automaton.CompiledAutomaton;
@@ -75,13 +74,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
@@ -109,6 +101,8 @@
* measures, this measure does not take deleted documents
* into account. */
public abstract int getDocCount() throws IOException;
+
+ // TODO: shouldn't we have hasFreq() as well?
/** Returns true if documents in this field store offsets. */
public abstract boolean hasOffsets();
Index: lucene/core/src/java/org/apache/lucene/index/MappingMultiDocsAndPositionsEnum.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/index/MappingMultiDocsAndPositionsEnum.java (working copy)
+++ lucene/core/src/java/org/apache/lucene/index/MappingMultiDocsAndPositionsEnum.java (working copy)
@@ -1,4 +1,4 @@
-package org.apache.lucene.codecs;
+package org.apache.lucene.index;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@@ -18,9 +18,6 @@
*/
import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.index.DocsAndPositionsEnum;
-import org.apache.lucene.index.MergeState;
-import org.apache.lucene.index.MultiDocsAndPositionsEnum;
import org.apache.lucene.index.MultiDocsAndPositionsEnum.EnumWithSlice;
import java.io.IOException;
@@ -32,7 +29,7 @@
* @lucene.experimental
*/
-public final class MappingMultiDocsAndPositionsEnum extends DocsAndPositionsEnum {
+final class MappingMultiDocsAndPositionsEnum extends DocsAndPositionsEnum {
private MultiDocsAndPositionsEnum.EnumWithSlice[] subs;
int numSubs;
int upto;
@@ -41,9 +38,11 @@
int currentBase;
int doc = -1;
private MergeState mergeState;
+ MultiDocsAndPositionsEnum multiDocsAndPositionsEnum;
/** Sole constructor. */
- public MappingMultiDocsAndPositionsEnum() {
+ public MappingMultiDocsAndPositionsEnum(MergeState mergeState) {
+ this.mergeState = mergeState;
}
MappingMultiDocsAndPositionsEnum reset(MultiDocsAndPositionsEnum postingsEnum) {
@@ -51,15 +50,10 @@
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() {
@@ -103,6 +97,13 @@
int doc = current.nextDoc();
if (doc != NO_MORE_DOCS) {
+
+ mergeState.checkAbortCount++;
+ if (mergeState.checkAbortCount > 60000) {
+ mergeState.checkAbort.work(mergeState.checkAbortCount/5.0);
+ mergeState.checkAbortCount = 0;
+ }
+
// compact deletions
doc = currentMap.get(doc);
if (doc == -1) {
Index: lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java (revision 1522947)
+++ lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java (working copy)
@@ -23,7 +23,6 @@
import java.io.IOException;
import java.util.Arrays;
-import java.util.Comparator;
/**
* Exposes {@link TermsEnum} API, merged from {@link TermsEnum} API of sub-segments.
@@ -47,7 +46,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 +93,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];
@@ -149,7 +129,7 @@
numTop = 0;
boolean seekOpt = false;
- if (lastSeek != null && termComp.compare(lastSeek, term) <= 0) {
+ if (lastSeek != null && lastSeek.compareTo(term) <= 0) {
seekOpt = true;
}
@@ -167,7 +147,7 @@
if (seekOpt) {
final BytesRef curTerm = currentSubs[i].current;
if (curTerm != null) {
- final int cmp = termComp.compare(term, curTerm);
+ final int cmp = term.compareTo(curTerm);
if (cmp == 0) {
status = true;
} else if (cmp < 0) {
@@ -201,7 +181,7 @@
lastSeekExact = false;
boolean seekOpt = false;
- if (lastSeek != null && termComp.compare(lastSeek, term) <= 0) {
+ if (lastSeek != null && lastSeek.compareTo(term) <= 0) {
seekOpt = true;
}
@@ -219,7 +199,7 @@
if (seekOpt) {
final BytesRef curTerm = currentSubs[i].current;
if (curTerm != null) {
- final int cmp = termComp.compare(term, curTerm);
+ final int cmp = term.compareTo(curTerm);
if (cmp == 0) {
status = SeekStatus.FOUND;
} else if (cmp < 0) {
@@ -519,14 +499,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/CheckIndex.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/index/CheckIndex.java (revision 1522947)
+++ 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/util/BytesRefIterator.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/util/BytesRefIterator.java (revision 1522947)
+++ 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;
- }
};
}
Index: lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java
===================================================================
--- lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java (revision 1522947)
+++ lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java (working copy)
@@ -19,7 +19,6 @@
import java.io.IOException;
import java.util.Collections;
-import java.util.Comparator;
import java.util.Iterator;
import java.util.Map;
import java.util.SortedMap;
@@ -272,11 +271,6 @@
}
@Override
- public Comparator getComparator() {
- return BytesRef.getUTF8SortedAsUnicodeComparator();
- }
-
- @Override
public long size() throws IOException {
return terms.size();
}
@@ -394,11 +388,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/simpletext/SimpleTextTermVectorsWriter.java
===================================================================
--- lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsWriter.java (revision 1522947)
+++ lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsWriter.java (working copy)
@@ -18,7 +18,6 @@
*/
import java.io.IOException;
-import java.util.Comparator;
import org.apache.lucene.codecs.TermVectorsWriter;
import org.apache.lucene.index.FieldInfo;
@@ -189,11 +188,6 @@
}
}
- @Override
- public Comparator getComparator() throws IOException {
- return BytesRef.getUTF8SortedAsUnicodeComparator();
- }
-
private void write(String s) throws IOException {
SimpleTextUtil.write(out, s, scratch);
}
Index: lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPostingsFormat.java
===================================================================
--- lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPostingsFormat.java (revision 1522947)
+++ 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.
Index: lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java
===================================================================
--- lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java (revision 1522947)
+++ lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java (working copy)
@@ -19,7 +19,6 @@
import java.io.IOException;
import java.util.Collections;
-import java.util.Comparator;
import java.util.HashMap;
import java.util.Iterator;
import java.util.Map;
@@ -218,11 +217,6 @@
}
return docsAndPositionsEnum.reset(docsStart, liveDocs, indexOptions, docFreq);
}
-
- @Override
- public Comparator getComparator() {
- return BytesRef.getUTF8SortedAsUnicodeComparator();
- }
}
private class SimpleTextDocsEnum extends DocsEnum {
@@ -590,11 +584,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 1522947)
+++ 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,173 @@
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);
+ if (terms == null) {
+ // Annoyingly, this can happen!
+ continue;
+ }
+ 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/blockterms/BlockTermsReader.java
===================================================================
--- lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsReader.java (revision 1522947)
+++ lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsReader.java (working copy)
@@ -19,7 +19,6 @@
import java.io.IOException;
import java.util.Collections;
-import java.util.Comparator;
import java.util.Iterator;
import java.util.TreeMap;
@@ -245,11 +244,6 @@
}
@Override
- public Comparator getComparator() {
- return BytesRef.getUTF8SortedAsUnicodeComparator();
- }
-
- @Override
public TermsEnum iterator(TermsEnum reuse) throws IOException {
return new SegmentTermsEnum();
}
@@ -349,11 +343,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 1522947)
+++ 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/FSTTermsReader.java
===================================================================
--- lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsReader.java (revision 1522947)
+++ lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsReader.java (working copy)
@@ -18,12 +18,9 @@
*/
import java.io.IOException;
-import java.io.PrintWriter;
-import java.io.File;
import java.util.ArrayList;
import java.util.BitSet;
import java.util.Collections;
-import java.util.Comparator;
import java.util.Iterator;
import java.util.TreeMap;
@@ -40,7 +37,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 +176,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 +243,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 1522947)
+++ lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsReader.java (working copy)
@@ -18,13 +18,10 @@
*/
import java.io.IOException;
-import java.io.PrintWriter;
-import java.io.File;
import java.util.Arrays;
import java.util.ArrayList;
import java.util.BitSet;
import java.util.Collections;
-import java.util.Comparator;
import java.util.Iterator;
import java.util.TreeMap;
@@ -41,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;
@@ -210,11 +206,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 +368,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/FSTTermsWriter.java
===================================================================
--- lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsWriter.java (revision 1522947)
+++ 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 1522947)
+++ lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java (working copy)
@@ -19,7 +19,6 @@
import java.io.IOException;
import java.util.Collections;
-import java.util.Comparator;
import java.util.Iterator;
import java.util.Map;
import java.util.SortedMap;
@@ -29,6 +28,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 +278,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 +288,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 +754,6 @@
}
@Override
- public Comparator getComparator() {
- return BytesRef.getUTF8SortedAsUnicodeComparator();
- }
-
- @Override
public void seekExact(long ord) {
// NOTE: we could add this...
throw new UnsupportedOperationException();
@@ -827,11 +817,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 1522947)
+++ 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 1522947)
+++ lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java (working copy)
@@ -19,7 +19,6 @@
import java.io.IOException;
import java.util.Collections;
-import java.util.Comparator;
import java.util.Iterator;
import java.util.Map;
import java.util.TreeMap;
@@ -661,11 +660,6 @@
}
@Override
- public Comparator getComparator() {
- return BytesRef.getUTF8SortedAsUnicodeComparator();
- }
-
- @Override
public boolean hasOffsets() {
return hasOffsets;
}
@@ -701,11 +695,6 @@
}
@Override
- public Comparator getComparator() {
- return BytesRef.getUTF8SortedAsUnicodeComparator();
- }
-
- @Override
public BytesRef next() {
termOrd++;
if (termOrd < terms.length) {
@@ -1096,11 +1085,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 1522947)
+++ 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/bloom/BloomFilteringPostingsFormat.java
===================================================================
--- lucene/codecs/src/java/org/apache/lucene/codecs/bloom/BloomFilteringPostingsFormat.java (revision 1522947)
+++ lucene/codecs/src/java/org/apache/lucene/codecs/bloom/BloomFilteringPostingsFormat.java (working copy)
@@ -19,18 +19,18 @@
import java.io.IOException;
import java.util.ArrayList;
-import java.util.Comparator;
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 +111,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,9 +143,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,
- delegatePostingsFormat);
+ (PushFieldsConsumer) fieldsConsumer, state);
}
@Override
@@ -252,11 +257,6 @@
}
@Override
- public Comparator getComparator() {
- return delegateTerms.getComparator();
- }
-
- @Override
public long size() throws IOException {
return delegateTerms.size();
}
@@ -327,11 +327,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 +383,6 @@
throws IOException {
return delegate().docs(liveDocs, reuse, flags);
}
-
-
}
@Override
@@ -401,17 +394,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,
- SegmentWriteState state, PostingsFormat delegatePostingsFormat) {
+ public BloomFilteredFieldsConsumer(PushFieldsConsumer fieldsConsumer,
+ SegmentWriteState state) {
+ super(state);
this.delegateFieldsConsumer = fieldsConsumer;
this.state = state;
}
@@ -422,7 +414,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 +502,5 @@
throws IOException {
delegateTermsConsumer.finish(sumTotalTermFreq, sumDocFreq, docCount);
}
-
- @Override
- public Comparator getComparator() throws IOException {
- return delegateTermsConsumer.getComparator();
- }
-
}
-
}
Index: lucene/spatial/src/java/org/apache/lucene/spatial/prefix/AbstractVisitingPrefixTreeFilter.java
===================================================================
--- lucene/spatial/src/java/org/apache/lucene/spatial/prefix/AbstractVisitingPrefixTreeFilter.java (revision 1522947)
+++ lucene/spatial/src/java/org/apache/lucene/spatial/prefix/AbstractVisitingPrefixTreeFilter.java (working copy)
@@ -17,7 +17,9 @@
* limitations under the License.
*/
-import com.spatial4j.core.shape.Shape;
+import java.io.IOException;
+import java.util.Iterator;
+
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.DocIdSet;
@@ -26,10 +28,8 @@
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.StringHelper;
+import com.spatial4j.core.shape.Shape;
-import java.io.IOException;
-import java.util.Iterator;
-
/**
* Traverses a {@link SpatialPrefixTree} indexed field, using the template &
* visitor design patterns for subclasses to guide the traversal and collect
@@ -176,7 +176,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 = thisTerm.compareTo(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/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
===================================================================
--- lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java (revision 1522947)
+++ 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/suggest/src/test/org/apache/lucene/search/suggest/TestHighFrequencyDictionary.java
===================================================================
--- lucene/suggest/src/test/org/apache/lucene/search/suggest/TestHighFrequencyDictionary.java (revision 1522947)
+++ 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/test/org/apache/lucene/search/suggest/TermFreqArrayIterator.java
===================================================================
--- lucene/suggest/src/test/org/apache/lucene/search/suggest/TermFreqArrayIterator.java (revision 1522947)
+++ 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 1522947)
+++ 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/java/org/apache/lucene/search/spell/HighFrequencyDictionary.java
===================================================================
--- lucene/suggest/src/java/org/apache/lucene/search/spell/HighFrequencyDictionary.java (revision 1522947)
+++ 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 1522947)
+++ 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 1522947)
+++ 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/FileDictionary.java
===================================================================
--- lucene/suggest/src/java/org/apache/lucene/search/suggest/FileDictionary.java (revision 1522947)
+++ 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 1522947)
+++ lucene/suggest/src/java/org/apache/lucene/search/suggest/jaspell/JaspellLookup.java (working copy)
@@ -28,7 +28,6 @@
import org.apache.lucene.search.spell.TermFreqIterator;
import org.apache.lucene.search.spell.TermFreqPayloadIterator;
import org.apache.lucene.search.suggest.Lookup;
-import org.apache.lucene.search.suggest.UnsortedTermFreqIteratorWrapper;
import org.apache.lucene.search.suggest.jaspell.JaspellTernarySearchTrie.TSTNode;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CharsRef;
@@ -57,11 +56,6 @@
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);
- }
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 1522947)
+++ 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 1522947)
+++ 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/suggest/src/java/org/apache/lucene/search/suggest/BytesRefArray.java
===================================================================
--- lucene/suggest/src/java/org/apache/lucene/search/suggest/BytesRefArray.java (revision 1522947)
+++ 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 1522947)
+++ 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 1522947)
+++ 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;