LUCENE-6766: merged

This commit is contained in:
Mike McCandless 2016-05-13 04:31:29 -04:00
commit d715210467
113 changed files with 4374 additions and 2778 deletions

View File

@ -108,7 +108,7 @@ public class Lucene50Codec extends Codec {
}
@Override
public final SegmentInfoFormat segmentInfoFormat() {
public SegmentInfoFormat segmentInfoFormat() {
return segmentInfosFormat;
}

View File

@ -0,0 +1,105 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene50;
import java.io.IOException;
import java.util.Collections;
import java.util.Map;
import java.util.Set;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexWriter; // javadocs
import org.apache.lucene.index.SegmentInfo; // javadocs
import org.apache.lucene.index.SegmentInfos; // javadocs
import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.DataOutput; // javadocs
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.Version;
/**
* Lucene 5.0 Segment info format.
* @deprecated Only for reading old 5.0-6.0 segments
*/
@Deprecated
public class Lucene50SegmentInfoFormat extends SegmentInfoFormat {
/** Sole constructor. */
public Lucene50SegmentInfoFormat() {
}
@Override
public SegmentInfo read(Directory dir, String segment, byte[] segmentID, IOContext context) throws IOException {
final String fileName = IndexFileNames.segmentFileName(segment, "", Lucene50SegmentInfoFormat.SI_EXTENSION);
try (ChecksumIndexInput input = dir.openChecksumInput(fileName, context)) {
Throwable priorE = null;
SegmentInfo si = null;
try {
int format = CodecUtil.checkIndexHeader(input, Lucene50SegmentInfoFormat.CODEC_NAME,
Lucene50SegmentInfoFormat.VERSION_START,
Lucene50SegmentInfoFormat.VERSION_CURRENT,
segmentID, "");
final Version version = Version.fromBits(input.readInt(), input.readInt(), input.readInt());
final int docCount = input.readInt();
if (docCount < 0) {
throw new CorruptIndexException("invalid docCount: " + docCount, input);
}
final boolean isCompoundFile = input.readByte() == SegmentInfo.YES;
final Map<String,String> diagnostics;
final Set<String> files;
final Map<String,String> attributes;
if (format >= VERSION_SAFE_MAPS) {
diagnostics = input.readMapOfStrings();
files = input.readSetOfStrings();
attributes = input.readMapOfStrings();
} else {
diagnostics = Collections.unmodifiableMap(input.readStringStringMap());
files = Collections.unmodifiableSet(input.readStringSet());
attributes = Collections.unmodifiableMap(input.readStringStringMap());
}
si = new SegmentInfo(dir, version, segment, docCount, isCompoundFile, null, diagnostics, segmentID, attributes, null);
si.setFiles(files);
} catch (Throwable exception) {
priorE = exception;
} finally {
CodecUtil.checkFooter(input, priorE);
}
return si;
}
}
@Override
public void write(Directory dir, SegmentInfo si, IOContext ioContext) throws IOException {
throw new UnsupportedOperationException("this codec can only be used for reading");
}
/** File extension used to store {@link SegmentInfo}. */
public final static String SI_EXTENSION = "si";
static final String CODEC_NAME = "Lucene50SegmentInfo";
static final int VERSION_START = 0;
static final int VERSION_SAFE_MAPS = 1;
static final int VERSION_CURRENT = VERSION_SAFE_MAPS;
}

View File

@ -16,7 +16,6 @@
*/
package org.apache.lucene.codecs.lucene60;
import java.util.Objects;
import org.apache.lucene.codecs.Codec;
@ -149,7 +148,7 @@ public class Lucene60Codec extends Codec {
/** Returns the docvalues format that should be used for writing
* new segments of <code>field</code>.
*
* The default implementation always returns "Lucene50".
* The default implementation always returns "Lucene54".
* <p>
* <b>WARNING:</b> if you subclass, you are responsible for index
* backwards compatibility: future version of Lucene are only

View File

@ -0,0 +1,25 @@
<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
<!--
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.
-->
<html>
<head>
<meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
</head>
<body>
Lucene 6.0 file format.
</body>
</html>

View File

@ -16,3 +16,4 @@
org.apache.lucene.codecs.lucene50.Lucene50Codec
org.apache.lucene.codecs.lucene53.Lucene53Codec
org.apache.lucene.codecs.lucene54.Lucene54Codec
org.apache.lucene.codecs.lucene60.Lucene60Codec

View File

@ -18,6 +18,7 @@ package org.apache.lucene.codecs.lucene50;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat;
/**
* Codec for testing 5.0 index format
@ -26,9 +27,15 @@ import org.apache.lucene.codecs.NormsFormat;
@Deprecated
final class Lucene50RWCodec extends Lucene50Codec {
private final NormsFormat normsFormat = new Lucene50RWNormsFormat();
private final SegmentInfoFormat segmentInfoFormat = new Lucene50RWSegmentInfoFormat();
@Override
public NormsFormat normsFormat() {
return normsFormat;
}
@Override
public SegmentInfoFormat segmentInfoFormat() {
return segmentInfoFormat;
}
}

View File

@ -37,43 +37,14 @@ import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.Version;
/**
* Lucene 5.0 Segment info format.
* <p>
* Files:
* <ul>
* <li><tt>.si</tt>: Header, SegVersion, SegSize, IsCompoundFile, Diagnostics, Files, Attributes, Footer
* </ul>
* Data types:
* <ul>
* <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}</li>
* <li>SegSize --&gt; {@link DataOutput#writeInt Int32}</li>
* <li>SegVersion --&gt; {@link DataOutput#writeString String}</li>
* <li>Files --&gt; {@link DataOutput#writeSetOfStrings Set&lt;String&gt;}</li>
* <li>Diagnostics,Attributes --&gt; {@link DataOutput#writeMapOfStrings Map&lt;String,String&gt;}</li>
* <li>IsCompoundFile --&gt; {@link DataOutput#writeByte Int8}</li>
* <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
* </ul>
* Field Descriptions:
* <ul>
* <li>SegVersion is the code version that created the segment.</li>
* <li>SegSize is the number of documents contained in the segment index.</li>
* <li>IsCompoundFile records whether the segment is written as a compound file or
* not. If this is -1, the segment is not a compound file. If it is 1, the segment
* is a compound file.</li>
* <li>The Diagnostics Map is privately written by {@link IndexWriter}, as a debugging aid,
* for each segment it creates. It includes metadata like the current Lucene
* version, OS, Java version, why the segment was created (merge, flush,
* addIndexes), etc.</li>
* <li>Files is a list of files referred to by this segment.</li>
* </ul>
*
* @see SegmentInfos
* @lucene.experimental
* Read-write version of 5.0 SegmentInfoFormat for testing
* @deprecated for test purposes only
*/
public class Lucene50SegmentInfoFormat extends SegmentInfoFormat {
@Deprecated
public class Lucene50RWSegmentInfoFormat extends Lucene50SegmentInfoFormat {
/** Sole constructor. */
public Lucene50SegmentInfoFormat() {
public Lucene50RWSegmentInfoFormat() {
}
@Override
@ -109,7 +80,7 @@ public class Lucene50SegmentInfoFormat extends SegmentInfoFormat {
attributes = Collections.unmodifiableMap(input.readStringStringMap());
}
si = new SegmentInfo(dir, version, segment, docCount, isCompoundFile, null, diagnostics, segmentID, attributes);
si = new SegmentInfo(dir, version, segment, docCount, isCompoundFile, null, diagnostics, segmentID, attributes, null);
si.setFiles(files);
} catch (Throwable exception) {
priorE = exception;
@ -124,6 +95,8 @@ public class Lucene50SegmentInfoFormat extends SegmentInfoFormat {
public void write(Directory dir, SegmentInfo si, IOContext ioContext) throws IOException {
final String fileName = IndexFileNames.segmentFileName(si.name, "", Lucene50SegmentInfoFormat.SI_EXTENSION);
assert si.getIndexSort() == null;
try (IndexOutput output = dir.createOutput(fileName, ioContext)) {
// Only add the file once we've successfully created it, else IFD assert can trip:
si.addFile(fileName);
@ -153,6 +126,7 @@ public class Lucene50SegmentInfoFormat extends SegmentInfoFormat {
}
output.writeSetOfStrings(files);
output.writeMapOfStrings(si.getAttributes());
CodecUtil.writeFooter(output);
}
}

View File

@ -29,7 +29,7 @@ import org.apache.lucene.benchmark.byTask.PerfRunData;
import org.apache.lucene.benchmark.byTask.utils.Config;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.lucene60.Lucene60Codec;
import org.apache.lucene.codecs.lucene62.Lucene62Codec;
import org.apache.lucene.index.ConcurrentMergeScheduler;
import org.apache.lucene.index.IndexCommit;
import org.apache.lucene.index.IndexDeletionPolicy;
@ -139,7 +139,7 @@ public class CreateIndexTask extends PerfTask {
if (defaultCodec == null && postingsFormat != null) {
try {
final PostingsFormat postingsFormatChosen = PostingsFormat.forName(postingsFormat);
iwConf.setCodec(new Lucene60Codec() {
iwConf.setCodec(new Lucene62Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return postingsFormatChosen;

View File

@ -28,7 +28,6 @@
<path refid="base.classpath"/>
<pathelement path="${queries.jar}"/>
<pathelement path="${grouping.jar}"/>
<pathelement path="${misc.jar}"/>
</path>
<path id="test.classpath">
@ -37,17 +36,16 @@
<path refid="test.base.classpath"/>
</path>
<target name="compile-core" depends="jar-misc,jar-grouping,jar-queries,jar-analyzers-common,common.compile-core" />
<target name="compile-core" depends="jar-grouping,jar-queries,jar-analyzers-common,common.compile-core" />
<target name="jar-core" depends="common.jar-core" />
<target name="javadocs" depends="javadocs-misc,javadocs-grouping,javadocs-misc,compile-core,check-javadocs-uptodate"
<target name="javadocs" depends="javadocs-grouping,compile-core,check-javadocs-uptodate"
unless="javadocs-uptodate-${name}">
<invoke-module-javadoc>
<links>
<link href="../queries"/>
<link href="../grouping"/>
<link href="../misc"/>
</links>
</invoke-module-javadoc>
</target>

View File

@ -29,6 +29,7 @@ import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.Terms;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.MatchAllDocsQuery;
@ -38,7 +39,6 @@ import org.apache.lucene.search.grouping.GroupDocs;
import org.apache.lucene.search.grouping.GroupingSearch;
import org.apache.lucene.search.grouping.TopGroups;
import org.apache.lucene.store.Directory;
import org.apache.lucene.uninverting.UninvertingReader;
/**
* Utility class for creating training / test / cross validation indexes from the original index.
@ -68,7 +68,7 @@ public class DatasetSplitter {
* @param crossValidationIndex a {@link Directory} used to write the cross validation index
* @param analyzer {@link Analyzer} used to create the new docs
* @param termVectors {@code true} if term vectors should be kept
* @param classFieldName names of the field used as the label for classification
* @param classFieldName name of the field used as the label for classification; this must be indexed with sorted doc values
* @param fieldNames names of fields that need to be put in the new indexes or <code>null</code> if all should be used
* @throws IOException if any writing operation fails on any of the indexes
*/
@ -80,30 +80,23 @@ public class DatasetSplitter {
IndexWriter cvWriter = new IndexWriter(crossValidationIndex, new IndexWriterConfig(analyzer));
IndexWriter trainingWriter = new IndexWriter(trainingIndex, new IndexWriterConfig(analyzer));
// try to get the exact no. of existing classes
Terms terms = originalIndex.terms(classFieldName);
long noOfClasses = -1;
if (terms != null) {
noOfClasses = terms.size();
}
if (noOfClasses == -1) {
noOfClasses = 10000; // fallback
// get the exact no. of existing classes
SortedDocValues classValues = originalIndex.getSortedDocValues(classFieldName);
if (classValues == null) {
throw new IllegalStateException("the classFieldName \"" + classFieldName + "\" must index sorted doc values");
}
HashMap<String, UninvertingReader.Type> mapping = new HashMap<>();
mapping.put(classFieldName, UninvertingReader.Type.SORTED);
UninvertingReader uninvertingReader = new UninvertingReader(originalIndex, mapping);
int noOfClasses = classValues.getValueCount();
try {
IndexSearcher indexSearcher = new IndexSearcher(uninvertingReader);
IndexSearcher indexSearcher = new IndexSearcher(originalIndex);
GroupingSearch gs = new GroupingSearch(classFieldName);
gs.setGroupSort(Sort.INDEXORDER);
gs.setSortWithinGroup(Sort.INDEXORDER);
gs.setAllGroups(true);
gs.setGroupDocsLimit(originalIndex.maxDoc());
TopGroups<Object> topGroups = gs.search(indexSearcher, new MatchAllDocsQuery(), 0, (int) noOfClasses);
TopGroups<Object> topGroups = gs.search(indexSearcher, new MatchAllDocsQuery(), 0, noOfClasses);
// set the type to be indexed, stored, with term vectors
FieldType ft = new FieldType(TextField.TYPE_STORED);
@ -156,7 +149,7 @@ public class DatasetSplitter {
testWriter.close();
cvWriter.close();
trainingWriter.close();
uninvertingReader.close();
originalIndex.close();
}
}

View File

@ -70,7 +70,9 @@ public class DataSplitterTest extends LuceneTestCase {
doc = new Document();
doc.add(new Field(idFieldName, "id" + Integer.toString(i), ft));
doc.add(new Field(textFieldName, TestUtil.randomUnicodeString(rnd, 1024), ft));
doc.add(new Field(classFieldName, Integer.toString(rnd.nextInt(10)), ft));
String className = Integer.toString(rnd.nextInt(10));
doc.add(new Field(classFieldName, className, ft));
doc.add(new SortedDocValuesField(classFieldName, new BytesRef(className)));
indexWriter.addDocument(doc);
}
@ -89,13 +91,11 @@ public class DataSplitterTest extends LuceneTestCase {
super.tearDown();
}
@Test
public void testSplitOnAllFields() throws Exception {
assertSplit(originalIndex, 0.1, 0.1);
}
@Test
public void testSplitOnSomeFields() throws Exception {
assertSplit(originalIndex, 0.2, 0.35, idFieldName, textFieldName);

View File

@ -36,6 +36,7 @@ class SimpleTextFieldsWriter extends FieldsConsumer {
private IndexOutput out;
private final BytesRefBuilder scratch = new BytesRefBuilder();
private final SegmentWriteState writeState;
final String segment;
final static BytesRef END = new BytesRef("END");
final static BytesRef FIELD = new BytesRef("field ");
@ -49,6 +50,7 @@ class SimpleTextFieldsWriter extends FieldsConsumer {
public SimpleTextFieldsWriter(SegmentWriteState writeState) throws IOException {
final String fileName = SimpleTextPostingsFormat.getPostingsFileName(writeState.segmentInfo.name, writeState.segmentSuffix);
segment = writeState.segmentInfo.name;
out = writeState.directory.createOutput(fileName, writeState.context);
this.writeState = writeState;
}

View File

@ -31,6 +31,8 @@ import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField;
import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
@ -59,6 +61,11 @@ public class SimpleTextSegmentInfoFormat extends SegmentInfoFormat {
final static BytesRef SI_NUM_FILES = new BytesRef(" files ");
final static BytesRef SI_FILE = new BytesRef(" file ");
final static BytesRef SI_ID = new BytesRef(" id ");
final static BytesRef SI_SORT = new BytesRef(" sort ");
final static BytesRef SI_SORT_FIELD = new BytesRef(" field ");
final static BytesRef SI_SORT_TYPE = new BytesRef(" type ");
final static BytesRef SI_SORT_REVERSE = new BytesRef(" reverse ");
final static BytesRef SI_SORT_MISSING = new BytesRef(" missing ");
public static final String SI_EXTENSION = "si";
@ -137,10 +144,119 @@ public class SimpleTextSegmentInfoFormat extends SegmentInfoFormat {
+ ", got: " + StringHelper.idToString(id), input);
}
SimpleTextUtil.readLine(input, scratch);
assert StringHelper.startsWith(scratch.get(), SI_SORT);
final int numSortFields = Integer.parseInt(readString(SI_SORT.length, scratch));
SortField[] sortField = new SortField[numSortFields];
for (int i = 0; i < numSortFields; ++i) {
SimpleTextUtil.readLine(input, scratch);
assert StringHelper.startsWith(scratch.get(), SI_SORT_FIELD);
final String field = readString(SI_SORT_FIELD.length, scratch);
SimpleTextUtil.readLine(input, scratch);
assert StringHelper.startsWith(scratch.get(), SI_SORT_TYPE);
final String typeAsString = readString(SI_SORT_TYPE.length, scratch);
final SortField.Type type;
switch (typeAsString) {
case "string":
type = SortField.Type.STRING;
break;
case "long":
type = SortField.Type.LONG;
break;
case "int":
type = SortField.Type.INT;
break;
case "double":
type = SortField.Type.DOUBLE;
break;
case "float":
type = SortField.Type.FLOAT;
break;
default:
throw new CorruptIndexException("unable to parse sort type string: " + typeAsString, input);
}
SimpleTextUtil.readLine(input, scratch);
assert StringHelper.startsWith(scratch.get(), SI_SORT_REVERSE);
final boolean reverse = Boolean.parseBoolean(readString(SI_SORT_REVERSE.length, scratch));
SimpleTextUtil.readLine(input, scratch);
assert StringHelper.startsWith(scratch.get(), SI_SORT_MISSING);
final String missingLastAsString = readString(SI_SORT_MISSING.length, scratch);
final Object missingValue;
switch (type) {
case STRING:
switch (missingLastAsString) {
case "null":
missingValue = null;
break;
case "first":
missingValue = SortField.STRING_FIRST;
break;
case "last":
missingValue = SortField.STRING_LAST;
break;
default:
throw new CorruptIndexException("unable to parse missing string: " + typeAsString, input);
}
break;
case LONG:
switch (missingLastAsString) {
case "null":
missingValue = null;
break;
default:
missingValue = Long.parseLong(missingLastAsString);
break;
}
break;
case INT:
switch (missingLastAsString) {
case "null":
missingValue = null;
break;
default:
missingValue = Integer.parseInt(missingLastAsString);
break;
}
break;
case DOUBLE:
switch (missingLastAsString) {
case "null":
missingValue = null;
break;
default:
missingValue = Double.parseDouble(missingLastAsString);
break;
}
break;
case FLOAT:
switch (missingLastAsString) {
case "null":
missingValue = null;
break;
default:
missingValue = Float.parseFloat(missingLastAsString);
break;
}
break;
default:
throw new AssertionError();
}
sortField[i] = new SortField(field, type, reverse);
if (missingValue != null) {
sortField[i].setMissingValue(missingValue);
}
}
Sort indexSort = sortField.length == 0 ? null : new Sort(sortField);
SimpleTextUtil.checkFooter(input);
SegmentInfo info = new SegmentInfo(directory, version, segmentName, docCount,
isCompoundFile, null, Collections.unmodifiableMap(diagnostics), id, Collections.unmodifiableMap(attributes));
isCompoundFile, null, Collections.unmodifiableMap(diagnostics),
id, Collections.unmodifiableMap(attributes), indexSort);
info.setFiles(files);
return info;
}
@ -223,6 +339,62 @@ public class SimpleTextSegmentInfoFormat extends SegmentInfoFormat {
SimpleTextUtil.write(output, new BytesRef(si.getId()));
SimpleTextUtil.writeNewline(output);
Sort indexSort = si.getIndexSort();
SimpleTextUtil.write(output, SI_SORT);
final int numSortFields = indexSort == null ? 0 : indexSort.getSort().length;
SimpleTextUtil.write(output, Integer.toString(numSortFields), scratch);
SimpleTextUtil.writeNewline(output);
for (int i = 0; i < numSortFields; ++i) {
final SortField sortField = indexSort.getSort()[i];
SimpleTextUtil.write(output, SI_SORT_FIELD);
SimpleTextUtil.write(output, sortField.getField(), scratch);
SimpleTextUtil.writeNewline(output);
SimpleTextUtil.write(output, SI_SORT_TYPE);
final String sortType;
switch (sortField.getType()) {
case STRING:
sortType = "string";
break;
case LONG:
sortType = "long";
break;
case INT:
sortType = "int";
break;
case DOUBLE:
sortType = "double";
break;
case FLOAT:
sortType = "float";
break;
default:
throw new IllegalStateException("Unexpected sort type: " + sortField.getType());
}
SimpleTextUtil.write(output, sortType, scratch);
SimpleTextUtil.writeNewline(output);
SimpleTextUtil.write(output, SI_SORT_REVERSE);
SimpleTextUtil.write(output, Boolean.toString(sortField.getReverse()), scratch);
SimpleTextUtil.writeNewline(output);
SimpleTextUtil.write(output, SI_SORT_MISSING);
final Object missingValue = sortField.getMissingValue();
final String missing;
if (missingValue == null) {
missing = "null";
} else if (missingValue == SortField.STRING_FIRST) {
missing = "first";
} else if (missingValue == SortField.STRING_LAST) {
missing = "last";
} else {
missing = missingValue.toString();
}
SimpleTextUtil.write(output, missing, scratch);
SimpleTextUtil.writeNewline(output);
}
SimpleTextUtil.writeChecksum(output, scratch);
}
}

View File

@ -143,7 +143,6 @@ public class SimpleTextStoredFieldsWriter extends StoredFieldsWriter {
} else {
write(TYPE_STRING);
newLine();
write(VALUE);
write(field.stringValue());
newLine();

View File

@ -57,7 +57,7 @@ public abstract class Codec implements NamedSPILoader.NamedSPI {
}
// TODO: should we use this, or maybe a system property is better?
static Codec defaultCodec = LOADER.lookup("Lucene60");
static Codec defaultCodec = LOADER.lookup("Lucene62");
}
private final String name;

View File

@ -16,7 +16,6 @@
*/
package org.apache.lucene.codecs;
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
@ -25,12 +24,13 @@ import java.util.List;
import java.util.NoSuchElementException;
import org.apache.lucene.index.BinaryDocValues;
import org.apache.lucene.index.DocIDMerger;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.DocValuesType;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FilteredTermsEnum;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.DocValuesType;
import org.apache.lucene.index.MultiDocValues.OrdinalMap;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentWriteState; // javadocs
import org.apache.lucene.index.SortedDocValues;
@ -44,6 +44,8 @@ import org.apache.lucene.util.LongBitSet;
import org.apache.lucene.util.LongValues;
import org.apache.lucene.util.packed.PackedInts;
import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
/**
* Abstract API that consumes numeric, binary and
* sorted docvalues. Concrete implementations of this
@ -240,6 +242,32 @@ public abstract class DocValuesConsumer implements Closeable {
}
}
}
/** Tracks state of one numeric sub-reader that we are merging */
private static class NumericDocValuesSub extends DocIDMerger.Sub {
private final NumericDocValues values;
private final Bits docsWithField;
private int docID = -1;
private final int maxDoc;
public NumericDocValuesSub(MergeState.DocMap docMap, NumericDocValues values, Bits docsWithField, int maxDoc) {
super(docMap);
this.values = values;
this.docsWithField = docsWithField;
this.maxDoc = maxDoc;
}
@Override
public int nextDoc() {
docID++;
if (docID == maxDoc) {
return NO_MORE_DOCS;
} else {
return docID;
}
}
}
/**
* Merges the numeric docvalues from <code>toMerge</code>.
@ -248,20 +276,23 @@ public abstract class DocValuesConsumer implements Closeable {
* an Iterable that merges and filters deleted documents on the fly.
*/
public void mergeNumericField(final FieldInfo fieldInfo, final MergeState mergeState, final List<NumericDocValues> toMerge, final List<Bits> docsWithField) throws IOException {
addNumericField(fieldInfo,
new Iterable<Number>() {
@Override
public Iterator<Number> iterator() {
// We must make a new DocIDMerger for each iterator:
List<NumericDocValuesSub> subs = new ArrayList<>();
assert mergeState.docMaps.length == toMerge.size();
for(int i=0;i<toMerge.size();i++) {
subs.add(new NumericDocValuesSub(mergeState.docMaps[i], toMerge.get(i), docsWithField.get(i), mergeState.maxDocs[i]));
}
final DocIDMerger<NumericDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
return new Iterator<Number>() {
int readerUpto = -1;
int docIDUpto;
long nextValue;
boolean nextHasValue;
int currentMaxDoc;
NumericDocValues currentValues;
Bits currentLiveDocs;
Bits currentDocsWithField;
boolean nextIsSet;
@Override
@ -276,7 +307,7 @@ public abstract class DocValuesConsumer implements Closeable {
@Override
public Number next() {
if (!hasNext()) {
if (hasNext() == false) {
throw new NoSuchElementException();
}
assert nextIsSet;
@ -285,43 +316,46 @@ public abstract class DocValuesConsumer implements Closeable {
}
private boolean setNext() {
while (true) {
if (readerUpto == toMerge.size()) {
return false;
}
if (docIDUpto == currentMaxDoc) {
readerUpto++;
if (readerUpto < toMerge.size()) {
currentValues = toMerge.get(readerUpto);
currentDocsWithField = docsWithField.get(readerUpto);
currentLiveDocs = mergeState.liveDocs[readerUpto];
currentMaxDoc = mergeState.maxDocs[readerUpto];
}
docIDUpto = 0;
continue;
}
if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
nextIsSet = true;
nextValue = currentValues.get(docIDUpto);
if (nextValue == 0 && currentDocsWithField.get(docIDUpto) == false) {
nextHasValue = false;
} else {
nextHasValue = true;
}
docIDUpto++;
return true;
}
docIDUpto++;
NumericDocValuesSub sub = docIDMerger.next();
if (sub == null) {
return false;
}
nextIsSet = true;
nextValue = sub.values.get(sub.docID);
nextHasValue = nextValue != 0 || sub.docsWithField.get(sub.docID);
return true;
}
};
}
});
}
/** Tracks state of one binary sub-reader that we are merging */
private static class BinaryDocValuesSub extends DocIDMerger.Sub {
private final BinaryDocValues values;
private final Bits docsWithField;
private int docID = -1;
private final int maxDoc;
public BinaryDocValuesSub(MergeState.DocMap docMap, BinaryDocValues values, Bits docsWithField, int maxDoc) {
super(docMap);
this.values = values;
this.docsWithField = docsWithField;
this.maxDoc = maxDoc;
}
@Override
public int nextDoc() {
docID++;
if (docID == maxDoc) {
return NO_MORE_DOCS;
} else {
return docID;
}
}
}
/**
* Merges the binary docvalues from <code>toMerge</code>.
* <p>
@ -329,20 +363,23 @@ public abstract class DocValuesConsumer implements Closeable {
* an Iterable that merges and filters deleted documents on the fly.
*/
public void mergeBinaryField(FieldInfo fieldInfo, final MergeState mergeState, final List<BinaryDocValues> toMerge, final List<Bits> docsWithField) throws IOException {
addBinaryField(fieldInfo,
new Iterable<BytesRef>() {
@Override
public Iterator<BytesRef> iterator() {
// We must make a new DocIDMerger for each iterator:
List<BinaryDocValuesSub> subs = new ArrayList<>();
assert mergeState.docMaps.length == toMerge.size();
for(int i=0;i<toMerge.size();i++) {
subs.add(new BinaryDocValuesSub(mergeState.docMaps[i], toMerge.get(i), docsWithField.get(i), mergeState.maxDocs[i]));
}
final DocIDMerger<BinaryDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
return new Iterator<BytesRef>() {
int readerUpto = -1;
int docIDUpto;
BytesRef nextValue;
BytesRef nextPointer; // points to null if missing, or nextValue
int currentMaxDoc;
BinaryDocValues currentValues;
Bits currentLiveDocs;
Bits currentDocsWithField;
boolean nextIsSet;
@Override
@ -357,7 +394,7 @@ public abstract class DocValuesConsumer implements Closeable {
@Override
public BytesRef next() {
if (!hasNext()) {
if (hasNext() == false) {
throw new NoSuchElementException();
}
assert nextIsSet;
@ -367,42 +404,49 @@ public abstract class DocValuesConsumer implements Closeable {
private boolean setNext() {
while (true) {
if (readerUpto == toMerge.size()) {
return false;
BinaryDocValuesSub sub = docIDMerger.next();
if (sub == null) {
return false;
}
nextIsSet = true;
if (sub.docsWithField.get(sub.docID)) {
nextPointer = nextValue = sub.values.get(sub.docID);
} else {
nextPointer = null;
}
return true;
}
if (docIDUpto == currentMaxDoc) {
readerUpto++;
if (readerUpto < toMerge.size()) {
currentValues = toMerge.get(readerUpto);
currentDocsWithField = docsWithField.get(readerUpto);
currentLiveDocs = mergeState.liveDocs[readerUpto];
currentMaxDoc = mergeState.maxDocs[readerUpto];
}
docIDUpto = 0;
continue;
}
if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
nextIsSet = true;
if (currentDocsWithField.get(docIDUpto)) {
nextValue = currentValues.get(docIDUpto);
nextPointer = nextValue;
} else {
nextPointer = null;
}
docIDUpto++;
return true;
}
docIDUpto++;
}
}
};
}
});
}
/** Tracks state of one sorted numeric sub-reader that we are merging */
private static class SortedNumericDocValuesSub extends DocIDMerger.Sub {
private final SortedNumericDocValues values;
private int docID = -1;
private final int maxDoc;
public SortedNumericDocValuesSub(MergeState.DocMap docMap, SortedNumericDocValues values, int maxDoc) {
super(docMap);
this.values = values;
this.maxDoc = maxDoc;
}
@Override
public int nextDoc() {
docID++;
if (docID == maxDoc) {
return NO_MORE_DOCS;
} else {
values.setDocument(docID);
return docID;
}
}
}
/**
* Merges the sorted docvalues from <code>toMerge</code>.
* <p>
@ -410,21 +454,24 @@ public abstract class DocValuesConsumer implements Closeable {
* iterables that filter deleted documents.
*/
public void mergeSortedNumericField(FieldInfo fieldInfo, final MergeState mergeState, List<SortedNumericDocValues> toMerge) throws IOException {
final int numReaders = toMerge.size();
final SortedNumericDocValues dvs[] = toMerge.toArray(new SortedNumericDocValues[numReaders]);
// step 3: add field
addSortedNumericField(fieldInfo,
// doc -> value count
new Iterable<Number>() {
@Override
public Iterator<Number> iterator() {
// We must make a new DocIDMerger for each iterator:
List<SortedNumericDocValuesSub> subs = new ArrayList<>();
assert mergeState.docMaps.length == toMerge.size();
for(int i=0;i<toMerge.size();i++) {
subs.add(new SortedNumericDocValuesSub(mergeState.docMaps[i], toMerge.get(i), mergeState.maxDocs[i]));
}
final DocIDMerger<SortedNumericDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
return new Iterator<Number>() {
int readerUpto = -1;
int docIDUpto;
int nextValue;
int currentMaxDoc;
Bits currentLiveDocs;
boolean nextIsSet;
@Override
@ -439,7 +486,7 @@ public abstract class DocValuesConsumer implements Closeable {
@Override
public Number next() {
if (!hasNext()) {
if (hasNext() == false) {
throw new NoSuchElementException();
}
assert nextIsSet;
@ -449,30 +496,13 @@ public abstract class DocValuesConsumer implements Closeable {
private boolean setNext() {
while (true) {
if (readerUpto == numReaders) {
SortedNumericDocValuesSub sub = docIDMerger.next();
if (sub == null) {
return false;
}
if (docIDUpto == currentMaxDoc) {
readerUpto++;
if (readerUpto < numReaders) {
currentLiveDocs = mergeState.liveDocs[readerUpto];
currentMaxDoc = mergeState.maxDocs[readerUpto];
}
docIDUpto = 0;
continue;
}
if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
nextIsSet = true;
SortedNumericDocValues dv = dvs[readerUpto];
dv.setDocument(docIDUpto);
nextValue = dv.count();
docIDUpto++;
return true;
}
docIDUpto++;
nextIsSet = true;
nextValue = sub.values.count();
return true;
}
}
};
@ -482,15 +512,21 @@ public abstract class DocValuesConsumer implements Closeable {
new Iterable<Number>() {
@Override
public Iterator<Number> iterator() {
// We must make a new DocIDMerger for each iterator:
List<SortedNumericDocValuesSub> subs = new ArrayList<>();
assert mergeState.docMaps.length == toMerge.size();
for(int i=0;i<toMerge.size();i++) {
subs.add(new SortedNumericDocValuesSub(mergeState.docMaps[i], toMerge.get(i), mergeState.maxDocs[i]));
}
final DocIDMerger<SortedNumericDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
return new Iterator<Number>() {
int readerUpto = -1;
int docIDUpto;
long nextValue;
int currentMaxDoc;
Bits currentLiveDocs;
boolean nextIsSet;
int valueUpto;
int valueLength;
SortedNumericDocValuesSub current;
@Override
public boolean hasNext() {
@ -504,7 +540,7 @@ public abstract class DocValuesConsumer implements Closeable {
@Override
public Number next() {
if (!hasNext()) {
if (hasNext() == false) {
throw new NoSuchElementException();
}
assert nextIsSet;
@ -514,38 +550,21 @@ public abstract class DocValuesConsumer implements Closeable {
private boolean setNext() {
while (true) {
if (readerUpto == numReaders) {
return false;
}
if (valueUpto < valueLength) {
nextValue = dvs[readerUpto].valueAt(valueUpto);
nextValue = current.values.valueAt(valueUpto);
valueUpto++;
nextIsSet = true;
return true;
}
if (docIDUpto == currentMaxDoc) {
readerUpto++;
if (readerUpto < numReaders) {
currentLiveDocs = mergeState.liveDocs[readerUpto];
currentMaxDoc = mergeState.maxDocs[readerUpto];
}
docIDUpto = 0;
continue;
current = docIDMerger.next();
if (current == null) {
return false;
}
if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
assert docIDUpto < currentMaxDoc;
SortedNumericDocValues dv = dvs[readerUpto];
dv.setDocument(docIDUpto);
valueUpto = 0;
valueLength = dv.count();
docIDUpto++;
continue;
}
docIDUpto++;
valueUpto = 0;
valueLength = current.values.count();
continue;
}
}
};
@ -554,6 +573,32 @@ public abstract class DocValuesConsumer implements Closeable {
);
}
/** Tracks state of one sorted sub-reader that we are merging */
private static class SortedDocValuesSub extends DocIDMerger.Sub {
private final SortedDocValues values;
private int docID = -1;
private final int maxDoc;
private final LongValues map;
public SortedDocValuesSub(MergeState.DocMap docMap, SortedDocValues values, int maxDoc, LongValues map) {
super(docMap);
this.values = values;
this.maxDoc = maxDoc;
this.map = map;
}
@Override
public int nextDoc() {
docID++;
if (docID == maxDoc) {
return NO_MORE_DOCS;
} else {
return docID;
}
}
}
/**
* Merges the sorted docvalues from <code>toMerge</code>.
* <p>
@ -608,7 +653,7 @@ public abstract class DocValuesConsumer implements Closeable {
@Override
public BytesRef next() {
if (!hasNext()) {
if (hasNext() == false) {
throw new NoSuchElementException();
}
int segmentNumber = map.getFirstSegmentNumber(currentOrd);
@ -629,13 +674,17 @@ public abstract class DocValuesConsumer implements Closeable {
new Iterable<Number>() {
@Override
public Iterator<Number> iterator() {
// We must make a new DocIDMerger for each iterator:
List<SortedDocValuesSub> subs = new ArrayList<>();
assert mergeState.docMaps.length == toMerge.size();
for(int i=0;i<toMerge.size();i++) {
subs.add(new SortedDocValuesSub(mergeState.docMaps[i], toMerge.get(i), mergeState.maxDocs[i], map.getGlobalOrds(i)));
}
final DocIDMerger<SortedDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
return new Iterator<Number>() {
int readerUpto = -1;
int docIDUpto;
int nextValue;
int currentMaxDoc;
Bits currentLiveDocs;
LongValues currentMap;
boolean nextIsSet;
@Override
@ -650,7 +699,7 @@ public abstract class DocValuesConsumer implements Closeable {
@Override
public Number next() {
if (!hasNext()) {
if (hasNext() == false) {
throw new NoSuchElementException();
}
assert nextIsSet;
@ -661,30 +710,15 @@ public abstract class DocValuesConsumer implements Closeable {
private boolean setNext() {
while (true) {
if (readerUpto == numReaders) {
SortedDocValuesSub sub = docIDMerger.next();
if (sub == null) {
return false;
}
if (docIDUpto == currentMaxDoc) {
readerUpto++;
if (readerUpto < numReaders) {
currentMap = map.getGlobalOrds(readerUpto);
currentLiveDocs = mergeState.liveDocs[readerUpto];
currentMaxDoc = mergeState.maxDocs[readerUpto];
}
docIDUpto = 0;
continue;
}
if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
nextIsSet = true;
int segOrd = dvs[readerUpto].getOrd(docIDUpto);
nextValue = segOrd == -1 ? -1 : (int) currentMap.get(segOrd);
docIDUpto++;
return true;
}
docIDUpto++;
nextIsSet = true;
int segOrd = sub.values.getOrd(sub.docID);
nextValue = segOrd == -1 ? -1 : (int) sub.map.get(segOrd);
return true;
}
}
};
@ -693,6 +727,37 @@ public abstract class DocValuesConsumer implements Closeable {
);
}
/** Tracks state of one sorted set sub-reader that we are merging */
private static class SortedSetDocValuesSub extends DocIDMerger.Sub {
private final SortedSetDocValues values;
int docID = -1;
private final int maxDoc;
private final LongValues map;
public SortedSetDocValuesSub(MergeState.DocMap docMap, SortedSetDocValues values, int maxDoc, LongValues map) {
super(docMap);
this.values = values;
this.maxDoc = maxDoc;
this.map = map;
}
@Override
public int nextDoc() {
docID++;
if (docID == maxDoc) {
return NO_MORE_DOCS;
} else {
return docID;
}
}
@Override
public String toString() {
return "SortedSetDocValuesSub(docID=" + docID + " mappedDocID=" + mappedDocID + " values=" + values + ")";
}
}
/**
* Merges the sortedset docvalues from <code>toMerge</code>.
* <p>
@ -700,14 +765,12 @@ public abstract class DocValuesConsumer implements Closeable {
* an Iterable that merges ordinals and values and filters deleted documents .
*/
public void mergeSortedSetField(FieldInfo fieldInfo, final MergeState mergeState, List<SortedSetDocValues> toMerge) throws IOException {
final SortedSetDocValues dvs[] = toMerge.toArray(new SortedSetDocValues[toMerge.size()]);
final int numReaders = mergeState.maxDocs.length;
// step 1: iterate thru each sub and mark terms still in use
TermsEnum liveTerms[] = new TermsEnum[dvs.length];
TermsEnum liveTerms[] = new TermsEnum[toMerge.size()];
long[] weights = new long[liveTerms.length];
for (int sub = 0; sub < liveTerms.length; sub++) {
SortedSetDocValues dv = dvs[sub];
SortedSetDocValues dv = toMerge.get(sub);
Bits liveDocs = mergeState.liveDocs[sub];
int maxDoc = mergeState.maxDocs[sub];
if (liveDocs == null) {
@ -748,12 +811,12 @@ public abstract class DocValuesConsumer implements Closeable {
@Override
public BytesRef next() {
if (!hasNext()) {
if (hasNext() == false) {
throw new NoSuchElementException();
}
int segmentNumber = map.getFirstSegmentNumber(currentOrd);
long segmentOrd = map.getFirstSegmentOrd(currentOrd);
final BytesRef term = dvs[segmentNumber].lookupOrd(segmentOrd);
final BytesRef term = toMerge.get(segmentNumber).lookupOrd(segmentOrd);
currentOrd++;
return term;
}
@ -769,12 +832,18 @@ public abstract class DocValuesConsumer implements Closeable {
new Iterable<Number>() {
@Override
public Iterator<Number> iterator() {
// We must make a new DocIDMerger for each iterator:
List<SortedSetDocValuesSub> subs = new ArrayList<>();
assert mergeState.docMaps.length == toMerge.size();
for(int i=0;i<toMerge.size();i++) {
subs.add(new SortedSetDocValuesSub(mergeState.docMaps[i], toMerge.get(i), mergeState.maxDocs[i], map.getGlobalOrds(i)));
}
final DocIDMerger<SortedSetDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
return new Iterator<Number>() {
int readerUpto = -1;
int docIDUpto;
int nextValue;
int currentMaxDoc;
Bits currentLiveDocs;
boolean nextIsSet;
@Override
@ -789,7 +858,7 @@ public abstract class DocValuesConsumer implements Closeable {
@Override
public Number next() {
if (!hasNext()) {
if (hasNext() == false) {
throw new NoSuchElementException();
}
assert nextIsSet;
@ -800,33 +869,18 @@ public abstract class DocValuesConsumer implements Closeable {
private boolean setNext() {
while (true) {
if (readerUpto == numReaders) {
SortedSetDocValuesSub sub = docIDMerger.next();
if (sub == null) {
return false;
}
if (docIDUpto == currentMaxDoc) {
readerUpto++;
if (readerUpto < numReaders) {
currentLiveDocs = mergeState.liveDocs[readerUpto];
currentMaxDoc = mergeState.maxDocs[readerUpto];
}
docIDUpto = 0;
continue;
sub.values.setDocument(sub.docID);
nextValue = 0;
while (sub.values.nextOrd() != SortedSetDocValues.NO_MORE_ORDS) {
nextValue++;
}
if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
nextIsSet = true;
SortedSetDocValues dv = dvs[readerUpto];
dv.setDocument(docIDUpto);
nextValue = 0;
while (dv.nextOrd() != SortedSetDocValues.NO_MORE_ORDS) {
nextValue++;
}
docIDUpto++;
return true;
}
docIDUpto++;
//System.out.println(" doc " + sub + " -> ord count = " + nextValue);
nextIsSet = true;
return true;
}
}
};
@ -836,13 +890,18 @@ public abstract class DocValuesConsumer implements Closeable {
new Iterable<Number>() {
@Override
public Iterator<Number> iterator() {
// We must make a new DocIDMerger for each iterator:
List<SortedSetDocValuesSub> subs = new ArrayList<>();
assert mergeState.docMaps.length == toMerge.size();
for(int i=0;i<toMerge.size();i++) {
subs.add(new SortedSetDocValuesSub(mergeState.docMaps[i], toMerge.get(i), mergeState.maxDocs[i], map.getGlobalOrds(i)));
}
final DocIDMerger<SortedSetDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
return new Iterator<Number>() {
int readerUpto = -1;
int docIDUpto;
long nextValue;
int currentMaxDoc;
Bits currentLiveDocs;
LongValues currentMap;
boolean nextIsSet;
long ords[] = new long[8];
int ordUpto;
@ -860,7 +919,7 @@ public abstract class DocValuesConsumer implements Closeable {
@Override
public Number next() {
if (!hasNext()) {
if (hasNext() == false) {
throw new NoSuchElementException();
}
assert nextIsSet;
@ -871,10 +930,6 @@ public abstract class DocValuesConsumer implements Closeable {
private boolean setNext() {
while (true) {
if (readerUpto == numReaders) {
return false;
}
if (ordUpto < ordLength) {
nextValue = ords[ordUpto];
ordUpto++;
@ -882,35 +937,22 @@ public abstract class DocValuesConsumer implements Closeable {
return true;
}
if (docIDUpto == currentMaxDoc) {
readerUpto++;
if (readerUpto < numReaders) {
currentMap = map.getGlobalOrds(readerUpto);
currentLiveDocs = mergeState.liveDocs[readerUpto];
currentMaxDoc = mergeState.maxDocs[readerUpto];
}
docIDUpto = 0;
continue;
}
if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
assert docIDUpto < currentMaxDoc;
SortedSetDocValues dv = dvs[readerUpto];
dv.setDocument(docIDUpto);
ordUpto = ordLength = 0;
long ord;
while ((ord = dv.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
if (ordLength == ords.length) {
ords = ArrayUtil.grow(ords, ordLength+1);
}
ords[ordLength] = currentMap.get(ord);
ordLength++;
}
docIDUpto++;
continue;
SortedSetDocValuesSub sub = docIDMerger.next();
if (sub == null) {
return false;
}
sub.values.setDocument(sub.docID);
docIDUpto++;
ordUpto = ordLength = 0;
long ord;
while ((ord = sub.values.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
if (ordLength == ords.length) {
ords = ArrayUtil.grow(ords, ordLength+1);
}
ords[ordLength] = sub.map.get(ord);
ordLength++;
}
continue;
}
}
};

View File

@ -16,7 +16,6 @@
*/
package org.apache.lucene.codecs;
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
@ -24,6 +23,7 @@ import java.util.Iterator;
import java.util.List;
import java.util.NoSuchElementException;
import org.apache.lucene.index.DocIDMerger;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.MergeState;
@ -31,6 +31,8 @@ import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.util.Bits;
import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
/**
* Abstract API that consumes normalization values.
* Concrete implementations of this
@ -98,6 +100,30 @@ public abstract class NormsConsumer implements Closeable {
}
}
/** Tracks state of one numeric sub-reader that we are merging */
private static class NumericDocValuesSub extends DocIDMerger.Sub {
private final NumericDocValues values;
private int docID = -1;
private final int maxDoc;
public NumericDocValuesSub(MergeState.DocMap docMap, NumericDocValues values, int maxDoc) {
super(docMap);
this.values = values;
this.maxDoc = maxDoc;
}
@Override
public int nextDoc() {
docID++;
if (docID == maxDoc) {
return NO_MORE_DOCS;
} else {
return docID;
}
}
}
/**
* Merges the norms from <code>toMerge</code>.
* <p>
@ -111,13 +137,18 @@ public abstract class NormsConsumer implements Closeable {
new Iterable<Number>() {
@Override
public Iterator<Number> iterator() {
// We must make a new DocIDMerger for each iterator:
List<NumericDocValuesSub> subs = new ArrayList<>();
assert mergeState.docMaps.length == toMerge.size();
for(int i=0;i<toMerge.size();i++) {
subs.add(new NumericDocValuesSub(mergeState.docMaps[i], toMerge.get(i), mergeState.maxDocs[i]));
}
final DocIDMerger<NumericDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
return new Iterator<Number>() {
int readerUpto = -1;
int docIDUpto;
long nextValue;
int maxDoc;
NumericDocValues currentValues;
Bits currentLiveDocs;
boolean nextIsSet;
@Override
@ -141,31 +172,13 @@ public abstract class NormsConsumer implements Closeable {
}
private boolean setNext() {
while (true) {
if (readerUpto == toMerge.size()) {
return false;
}
if (currentValues == null || docIDUpto == maxDoc) {
readerUpto++;
if (readerUpto < toMerge.size()) {
currentValues = toMerge.get(readerUpto);
currentLiveDocs = mergeState.liveDocs[readerUpto];
maxDoc = mergeState.maxDocs[readerUpto];
}
docIDUpto = 0;
continue;
}
if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
nextIsSet = true;
nextValue = currentValues.get(docIDUpto);
docIDUpto++;
return true;
}
docIDUpto++;
NumericDocValuesSub sub = docIDMerger.next();
if (sub == null) {
return false;
}
nextIsSet = true;
nextValue = sub.values.get(sub.docID);
return true;
}
};
}

View File

@ -76,7 +76,6 @@ public abstract class PointsWriter implements Closeable {
}
MergeState.DocMap docMap = mergeState.docMaps[i];
int docBase = mergeState.docBase[i];
pointsReader.intersect(fieldInfo.name,
new IntersectVisitor() {
@Override
@ -90,7 +89,7 @@ public abstract class PointsWriter implements Closeable {
int newDocID = docMap.get(docID);
if (newDocID != -1) {
// Not deleted:
mergedVisitor.visit(docBase + newDocID, packedValue);
mergedVisitor.visit(newDocID, packedValue);
}
}

View File

@ -20,10 +20,13 @@ import java.io.Closeable;
import java.io.IOException;
import java.io.Reader;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.List;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.document.StoredField;
import org.apache.lucene.index.DocIDMerger;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexableField;
@ -33,6 +36,8 @@ import org.apache.lucene.index.StoredFieldVisitor;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
/**
* Codec API for writing stored fields:
* <ol>
@ -73,6 +78,30 @@ public abstract class StoredFieldsWriter implements Closeable {
* check that this is the case to detect the JRE bug described
* in LUCENE-1282. */
public abstract void finish(FieldInfos fis, int numDocs) throws IOException;
private static class StoredFieldsMergeSub extends DocIDMerger.Sub {
private final StoredFieldsReader reader;
private final int maxDoc;
private final MergeVisitor visitor;
int docID = -1;
public StoredFieldsMergeSub(MergeVisitor visitor, MergeState.DocMap docMap, StoredFieldsReader reader, int maxDoc) {
super(docMap);
this.maxDoc = maxDoc;
this.reader = reader;
this.visitor = visitor;
}
@Override
public int nextDoc() {
docID++;
if (docID == maxDoc) {
return NO_MORE_DOCS;
} else {
return docID;
}
}
}
/** Merges in the stored fields from the readers in
* <code>mergeState</code>. The default implementation skips
@ -82,23 +111,26 @@ public abstract class StoredFieldsWriter implements Closeable {
* Implementations can override this method for more sophisticated
* merging (bulk-byte copying, etc). */
public int merge(MergeState mergeState) throws IOException {
int docCount = 0;
for (int i=0;i<mergeState.storedFieldsReaders.length;i++) {
List<StoredFieldsMergeSub> subs = new ArrayList<>();
for(int i=0;i<mergeState.storedFieldsReaders.length;i++) {
StoredFieldsReader storedFieldsReader = mergeState.storedFieldsReaders[i];
storedFieldsReader.checkIntegrity();
MergeVisitor visitor = new MergeVisitor(mergeState, i);
int maxDoc = mergeState.maxDocs[i];
Bits liveDocs = mergeState.liveDocs[i];
for (int docID=0;docID<maxDoc;docID++) {
if (liveDocs != null && !liveDocs.get(docID)) {
// skip deleted docs
continue;
}
startDocument();
storedFieldsReader.visitDocument(docID, visitor);
finishDocument();
docCount++;
subs.add(new StoredFieldsMergeSub(new MergeVisitor(mergeState, i), mergeState.docMaps[i], storedFieldsReader, mergeState.maxDocs[i]));
}
final DocIDMerger<StoredFieldsMergeSub> docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
int docCount = 0;
while (true) {
StoredFieldsMergeSub sub = docIDMerger.next();
if (sub == null) {
break;
}
assert sub.mappedDocID == docCount;
startDocument();
sub.reader.visitDocument(sub.docID, sub.visitor);
finishDocument();
docCount++;
}
finish(mergeState.mergeFieldInfos, docCount);
return docCount;

View File

@ -16,16 +16,18 @@
*/
package org.apache.lucene.codecs;
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.DocIDMerger;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.DocIdSetIterator;
@ -34,6 +36,8 @@ import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
/**
* Codec API for writing term vectors:
* <ol>
@ -160,6 +164,28 @@ public abstract class TermVectorsWriter implements Closeable {
}
}
private static class TermVectorsMergeSub extends DocIDMerger.Sub {
private final TermVectorsReader reader;
private final int maxDoc;
int docID = -1;
public TermVectorsMergeSub(MergeState.DocMap docMap, TermVectorsReader reader, int maxDoc) {
super(docMap);
this.maxDoc = maxDoc;
this.reader = reader;
}
@Override
public int nextDoc() {
docID++;
if (docID == maxDoc) {
return NO_MORE_DOCS;
} else {
return docID;
}
}
}
/** Merges in the term vectors from the readers in
* <code>mergeState</code>. The default implementation skips
* over deleted documents, and uses {@link #startDocument(int)},
@ -170,32 +196,35 @@ public abstract class TermVectorsWriter implements Closeable {
* Implementations can override this method for more sophisticated
* merging (bulk-byte copying, etc). */
public int merge(MergeState mergeState) throws IOException {
List<TermVectorsMergeSub> subs = new ArrayList<>();
for(int i=0;i<mergeState.termVectorsReaders.length;i++) {
TermVectorsReader reader = mergeState.termVectorsReaders[i];
if (reader != null) {
reader.checkIntegrity();
}
subs.add(new TermVectorsMergeSub(mergeState.docMaps[i], reader, mergeState.maxDocs[i]));
}
final DocIDMerger<TermVectorsMergeSub> docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
int docCount = 0;
int numReaders = mergeState.maxDocs.length;
for (int i = 0; i < numReaders; i++) {
int maxDoc = mergeState.maxDocs[i];
Bits liveDocs = mergeState.liveDocs[i];
TermVectorsReader termVectorsReader = mergeState.termVectorsReaders[i];
if (termVectorsReader != null) {
termVectorsReader.checkIntegrity();
while (true) {
TermVectorsMergeSub sub = docIDMerger.next();
if (sub == null) {
break;
}
for (int docID=0;docID<maxDoc;docID++) {
if (liveDocs != null && !liveDocs.get(docID)) {
// skip deleted docs
continue;
}
// NOTE: it's very important to first assign to vectors then pass it to
// termVectorsWriter.addAllDocVectors; see LUCENE-1282
Fields vectors;
if (termVectorsReader == null) {
vectors = null;
} else {
vectors = termVectorsReader.get(docID);
}
addAllDocVectors(vectors, mergeState);
docCount++;
// NOTE: it's very important to first assign to vectors then pass it to
// termVectorsWriter.addAllDocVectors; see LUCENE-1282
Fields vectors;
if (sub.reader == null) {
vectors = null;
} else {
vectors = sub.reader.get(sub.docID);
}
addAllDocVectors(vectors, mergeState);
docCount++;
}
finish(mergeState.mergeFieldInfos, docCount);
return docCount;

View File

@ -486,6 +486,12 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
@Override
public int merge(MergeState mergeState) throws IOException {
if (mergeState.segmentInfo.getIndexSort() != null) {
// TODO: can we gain back some optos even if index is sorted? E.g. if sort results in large chunks of contiguous docs from one sub
// being copied over...?
return super.merge(mergeState);
}
int docCount = 0;
int numReaders = mergeState.maxDocs.length;

View File

@ -730,6 +730,11 @@ public final class CompressingTermVectorsWriter extends TermVectorsWriter {
@Override
public int merge(MergeState mergeState) throws IOException {
if (mergeState.segmentInfo.getIndexSort() != null) {
// TODO: can we gain back some optos even if index is sorted? E.g. if sort results in large chunks of contiguous docs from one sub
// being copied over...?
return super.merge(mergeState);
}
int docCount = 0;
int numReaders = mergeState.maxDocs.length;

View File

@ -135,7 +135,7 @@
* <p>Each segment index maintains the following:</p>
* <ul>
* <li>
* {@link org.apache.lucene.codecs.lucene50.Lucene50SegmentInfoFormat Segment info}.
* {@link org.apache.lucene.codecs.lucene62.Lucene62SegmentInfoFormat Segment info}.
* This contains metadata about a segment, such as the number of documents,
* what files it uses,
* </li>
@ -235,7 +235,7 @@
* file.</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50SegmentInfoFormat Segment Info}</td>
* <td>{@link org.apache.lucene.codecs.lucene62.Lucene62SegmentInfoFormat Segment Info}</td>
* <td>.si</td>
* <td>Stores metadata about a segment</td>
* </tr>

View File

@ -123,6 +123,13 @@ public class Lucene60PointsWriter extends PointsWriter implements Closeable {
@Override
public void merge(MergeState mergeState) throws IOException {
if (mergeState.segmentInfo.getIndexSort() != null) {
// TODO: can we gain back some optos even if index is sorted? E.g. if sort results in large chunks of contiguous docs from one sub
// being copied over...?
super.merge(mergeState);
return;
}
for(PointsReader reader : mergeState.pointsReaders) {
if (reader instanceof Lucene60PointsReader == false) {
// We can only bulk merge when all to-be-merged segments use our format:
@ -171,7 +178,6 @@ public class Lucene60PointsWriter extends PointsWriter implements Closeable {
singleValuePerDoc)) {
List<BKDReader> bkdReaders = new ArrayList<>();
List<MergeState.DocMap> docMaps = new ArrayList<>();
List<Integer> docIDBases = new ArrayList<>();
for(int i=0;i<mergeState.pointsReaders.length;i++) {
PointsReader reader = mergeState.pointsReaders[i];
@ -191,7 +197,6 @@ public class Lucene60PointsWriter extends PointsWriter implements Closeable {
if (readerFieldInfo != null) {
BKDReader bkdReader = reader60.readers.get(readerFieldInfo.number);
if (bkdReader != null) {
docIDBases.add(mergeState.docBase[i]);
bkdReaders.add(bkdReader);
docMaps.add(mergeState.docMaps[i]);
}
@ -199,7 +204,7 @@ public class Lucene60PointsWriter extends PointsWriter implements Closeable {
}
}
long fp = writer.merge(dataOut, docMaps, bkdReaders, docIDBases);
long fp = writer.merge(dataOut, docMaps, bkdReaders);
if (fp != -1) {
indexFPs.put(fieldInfo.name, fp);
}

View File

@ -16,400 +16,7 @@
*/
/**
* Lucene 6.0 file format.
*
* <h1>Apache Lucene - Index File Formats</h1>
* <div>
* <ul>
* <li><a href="#Introduction">Introduction</a></li>
* <li><a href="#Definitions">Definitions</a>
* <ul>
* <li><a href="#Inverted_Indexing">Inverted Indexing</a></li>
* <li><a href="#Types_of_Fields">Types of Fields</a></li>
* <li><a href="#Segments">Segments</a></li>
* <li><a href="#Document_Numbers">Document Numbers</a></li>
* </ul>
* </li>
* <li><a href="#Overview">Index Structure Overview</a></li>
* <li><a href="#File_Naming">File Naming</a></li>
* <li><a href="#file-names">Summary of File Extensions</a>
* <ul>
* <li><a href="#Lock_File">Lock File</a></li>
* <li><a href="#History">History</a></li>
* <li><a href="#Limitations">Limitations</a></li>
* </ul>
* </li>
* </ul>
* </div>
* <a name="Introduction"></a>
* <h2>Introduction</h2>
* <div>
* <p>This document defines the index file formats used in this version of Lucene.
* If you are using a different version of Lucene, please consult the copy of
* <code>docs/</code> that was distributed with
* the version you are using.</p>
* <p>Apache Lucene is written in Java, but several efforts are underway to write
* <a href="http://wiki.apache.org/lucene-java/LuceneImplementations">versions of
* Lucene in other programming languages</a>. If these versions are to remain
* compatible with Apache Lucene, then a language-independent definition of the
* Lucene index format is required. This document thus attempts to provide a
* complete and independent definition of the Apache Lucene file formats.</p>
* <p>As Lucene evolves, this document should evolve. Versions of Lucene in
* different programming languages should endeavor to agree on file formats, and
* generate new versions of this document.</p>
* </div>
* <a name="Definitions"></a>
* <h2>Definitions</h2>
* <div>
* <p>The fundamental concepts in Lucene are index, document, field and term.</p>
* <p>An index contains a sequence of documents.</p>
* <ul>
* <li>A document is a sequence of fields.</li>
* <li>A field is a named sequence of terms.</li>
* <li>A term is a sequence of bytes.</li>
* </ul>
* <p>The same sequence of bytes in two different fields is considered a different
* term. Thus terms are represented as a pair: the string naming the field, and the
* bytes within the field.</p>
* <a name="Inverted_Indexing"></a>
* <h3>Inverted Indexing</h3>
* <p>The index stores statistics about terms in order to make term-based search
* more efficient. Lucene's index falls into the family of indexes known as an
* <i>inverted index.</i> This is because it can list, for a term, the documents
* that contain it. This is the inverse of the natural relationship, in which
* documents list terms.</p>
* <a name="Types_of_Fields"></a>
* <h3>Types of Fields</h3>
* <p>In Lucene, fields may be <i>stored</i>, in which case their text is stored
* in the index literally, in a non-inverted manner. Fields that are inverted are
* called <i>indexed</i>. A field may be both stored and indexed.</p>
* <p>The text of a field may be <i>tokenized</i> into terms to be indexed, or the
* text of a field may be used literally as a term to be indexed. Most fields are
* tokenized, but sometimes it is useful for certain identifier fields to be
* indexed literally.</p>
* <p>See the {@link org.apache.lucene.document.Field Field}
* java docs for more information on Fields.</p>
* <a name="Segments"></a>
* <h3>Segments</h3>
* <p>Lucene indexes may be composed of multiple sub-indexes, or <i>segments</i>.
* Each segment is a fully independent index, which could be searched separately.
* Indexes evolve by:</p>
* <ol>
* <li>Creating new segments for newly added documents.</li>
* <li>Merging existing segments.</li>
* </ol>
* <p>Searches may involve multiple segments and/or multiple indexes, each index
* potentially composed of a set of segments.</p>
* <a name="Document_Numbers"></a>
* <h3>Document Numbers</h3>
* <p>Internally, Lucene refers to documents by an integer <i>document number</i>.
* The first document added to an index is numbered zero, and each subsequent
* document added gets a number one greater than the previous.</p>
* <p>Note that a document's number may change, so caution should be taken when
* storing these numbers outside of Lucene. In particular, numbers may change in
* the following situations:</p>
* <ul>
* <li>
* <p>The numbers stored in each segment are unique only within the segment, and
* must be converted before they can be used in a larger context. The standard
* technique is to allocate each segment a range of values, based on the range of
* numbers used in that segment. To convert a document number from a segment to an
* external value, the segment's <i>base</i> document number is added. To convert
* an external value back to a segment-specific value, the segment is identified
* by the range that the external value is in, and the segment's base value is
* subtracted. For example two five document segments might be combined, so that
* the first segment has a base value of zero, and the second of five. Document
* three from the second segment would have an external value of eight.</p>
* </li>
* <li>
* <p>When documents are deleted, gaps are created in the numbering. These are
* eventually removed as the index evolves through merging. Deleted documents are
* dropped when segments are merged. A freshly-merged segment thus has no gaps in
* its numbering.</p>
* </li>
* </ul>
* </div>
* <a name="Overview"></a>
* <h2>Index Structure Overview</h2>
* <div>
* <p>Each segment index maintains the following:</p>
* <ul>
* <li>
* {@link org.apache.lucene.codecs.lucene50.Lucene50SegmentInfoFormat Segment info}.
* This contains metadata about a segment, such as the number of documents,
* what files it uses,
* </li>
* <li>
* {@link org.apache.lucene.codecs.lucene50.Lucene50FieldInfosFormat Field names}.
* This contains the set of field names used in the index.
* </li>
* <li>
* {@link org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat Stored Field values}.
* This contains, for each document, a list of attribute-value pairs, where the attributes
* are field names. These are used to store auxiliary information about the document, such as
* its title, url, or an identifier to access a database. The set of stored fields are what is
* returned for each hit when searching. This is keyed by document number.
* </li>
* <li>
* {@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term dictionary}.
* A dictionary containing all of the terms used in all of the
* indexed fields of all of the documents. The dictionary also contains the number
* of documents which contain the term, and pointers to the term's frequency and
* proximity data.
* </li>
* <li>
* {@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Frequency data}.
* For each term in the dictionary, the numbers of all the
* documents that contain that term, and the frequency of the term in that
* document, unless frequencies are omitted (IndexOptions.DOCS_ONLY)
* </li>
* <li>
* {@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Proximity data}.
* For each term in the dictionary, the positions that the
* term occurs in each document. Note that this will not exist if all fields in
* all documents omit position data.
* </li>
* <li>
* {@link org.apache.lucene.codecs.lucene53.Lucene53NormsFormat Normalization factors}.
* For each field in each document, a value is stored
* that is multiplied into the score for hits on that field.
* </li>
* <li>
* {@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vectors}.
* For each field in each document, the term vector (sometimes
* called document vector) may be stored. A term vector consists of term text and
* term frequency. To add Term Vectors to your index see the
* {@link org.apache.lucene.document.Field Field} constructors
* </li>
* <li>
* {@link org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat Per-document values}.
* Like stored values, these are also keyed by document
* number, but are generally intended to be loaded into main memory for fast
* access. Whereas stored values are generally intended for summary results from
* searches, per-document values are useful for things like scoring factors.
* </li>
* <li>
* {@link org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat Live documents}.
* An optional file indicating which documents are live.
* </li>
* <li>
* {@link org.apache.lucene.codecs.lucene60.Lucene60PointsFormat Point values}.
* Optional pair of files, recording dimensionally indexed fields, to enable fast
* numeric range filtering and large numeric values like BigInteger and BigDecimal (1D)
* and geographic shape intersection (2D, 3D).
* </li>
* </ul>
* <p>Details on each of these are provided in their linked pages.</p>
* </div>
* <a name="File_Naming"></a>
* <h2>File Naming</h2>
* <div>
* <p>All files belonging to a segment have the same name with varying extensions.
* The extensions correspond to the different file formats described below. When
* using the Compound File format (default in 1.4 and greater) these files (except
* for the Segment info file, the Lock file, and Deleted documents file) are collapsed
* into a single .cfs file (see below for details)</p>
* <p>Typically, all segments in an index are stored in a single directory,
* although this is not required.</p>
* <p>As of version 2.1 (lock-less commits), file names are never re-used.
* That is, when any file is saved
* to the Directory it is given a never before used filename. This is achieved
* using a simple generations approach. For example, the first segments file is
* segments_1, then segments_2, etc. The generation is a sequential long integer
* represented in alpha-numeric (base 36) form.</p>
* </div>
* <a name="file-names"></a>
* <h2>Summary of File Extensions</h2>
* <div>
* <p>The following table summarizes the names and extensions of the files in
* Lucene:</p>
* <table cellspacing="1" cellpadding="4" summary="lucene filenames by extension">
* <tr>
* <th>Name</th>
* <th>Extension</th>
* <th>Brief Description</th>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.index.SegmentInfos Segments File}</td>
* <td>segments_N</td>
* <td>Stores information about a commit point</td>
* </tr>
* <tr>
* <td><a href="#Lock_File">Lock File</a></td>
* <td>write.lock</td>
* <td>The Write lock prevents multiple IndexWriters from writing to the same
* file.</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50SegmentInfoFormat Segment Info}</td>
* <td>.si</td>
* <td>Stores metadata about a segment</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50CompoundFormat Compound File}</td>
* <td>.cfs, .cfe</td>
* <td>An optional "virtual" file consisting of all the other index files for
* systems that frequently run out of file handles.</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50FieldInfosFormat Fields}</td>
* <td>.fnm</td>
* <td>Stores information about the fields</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat Field Index}</td>
* <td>.fdx</td>
* <td>Contains pointers to field data</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat Field Data}</td>
* <td>.fdt</td>
* <td>The stored fields for documents</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Dictionary}</td>
* <td>.tim</td>
* <td>The term dictionary, stores term info</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Index}</td>
* <td>.tip</td>
* <td>The index into the Term Dictionary</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Frequencies}</td>
* <td>.doc</td>
* <td>Contains the list of docs which contain each term along with frequency</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Positions}</td>
* <td>.pos</td>
* <td>Stores position information about where a term occurs in the index</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Payloads}</td>
* <td>.pay</td>
* <td>Stores additional per-position metadata information such as character offsets and user payloads</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene53.Lucene53NormsFormat Norms}</td>
* <td>.nvd, .nvm</td>
* <td>Encodes length and boost factors for docs and fields</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat Per-Document Values}</td>
* <td>.dvd, .dvm</td>
* <td>Encodes additional scoring factors or other per-document information.</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vector Index}</td>
* <td>.tvx</td>
* <td>Stores offset into the document data file</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vector Documents}</td>
* <td>.tvd</td>
* <td>Contains information about each document that has term vectors</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vector Fields}</td>
* <td>.tvf</td>
* <td>The field level info about term vectors</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat Live Documents}</td>
* <td>.liv</td>
* <td>Info about what files are live</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene60.Lucene60PointsFormat Point values}</td>
* <td>.dii, .dim</td>
* <td>Holds indexed points, if any</td>
* </tr>
* </table>
* </div>
* <a name="Lock_File"></a>
* <h2>Lock File</h2>
* The write lock, which is stored in the index directory by default, is named
* "write.lock". If the lock directory is different from the index directory then
* the write lock will be named "XXXX-write.lock" where XXXX is a unique prefix
* derived from the full path to the index directory. When this file is present, a
* writer is currently modifying the index (adding or removing documents). This
* lock file ensures that only one writer is modifying the index at a time.
* <a name="History"></a>
* <h2>History</h2>
* <p>Compatibility notes are provided in this document, describing how file
* formats have changed from prior versions:</p>
* <ul>
* <li>In version 2.1, the file format was changed to allow lock-less commits (ie,
* no more commit lock). The change is fully backwards compatible: you can open a
* pre-2.1 index for searching or adding/deleting of docs. When the new segments
* file is saved (committed), it will be written in the new file format (meaning
* no specific "upgrade" process is needed). But note that once a commit has
* occurred, pre-2.1 Lucene will not be able to read the index.</li>
* <li>In version 2.3, the file format was changed to allow segments to share a
* single set of doc store (vectors &amp; stored fields) files. This allows for
* faster indexing in certain cases. The change is fully backwards compatible (in
* the same way as the lock-less commits change in 2.1).</li>
* <li>In version 2.4, Strings are now written as true UTF-8 byte sequence, not
* Java's modified UTF-8. See <a href="http://issues.apache.org/jira/browse/LUCENE-510">
* LUCENE-510</a> for details.</li>
* <li>In version 2.9, an optional opaque Map&lt;String,String&gt; CommitUserData
* may be passed to IndexWriter's commit methods (and later retrieved), which is
* recorded in the segments_N file. See <a href="http://issues.apache.org/jira/browse/LUCENE-1382">
* LUCENE-1382</a> for details. Also,
* diagnostics were added to each segment written recording details about why it
* was written (due to flush, merge; which OS/JRE was used; etc.). See issue
* <a href="http://issues.apache.org/jira/browse/LUCENE-1654">LUCENE-1654</a> for details.</li>
* <li>In version 3.0, compressed fields are no longer written to the index (they
* can still be read, but on merge the new segment will write them, uncompressed).
* See issue <a href="http://issues.apache.org/jira/browse/LUCENE-1960">LUCENE-1960</a>
* for details.</li>
* <li>In version 3.1, segments records the code version that created them. See
* <a href="http://issues.apache.org/jira/browse/LUCENE-2720">LUCENE-2720</a> for details.
* Additionally segments track explicitly whether or not they have term vectors.
* See <a href="http://issues.apache.org/jira/browse/LUCENE-2811">LUCENE-2811</a>
* for details.</li>
* <li>In version 3.2, numeric fields are written as natively to stored fields
* file, previously they were stored in text format only.</li>
* <li>In version 3.4, fields can omit position data while still indexing term
* frequencies.</li>
* <li>In version 4.0, the format of the inverted index became extensible via
* the {@link org.apache.lucene.codecs.Codec Codec} api. Fast per-document storage
* ({@code DocValues}) was introduced. Normalization factors need no longer be a
* single byte, they can be any {@link org.apache.lucene.index.NumericDocValues NumericDocValues}.
* Terms need not be unicode strings, they can be any byte sequence. Term offsets
* can optionally be indexed into the postings lists. Payloads can be stored in the
* term vectors.</li>
* <li>In version 4.1, the format of the postings list changed to use either
* of FOR compression or variable-byte encoding, depending upon the frequency
* of the term. Terms appearing only once were changed to inline directly into
* the term dictionary. Stored fields are compressed by default. </li>
* <li>In version 4.2, term vectors are compressed by default. DocValues has
* a new multi-valued type (SortedSet), that can be used for faceting/grouping/joining
* on multi-valued fields.</li>
* <li>In version 4.5, DocValues were extended to explicitly represent missing values.</li>
* <li>In version 4.6, FieldInfos were extended to support per-field DocValues generation, to
* allow updating NumericDocValues fields.</li>
* <li>In version 4.8, checksum footers were added to the end of each index file
* for improved data integrity. Specifically, the last 8 bytes of every index file
* contain the zlib-crc32 checksum of the file.</li>
* <li>In version 4.9, DocValues has a new multi-valued numeric type (SortedNumeric)
* that is suitable for faceting/sorting/analytics.
* <li>In version 5.4, DocValues have been improved to store more information on disk:
* addresses for binary fields and ord indexes for multi-valued fields.
* <li>In version 6.0, Points were added, for multi-dimensional range/distance search.
* </li>
* </ul>
* <a name="Limitations"></a>
* <h2>Limitations</h2>
* <div>
* <p>Lucene uses a Java <code>int</code> to refer to
* document numbers, and the index file format uses an <code>Int32</code>
* on-disk to store document numbers. This is a limitation
* of both the index file format and the current implementation. Eventually these
* should be replaced with either <code>UInt64</code> values, or
* better yet, {@link org.apache.lucene.store.DataOutput#writeVInt VInt} values which have no limit.</p>
* </div>
* Components from the Lucene 6.0 index format. See {@link org.apache.lucene.codecs.lucene62}
* for an overview of the index format.
*/
package org.apache.lucene.codecs.lucene60;

View File

@ -0,0 +1,176 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene62;
import java.util.Objects;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.CompoundFormat;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PointsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene50.Lucene50CompoundFormat;
import org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat;
import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat;
import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat;
import org.apache.lucene.codecs.lucene53.Lucene53NormsFormat;
import org.apache.lucene.codecs.lucene60.Lucene60FieldInfosFormat;
import org.apache.lucene.codecs.lucene60.Lucene60PointsFormat;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
/**
* Implements the Lucene 6.2 index format, with configurable per-field postings
* and docvalues formats.
* <p>
* If you want to reuse functionality of this codec in another codec, extend
* {@link FilterCodec}.
*
* @see org.apache.lucene.codecs.lucene60 package documentation for file format details.
*
* @lucene.experimental
*/
public class Lucene62Codec extends Codec {
private final TermVectorsFormat vectorsFormat = new Lucene50TermVectorsFormat();
private final FieldInfosFormat fieldInfosFormat = new Lucene60FieldInfosFormat();
private final SegmentInfoFormat segmentInfosFormat = new Lucene62SegmentInfoFormat();
private final LiveDocsFormat liveDocsFormat = new Lucene50LiveDocsFormat();
private final CompoundFormat compoundFormat = new Lucene50CompoundFormat();
private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return Lucene62Codec.this.getPostingsFormatForField(field);
}
};
private final DocValuesFormat docValuesFormat = new PerFieldDocValuesFormat() {
@Override
public DocValuesFormat getDocValuesFormatForField(String field) {
return Lucene62Codec.this.getDocValuesFormatForField(field);
}
};
private final StoredFieldsFormat storedFieldsFormat;
/**
* Instantiates a new codec.
*/
public Lucene62Codec() {
this(Mode.BEST_SPEED);
}
/**
* Instantiates a new codec, specifying the stored fields compression
* mode to use.
* @param mode stored fields compression mode to use for newly
* flushed/merged segments.
*/
public Lucene62Codec(Mode mode) {
super("Lucene62");
this.storedFieldsFormat = new Lucene50StoredFieldsFormat(Objects.requireNonNull(mode));
}
@Override
public final StoredFieldsFormat storedFieldsFormat() {
return storedFieldsFormat;
}
@Override
public final TermVectorsFormat termVectorsFormat() {
return vectorsFormat;
}
@Override
public final PostingsFormat postingsFormat() {
return postingsFormat;
}
@Override
public final FieldInfosFormat fieldInfosFormat() {
return fieldInfosFormat;
}
@Override
public final SegmentInfoFormat segmentInfoFormat() {
return segmentInfosFormat;
}
@Override
public final LiveDocsFormat liveDocsFormat() {
return liveDocsFormat;
}
@Override
public final CompoundFormat compoundFormat() {
return compoundFormat;
}
@Override
public final PointsFormat pointsFormat() {
return new Lucene60PointsFormat();
}
/** Returns the postings format that should be used for writing
* new segments of <code>field</code>.
*
* The default implementation always returns "Lucene50".
* <p>
* <b>WARNING:</b> if you subclass, you are responsible for index
* backwards compatibility: future version of Lucene are only
* guaranteed to be able to read the default implementation.
*/
public PostingsFormat getPostingsFormatForField(String field) {
return defaultFormat;
}
/** Returns the docvalues format that should be used for writing
* new segments of <code>field</code>.
*
* The default implementation always returns "Lucene54".
* <p>
* <b>WARNING:</b> if you subclass, you are responsible for index
* backwards compatibility: future version of Lucene are only
* guaranteed to be able to read the default implementation.
*/
public DocValuesFormat getDocValuesFormatForField(String field) {
return defaultDVFormat;
}
@Override
public final DocValuesFormat docValuesFormat() {
return docValuesFormat;
}
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene50");
private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene54");
private final NormsFormat normsFormat = new Lucene53NormsFormat();
@Override
public final NormsFormat normsFormat() {
return normsFormat;
}
}

View File

@ -0,0 +1,319 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene62;
import java.io.IOException;
import java.util.Collections;
import java.util.Map;
import java.util.Set;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexWriter; // javadocs
import org.apache.lucene.index.SegmentInfo; // javadocs
import org.apache.lucene.index.SegmentInfos; // javadocs
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField;
import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.DataOutput; // javadocs
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.Version;
/**
* Lucene 6.2 Segment info format.
* <p>
* Files:
* <ul>
* <li><tt>.si</tt>: Header, SegVersion, SegSize, IsCompoundFile, Diagnostics, Files, Attributes, IndexSort, Footer
* </ul>
* Data types:
* <ul>
* <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}</li>
* <li>SegSize --&gt; {@link DataOutput#writeInt Int32}</li>
* <li>SegVersion --&gt; {@link DataOutput#writeString String}</li>
* <li>Files --&gt; {@link DataOutput#writeSetOfStrings Set&lt;String&gt;}</li>
* <li>Diagnostics,Attributes --&gt; {@link DataOutput#writeMapOfStrings Map&lt;String,String&gt;}</li>
* <li>IsCompoundFile --&gt; {@link DataOutput#writeByte Int8}</li>
* <li>IndexSort --&gt; {@link DataOutput#writeVInt Int32} count, followed by {@code count} SortField</li>
* <li>SortField --&gt; {@link DataOutput#writeString String} field name, followed by {@link DataOutput#writeVInt Int32} sort type ID,
* followed by {@link DataOutput#writeByte Int8} indicatating reversed sort, followed by a type-specific encoding of the optional missing value
* <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
* </ul>
* Field Descriptions:
* <ul>
* <li>SegVersion is the code version that created the segment.</li>
* <li>SegSize is the number of documents contained in the segment index.</li>
* <li>IsCompoundFile records whether the segment is written as a compound file or
* not. If this is -1, the segment is not a compound file. If it is 1, the segment
* is a compound file.</li>
* <li>The Diagnostics Map is privately written by {@link IndexWriter}, as a debugging aid,
* for each segment it creates. It includes metadata like the current Lucene
* version, OS, Java version, why the segment was created (merge, flush,
* addIndexes), etc.</li>
* <li>Files is a list of files referred to by this segment.</li>
* </ul>
*
* @see SegmentInfos
* @lucene.experimental
*/
public class Lucene62SegmentInfoFormat extends SegmentInfoFormat {
/** Sole constructor. */
public Lucene62SegmentInfoFormat() {
}
@Override
public SegmentInfo read(Directory dir, String segment, byte[] segmentID, IOContext context) throws IOException {
final String fileName = IndexFileNames.segmentFileName(segment, "", Lucene62SegmentInfoFormat.SI_EXTENSION);
try (ChecksumIndexInput input = dir.openChecksumInput(fileName, context)) {
Throwable priorE = null;
SegmentInfo si = null;
try {
int format = CodecUtil.checkIndexHeader(input, Lucene62SegmentInfoFormat.CODEC_NAME,
Lucene62SegmentInfoFormat.VERSION_START,
Lucene62SegmentInfoFormat.VERSION_CURRENT,
segmentID, "");
final Version version = Version.fromBits(input.readInt(), input.readInt(), input.readInt());
final int docCount = input.readInt();
if (docCount < 0) {
throw new CorruptIndexException("invalid docCount: " + docCount, input);
}
final boolean isCompoundFile = input.readByte() == SegmentInfo.YES;
final Map<String,String> diagnostics = input.readMapOfStrings();
final Set<String> files = input.readSetOfStrings();
final Map<String,String> attributes = input.readMapOfStrings();
int numSortFields = input.readVInt();
Sort indexSort;
if (numSortFields > 0) {
SortField[] sortFields = new SortField[numSortFields];
for(int i=0;i<numSortFields;i++) {
String fieldName = input.readString();
int sortTypeID = input.readVInt();
SortField.Type sortType;
switch(sortTypeID) {
case 0:
sortType = SortField.Type.STRING;
break;
case 1:
sortType = SortField.Type.LONG;
break;
case 2:
sortType = SortField.Type.INT;
break;
case 3:
sortType = SortField.Type.DOUBLE;
break;
case 4:
sortType = SortField.Type.FLOAT;
break;
default:
throw new CorruptIndexException("invalid index sort field type ID: " + sortTypeID, input);
}
byte b = input.readByte();
boolean reverse;
if (b == 0) {
reverse = true;
} else if (b == 1) {
reverse = false;
} else {
throw new CorruptIndexException("invalid index sort reverse: " + b, input);
}
sortFields[i] = new SortField(fieldName, sortType, reverse);
Object missingValue;
b = input.readByte();
if (b == 0) {
missingValue = null;
} else {
switch(sortType) {
case STRING:
if (b == 1) {
missingValue = SortField.STRING_LAST;
} else if (b == 2) {
missingValue = SortField.STRING_FIRST;
} else {
throw new CorruptIndexException("invalid missing value flag: " + b, input);
}
break;
case LONG:
if (b != 1) {
throw new CorruptIndexException("invalid missing value flag: " + b, input);
}
missingValue = input.readLong();
break;
case INT:
if (b != 1) {
throw new CorruptIndexException("invalid missing value flag: " + b, input);
}
missingValue = input.readInt();
break;
case DOUBLE:
if (b != 1) {
throw new CorruptIndexException("invalid missing value flag: " + b, input);
}
missingValue = Double.longBitsToDouble(input.readLong());
break;
case FLOAT:
if (b != 1) {
throw new CorruptIndexException("invalid missing value flag: " + b, input);
}
missingValue = Float.intBitsToFloat(input.readInt());
break;
default:
throw new AssertionError("unhandled sortType=" + sortType);
}
}
if (missingValue != null) {
sortFields[i].setMissingValue(missingValue);
}
}
indexSort = new Sort(sortFields);
} else if (numSortFields < 0) {
throw new CorruptIndexException("invalid index sort field count: " + numSortFields, input);
} else {
indexSort = null;
}
si = new SegmentInfo(dir, version, segment, docCount, isCompoundFile, null, diagnostics, segmentID, attributes, indexSort);
si.setFiles(files);
} catch (Throwable exception) {
priorE = exception;
} finally {
CodecUtil.checkFooter(input, priorE);
}
return si;
}
}
@Override
public void write(Directory dir, SegmentInfo si, IOContext ioContext) throws IOException {
final String fileName = IndexFileNames.segmentFileName(si.name, "", Lucene62SegmentInfoFormat.SI_EXTENSION);
try (IndexOutput output = dir.createOutput(fileName, ioContext)) {
// Only add the file once we've successfully created it, else IFD assert can trip:
si.addFile(fileName);
CodecUtil.writeIndexHeader(output,
Lucene62SegmentInfoFormat.CODEC_NAME,
Lucene62SegmentInfoFormat.VERSION_CURRENT,
si.getId(),
"");
Version version = si.getVersion();
if (version.major < 5) {
throw new IllegalArgumentException("invalid major version: should be >= 5 but got: " + version.major + " segment=" + si);
}
// Write the Lucene version that created this segment, since 3.1
output.writeInt(version.major);
output.writeInt(version.minor);
output.writeInt(version.bugfix);
assert version.prerelease == 0;
output.writeInt(si.maxDoc());
output.writeByte((byte) (si.getUseCompoundFile() ? SegmentInfo.YES : SegmentInfo.NO));
output.writeMapOfStrings(si.getDiagnostics());
Set<String> files = si.files();
for (String file : files) {
if (!IndexFileNames.parseSegmentName(file).equals(si.name)) {
throw new IllegalArgumentException("invalid files: expected segment=" + si.name + ", got=" + files);
}
}
output.writeSetOfStrings(files);
output.writeMapOfStrings(si.getAttributes());
Sort indexSort = si.getIndexSort();
int numSortFields = indexSort == null ? 0 : indexSort.getSort().length;
output.writeVInt(numSortFields);
for (int i = 0; i < numSortFields; ++i) {
SortField sortField = indexSort.getSort()[i];
output.writeString(sortField.getField());
int sortTypeID;
switch (sortField.getType()) {
case STRING:
sortTypeID = 0;
break;
case LONG:
sortTypeID = 1;
break;
case INT:
sortTypeID = 2;
break;
case DOUBLE:
sortTypeID = 3;
break;
case FLOAT:
sortTypeID = 4;
break;
default:
throw new IllegalStateException("Unexpected sort type: " + sortField.getType());
}
output.writeVInt(sortTypeID);
output.writeByte((byte) (sortField.getReverse() ? 0 : 1));
// write missing value
Object missingValue = sortField.getMissingValue();
if (missingValue == null) {
output.writeByte((byte) 0);
} else {
switch(sortField.getType()) {
case STRING:
if (missingValue == SortField.STRING_LAST) {
output.writeByte((byte) 1);
} else if (missingValue == SortField.STRING_FIRST) {
output.writeByte((byte) 2);
} else {
throw new AssertionError("unrecognized missing value for STRING field \"" + sortField.getField() + "\": " + missingValue);
}
break;
case LONG:
output.writeByte((byte) 1);
output.writeLong(((Long) missingValue).longValue());
break;
case INT:
output.writeByte((byte) 1);
output.writeInt(((Integer) missingValue).intValue());
break;
case DOUBLE:
output.writeByte((byte) 1);
output.writeLong(Double.doubleToLongBits(((Double) missingValue).doubleValue()));
break;
case FLOAT:
output.writeByte((byte) 1);
output.writeInt(Float.floatToIntBits(((Float) missingValue).floatValue()));
break;
default:
throw new IllegalStateException("Unexpected sort type: " + sortField.getType());
}
}
}
CodecUtil.writeFooter(output);
}
}
/** File extension used to store {@link SegmentInfo}. */
public final static String SI_EXTENSION = "si";
static final String CODEC_NAME = "Lucene62SegmentInfo";
static final int VERSION_START = 0;
static final int VERSION_CURRENT = VERSION_START;
}

View File

@ -0,0 +1,24 @@
/*
* 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.
*/
/**
* Components from the Lucene 6.2 index format
* See {@link org.apache.lucene.codecs.lucene62} for an overview
* of the index format.
*/
package org.apache.lucene.codecs.lucene62;

View File

@ -43,6 +43,9 @@ import org.apache.lucene.document.Document;
import org.apache.lucene.document.DocumentStoredFieldVisitor;
import org.apache.lucene.index.CheckIndex.Status.DocValuesStatus;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.LeafFieldComparator;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FSDirectory;
@ -217,6 +220,9 @@ public final class CheckIndex implements Closeable {
/** Status for testing of PointValues (null if PointValues could not be tested). */
public PointsStatus pointsStatus;
/** Status of index sort */
public IndexSortStatus indexSortStatus;
}
/**
@ -374,6 +380,18 @@ public final class CheckIndex implements Closeable {
/** Exception thrown during doc values test (null on success) */
public Throwable error = null;
}
/**
* Status from testing index sort
*/
public static final class IndexSortStatus {
IndexSortStatus() {
}
/** Exception thrown during term index test (null on success) */
public Throwable error = null;
}
}
/** Create a new CheckIndex on the directory. */
@ -632,6 +650,7 @@ public final class CheckIndex implements Closeable {
int toLoseDocCount = info.info.maxDoc();
SegmentReader reader = null;
Sort previousIndexSort = null;
try {
msg(infoStream, " version=" + (version == null ? "3.0" : version));
@ -642,6 +661,17 @@ public final class CheckIndex implements Closeable {
msg(infoStream, " compound=" + info.info.getUseCompoundFile());
segInfoStat.compound = info.info.getUseCompoundFile();
msg(infoStream, " numFiles=" + info.files().size());
Sort indexSort = info.info.getIndexSort();
if (indexSort != null) {
msg(infoStream, " sort=" + indexSort);
if (previousIndexSort != null) {
if (previousIndexSort.equals(indexSort) == false) {
throw new RuntimeException("index sort changed from " + previousIndexSort + " to " + indexSort);
}
} else {
previousIndexSort = indexSort;
}
}
segInfoStat.numFiles = info.files().size();
segInfoStat.sizeMB = info.sizeInBytes()/(1024.*1024.);
msg(infoStream, " size (MB)=" + nf.format(segInfoStat.sizeMB));
@ -722,6 +752,9 @@ public final class CheckIndex implements Closeable {
// Test PointValues
segInfoStat.pointsStatus = testPoints(reader, infoStream, failFast);
// Test index sort
segInfoStat.indexSortStatus = testSort(reader, indexSort, infoStream, failFast);
// Rethrow the first exception we encountered
// This will cause stats for failed segments to be incremented properly
if (segInfoStat.liveDocStatus.error != null) {
@ -790,6 +823,72 @@ public final class CheckIndex implements Closeable {
return result;
}
/**
* Tests index sort order.
* @lucene.experimental
*/
public static Status.IndexSortStatus testSort(CodecReader reader, Sort sort, PrintStream infoStream, boolean failFast) throws IOException {
// This segment claims its documents are sorted according to the incoming sort ... let's make sure:
long startNS = System.nanoTime();
Status.IndexSortStatus status = new Status.IndexSortStatus();
if (sort != null) {
if (infoStream != null) {
infoStream.print(" test: index sort..........");
}
SortField fields[] = sort.getSort();
final int reverseMul[] = new int[fields.length];
final LeafFieldComparator comparators[] = new LeafFieldComparator[fields.length];
LeafReaderContext readerContext = new LeafReaderContext(reader);
for (int i = 0; i < fields.length; i++) {
reverseMul[i] = fields[i].getReverse() ? -1 : 1;
comparators[i] = fields[i].getComparator(1, i).getLeafComparator(readerContext);
}
int maxDoc = reader.maxDoc();
try {
for(int docID=1;docID < maxDoc;docID++) {
int cmp = 0;
for (int i = 0; i < comparators.length; i++) {
// TODO: would be better if copy() didnt cause a term lookup in TermOrdVal & co,
// the segments are always the same here...
comparators[i].copy(0, docID-1);
comparators[i].setBottom(0);
cmp = reverseMul[i] * comparators[i].compareBottom(docID);
if (cmp != 0) {
break;
}
}
if (cmp > 0) {
throw new RuntimeException("segment has indexSort=" + sort + " but docID=" + (docID-1) + " sorts after docID=" + docID);
}
}
msg(infoStream, String.format(Locale.ROOT, "OK [took %.3f sec]", nsToSec(System.nanoTime()-startNS)));
} catch (Throwable e) {
if (failFast) {
IOUtils.reThrow(e);
}
msg(infoStream, "ERROR [" + String.valueOf(e.getMessage()) + "]");
status.error = e;
if (infoStream != null) {
e.printStackTrace(infoStream);
}
}
}
return status;
}
/**
* Test live docs.

View File

@ -0,0 +1,181 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.index;
import java.io.IOException;
import java.util.List;
import org.apache.lucene.search.DocIdSetIterator; // javadocs
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.PriorityQueue;
import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
/** Utility class to help merging documents from sub-readers according to either simple
* concatenated (unsorted) order, or by a specified index-time sort, skipping
* deleted documents and remapping non-deleted documents. */
public class DocIDMerger<T extends DocIDMerger.Sub> {
private final List<T> subs;
// Used when indexSort != null:
private final PriorityQueue<T> queue;
private boolean first;
// Used when indexIsSorted
private T current;
private int nextIndex;
/** Represents one sub-reader being merged */
public static abstract class Sub {
/** Mapped doc ID */
public int mappedDocID;
final MergeState.DocMap docMap;
/** Sole constructor */
public Sub(MergeState.DocMap docMap) {
this.docMap = docMap;
}
/** Returns the next document ID from this sub reader, and {@link DocIdSetIterator#NO_MORE_DOCS} when done */
public abstract int nextDoc();
}
/** Construct this from the provided subs, specifying the maximum sub count */
public DocIDMerger(List<T> subs, int maxCount, boolean indexIsSorted) {
this.subs = subs;
if (indexIsSorted && maxCount > 1) {
queue = new PriorityQueue<T>(maxCount) {
@Override
protected boolean lessThan(Sub a, Sub b) {
assert a.mappedDocID != b.mappedDocID;
return a.mappedDocID < b.mappedDocID;
}
};
} else {
// We simply concatentate
queue = null;
}
reset();
}
/** Construct this from the provided subs */
public DocIDMerger(List<T> subs, boolean indexIsSorted) {
this(subs, subs.size(), indexIsSorted);
}
/** Reuse API, currently only used by postings during merge */
public void reset() {
if (queue != null) {
// caller may not have fully consumed the queue:
queue.clear();
for(T sub : subs) {
while (true) {
int docID = sub.nextDoc();
if (docID == NO_MORE_DOCS) {
// all docs in this sub were deleted; do not add it to the queue!
break;
}
int mappedDocID = sub.docMap.get(docID);
if (mappedDocID == -1) {
// doc was deleted
continue;
} else {
sub.mappedDocID = mappedDocID;
queue.add(sub);
break;
}
}
}
first = true;
} else if (subs.size() > 0) {
current = subs.get(0);
nextIndex = 1;
} else {
current = null;
nextIndex = 0;
}
}
/** Returns null when done */
public T next() {
// Loop until we find a non-deleted document
if (queue != null) {
T top = queue.top();
if (top == null) {
// NOTE: it's annoying that caller is allowed to call us again even after we returned null before
return null;
}
if (first == false) {
while (true) {
int docID = top.nextDoc();
if (docID == NO_MORE_DOCS) {
queue.pop();
top = queue.top();
break;
}
int mappedDocID = top.docMap.get(docID);
if (mappedDocID == -1) {
// doc was deleted
continue;
} else {
top.mappedDocID = mappedDocID;
top = queue.updateTop();
break;
}
}
}
first = false;
return top;
} else {
while (true) {
if (current == null) {
// NOTE: it's annoying that caller is allowed to call us again even after we returned null before
return null;
}
int docID = current.nextDoc();
if (docID == NO_MORE_DOCS) {
if (nextIndex == subs.size()) {
current = null;
return null;
}
current = subs.get(nextIndex);
nextIndex++;
continue;
}
int mappedDocID = current.docMap.get(docID);
if (mappedDocID == -1) {
// doc is deleted
continue;
}
current.mappedDocID = mappedDocID;
return current;
}
}
}
}

View File

@ -210,7 +210,7 @@ public final class DocValues {
(expected.length == 1
? "(expected=" + expected[0]
: "(expected one of " + Arrays.toString(expected)) + "). " +
"Use UninvertingReader or index with docvalues.");
"Re-index with correct docvalues type.");
}
}

View File

@ -178,7 +178,7 @@ class DocumentsWriterPerThread {
pendingUpdates.clear();
deleteSlice = deleteQueue.newSlice();
segmentInfo = new SegmentInfo(directoryOrig, Version.LATEST, segmentName, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
segmentInfo = new SegmentInfo(directoryOrig, Version.LATEST, segmentName, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
assert numDocsInRAM == 0;
if (INFO_VERBOSE && infoStream.isEnabled("DWPT")) {
infoStream.message("DWPT", Thread.currentThread().getName() + " init seg=" + segmentName + " delQueue=" + deleteQueue);

View File

@ -25,6 +25,7 @@ import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.codecs.PointsReader;
import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.search.Sort;
import org.apache.lucene.util.Bits;
/**
@ -101,6 +102,11 @@ public abstract class FilterCodecReader extends CodecReader {
return in.maxDoc();
}
@Override
public Sort getIndexSort() {
return in.getIndexSort();
}
@Override
public void addCoreClosedListener(CoreClosedListener listener) {
in.addCoreClosedListener(listener);

View File

@ -22,6 +22,7 @@ import java.util.Iterator;
import java.util.Objects;
import org.apache.lucene.search.QueryCache;
import org.apache.lucene.search.Sort;
import org.apache.lucene.util.AttributeSource;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
@ -471,6 +472,12 @@ public abstract class FilterLeafReader extends LeafReader {
return in.getDocsWithField(field);
}
@Override
public Sort getIndexSort() {
ensureOpen();
return in.getIndexSort();
}
@Override
public void checkIntegrity() throws IOException {
ensureOpen();

View File

@ -16,7 +16,6 @@
*/
package org.apache.lucene.index;
import java.io.Closeable;
import java.io.FileNotFoundException;
import java.io.IOException;
@ -32,8 +31,8 @@ import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Map;
import java.util.Queue;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
@ -49,6 +48,7 @@ import org.apache.lucene.index.FieldInfos.FieldNumbers;
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.Sort;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FSDirectory;
@ -937,6 +937,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
// NOTE: this is correct even for an NRT reader because we'll pull FieldInfos even for the un-committed segments:
globalFieldNumberMap = getFieldNumberMap();
validateIndexSort();
config.getFlushPolicy().init(config);
docWriter = new DocumentsWriter(this, config, directoryOrig, directory);
eventQueue = docWriter.eventQueue();
@ -1000,6 +1002,20 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
}
}
/** Confirms that the incoming index sort (if any) matches the existing index sort (if any). This is unfortunately just best effort,
* because it could be the old index only has flushed segments. */
private void validateIndexSort() {
Sort indexSort = config.getIndexSort();
if (indexSort != null) {
for(SegmentCommitInfo info : segmentInfos) {
Sort segmentIndexSort = info.info.getIndexSort();
if (segmentIndexSort != null && indexSort.equals(segmentIndexSort) == false) {
throw new IllegalArgumentException("cannot change previous indexSort=" + segmentIndexSort + " (from segment=" + info + ") to new indexSort=" + indexSort);
}
}
}
}
// reads latest field infos for the commit
// this is used on IW init and addIndexes(Dir) to create/update the global field map.
// TODO: fix tests abusing this method!
@ -2472,7 +2488,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
* @throws CorruptIndexException if the index is corrupt
* @throws IOException if there is a low-level IO error
* @throws IllegalArgumentException if addIndexes would cause
* the index to exceed {@link #MAX_DOCS}
* the index to exceed {@link #MAX_DOCS}, or if the indoming
* index sort does not match this index's index sort
*/
public void addIndexes(Directory... dirs) throws IOException {
ensureOpen();
@ -2481,6 +2498,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
List<Lock> locks = acquireWriteLocks(dirs);
Sort indexSort = config.getIndexSort();
boolean successTop = false;
try {
@ -2513,6 +2532,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
for (SegmentCommitInfo info : sis) {
assert !infos.contains(info): "dup info dir=" + info.info.dir + " name=" + info.info.name;
Sort segmentIndexSort = info.info.getIndexSort();
if (indexSort != null && segmentIndexSort != null && indexSort.equals(segmentIndexSort) == false) {
// TODO: we could make this smarter, e.g. if the incoming indexSort is congruent with our sort ("starts with") then it's OK
throw new IllegalArgumentException("cannot change index sort from " + segmentIndexSort + " to " + indexSort);
}
String newSegName = newSegmentName();
if (infoStream.isEnabled("IW")) {
@ -2609,6 +2635,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
// long so we can detect int overflow:
long numDocs = 0;
Sort indexSort = config.getIndexSort();
try {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "flush at addIndexes(CodecReader...)");
@ -2618,6 +2646,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
String mergedName = newSegmentName();
for (CodecReader leaf : readers) {
numDocs += leaf.numDocs();
Sort leafIndexSort = leaf.getIndexSort();
if (indexSort != null && leafIndexSort != null && indexSort.equals(leafIndexSort) == false) {
throw new IllegalArgumentException("cannot change index sort from " + leafIndexSort + " to " + indexSort);
}
}
// Best-effort up front check:
@ -2630,7 +2662,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(directory);
SegmentInfo info = new SegmentInfo(directoryOrig, Version.LATEST, mergedName, -1,
false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), config.getIndexSort());
SegmentMerger merger = new SegmentMerger(Arrays.asList(readers), info, infoStream, trackingDir,
globalFieldNumberMap,
@ -2715,7 +2747,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
// Same SI as before but we change directory and name
SegmentInfo newInfo = new SegmentInfo(directoryOrig, info.info.getVersion(), segName, info.info.maxDoc(),
info.info.getUseCompoundFile(), info.info.getCodec(),
info.info.getDiagnostics(), info.info.getId(), info.info.getAttributes());
info.info.getDiagnostics(), info.info.getId(), info.info.getAttributes(), info.info.getIndexSort());
SegmentCommitInfo newInfoPerCommit = new SegmentCommitInfo(newInfo, info.getDelCount(), info.getDelGen(),
info.getFieldInfosGen(), info.getDocValuesGen());
@ -3243,16 +3275,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
private static class MergedDeletesAndUpdates {
ReadersAndUpdates mergedDeletesAndUpdates = null;
MergePolicy.DocMap docMap = null;
boolean initializedWritableLiveDocs = false;
MergedDeletesAndUpdates() {}
final void init(ReaderPool readerPool, MergePolicy.OneMerge merge, MergeState mergeState, boolean initWritableLiveDocs) throws IOException {
final void init(ReaderPool readerPool, MergePolicy.OneMerge merge, boolean initWritableLiveDocs) throws IOException {
if (mergedDeletesAndUpdates == null) {
mergedDeletesAndUpdates = readerPool.get(merge.info, true);
docMap = merge.getDocMap(mergeState);
assert docMap.isConsistent(merge.info.info.maxDoc());
}
if (initWritableLiveDocs && !initializedWritableLiveDocs) {
mergedDeletesAndUpdates.initWritableLiveDocs();
@ -3262,18 +3291,18 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
}
private void maybeApplyMergedDVUpdates(MergePolicy.OneMerge merge, MergeState mergeState, int docUpto,
private void maybeApplyMergedDVUpdates(MergePolicy.OneMerge merge, MergeState mergeState,
MergedDeletesAndUpdates holder, String[] mergingFields, DocValuesFieldUpdates[] dvFieldUpdates,
DocValuesFieldUpdates.Iterator[] updatesIters, int curDoc) throws IOException {
DocValuesFieldUpdates.Iterator[] updatesIters, int segment, int curDoc) throws IOException {
int newDoc = -1;
for (int idx = 0; idx < mergingFields.length; idx++) {
DocValuesFieldUpdates.Iterator updatesIter = updatesIters[idx];
if (updatesIter.doc() == curDoc) { // document has an update
if (holder.mergedDeletesAndUpdates == null) {
holder.init(readerPool, merge, mergeState, false);
holder.init(readerPool, merge, false);
}
if (newDoc == -1) { // map once per all field updates, but only if there are any updates
newDoc = holder.docMap.map(docUpto);
newDoc = mergeState.docMaps[segment].get(curDoc);
}
DocValuesFieldUpdates dvUpdates = dvFieldUpdates[idx];
dvUpdates.add(newDoc, updatesIter.value());
@ -3306,13 +3335,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
// Carefully merge deletes that occurred after we
// started merging:
int docUpto = 0;
long minGen = Long.MAX_VALUE;
// Lazy init (only when we find a delete to carry over):
final MergedDeletesAndUpdates holder = new MergedDeletesAndUpdates();
final DocValuesFieldUpdates.Container mergedDVUpdates = new DocValuesFieldUpdates.Container();
assert sourceSegments.size() == mergeState.docMaps.length;
for (int i = 0; i < sourceSegments.size(); i++) {
SegmentCommitInfo info = sourceSegments.get(i);
minGen = Math.min(info.getBufferedDeletesGen(), minGen);
@ -3375,21 +3404,20 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
// since we started the merge, so we
// must merge them:
for (int j = 0; j < maxDoc; j++) {
if (!prevLiveDocs.get(j)) {
assert !currentLiveDocs.get(j);
} else {
if (!currentLiveDocs.get(j)) {
if (holder.mergedDeletesAndUpdates == null || !holder.initializedWritableLiveDocs) {
holder.init(readerPool, merge, mergeState, true);
}
holder.mergedDeletesAndUpdates.delete(holder.docMap.map(docUpto));
if (mergingFields != null) { // advance all iters beyond the deleted document
skipDeletedDoc(updatesIters, j);
}
} else if (mergingFields != null) {
maybeApplyMergedDVUpdates(merge, mergeState, docUpto, holder, mergingFields, dvFieldUpdates, updatesIters, j);
if (prevLiveDocs.get(j) == false) {
// if the document was deleted before, it better still be deleted!
assert currentLiveDocs.get(j) == false;
} else if (currentLiveDocs.get(j) == false) {
// the document was deleted while we were merging:
if (holder.mergedDeletesAndUpdates == null || holder.initializedWritableLiveDocs == false) {
holder.init(readerPool, merge, true);
}
docUpto++;
holder.mergedDeletesAndUpdates.delete(mergeState.docMaps[i].get(mergeState.leafDocMaps[i].get(j)));
if (mergingFields != null) { // advance all iters beyond the deleted document
skipDeletedDoc(updatesIters, j);
}
} else if (mergingFields != null) {
maybeApplyMergedDVUpdates(merge, mergeState, holder, mergingFields, dvFieldUpdates, updatesIters, i, j);
}
}
} else if (mergingFields != null) {
@ -3397,50 +3425,38 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
for (int j = 0; j < maxDoc; j++) {
if (prevLiveDocs.get(j)) {
// document isn't deleted, check if any of the fields have an update to it
maybeApplyMergedDVUpdates(merge, mergeState, docUpto, holder, mergingFields, dvFieldUpdates, updatesIters, j);
// advance docUpto for every non-deleted document
docUpto++;
maybeApplyMergedDVUpdates(merge, mergeState, holder, mergingFields, dvFieldUpdates, updatesIters, i, j);
} else {
// advance all iters beyond the deleted document
skipDeletedDoc(updatesIters, j);
}
}
} else {
docUpto += info.info.maxDoc() - info.getDelCount() - rld.getPendingDeleteCount();
}
} else if (currentLiveDocs != null) {
assert currentLiveDocs.length() == maxDoc;
// This segment had no deletes before but now it
// does:
for (int j = 0; j < maxDoc; j++) {
if (!currentLiveDocs.get(j)) {
if (currentLiveDocs.get(j) == false) {
if (holder.mergedDeletesAndUpdates == null || !holder.initializedWritableLiveDocs) {
holder.init(readerPool, merge, mergeState, true);
holder.init(readerPool, merge, true);
}
holder.mergedDeletesAndUpdates.delete(holder.docMap.map(docUpto));
holder.mergedDeletesAndUpdates.delete(mergeState.docMaps[i].get(mergeState.leafDocMaps[i].get(j)));
if (mergingFields != null) { // advance all iters beyond the deleted document
skipDeletedDoc(updatesIters, j);
}
} else if (mergingFields != null) {
maybeApplyMergedDVUpdates(merge, mergeState, docUpto, holder, mergingFields, dvFieldUpdates, updatesIters, j);
maybeApplyMergedDVUpdates(merge, mergeState, holder, mergingFields, dvFieldUpdates, updatesIters, i, j);
}
docUpto++;
}
} else if (mergingFields != null) {
// no deletions before or after, but there were updates
for (int j = 0; j < maxDoc; j++) {
maybeApplyMergedDVUpdates(merge, mergeState, docUpto, holder, mergingFields, dvFieldUpdates, updatesIters, j);
// advance docUpto for every non-deleted document
docUpto++;
maybeApplyMergedDVUpdates(merge, mergeState, holder, mergingFields, dvFieldUpdates, updatesIters, i, j);
}
} else {
// No deletes or updates before or after
docUpto += info.info.maxDoc();
}
}
assert docUpto == merge.info.info.maxDoc();
if (mergedDVUpdates.any()) {
// System.out.println("[" + Thread.currentThread().getName() + "] IW.commitMergedDeletes: mergedDeletes.info=" + mergedDeletes.info + ", mergedFieldUpdates=" + mergedFieldUpdates);
boolean success = false;
@ -3881,7 +3897,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
// ConcurrentMergePolicy we keep deterministic segment
// names.
final String mergeSegmentName = newSegmentName();
SegmentInfo si = new SegmentInfo(directoryOrig, Version.LATEST, mergeSegmentName, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
SegmentInfo si = new SegmentInfo(directoryOrig, Version.LATEST, mergeSegmentName, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), config.getIndexSort());
Map<String,String> details = new HashMap<>();
details.put("mergeMaxNumSegments", "" + merge.maxNumSegments);
details.put("mergeFactor", Integer.toString(merge.segments.size()));
@ -4082,10 +4098,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
}
// System.out.println("[" + Thread.currentThread().getName() + "] IW.mergeMiddle: merging " + merge.getMergeReaders());
// we pass merge.getMergeReaders() instead of merge.readers to allow the
// OneMerge to return a view over the actual segments to merge
final SegmentMerger merger = new SegmentMerger(merge.getMergeReaders(),
// Let the merge wrap readers
List<CodecReader> mergeReaders = new ArrayList<>();
for (SegmentReader reader : merge.readers) {
mergeReaders.add(merge.wrapForMerge(reader));
}
final SegmentMerger merger = new SegmentMerger(mergeReaders,
merge.info.info, infoStream, dirWrapper,
globalFieldNumberMap,
context);

View File

@ -18,16 +18,19 @@ package org.apache.lucene.index;
import java.io.PrintStream;
import java.util.EnumSet;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.index.DocumentsWriterPerThread.IndexingChain;
import org.apache.lucene.index.IndexWriter.IndexReaderWarmer;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField;
import org.apache.lucene.search.similarities.Similarity;
import org.apache.lucene.util.InfoStream;
import org.apache.lucene.util.PrintStreamInfoStream;
import org.apache.lucene.util.SetOnce;
import org.apache.lucene.util.SetOnce.AlreadySetException;
import org.apache.lucene.util.SetOnce;
/**
* Holds all the configuration that is used to create an {@link IndexWriter}.
@ -439,6 +442,26 @@ public final class IndexWriterConfig extends LiveIndexWriterConfig {
return this;
}
/** We only allow sorting on these types */
private static final EnumSet<SortField.Type> ALLOWED_INDEX_SORT_TYPES = EnumSet.of(SortField.Type.STRING,
SortField.Type.LONG,
SortField.Type.INT,
SortField.Type.DOUBLE,
SortField.Type.FLOAT);
/**
* Set the {@link Sort} order to use when merging segments. Note that newly flushed segments will remain unsorted.
*/
public IndexWriterConfig setIndexSort(Sort sort) {
for(SortField sortField : sort.getSort()) {
if (ALLOWED_INDEX_SORT_TYPES.contains(sortField.getType()) == false) {
throw new IllegalArgumentException("invalid SortField type: must be one of " + ALLOWED_INDEX_SORT_TYPES + " but got: " + sortField);
}
}
this.indexSort = sort;
return this;
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder(super.toString());

View File

@ -20,6 +20,7 @@ package org.apache.lucene.index;
import java.io.IOException;
import org.apache.lucene.index.IndexReader.ReaderClosedListener;
import org.apache.lucene.search.Sort;
import org.apache.lucene.util.Bits;
/** {@code LeafReader} is an abstract class, providing an interface for accessing an
@ -312,4 +313,7 @@ public abstract class LeafReader extends IndexReader {
* @lucene.internal
*/
public abstract void checkIntegrity() throws IOException;
/** Returns null if this leaf is unsorted, or the {@link Sort} that it was sorted by */
public abstract Sort getIndexSort();
}

View File

@ -23,6 +23,7 @@ import org.apache.lucene.index.DocumentsWriterPerThread.IndexingChain;
import org.apache.lucene.index.IndexWriter.IndexReaderWarmer;
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.similarities.Similarity;
import org.apache.lucene.util.InfoStream;
@ -94,6 +95,9 @@ public class LiveIndexWriterConfig {
/** True if calls to {@link IndexWriter#close()} should first do a commit. */
protected boolean commitOnClose = IndexWriterConfig.DEFAULT_COMMIT_ON_CLOSE;
/** The sort order to use to write merged segments. */
protected Sort indexSort = null;
// used by IndexWriterConfig
LiveIndexWriterConfig(Analyzer analyzer) {
this.analyzer = analyzer;
@ -445,6 +449,14 @@ public class LiveIndexWriterConfig {
return commitOnClose;
}
/**
* Set the index-time {@link Sort} order. Merged segments will be written
* in this order.
*/
public Sort getIndexSort() {
return indexSort;
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder();
@ -467,6 +479,7 @@ public class LiveIndexWriterConfig {
sb.append("perThreadHardLimitMB=").append(getRAMPerThreadHardLimitMB()).append("\n");
sb.append("useCompoundFile=").append(getUseCompoundFile()).append("\n");
sb.append("commitOnClose=").append(getCommitOnClose()).append("\n");
sb.append("indexSort=").append(getIndexSort()).append("\n");
return sb.toString();
}
}

View File

@ -18,8 +18,11 @@ package org.apache.lucene.index;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.lucene.index.MultiPostingsEnum.EnumWithSlice;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
/**
@ -30,52 +33,66 @@ import org.apache.lucene.util.BytesRef;
*/
final class MappingMultiPostingsEnum extends PostingsEnum {
private MultiPostingsEnum.EnumWithSlice[] subs;
int numSubs;
int upto;
MergeState.DocMap currentMap;
PostingsEnum current;
int currentBase;
int doc = -1;
private MergeState mergeState;
MultiPostingsEnum multiDocsAndPositionsEnum;
final String field;
final DocIDMerger<MappingPostingsSub> docIDMerger;
private MappingPostingsSub current;
private final MappingPostingsSub[] allSubs;
private final List<MappingPostingsSub> subs = new ArrayList<>();
private static class MappingPostingsSub extends DocIDMerger.Sub {
public PostingsEnum postings;
public MappingPostingsSub(MergeState.DocMap docMap) {
super(docMap);
}
@Override
public int nextDoc() {
try {
return postings.nextDoc();
} catch (IOException ioe) {
throw new RuntimeException(ioe);
}
}
}
/** Sole constructor. */
public MappingMultiPostingsEnum(String field, MergeState mergeState) {
public MappingMultiPostingsEnum(String field, MergeState mergeState) throws IOException {
this.field = field;
this.mergeState = mergeState;
allSubs = new MappingPostingsSub[mergeState.fieldsProducers.length];
for(int i=0;i<allSubs.length;i++) {
allSubs[i] = new MappingPostingsSub(mergeState.docMaps[i]);
}
this.docIDMerger = new DocIDMerger<MappingPostingsSub>(subs, allSubs.length, mergeState.segmentInfo.getIndexSort() != null);
}
MappingMultiPostingsEnum reset(MultiPostingsEnum postingsEnum) {
this.numSubs = postingsEnum.getNumSubs();
this.subs = postingsEnum.getSubs();
upto = -1;
doc = -1;
current = null;
MappingMultiPostingsEnum reset(MultiPostingsEnum postingsEnum) throws IOException {
this.multiDocsAndPositionsEnum = postingsEnum;
MultiPostingsEnum.EnumWithSlice[] subsArray = postingsEnum.getSubs();
int count = postingsEnum.getNumSubs();
subs.clear();
for(int i=0;i<count;i++) {
MappingPostingsSub sub = allSubs[subsArray[i].slice.readerIndex];
sub.postings = subsArray[i].postingsEnum;
subs.add(sub);
}
docIDMerger.reset();
return this;
}
/** How many sub-readers we are merging.
* @see #getSubs */
public int getNumSubs() {
return numSubs;
}
/** Returns sub-readers we are merging. */
public EnumWithSlice[] getSubs() {
return subs;
}
@Override
public int freq() throws IOException {
return current.freq();
return current.postings.freq();
}
@Override
public int docID() {
return doc;
if (current == null) {
return -1;
} else {
return current.mappedDocID;
}
}
@Override
@ -85,66 +102,47 @@ final class MappingMultiPostingsEnum extends PostingsEnum {
@Override
public int nextDoc() throws IOException {
while(true) {
if (current == null) {
if (upto == numSubs-1) {
return this.doc = NO_MORE_DOCS;
} else {
upto++;
final int reader = subs[upto].slice.readerIndex;
current = subs[upto].postingsEnum;
currentBase = mergeState.docBase[reader];
currentMap = mergeState.docMaps[reader];
}
}
int doc = current.nextDoc();
if (doc != NO_MORE_DOCS) {
// compact deletions
doc = currentMap.get(doc);
if (doc == -1) {
continue;
}
return this.doc = currentBase + doc;
} else {
current = null;
}
current = docIDMerger.next();
if (current == null) {
return NO_MORE_DOCS;
} else {
return current.mappedDocID;
}
}
@Override
public int nextPosition() throws IOException {
int pos = current.nextPosition();
int pos = current.postings.nextPosition();
if (pos < 0) {
throw new CorruptIndexException("position=" + pos + " is negative, field=\"" + field + " doc=" + doc,
mergeState.fieldsProducers[upto].toString());
throw new CorruptIndexException("position=" + pos + " is negative, field=\"" + field + " doc=" + current.mappedDocID,
current.postings.toString());
} else if (pos > IndexWriter.MAX_POSITION) {
throw new CorruptIndexException("position=" + pos + " is too large (> IndexWriter.MAX_POSITION=" + IndexWriter.MAX_POSITION + "), field=\"" + field + "\" doc=" + doc,
mergeState.fieldsProducers[upto].toString());
throw new CorruptIndexException("position=" + pos + " is too large (> IndexWriter.MAX_POSITION=" + IndexWriter.MAX_POSITION + "), field=\"" + field + "\" doc=" + current.mappedDocID,
current.postings.toString());
}
return pos;
}
@Override
public int startOffset() throws IOException {
return current.startOffset();
return current.postings.startOffset();
}
@Override
public int endOffset() throws IOException {
return current.endOffset();
return current.postings.endOffset();
}
@Override
public BytesRef getPayload() throws IOException {
return current.getPayload();
return current.postings.getPayload();
}
@Override
public long cost() {
long cost = 0;
for (EnumWithSlice enumWithSlice : subs) {
cost += enumWithSlice.postingsEnum.cost();
for (MappingPostingsSub sub : subs) {
cost += sub.postings.cost();
}
return cost;
}

View File

@ -58,31 +58,6 @@ import org.apache.lucene.util.FixedBitSet;
*/
public abstract class MergePolicy {
/** A map of doc IDs. */
public static abstract class DocMap {
/** Sole constructor, typically invoked from sub-classes constructors. */
protected DocMap() {}
/** Return the new doc ID according to its old value. */
public abstract int map(int old);
/** Useful from an assert. */
boolean isConsistent(int maxDoc) {
final FixedBitSet targets = new FixedBitSet(maxDoc);
for (int i = 0; i < maxDoc; ++i) {
final int target = map(i);
if (target < 0 || target >= maxDoc) {
assert false : "out of range: " + target + " not in [0-" + maxDoc + "[";
return false;
} else if (targets.get(target)) {
assert false : target + " is already taken (" + i + ")";
return false;
}
}
return true;
}
}
/** OneMerge provides the information necessary to perform
* an individual primitive merge operation, resulting in
* a single new segment. The merge spec includes the
@ -140,25 +115,11 @@ public abstract class MergePolicy {
public void mergeFinished() throws IOException {
}
/** Expert: Get the list of readers to merge. Note that this list does not
* necessarily match the list of segments to merge and should only be used
* to feed SegmentMerger to initialize a merge. When a {@link OneMerge}
* reorders doc IDs, it must override {@link #getDocMap} too so that
* deletes that happened during the merge can be applied to the newly
* merged segment. */
public List<CodecReader> getMergeReaders() throws IOException {
if (readers == null) {
throw new IllegalStateException("IndexWriter has not initialized readers from the segment infos yet");
}
final List<CodecReader> readers = new ArrayList<>(this.readers.size());
for (SegmentReader reader : this.readers) {
if (reader.numDocs() > 0) {
readers.add(reader);
}
}
return Collections.unmodifiableList(readers);
/** Wrap the reader in order to add/remove information to the merged segment. */
public CodecReader wrapForMerge(CodecReader reader) throws IOException {
return reader;
}
/**
* Expert: Sets the {@link SegmentCommitInfo} of the merged segment.
* Allows sub-classes to e.g. set diagnostics properties.
@ -175,20 +136,6 @@ public abstract class MergePolicy {
return info;
}
/** Expert: If {@link #getMergeReaders()} reorders document IDs, this method
* must be overridden to return a mapping from the <i>natural</i> doc ID
* (the doc ID that would result from a natural merge) to the actual doc
* ID. This mapping is used to apply deletions that happened during the
* merge to the new segment. */
public DocMap getDocMap(MergeState mergeState) {
return new DocMap() {
@Override
public int map(int docID) {
return docID;
}
};
}
/** Record that an exception occurred while executing
* this merge */
synchronized void setException(Throwable error) {

View File

@ -1,3 +1,5 @@
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
@ -14,7 +16,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.index;
import java.io.IOException;
@ -23,18 +24,19 @@ import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.search.Sort;
import org.apache.lucene.util.Bits;
/** this is a hack to make SortingMP fast! */
/** This is a hack to make index sorting fast, with a {@link LeafReader} that always returns merge instances when you ask for the codec readers. */
class MergeReaderWrapper extends LeafReader {
final SegmentReader in;
final CodecReader in;
final FieldsProducer fields;
final NormsProducer norms;
final DocValuesProducer docValues;
final StoredFieldsReader store;
final TermVectorsReader vectors;
MergeReaderWrapper(SegmentReader in) throws IOException {
MergeReaderWrapper(CodecReader in) throws IOException {
this.in = in;
FieldsProducer fields = in.getPostingsReader();
@ -256,4 +258,9 @@ class MergeReaderWrapper extends LeafReader {
public String toString() {
return "MergeReaderWrapper(" + in + ")";
}
@Override
public Sort getIndexSort() {
return in.getIndexSort();
}
}

View File

@ -18,7 +18,10 @@ package org.apache.lucene.index;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Locale;
import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.codecs.FieldsProducer;
@ -26,6 +29,7 @@ import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.codecs.PointsReader;
import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.search.Sort;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.InfoStream;
import org.apache.lucene.util.packed.PackedInts;
@ -36,6 +40,12 @@ import org.apache.lucene.util.packed.PackedLongValues;
* @lucene.experimental */
public class MergeState {
/** Maps document IDs from old segments to document IDs in the new segment */
public final DocMap[] docMaps;
// Only used by IW when it must remap deletes that arrived against the merging segmetns while a merge was running:
final DocMap[] leafDocMaps;
/** {@link SegmentInfo} of the newly merged segment. */
public final SegmentInfo segmentInfo;
@ -60,18 +70,12 @@ public class MergeState {
/** Live docs for each reader */
public final Bits[] liveDocs;
/** Maps docIDs around deletions. */
public final DocMap[] docMaps;
/** Postings to merge */
public final FieldsProducer[] fieldsProducers;
/** Point readers to merge */
public final PointsReader[] pointsReaders;
/** New docID base per reader. */
public final int[] docBase;
/** Max docs per reader */
public final int[] maxDocs;
@ -79,11 +83,15 @@ public class MergeState {
public final InfoStream infoStream;
/** Sole constructor. */
MergeState(List<CodecReader> readers, SegmentInfo segmentInfo, InfoStream infoStream) throws IOException {
MergeState(List<CodecReader> originalReaders, SegmentInfo segmentInfo, InfoStream infoStream) throws IOException {
this.infoStream = infoStream;
final Sort indexSort = segmentInfo.getIndexSort();
int numReaders = originalReaders.size();
leafDocMaps = new DocMap[numReaders];
List<CodecReader> readers = maybeSortReaders(originalReaders, segmentInfo);
int numReaders = readers.size();
docMaps = new DocMap[numReaders];
docBase = new int[numReaders];
maxDocs = new int[numReaders];
fieldsProducers = new FieldsProducer[numReaders];
normsProducers = new NormsProducer[numReaders];
@ -94,6 +102,7 @@ public class MergeState {
fieldInfos = new FieldInfos[numReaders];
liveDocs = new Bits[numReaders];
int numDocs = 0;
for(int i=0;i<numReaders;i++) {
final CodecReader reader = readers.get(i);
@ -126,126 +135,152 @@ public class MergeState {
if (pointsReaders[i] != null) {
pointsReaders[i] = pointsReaders[i].getMergeInstance();
}
numDocs += reader.numDocs();
}
segmentInfo.setMaxDoc(numDocs);
this.segmentInfo = segmentInfo;
this.infoStream = infoStream;
setDocMaps(readers);
this.docMaps = buildDocMaps(readers, indexSort);
}
// NOTE: removes any "all deleted" readers from mergeState.readers
private void setDocMaps(List<CodecReader> readers) throws IOException {
final int numReaders = maxDocs.length;
private DocMap[] buildDocMaps(List<CodecReader> readers, Sort indexSort) throws IOException {
// Remap docIDs
int docBase = 0;
for(int i=0;i<numReaders;i++) {
final CodecReader reader = readers.get(i);
this.docBase[i] = docBase;
final DocMap docMap = DocMap.build(reader);
docMaps[i] = docMap;
docBase += docMap.numDocs();
}
int numReaders = readers.size();
segmentInfo.setMaxDoc(docBase);
}
if (indexSort == null) {
// no index sort ... we only must map around deletions, and rebase to the merged segment's docID space
/**
* Remaps docids around deletes during merge
*/
public static abstract class DocMap {
int totalDocs = 0;
DocMap[] docMaps = new DocMap[numReaders];
DocMap() {}
// Remap docIDs around deletions:
for (int i = 0; i < numReaders; i++) {
LeafReader reader = readers.get(i);
Bits liveDocs = reader.getLiveDocs();
/** Returns the mapped docID corresponding to the provided one. */
public abstract int get(int docID);
/** Returns the total number of documents, ignoring
* deletions. */
public abstract int maxDoc();
/** Returns the number of not-deleted documents. */
public final int numDocs() {
return maxDoc() - numDeletedDocs();
}
/** Returns the number of deleted documents. */
public abstract int numDeletedDocs();
/** Returns true if there are any deletions. */
public boolean hasDeletions() {
return numDeletedDocs() > 0;
}
/** Creates a {@link DocMap} instance appropriate for
* this reader. */
public static DocMap build(CodecReader reader) {
final int maxDoc = reader.maxDoc();
if (!reader.hasDeletions()) {
return new NoDelDocMap(maxDoc);
}
final Bits liveDocs = reader.getLiveDocs();
return build(maxDoc, liveDocs);
}
static DocMap build(final int maxDoc, final Bits liveDocs) {
assert liveDocs != null;
final PackedLongValues.Builder docMapBuilder = PackedLongValues.monotonicBuilder(PackedInts.COMPACT);
int del = 0;
for (int i = 0; i < maxDoc; ++i) {
docMapBuilder.add(i - del);
if (!liveDocs.get(i)) {
++del;
final PackedLongValues delDocMap;
if (liveDocs != null) {
delDocMap = removeDeletes(reader.maxDoc(), liveDocs);
} else {
delDocMap = null;
}
}
final PackedLongValues docMap = docMapBuilder.build();
final int numDeletedDocs = del;
assert docMap.size() == maxDoc;
return new DocMap() {
@Override
public int get(int docID) {
if (!liveDocs.get(docID)) {
return -1;
final int docBase = totalDocs;
docMaps[i] = new DocMap() {
@Override
public int get(int docID) {
if (liveDocs == null) {
return docBase + docID;
} else if (liveDocs.get(docID)) {
return docBase + (int) delDocMap.get(docID);
} else {
return -1;
}
}
return (int) docMap.get(docID);
}
};
totalDocs += reader.numDocs();
}
@Override
public int maxDoc() {
return maxDoc;
}
return docMaps;
@Override
public int numDeletedDocs() {
return numDeletedDocs;
}
};
} else {
// do a merge sort of the incoming leaves:
long t0 = System.nanoTime();
DocMap[] result = MultiSorter.sort(indexSort, readers);
long t1 = System.nanoTime();
if (infoStream.isEnabled("SM")) {
infoStream.message("SM", String.format(Locale.ROOT, "%.2f msec to build merge sorted DocMaps", (t1-t0)/1000000.0));
}
return result;
}
}
private static final class NoDelDocMap extends DocMap {
private List<CodecReader> maybeSortReaders(List<CodecReader> originalReaders, SegmentInfo segmentInfo) throws IOException {
private final int maxDoc;
NoDelDocMap(int maxDoc) {
this.maxDoc = maxDoc;
// Default to identity:
for(int i=0;i<originalReaders.size();i++) {
leafDocMaps[i] = new DocMap() {
@Override
public int get(int docID) {
return docID;
}
};
}
@Override
public int get(int docID) {
return docID;
Sort indexSort = segmentInfo.getIndexSort();
if (indexSort == null) {
return originalReaders;
}
@Override
public int maxDoc() {
return maxDoc;
// If an incoming reader is not sorted, because it was flushed by IW, we sort it here:
final Sorter sorter = new Sorter(indexSort);
List<CodecReader> readers = new ArrayList<>(originalReaders.size());
for (CodecReader leaf : originalReaders) {
Sort segmentSort = leaf.getIndexSort();
if (segmentSort == null) {
// TODO: fix IW to also sort when flushing? It's somewhat tricky because of stored fields and term vectors, which write "live"
// to their index files on each indexed document:
// This segment was written by flush, so documents are not yet sorted, so we sort them now:
long t0 = System.nanoTime();
Sorter.DocMap sortDocMap = sorter.sort(leaf);
long t1 = System.nanoTime();
double msec = (t1-t0)/1000000.0;
if (sortDocMap != null) {
if (infoStream.isEnabled("SM")) {
infoStream.message("SM", String.format(Locale.ROOT, "segment %s is not sorted; wrapping for sort %s now (%.2f msec to sort)", leaf, indexSort, msec));
}
leaf = SlowCodecReaderWrapper.wrap(SortingLeafReader.wrap(new MergeReaderWrapper(leaf), sortDocMap));
leafDocMaps[readers.size()] = new DocMap() {
@Override
public int get(int docID) {
return sortDocMap.oldToNew(docID);
}
};
} else {
if (infoStream.isEnabled("SM")) {
infoStream.message("SM", String.format(Locale.ROOT, "segment %s is not sorted, but is already accidentally in sort %s order (%.2f msec to sort)", leaf, indexSort, msec));
}
}
} else {
if (segmentSort.equals(indexSort) == false) {
throw new IllegalArgumentException("index sort mismatch: merged segment has sort=" + indexSort + " but to-be-merged segment has sort=" + segmentSort);
}
if (infoStream.isEnabled("SM")) {
infoStream.message("SM", "segment " + leaf + " already sorted");
}
}
readers.add(leaf);
}
@Override
public int numDeletedDocs() {
return 0;
return readers;
}
/** A map of doc IDs. */
public static abstract class DocMap {
/** Sole constructor */
public DocMap() {
}
/** Return the mapped docID or -1 if the given doc is not mapped. */
public abstract int get(int docID);
}
static PackedLongValues removeDeletes(final int maxDoc, final Bits liveDocs) {
final PackedLongValues.Builder docMapBuilder = PackedLongValues.monotonicBuilder(PackedInts.COMPACT);
int del = 0;
for (int i = 0; i < maxDoc; ++i) {
docMapBuilder.add(i - del);
if (liveDocs.get(i) == false) {
++del;
}
}
return docMapBuilder.build();
}
}

View File

@ -57,7 +57,7 @@ public final class MultiPostingsEnum extends PostingsEnum {
return this.parent == parent;
}
/** Rre-use and reset this instance on the provided slices. */
/** Re-use and reset this instance on the provided slices. */
public MultiPostingsEnum reset(final EnumWithSlice[] subs, final int numSubs) {
this.numSubs = numSubs;
for(int i=0;i<numSubs;i++) {
@ -165,15 +165,15 @@ public final class MultiPostingsEnum extends PostingsEnum {
/** Holds a {@link PostingsEnum} along with the
* corresponding {@link ReaderSlice}. */
public final static class EnumWithSlice {
EnumWithSlice() {
}
/** {@link PostingsEnum} for this sub-reader. */
public PostingsEnum postingsEnum;
/** {@link ReaderSlice} describing how this sub-reader
* fits into the composite reader. */
public ReaderSlice slice;
EnumWithSlice() {
}
@Override
public String toString() {

View File

@ -0,0 +1,354 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.index;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.lucene.index.MergeState.DocMap;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.search.LeafFieldComparator;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.PriorityQueue;
import org.apache.lucene.util.packed.PackedInts;
import org.apache.lucene.util.packed.PackedLongValues;
final class MultiSorter {
/** Does a merge sort of the leaves of the incoming reader, returning {@link DocMap} to map each leaf's
* documents into the merged segment. The documents for each incoming leaf reader must already be sorted by the same sort! */
static MergeState.DocMap[] sort(Sort sort, List<CodecReader> readers) throws IOException {
// TODO: optimize if only 1 reader is incoming, though that's a rare case
SortField fields[] = sort.getSort();
final CrossReaderComparator[] comparators = new CrossReaderComparator[fields.length];
for(int i=0;i<fields.length;i++) {
comparators[i] = getComparator(readers, fields[i]);
}
int leafCount = readers.size();
PriorityQueue<LeafAndDocID> queue = new PriorityQueue<LeafAndDocID>(leafCount) {
@Override
public boolean lessThan(LeafAndDocID a, LeafAndDocID b) {
for(int i=0;i<comparators.length;i++) {
int cmp = comparators[i].compare(a.readerIndex, a.docID, b.readerIndex, b.docID);
if (cmp != 0) {
return cmp < 0;
}
}
// tie-break by docID natural order:
if (a.readerIndex != b.readerIndex) {
return a.readerIndex < b.readerIndex;
}
return a.docID < b.docID;
}
};
PackedLongValues.Builder[] builders = new PackedLongValues.Builder[leafCount];
for(int i=0;i<leafCount;i++) {
CodecReader reader = readers.get(i);
queue.add(new LeafAndDocID(i, reader.getLiveDocs(), reader.maxDoc()));
builders[i] = PackedLongValues.monotonicBuilder(PackedInts.COMPACT);
}
int mappedDocID = 0;
while (queue.size() != 0) {
LeafAndDocID top = queue.top();
builders[top.readerIndex].add(mappedDocID);
if (top.liveDocs == null || top.liveDocs.get(top.docID)) {
mappedDocID++;
}
top.docID++;
if (top.docID < top.maxDoc) {
queue.updateTop();
} else {
queue.pop();
}
}
MergeState.DocMap[] docMaps = new MergeState.DocMap[leafCount];
for(int i=0;i<leafCount;i++) {
final PackedLongValues remapped = builders[i].build();
final Bits liveDocs = readers.get(i).getLiveDocs();
docMaps[i] = new MergeState.DocMap() {
@Override
public int get(int docID) {
if (liveDocs == null || liveDocs.get(docID)) {
return (int) remapped.get(docID);
} else {
return -1;
}
}
};
}
return docMaps;
}
private static class LeafAndDocID {
final int readerIndex;
final Bits liveDocs;
final int maxDoc;
int docID;
public LeafAndDocID(int readerIndex, Bits liveDocs, int maxDoc) {
this.readerIndex = readerIndex;
this.liveDocs = liveDocs;
this.maxDoc = maxDoc;
}
}
private interface CrossReaderComparator {
public int compare(int readerIndexA, int docIDA, int readerIndexB, int docIDB);
}
private static CrossReaderComparator getComparator(List<CodecReader> readers, SortField sortField) throws IOException {
switch(sortField.getType()) {
case STRING:
{
// this uses the efficient segment-local ordinal map:
MultiReader multiReader = new MultiReader(readers.toArray(new LeafReader[readers.size()]));
final SortedDocValues sorted = MultiDocValues.getSortedValues(multiReader, sortField.getField());
final int[] docStarts = new int[readers.size()];
List<LeafReaderContext> leaves = multiReader.leaves();
for(int i=0;i<readers.size();i++) {
docStarts[i] = leaves.get(i).docBase;
}
final int missingOrd;
if (sortField.getMissingValue() == SortField.STRING_LAST) {
missingOrd = Integer.MAX_VALUE;
} else {
missingOrd = Integer.MIN_VALUE;
}
final int reverseMul;
if (sortField.getReverse()) {
reverseMul = -1;
} else {
reverseMul = 1;
}
return new CrossReaderComparator() {
@Override
public int compare(int readerIndexA, int docIDA, int readerIndexB, int docIDB) {
int ordA = sorted.getOrd(docStarts[readerIndexA] + docIDA);
if (ordA == -1) {
ordA = missingOrd;
}
int ordB = sorted.getOrd(docStarts[readerIndexB] + docIDB);
if (ordB == -1) {
ordB = missingOrd;
}
return reverseMul * Integer.compare(ordA, ordB);
}
};
}
case LONG:
{
List<NumericDocValues> values = new ArrayList<>();
List<Bits> docsWithFields = new ArrayList<>();
for(CodecReader reader : readers) {
values.add(DocValues.getNumeric(reader, sortField.getField()));
docsWithFields.add(DocValues.getDocsWithField(reader, sortField.getField()));
}
final int reverseMul;
if (sortField.getReverse()) {
reverseMul = -1;
} else {
reverseMul = 1;
}
final long missingValue;
if (sortField.getMissingValue() != null) {
missingValue = (Long) sortField.getMissingValue();
} else {
missingValue = 0;
}
return new CrossReaderComparator() {
@Override
public int compare(int readerIndexA, int docIDA, int readerIndexB, int docIDB) {
long valueA;
if (docsWithFields.get(readerIndexA).get(docIDA)) {
valueA = values.get(readerIndexA).get(docIDA);
} else {
valueA = missingValue;
}
long valueB;
if (docsWithFields.get(readerIndexB).get(docIDB)) {
valueB = values.get(readerIndexB).get(docIDB);
} else {
valueB = missingValue;
}
return reverseMul * Long.compare(valueA, valueB);
}
};
}
case INT:
{
List<NumericDocValues> values = new ArrayList<>();
List<Bits> docsWithFields = new ArrayList<>();
for(CodecReader reader : readers) {
values.add(DocValues.getNumeric(reader, sortField.getField()));
docsWithFields.add(DocValues.getDocsWithField(reader, sortField.getField()));
}
final int reverseMul;
if (sortField.getReverse()) {
reverseMul = -1;
} else {
reverseMul = 1;
}
final int missingValue;
if (sortField.getMissingValue() != null) {
missingValue = (Integer) sortField.getMissingValue();
} else {
missingValue = 0;
}
return new CrossReaderComparator() {
@Override
public int compare(int readerIndexA, int docIDA, int readerIndexB, int docIDB) {
int valueA;
if (docsWithFields.get(readerIndexA).get(docIDA)) {
valueA = (int) values.get(readerIndexA).get(docIDA);
} else {
valueA = missingValue;
}
int valueB;
if (docsWithFields.get(readerIndexB).get(docIDB)) {
valueB = (int) values.get(readerIndexB).get(docIDB);
} else {
valueB = missingValue;
}
return reverseMul * Integer.compare(valueA, valueB);
}
};
}
case DOUBLE:
{
List<NumericDocValues> values = new ArrayList<>();
List<Bits> docsWithFields = new ArrayList<>();
for(CodecReader reader : readers) {
values.add(DocValues.getNumeric(reader, sortField.getField()));
docsWithFields.add(DocValues.getDocsWithField(reader, sortField.getField()));
}
final int reverseMul;
if (sortField.getReverse()) {
reverseMul = -1;
} else {
reverseMul = 1;
}
final double missingValue;
if (sortField.getMissingValue() != null) {
missingValue = (Double) sortField.getMissingValue();
} else {
missingValue = 0.0;
}
return new CrossReaderComparator() {
@Override
public int compare(int readerIndexA, int docIDA, int readerIndexB, int docIDB) {
double valueA;
if (docsWithFields.get(readerIndexA).get(docIDA)) {
valueA = Double.longBitsToDouble(values.get(readerIndexA).get(docIDA));
} else {
valueA = missingValue;
}
double valueB;
if (docsWithFields.get(readerIndexB).get(docIDB)) {
valueB = Double.longBitsToDouble(values.get(readerIndexB).get(docIDB));
} else {
valueB = missingValue;
}
return reverseMul * Double.compare(valueA, valueB);
}
};
}
case FLOAT:
{
List<NumericDocValues> values = new ArrayList<>();
List<Bits> docsWithFields = new ArrayList<>();
for(CodecReader reader : readers) {
values.add(DocValues.getNumeric(reader, sortField.getField()));
docsWithFields.add(DocValues.getDocsWithField(reader, sortField.getField()));
}
final int reverseMul;
if (sortField.getReverse()) {
reverseMul = -1;
} else {
reverseMul = 1;
}
final float missingValue;
if (sortField.getMissingValue() != null) {
missingValue = (Float) sortField.getMissingValue();
} else {
missingValue = 0.0f;
}
return new CrossReaderComparator() {
@Override
public int compare(int readerIndexA, int docIDA, int readerIndexB, int docIDB) {
float valueA;
if (docsWithFields.get(readerIndexA).get(docIDA)) {
valueA = Float.intBitsToFloat((int) values.get(readerIndexA).get(docIDA));
} else {
valueA = missingValue;
}
float valueB;
if (docsWithFields.get(readerIndexB).get(docIDB)) {
valueB = Float.intBitsToFloat((int) values.get(readerIndexB).get(docIDB));
} else {
valueB = missingValue;
}
return reverseMul * Float.compare(valueA, valueB);
}
};
}
default:
throw new IllegalArgumentException("unhandled SortField.getType()=" + sortField.getType());
}
}
}

View File

@ -26,6 +26,7 @@ import java.util.Set;
import java.util.SortedMap;
import java.util.TreeMap;
import org.apache.lucene.search.Sort;
import org.apache.lucene.util.Bits;
/** An {@link LeafReader} which reads multiple, parallel indexes. Each index
@ -55,6 +56,7 @@ public class ParallelLeafReader extends LeafReader {
private final boolean closeSubReaders;
private final int maxDoc, numDocs;
private final boolean hasDeletions;
private final Sort indexSort;
private final SortedMap<String,LeafReader> fieldToReader = new TreeMap<>();
private final SortedMap<String,LeafReader> tvFieldToReader = new TreeMap<>();
@ -100,8 +102,18 @@ public class ParallelLeafReader extends LeafReader {
// TODO: make this read-only in a cleaner way?
FieldInfos.Builder builder = new FieldInfos.Builder();
Sort indexSort = null;
// build FieldInfos and fieldToReader map:
for (final LeafReader reader : this.parallelReaders) {
Sort leafIndexSort = reader.getIndexSort();
if (indexSort == null) {
indexSort = leafIndexSort;
} else if (leafIndexSort != null && indexSort.equals(leafIndexSort) == false) {
throw new IllegalArgumentException("cannot combine LeafReaders that have different index sorts: saw both sort=" + indexSort + " and " + leafIndexSort);
}
final FieldInfos readerFieldInfos = reader.getFieldInfos();
for (FieldInfo fieldInfo : readerFieldInfos) {
// NOTE: first reader having a given field "wins":
@ -115,6 +127,7 @@ public class ParallelLeafReader extends LeafReader {
}
}
fieldInfos = builder.finish();
this.indexSort = indexSort;
// build Fields instance
for (final LeafReader reader : this.parallelReaders) {
@ -423,4 +436,10 @@ public class ParallelLeafReader extends LeafReader {
ensureOpen();
return parallelReaders;
}
@Override
public Sort getIndexSort() {
return indexSort;
}
}

View File

@ -28,6 +28,7 @@ import java.util.Set;
import java.util.regex.Matcher;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.search.Sort;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.TrackingDirectoryWrapper;
import org.apache.lucene.util.StringHelper;
@ -69,6 +70,8 @@ public final class SegmentInfo {
private final Map<String,String> attributes;
private final Sort indexSort;
// Tracks the Lucene version this segment was created with, since 3.1. Null
// indicates an older than 3.0 index, and it's used to detect a too old index.
// The format expected is "x.y" - "2.x" for pre-3.0 indexes (or null), and
@ -93,7 +96,7 @@ public final class SegmentInfo {
*/
public SegmentInfo(Directory dir, Version version, String name, int maxDoc,
boolean isCompoundFile, Codec codec, Map<String,String> diagnostics,
byte[] id, Map<String,String> attributes) {
byte[] id, Map<String,String> attributes, Sort indexSort) {
assert !(dir instanceof TrackingDirectoryWrapper);
this.dir = Objects.requireNonNull(dir);
this.version = Objects.requireNonNull(version);
@ -107,6 +110,7 @@ public final class SegmentInfo {
throw new IllegalArgumentException("invalid id: " + Arrays.toString(id));
}
this.attributes = Objects.requireNonNull(attributes);
this.indexSort = indexSort;
}
/**
@ -194,13 +198,9 @@ public final class SegmentInfo {
s.append('/').append(delCount);
}
final String sorter_key = "sorter"; // SortingMergePolicy.SORTER_ID_PROP; // TODO: use this once we can import SortingMergePolicy (currently located in 'misc' instead of 'core')
final String sorter_val = diagnostics.get(sorter_key);
if (sorter_val != null) {
s.append(":[");
s.append(sorter_key);
s.append('=');
s.append(sorter_val);
if (indexSort != null) {
s.append(":[indexSort=");
s.append(indexSort);
s.append(']');
}
@ -311,5 +311,10 @@ public final class SegmentInfo {
public Map<String,String> getAttributes() {
return attributes;
}
/** Return the sort order of this segment, or null if the index has no sort. */
public Sort getIndexSort() {
return indexSort;
}
}

View File

@ -59,6 +59,11 @@ final class SegmentMerger {
this.codec = segmentInfo.getCodec();
this.context = context;
this.fieldInfosBuilder = new FieldInfos.Builder(fieldNumbers);
if (mergeState.infoStream.isEnabled("SM")) {
if (segmentInfo.getIndexSort() != null) {
mergeState.infoStream.message("SM", "index sort during merge: " + segmentInfo.getIndexSort());
}
}
}
/** True if any merging should happen */

View File

@ -28,6 +28,7 @@ import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.codecs.PointsReader;
import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.search.Sort;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.Bits;
@ -303,4 +304,9 @@ public final class SegmentReader extends CodecReader {
ensureOpen();
core.removeCoreClosedListener(listener);
}
@Override
public Sort getIndexSort() {
return si.info.getIndexSort();
}
}

View File

@ -26,6 +26,7 @@ import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.codecs.PointsReader;
import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.search.Sort;
import org.apache.lucene.util.Bits;
/**
@ -125,6 +126,16 @@ public final class SlowCodecReaderWrapper {
public void removeCoreClosedListener(CoreClosedListener listener) {
reader.removeCoreClosedListener(listener);
}
@Override
public String toString() {
return "SlowCodecReaderWrapper(" + reader + ")";
}
@Override
public Sort getIndexSort() {
return reader.getIndexSort();
}
};
}
}

View File

@ -168,6 +168,7 @@ final class Sorter {
}
final PackedLongValues newToOld = newToOldBuilder.build();
// invert the docs mapping:
for (int i = 0; i < maxDoc; ++i) {
docs[(int) newToOld.get(i)] = i;
} // docs is now the oldToNew mapping
@ -196,7 +197,7 @@ final class Sorter {
}
};
}
/**
* Returns a mapping from the old document ID to its new location in the
* sorted index. Implementations can use the auxiliary

View File

@ -1,3 +1,5 @@
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
@ -14,7 +16,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.index;
import java.io.IOException;
import java.util.Arrays;
@ -35,21 +36,13 @@ import org.apache.lucene.util.automaton.CompiledAutomaton;
/**
* An {@link org.apache.lucene.index.LeafReader} which supports sorting documents by a given
* {@link Sort}. You can use this class to sort an index as follows:
*
* <pre class="prettyprint">
* IndexWriter writer; // writer to which the sorted index will be added
* DirectoryReader reader; // reader on the input index
* Sort sort; // determines how the documents are sorted
* LeafReader sortingReader = SortingLeafReader.wrap(SlowCompositeReaderWrapper.wrap(reader), sort);
* writer.addIndexes(reader);
* writer.close();
* reader.close();
* </pre>
* {@link Sort}. This is package private and is only used by Lucene when it needs to merge
* a newly flushed (unsorted) segment.
*
* @lucene.experimental
*/
public class SortingLeafReader extends FilterLeafReader {
class SortingLeafReader extends FilterLeafReader {
private static class SortingFields extends FilterFields {
@ -111,25 +104,6 @@ public class SortingLeafReader extends FilterLeafReader {
this.hasPositions = hasPositions;
}
Bits newToOld(final Bits liveDocs) {
if (liveDocs == null) {
return null;
}
return new Bits() {
@Override
public boolean get(int index) {
return liveDocs.get(docMap.oldToNew(index));
}
@Override
public int length() {
return liveDocs.length();
}
};
}
@Override
public PostingsEnum postings( PostingsEnum reuse, final int flags) throws IOException {
@ -368,6 +342,7 @@ public class SortingLeafReader extends FilterLeafReader {
@Override
public void setDocument(int docID) {
//System.out.println(" slr.sssdv.setDocument docID=" + docID + " this=" + this);
in.setDocument(docMap.newToOld(docID));
}
@ -865,7 +840,6 @@ public class SortingLeafReader extends FilterLeafReader {
if (inPointValues == null) {
return null;
} else {
// TODO: this is untested!
return new SortingPointValues(inPointValues, docMap);
}
}

View File

@ -20,14 +20,14 @@ import java.io.IOException;
import java.util.Arrays;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.SortingMergePolicy;
import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.CollectionTerminatedException;
import org.apache.lucene.search.Collector;
import org.apache.lucene.search.FilterLeafCollector;
import org.apache.lucene.search.FilterCollector;
import org.apache.lucene.search.FilterLeafCollector;
import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.TopDocsCollector;
import org.apache.lucene.search.TotalHitCountCollector;
@ -39,8 +39,7 @@ import org.apache.lucene.search.TotalHitCountCollector;
*
* <p>
* <b>NOTE:</b> the {@code Collector} detects segments sorted according to a
* {@link SortingMergePolicy}'s {@link Sort} and so it's best used in conjunction
* with a {@link SortingMergePolicy}. Also,it collects up to a specified
* an {@link IndexWriterConfig#setIndexSort}. Also, it collects up to a specified
* {@code numDocsToCollect} from each segment, and therefore is mostly suitable
* for use in conjunction with collectors such as {@link TopDocsCollector}, and
* not e.g. {@link TotalHitCountCollector}.
@ -48,24 +47,12 @@ import org.apache.lucene.search.TotalHitCountCollector;
* <b>NOTE</b>: If you wrap a {@code TopDocsCollector} that sorts in the same
* order as the index order, the returned {@link TopDocsCollector#topDocs() TopDocs}
* will be correct. However the total of {@link TopDocsCollector#getTotalHits()
* hit count} will be underestimated since not all matching documents will have
* hit count} will be vastly underestimated since not all matching documents will have
* been collected.
* <p>
* <b>NOTE</b>: This {@code Collector} uses {@link Sort#toString()} to detect
* whether a segment was sorted with the same {@code Sort}. This has
* two implications:
* <ul>
* <li>if a custom comparator is not implemented correctly and returns
* different identifiers for equivalent instances, this collector will not
* detect sorted segments,</li>
* <li>if you suddenly change the {@link IndexWriter}'s
* {@code SortingMergePolicy} to sort according to another criterion and if both
* the old and the new {@code Sort}s have the same identifier, this
* {@code Collector} will incorrectly detect sorted segments.</li>
* </ul>
*
* @lucene.experimental
*/
public class EarlyTerminatingSortingCollector extends FilterCollector {
/** Returns whether collection can be early-terminated if it sorts with the
@ -85,7 +72,6 @@ public class EarlyTerminatingSortingCollector extends FilterCollector {
protected final Sort sort;
/** Number of documents to collect in each segment */
protected final int numDocsToCollect;
private final Sort mergePolicySort;
private final AtomicBoolean terminatedEarly = new AtomicBoolean(false);
/**
@ -99,27 +85,26 @@ public class EarlyTerminatingSortingCollector extends FilterCollector {
* the number of documents to collect on each segment. When wrapping
* a {@link TopDocsCollector}, this number should be the number of
* hits.
* @param mergePolicySort
* the sort your {@link SortingMergePolicy} uses
* @throws IllegalArgumentException if the sort order doesn't allow for early
* termination with the given merge policy.
*/
public EarlyTerminatingSortingCollector(Collector in, Sort sort, int numDocsToCollect, Sort mergePolicySort) {
public EarlyTerminatingSortingCollector(Collector in, Sort sort, int numDocsToCollect) {
super(in);
if (numDocsToCollect <= 0) {
throw new IllegalArgumentException("numDocsToCollect must always be > 0, got " + numDocsToCollect);
}
if (canEarlyTerminate(sort, mergePolicySort) == false) {
throw new IllegalStateException("Cannot early terminate with sort order " + sort + " if segments are sorted with " + mergePolicySort);
}
this.sort = sort;
this.numDocsToCollect = numDocsToCollect;
this.mergePolicySort = mergePolicySort;
}
@Override
public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
if (SortingMergePolicy.isSorted(context.reader(), mergePolicySort)) {
Sort segmentSort = context.reader().getIndexSort();
if (segmentSort != null && canEarlyTerminate(sort, segmentSort) == false) {
throw new IllegalStateException("Cannot early terminate with sort order " + sort + " if segments are sorted with " + segmentSort);
}
if (segmentSort != null) {
// segment is sorted, can early-terminate
return new FilterLeafCollector(super.getLeafCollector(context)) {
private int numCollected;
@ -142,5 +127,4 @@ public class EarlyTerminatingSortingCollector extends FilterCollector {
public boolean terminatedEarly() {
return terminatedEarly.get();
}
}

View File

@ -819,6 +819,7 @@ public class IndexSearcher {
sumTotalTermFreq = terms.getSumTotalTermFreq();
sumDocFreq = terms.getSumDocFreq();
}
return new CollectionStatistics(field, reader.maxDoc(), docCount, sumTotalTermFreq, sumDocFreq);
}
}

View File

@ -147,6 +147,9 @@ public class Sort {
* etc. Finally, if there is still a tie after all SortFields
* are checked, the internal Lucene docid is used to break it. */
public void setSort(SortField... fields) {
if (fields.length == 0) {
throw new IllegalArgumentException("There must be at least 1 sort field");
}
this.fields = fields;
}

View File

@ -77,9 +77,6 @@ public class SortField {
* uses ordinals to do the sorting. */
STRING_VAL,
/** Sort use byte[] index values. */
BYTES,
/** Force rewriting of SortField using {@link SortField#rewrite(IndexSearcher)}
* before it can be used for sorting */
REWRITEABLE

View File

@ -299,9 +299,6 @@ public class BKDWriter implements Closeable {
final BKDReader.IntersectState state;
final MergeState.DocMap docMap;
/** Base offset for all our docIDs */
final int docIDBase;
/** Current doc ID */
public int docID;
@ -314,7 +311,7 @@ public class BKDWriter implements Closeable {
/** Which leaf block we are up to */
private int blockID;
public MergeReader(BKDReader bkd, MergeState.DocMap docMap, int docIDBase) throws IOException {
public MergeReader(BKDReader bkd, MergeState.DocMap docMap) throws IOException {
this.bkd = bkd;
state = new BKDReader.IntersectState(bkd.in.clone(),
bkd.numDims,
@ -322,7 +319,6 @@ public class BKDWriter implements Closeable {
bkd.maxPointsInLeafNode,
null);
this.docMap = docMap;
this.docIDBase = docIDBase;
long minFP = Long.MAX_VALUE;
//System.out.println("MR.init " + this + " bkdreader=" + bkd + " leafBlockFPs.length=" + bkd.leafBlockFPs.length);
for(long fp : bkd.leafBlockFPs) {
@ -396,14 +392,14 @@ public class BKDWriter implements Closeable {
}
// Tie break by sorting smaller docIDs earlier:
return a.docIDBase < b.docIDBase;
return a.docID < b.docID;
}
}
/** More efficient bulk-add for incoming {@link BKDReader}s. This does a merge sort of the already
* sorted values and currently only works when numDims==1. This returns -1 if all documents containing
* dimensional values were deleted. */
public long merge(IndexOutput out, List<MergeState.DocMap> docMaps, List<BKDReader> readers, List<Integer> docIDBases) throws IOException {
public long merge(IndexOutput out, List<MergeState.DocMap> docMaps, List<BKDReader> readers) throws IOException {
if (numDims != 1) {
throw new UnsupportedOperationException("numDims must be 1 but got " + numDims);
}
@ -411,8 +407,6 @@ public class BKDWriter implements Closeable {
throw new IllegalStateException("cannot mix add and merge");
}
//System.out.println("BKDW.merge segs=" + readers.size());
// Catch user silliness:
if (heapPointWriter == null && tempInput == null) {
throw new IllegalStateException("already finished");
@ -433,7 +427,7 @@ public class BKDWriter implements Closeable {
} else {
docMap = docMaps.get(i);
}
MergeReader reader = new MergeReader(bkd, docMap, docIDBases.get(i));
MergeReader reader = new MergeReader(bkd, docMap);
if (reader.next()) {
queue.add(reader);
}
@ -468,7 +462,7 @@ public class BKDWriter implements Closeable {
// System.out.println("iter reader=" + reader);
// NOTE: doesn't work with subclasses (e.g. SimpleText!)
int docID = reader.docIDBase + reader.docID;
int docID = reader.docID;
leafBlockDocIDs[leafCount] = docID;
System.arraycopy(reader.state.scratchPackedValue, 0, leafBlockPackedValues[leafCount], 0, packedBytesLength);
docsSeen.set(docID);

View File

@ -13,4 +13,4 @@
# See the License for the specific language governing permissions and
# limitations under the License.
org.apache.lucene.codecs.lucene60.Lucene60Codec
org.apache.lucene.codecs.lucene62.Lucene62Codec

View File

@ -19,7 +19,7 @@ package org.apache.lucene.codecs.lucene50;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
import org.apache.lucene.codecs.lucene60.Lucene60Codec;
import org.apache.lucene.codecs.lucene62.Lucene62Codec;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.StoredField;
import org.apache.lucene.index.BaseStoredFieldsFormatTestCase;
@ -33,7 +33,7 @@ import com.carrotsearch.randomizedtesting.generators.RandomPicks;
public class TestLucene50StoredFieldsFormatHighCompression extends BaseStoredFieldsFormatTestCase {
@Override
protected Codec getCodec() {
return new Lucene60Codec(Mode.BEST_COMPRESSION);
return new Lucene62Codec(Mode.BEST_COMPRESSION);
}
/**
@ -44,7 +44,7 @@ public class TestLucene50StoredFieldsFormatHighCompression extends BaseStoredFie
Directory dir = newDirectory();
for (int i = 0; i < 10; i++) {
IndexWriterConfig iwc = newIndexWriterConfig();
iwc.setCodec(new Lucene60Codec(RandomPicks.randomFrom(random(), Mode.values())));
iwc.setCodec(new Lucene62Codec(RandomPicks.randomFrom(random(), Mode.values())));
IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig());
Document doc = new Document();
doc.add(new StoredField("field1", "value1"));
@ -71,7 +71,7 @@ public class TestLucene50StoredFieldsFormatHighCompression extends BaseStoredFie
public void testInvalidOptions() throws Exception {
expectThrows(NullPointerException.class, () -> {
new Lucene60Codec(null);
new Lucene62Codec(null);
});
expectThrows(NullPointerException.class, () -> {

View File

@ -18,14 +18,14 @@ package org.apache.lucene.codecs.lucene53;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.lucene60.Lucene60Codec;
import org.apache.lucene.codecs.lucene62.Lucene62Codec;
import org.apache.lucene.index.BaseNormsFormatTestCase;
/**
* Tests Lucene53NormsFormat
*/
public class TestLucene53NormsFormat extends BaseNormsFormatTestCase {
private final Codec codec = new Lucene60Codec();
private final Codec codec = new Lucene62Codec();
@Override
protected Codec getCodec() {

View File

@ -1,3 +1,5 @@
package org.apache.lucene.codecs.lucene62;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
@ -14,8 +16,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene50;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.index.BaseSegmentInfoFormatTestCase;
@ -23,9 +23,9 @@ import org.apache.lucene.util.TestUtil;
import org.apache.lucene.util.Version;
/**
* Tests Lucene50SegmentInfoFormat
* Tests Lucene62SegmentInfoFormat
*/
public class TestLucene50SegmentInfoFormat extends BaseSegmentInfoFormatTestCase {
public class TestLucene62SegmentInfoFormat extends BaseSegmentInfoFormatTestCase {
@Override
protected Version[] getVersions() {

View File

@ -24,8 +24,6 @@ import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.PointsFormat;
import org.apache.lucene.codecs.PointsReader;
import org.apache.lucene.codecs.PointsWriter;
import org.apache.lucene.codecs.lucene60.Lucene60PointsReader;
import org.apache.lucene.codecs.lucene60.Lucene60PointsWriter;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.LongPoint;
import org.apache.lucene.search.IndexSearcher;
@ -143,6 +141,6 @@ public class Test2BPoints extends LuceneTestCase {
}
private static Codec getCodec() {
return Codec.forName("Lucene60");
return Codec.forName("Lucene62");
}
}

View File

@ -53,7 +53,7 @@ import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
// disk (but, should run successfully). Best to run w/
// -Dtests.codec=<current codec>, and w/ plenty of RAM, eg:
//
// ant test -Dtests.monster=true -Dtests.heapsize=8g -Dtests.codec=Lucene60 -Dtestcase=Test2BTerms
// ant test -Dtests.monster=true -Dtests.heapsize=8g -Dtests.codec=Lucene62 -Dtestcase=Test2BTerms
//
@SuppressCodecs({ "SimpleText", "Memory", "Direct" })
@Monster("very slow, use 5g minimum heap")

View File

@ -39,6 +39,8 @@ import org.apache.lucene.document.TextField;
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.PhraseQuery;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.BaseDirectoryWrapper;
import org.apache.lucene.store.Directory;
@ -1281,4 +1283,53 @@ public class TestAddIndexes extends LuceneTestCase {
w2.close();
IOUtils.close(src, dest);
}
public void testIllegalIndexSortChange1() throws Exception {
Directory dir1 = newDirectory();
IndexWriterConfig iwc1 = newIndexWriterConfig(new MockAnalyzer(random()));
iwc1.setIndexSort(new Sort(new SortField("foo", SortField.Type.INT)));
RandomIndexWriter w1 = new RandomIndexWriter(random(), dir1, iwc1);
w1.addDocument(new Document());
w1.commit();
w1.addDocument(new Document());
w1.commit();
// so the index sort is in fact burned into the index:
w1.forceMerge(1);
w1.close();
Directory dir2 = newDirectory();
IndexWriterConfig iwc2 = newIndexWriterConfig(new MockAnalyzer(random()));
iwc2.setIndexSort(new Sort(new SortField("foo", SortField.Type.STRING)));
RandomIndexWriter w2 = new RandomIndexWriter(random(), dir2, iwc2);
String message = expectThrows(IllegalArgumentException.class, () -> {
w2.addIndexes(dir1);
}).getMessage();
assertEquals("cannot change index sort from <int: \"foo\"> to <string: \"foo\">", message);
IOUtils.close(dir1, w2, dir2);
}
public void testIllegalIndexSortChange2() throws Exception {
Directory dir1 = newDirectory();
IndexWriterConfig iwc1 = newIndexWriterConfig(new MockAnalyzer(random()));
iwc1.setIndexSort(new Sort(new SortField("foo", SortField.Type.INT)));
RandomIndexWriter w1 = new RandomIndexWriter(random(), dir1, iwc1);
w1.addDocument(new Document());
w1.commit();
w1.addDocument(new Document());
w1.commit();
// so the index sort is in fact burned into the index:
w1.forceMerge(1);
w1.close();
Directory dir2 = newDirectory();
IndexWriterConfig iwc2 = newIndexWriterConfig(new MockAnalyzer(random()));
iwc2.setIndexSort(new Sort(new SortField("foo", SortField.Type.STRING)));
RandomIndexWriter w2 = new RandomIndexWriter(random(), dir2, iwc2);
IndexReader r1 = DirectoryReader.open(dir1);
String message = expectThrows(IllegalArgumentException.class, () -> {
w2.addIndexes((SegmentReader) getOnlyLeafReader(r1));
}).getMessage();
assertEquals("cannot change index sort from <int: \"foo\"> to <string: \"foo\">", message);
IOUtils.close(r1, dir1, w2, dir2);
}
}

View File

@ -222,7 +222,7 @@ public class TestCodecs extends LuceneTestCase {
final FieldInfos fieldInfos = builder.finish();
final Directory dir = newDirectory();
Codec codec = Codec.getDefault();
final SegmentInfo si = new SegmentInfo(dir, Version.LATEST, SEGMENT, 10000, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
final SegmentInfo si = new SegmentInfo(dir, Version.LATEST, SEGMENT, 10000, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
this.write(si, fieldInfos, dir, fields);
final FieldsProducer reader = codec.postingsFormat().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, newIOContext(random())));
@ -279,7 +279,7 @@ public class TestCodecs extends LuceneTestCase {
}
Codec codec = Codec.getDefault();
final SegmentInfo si = new SegmentInfo(dir, Version.LATEST, SEGMENT, 10000, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
final SegmentInfo si = new SegmentInfo(dir, Version.LATEST, SEGMENT, 10000, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
this.write(si, fieldInfos, dir, fields);
if (VERBOSE) {

View File

@ -503,7 +503,7 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
class ReindexingOneMerge extends OneMerge {
List<LeafReader> parallelReaders;
final List<ParallelLeafReader> parallelReaders = new ArrayList<>();
final long schemaGen;
ReindexingOneMerge(List<SegmentCommitInfo> segments) {
@ -519,33 +519,23 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
}
@Override
public List<CodecReader> getMergeReaders() throws IOException {
if (parallelReaders == null) {
parallelReaders = new ArrayList<>();
for (CodecReader reader : super.getMergeReaders()) {
parallelReaders.add(getCurrentReader((SegmentReader)reader, schemaGen));
}
public CodecReader wrapForMerge(CodecReader reader) throws IOException {
LeafReader wrapped = getCurrentReader((SegmentReader)reader, schemaGen);
if (wrapped instanceof ParallelLeafReader) {
parallelReaders.add((ParallelLeafReader) wrapped);
}
// TODO: fix ParallelLeafReader, if this is a good use case
List<CodecReader> mergeReaders = new ArrayList<>();
for (LeafReader reader : parallelReaders) {
mergeReaders.add(SlowCodecReaderWrapper.wrap(reader));
}
return mergeReaders;
return SlowCodecReaderWrapper.wrap(wrapped);
}
@Override
public void mergeFinished() throws IOException {
Throwable th = null;
for(LeafReader r : parallelReaders) {
if (r instanceof ParallelLeafReader) {
try {
r.decRef();
} catch (Throwable t) {
if (th == null) {
th = t;
}
for (ParallelLeafReader r : parallelReaders) {
try {
r.decRef();
} catch (Throwable t) {
if (th == null) {
th = t;
}
}
}
@ -561,10 +551,6 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
super.setMergeInfo(info);
}
@Override
public MergePolicy.DocMap getDocMap(final MergeState mergeState) {
return super.getDocMap(mergeState);
}
}
class ReindexingMergeSpecification extends MergeSpecification {

View File

@ -218,7 +218,7 @@ public class TestDoc extends LuceneTestCase {
final Codec codec = Codec.getDefault();
TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(si1.info.dir);
final SegmentInfo si = new SegmentInfo(si1.info.dir, Version.LATEST, merged, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
final SegmentInfo si = new SegmentInfo(si1.info.dir, Version.LATEST, merged, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
SegmentMerger merger = new SegmentMerger(Arrays.<CodecReader>asList(r1, r2),
si, InfoStream.getDefault(), trackingDir,

View File

@ -0,0 +1,205 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.index;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
public class TestDocIDMerger extends LuceneTestCase {
private static class TestSubUnsorted extends DocIDMerger.Sub {
private int docID = -1;
final int valueStart;
final int maxDoc;
public TestSubUnsorted(MergeState.DocMap docMap, int maxDoc, int valueStart) {
super(docMap);
this.maxDoc = maxDoc;
this.valueStart = valueStart;
}
@Override
public int nextDoc() {
docID++;
if (docID == maxDoc) {
return NO_MORE_DOCS;
} else {
return docID;
}
}
public int getValue() {
return valueStart + docID;
}
}
public void testNoSort() throws Exception {
int subCount = TestUtil.nextInt(random(), 1, 20);
List<TestSubUnsorted> subs = new ArrayList<>();
int valueStart = 0;
for(int i=0;i<subCount;i++) {
int maxDoc = TestUtil.nextInt(random(), 1, 1000);
final int docBase = valueStart;
subs.add(new TestSubUnsorted(new MergeState.DocMap() {
@Override
public int get(int docID) {
return docBase + docID;
}
}, maxDoc, valueStart));
valueStart += maxDoc;
}
DocIDMerger<TestSubUnsorted> merger = new DocIDMerger<>(subs, false);
int count = 0;
while (true) {
TestSubUnsorted sub = merger.next();
if (sub == null) {
break;
}
assertEquals(count, sub.mappedDocID);
assertEquals(count, sub.getValue());
count++;
}
assertEquals(valueStart, count);
}
private static class TestSubSorted extends DocIDMerger.Sub {
private int docID = -1;
final int maxDoc;
final int index;
public TestSubSorted(MergeState.DocMap docMap, int maxDoc, int index) {
super(docMap);
this.maxDoc = maxDoc;
this.index = index;
}
@Override
public int nextDoc() {
docID++;
if (docID == maxDoc) {
return NO_MORE_DOCS;
} else {
return docID;
}
}
@Override
public String toString() {
return "TestSubSorted(index=" + index + ", mappedDocID=" + mappedDocID+ ")";
}
}
public void testWithSort() throws Exception {
int subCount = TestUtil.nextInt(random(), 1, 20);
List<int[]> oldToNew = new ArrayList<>();
// how many docs we've written to each sub:
List<Integer> uptos = new ArrayList<>();
int totDocCount = 0;
for(int i=0;i<subCount;i++) {
int maxDoc = TestUtil.nextInt(random(), 1, 1000);
uptos.add(0);
oldToNew.add(new int[maxDoc]);
totDocCount += maxDoc;
}
List<int[]> completedSubs = new ArrayList<>();
// randomly distribute target docIDs into the segments:
for(int docID=0;docID<totDocCount;docID++) {
int sub = random().nextInt(oldToNew.size());
int upto = uptos.get(sub);
int[] subDocs = oldToNew.get(sub);
subDocs[upto] = docID;
upto++;
if (upto == subDocs.length) {
completedSubs.add(subDocs);
oldToNew.remove(sub);
uptos.remove(sub);
} else {
uptos.set(sub, upto);
}
}
assertEquals(0, oldToNew.size());
// sometimes do some deletions:
final FixedBitSet liveDocs;
if (random().nextBoolean()) {
liveDocs = new FixedBitSet(totDocCount);
liveDocs.set(0, totDocCount);
int deleteAttemptCount = TestUtil.nextInt(random(), 1, totDocCount);
for(int i=0;i<deleteAttemptCount;i++) {
liveDocs.clear(random().nextInt(totDocCount));
}
} else {
liveDocs = null;
}
List<TestSubSorted> subs = new ArrayList<>();
for(int i=0;i<subCount;i++) {
final int[] docMap = completedSubs.get(i);
subs.add(new TestSubSorted(new MergeState.DocMap() {
@Override
public int get(int docID) {
int mapped = docMap[docID];
if (liveDocs == null || liveDocs.get(mapped)) {
return mapped;
} else {
return -1;
}
}
}, docMap.length, i));
}
DocIDMerger<TestSubSorted> merger = new DocIDMerger<>(subs, true);
int count = 0;
while (true) {
TestSubSorted sub = merger.next();
if (sub == null) {
break;
}
if (liveDocs != null) {
count = liveDocs.nextSetBit(count);
}
assertEquals(count, sub.mappedDocID);
count++;
}
if (liveDocs != null) {
if (count < totDocCount) {
assertEquals(NO_MORE_DOCS, liveDocs.nextSetBit(count));
} else {
assertEquals(totDocCount, count);
}
} else {
assertEquals(totDocCount, count);
}
}
}

File diff suppressed because it is too large Load Diff

View File

@ -69,6 +69,8 @@ import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.PhraseQuery;
import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.BaseDirectoryWrapper;
@ -2759,5 +2761,6 @@ public class TestIndexWriter extends LuceneTestCase {
w.close();
dir.close();
}
}

View File

@ -23,10 +23,11 @@ import java.util.Random;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.search.BooleanClause.Occur;
import org.apache.lucene.search.*;
import org.apache.lucene.search.BooleanClause.Occur;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
@ -314,4 +315,60 @@ public class TestParallelLeafReader extends LuceneTestCase {
return dir2;
}
// not ok to have one leaf w/ index sort and another with a different index sort
public void testWithIndexSort1() throws Exception {
Directory dir1 = newDirectory();
IndexWriterConfig iwc1 = newIndexWriterConfig(new MockAnalyzer(random()));
iwc1.setIndexSort(new Sort(new SortField("foo", SortField.Type.INT)));
IndexWriter w1 = new IndexWriter(dir1, iwc1);
w1.addDocument(new Document());
w1.commit();
w1.addDocument(new Document());
w1.forceMerge(1);
w1.close();
IndexReader r1 = DirectoryReader.open(dir1);
Directory dir2 = newDirectory();
IndexWriterConfig iwc2 = newIndexWriterConfig(new MockAnalyzer(random()));
iwc2.setIndexSort(new Sort(new SortField("bar", SortField.Type.INT)));
IndexWriter w2 = new IndexWriter(dir2, iwc2);
w2.addDocument(new Document());
w2.commit();
w2.addDocument(new Document());
w2.forceMerge(1);
w2.close();
IndexReader r2 = DirectoryReader.open(dir2);
String message = expectThrows(IllegalArgumentException.class, () -> {
new ParallelLeafReader(getOnlyLeafReader(r1), getOnlyLeafReader(r2));
}).getMessage();
assertEquals("cannot combine LeafReaders that have different index sorts: saw both sort=<int: \"foo\"> and <int: \"bar\">", message);
IOUtils.close(r1, dir1, r2, dir2);
}
// ok to have one leaf w/ index sort and the other with no sort
public void testWithIndexSort2() throws Exception {
Directory dir1 = newDirectory();
IndexWriterConfig iwc1 = newIndexWriterConfig(new MockAnalyzer(random()));
iwc1.setIndexSort(new Sort(new SortField("foo", SortField.Type.INT)));
IndexWriter w1 = new IndexWriter(dir1, iwc1);
w1.addDocument(new Document());
w1.commit();
w1.addDocument(new Document());
w1.forceMerge(1);
w1.close();
IndexReader r1 = DirectoryReader.open(dir1);
Directory dir2 = newDirectory();
IndexWriterConfig iwc2 = newIndexWriterConfig(new MockAnalyzer(random()));
IndexWriter w2 = new IndexWriter(dir2, iwc2);
w2.addDocument(new Document());
w2.addDocument(new Document());
w2.close();
IndexReader r2 = DirectoryReader.open(dir2);
new ParallelLeafReader(false, getOnlyLeafReader(r1), getOnlyLeafReader(r2)).close();
new ParallelLeafReader(false, getOnlyLeafReader(r2), getOnlyLeafReader(r1)).close();
IOUtils.close(r1, dir1, r2, dir2);
}
}

View File

@ -394,11 +394,11 @@ public class TestPointValues extends LuceneTestCase {
dir.close();
}
// Write point values, one segment with Lucene60, another with SimpleText, then forceMerge with SimpleText
// Write point values, one segment with Lucene62, another with SimpleText, then forceMerge with SimpleText
public void testDifferentCodecs1() throws Exception {
Directory dir = newDirectory();
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
iwc.setCodec(Codec.forName("Lucene60"));
iwc.setCodec(Codec.forName("Lucene62"));
IndexWriter w = new IndexWriter(dir, iwc);
Document doc = new Document();
doc.add(new IntPoint("int", 1));
@ -417,7 +417,7 @@ public class TestPointValues extends LuceneTestCase {
dir.close();
}
// Write point values, one segment with Lucene60, another with SimpleText, then forceMerge with Lucene60
// Write point values, one segment with Lucene62, another with SimpleText, then forceMerge with Lucene60
public void testDifferentCodecs2() throws Exception {
Directory dir = newDirectory();
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
@ -429,7 +429,7 @@ public class TestPointValues extends LuceneTestCase {
w.close();
iwc = new IndexWriterConfig(new MockAnalyzer(random()));
iwc.setCodec(Codec.forName("Lucene60"));
iwc.setCodec(Codec.forName("Lucene62"));
w = new IndexWriter(dir, iwc);
doc = new Document();
doc.add(new IntPoint("int", 1));

View File

@ -51,7 +51,7 @@ public class TestSegmentInfos extends LuceneTestCase {
SegmentInfos sis = new SegmentInfos();
SegmentInfo info = new SegmentInfo(dir, Version.LUCENE_6_0_0, "_0", 1, false, Codec.getDefault(),
Collections.<String,String>emptyMap(), id, Collections.<String,String>emptyMap());
Collections.<String,String>emptyMap(), id, Collections.<String,String>emptyMap(), null);
info.setFiles(Collections.<String>emptySet());
codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
SegmentCommitInfo commitInfo = new SegmentCommitInfo(info, 0, -1, -1, -1);
@ -73,14 +73,14 @@ public class TestSegmentInfos extends LuceneTestCase {
SegmentInfos sis = new SegmentInfos();
SegmentInfo info = new SegmentInfo(dir, Version.LUCENE_6_0_0, "_0", 1, false, Codec.getDefault(),
Collections.<String,String>emptyMap(), id, Collections.<String,String>emptyMap());
Collections.<String,String>emptyMap(), id, Collections.<String,String>emptyMap(), null);
info.setFiles(Collections.<String>emptySet());
codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
SegmentCommitInfo commitInfo = new SegmentCommitInfo(info, 0, -1, -1, -1);
sis.add(commitInfo);
info = new SegmentInfo(dir, Version.LUCENE_6_0_0, "_1", 1, false, Codec.getDefault(),
Collections.<String,String>emptyMap(), id, Collections.<String,String>emptyMap());
Collections.<String,String>emptyMap(), id, Collections.<String,String>emptyMap(), null);
info.setFiles(Collections.<String>emptySet());
codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
commitInfo = new SegmentCommitInfo(info, 0, -1, -1, -1);

View File

@ -35,6 +35,7 @@ import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.StringHelper;
import org.apache.lucene.util.TestUtil;
import org.apache.lucene.util.Version;
import org.apache.lucene.util.packed.PackedLongValues;
public class TestSegmentMerger extends LuceneTestCase {
//The variables for the new merged segment
@ -83,7 +84,7 @@ public class TestSegmentMerger extends LuceneTestCase {
public void testMerge() throws IOException {
final Codec codec = Codec.getDefault();
final SegmentInfo si = new SegmentInfo(mergedDir, Version.LATEST, mergedSegment, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
final SegmentInfo si = new SegmentInfo(mergedDir, Version.LATEST, mergedSegment, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
SegmentMerger merger = new SegmentMerger(Arrays.<CodecReader>asList(reader1, reader2),
si, InfoStream.getDefault(), mergedDir,
@ -144,22 +145,9 @@ public class TestSegmentMerger extends LuceneTestCase {
mergedReader.close();
}
private static boolean equals(MergeState.DocMap map1, MergeState.DocMap map2) {
if (map1.maxDoc() != map2.maxDoc()) {
return false;
}
for (int i = 0; i < map1.maxDoc(); ++i) {
if (map1.get(i) != map2.get(i)) {
return false;
}
}
return true;
}
public void testBuildDocMap() {
final int maxDoc = TestUtil.nextInt(random(), 1, 128);
final int numDocs = TestUtil.nextInt(random(), 0, maxDoc);
final int numDeletedDocs = maxDoc - numDocs;
final FixedBitSet liveDocs = new FixedBitSet(maxDoc);
for (int i = 0; i < numDocs; ++i) {
while (true) {
@ -171,15 +159,11 @@ public class TestSegmentMerger extends LuceneTestCase {
}
}
final MergeState.DocMap docMap = MergeState.DocMap.build(maxDoc, liveDocs);
final PackedLongValues docMap = MergeState.removeDeletes(maxDoc, liveDocs);
assertEquals(maxDoc, docMap.maxDoc());
assertEquals(numDocs, docMap.numDocs());
assertEquals(numDeletedDocs, docMap.numDeletedDocs());
// assert the mapping is compact
for (int i = 0, del = 0; i < maxDoc; ++i) {
if (!liveDocs.get(i)) {
assertEquals(-1, docMap.get(i));
if (liveDocs.get(i) == false) {
++del;
} else {
assertEquals(i - del, docMap.get(i));

View File

@ -25,23 +25,23 @@ import java.util.Random;
import java.util.Set;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field.Store;
import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.document.StringField;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.ExitableDirectoryReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.MockRandomMergePolicy;
import org.apache.lucene.index.QueryTimeout;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.SerialMergeScheduler;
import org.apache.lucene.index.SortingMergePolicy;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.TestSortingMergePolicy;
import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.ScoreDoc;
@ -50,8 +50,6 @@ import org.apache.lucene.search.SortField;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.TopFieldCollector;
import org.apache.lucene.store.Directory;
import org.apache.lucene.uninverting.UninvertingReader;
import org.apache.lucene.uninverting.UninvertingReader.Type;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
@ -62,18 +60,11 @@ public class TestEarlyTerminatingSortingCollector extends LuceneTestCase {
private int numDocs;
private List<String> terms;
private Directory dir;
private Sort sort;
private final Sort sort = new Sort(new SortField("ndv1", SortField.Type.LONG));
private RandomIndexWriter iw;
private IndexReader reader;
private SortingMergePolicy mergePolicy;
private final int forceMergeMaxSegmentCount = 5;
@Override
public void setUp() throws Exception {
super.setUp();
sort = new Sort(new SortField("ndv1", SortField.Type.LONG));
}
private Document randomDocument() {
final Document doc = new Document();
doc.add(new NumericDocValuesField("ndv1", random().nextInt(10)));
@ -93,9 +84,12 @@ public class TestEarlyTerminatingSortingCollector extends LuceneTestCase {
terms = new ArrayList<>(randomTerms);
final long seed = random().nextLong();
final IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(new Random(seed)));
if (iwc.getMergePolicy() instanceof MockRandomMergePolicy) {
// MockRandomMP randomly wraps the leaf readers which makes merging angry
iwc.setMergePolicy(newTieredMergePolicy());
}
iwc.setMergeScheduler(new SerialMergeScheduler()); // for reproducible tests
mergePolicy = TestSortingMergePolicy.newSortingMergePolicy(sort);
iwc.setMergePolicy(mergePolicy);
iwc.setIndexSort(sort);
iw = new RandomIndexWriter(new Random(seed), dir, iwc);
iw.setDoRandomForceMerge(false); // don't do this, it may happen anyway with MockRandomMP
for (int i = 0; i < numDocs; ++i) {
@ -151,7 +145,7 @@ public class TestEarlyTerminatingSortingCollector extends LuceneTestCase {
query = new MatchAllDocsQuery();
}
searcher.search(query, collector1);
searcher.search(query, new EarlyTerminatingSortingCollector(collector2, sort, numHits, mergePolicy.getSort()));
searcher.search(query, new EarlyTerminatingSortingCollector(collector2, sort, numHits));
assertTrue(collector1.getTotalHits() >= collector2.getTotalHits());
assertTopDocsEquals(collector1.topDocs().scoreDocs, collector2.topDocs().scoreDocs);
}
@ -190,40 +184,16 @@ public class TestEarlyTerminatingSortingCollector extends LuceneTestCase {
}
public void testEarlyTerminationDifferentSorter() throws IOException {
createRandomIndex(false);
final int iters = atLeast(3);
for (int i = 0; i < iters; ++i) {
final IndexSearcher searcher = newSearcher(reader);
// test that the collector works correctly when the index was sorted by a
// different sorter than the one specified in the ctor.
final int numHits = TestUtil.nextInt(random(), 1, numDocs);
final Sort sort = new Sort(new SortField("ndv2", SortField.Type.LONG, false));
final boolean fillFields = random().nextBoolean();
final boolean trackDocScores = random().nextBoolean();
final boolean trackMaxScore = random().nextBoolean();
final TopFieldCollector collector1 = TopFieldCollector.create(sort, numHits, fillFields, trackDocScores, trackMaxScore);
final TopFieldCollector collector2 = TopFieldCollector.create(sort, numHits, fillFields, trackDocScores, trackMaxScore);
final Query query;
if (random().nextBoolean()) {
query = new TermQuery(new Term("s", RandomPicks.randomFrom(random(), terms)));
} else {
query = new MatchAllDocsQuery();
}
searcher.search(query, collector1);
Sort different = new Sort(new SortField("ndv2", SortField.Type.LONG));
createRandomIndex(true);
searcher.search(query, new EarlyTerminatingSortingCollector(collector2, different, numHits, different) {
@Override
public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
final LeafCollector ret = super.getLeafCollector(context);
assertTrue("segment should not be recognized as sorted as different sorter was used", ret.getClass() == in.getLeafCollector(context).getClass());
return ret;
}
});
assertTrue(collector1.getTotalHits() >= collector2.getTotalHits());
assertTopDocsEquals(collector1.topDocs().scoreDocs, collector2.topDocs().scoreDocs);
}
Sort sort = new Sort(new SortField("ndv2", SortField.Type.LONG, false));
Collector c = new EarlyTerminatingSortingCollector(TopFieldCollector.create(sort, 10, true, true, true), sort, 10);
IndexSearcher searcher = newSearcher(reader);
Exception e = expectThrows(IllegalStateException.class,
() -> {
searcher.search(new MatchAllDocsQuery(), c);
});
assertEquals("Cannot early terminate with sort order <long: \"ndv2\"> if segments are sorted with <long: \"ndv1\">", e.getMessage());
closeIndex();
}
@ -262,34 +232,19 @@ public class TestEarlyTerminatingSortingCollector extends LuceneTestCase {
}
}
private IndexSearcher newSearcherForTestTerminatedEarly(IndexReader r) throws IOException {
switch(random().nextInt(2)) {
case 0:
return new IndexSearcher(r);
case 1:
assertTrue(r+" is not a DirectoryReader", (r instanceof DirectoryReader));
final DirectoryReader directoryReader = ExitableDirectoryReader.wrap(
UninvertingReader.wrap((DirectoryReader) r, new HashMap<String,Type>()),
new TestEarlyTerminatingSortingcollectorQueryTimeout(false));
return new IndexSearcher(directoryReader);
}
fail("newSearcherForTestTerminatedEarly("+r+") fell through switch");
return null;
}
public void testTerminatedEarly() throws IOException {
final int iters = atLeast(8);
for (int i = 0; i < iters; ++i) {
createRandomIndex(true);
final IndexSearcher searcher = newSearcherForTestTerminatedEarly(reader); // future TODO: use newSearcher(reader);
final IndexSearcher searcher = new IndexSearcher(reader); // future TODO: use newSearcher(reader);
final Query query = new MatchAllDocsQuery(); // search for everything/anything
final TestTerminatedEarlySimpleCollector collector1 = new TestTerminatedEarlySimpleCollector();
searcher.search(query, collector1);
final TestTerminatedEarlySimpleCollector collector2 = new TestTerminatedEarlySimpleCollector();
final EarlyTerminatingSortingCollector etsCollector = new EarlyTerminatingSortingCollector(collector2, sort, 1, mergePolicy.getSort());
final EarlyTerminatingSortingCollector etsCollector = new EarlyTerminatingSortingCollector(collector2, sort, 1);
searcher.search(query, etsCollector);
assertTrue("collector1="+collector1.collectedSomething()+" vs. collector2="+collector2.collectedSomething(), collector1.collectedSomething() == collector2.collectedSomething());

View File

@ -1151,14 +1151,14 @@ public class TestPointQueries extends LuceneTestCase {
}
private static Codec getCodec() {
if (Codec.getDefault().getName().equals("Lucene60")) {
if (Codec.getDefault().getName().equals("Lucene62")) {
int maxPointsInLeafNode = TestUtil.nextInt(random(), 16, 2048);
double maxMBSortInHeap = 5.0 + (3*random().nextDouble());
if (VERBOSE) {
System.out.println("TEST: using Lucene60PointsFormat with maxPointsInLeafNode=" + maxPointsInLeafNode + " and maxMBSortInHeap=" + maxMBSortInHeap);
}
return new FilterCodec("Lucene60", Codec.getDefault()) {
return new FilterCodec("Lucene62", Codec.getDefault()) {
@Override
public PointsFormat pointsFormat() {
return new PointsFormat() {

View File

@ -25,6 +25,7 @@ import java.util.BitSet;
import java.util.List;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.PointValues.IntersectVisitor;
import org.apache.lucene.index.PointValues.Relation;
import org.apache.lucene.store.CorruptingIndexOutput;
@ -554,7 +555,7 @@ public class TestBKD extends LuceneTestCase {
}
List<Long> toMerge = null;
List<Integer> docIDBases = null;
List<MergeState.DocMap> docMaps = null;
int seg = 0;
BKDWriter w = new BKDWriter(numValues, dir, "_" + seg, numDims, numBytesPerDim, maxPointsInLeafNode, maxMB, docValues.length, false);
@ -601,9 +602,15 @@ public class TestBKD extends LuceneTestCase {
if (useMerge && segCount == valuesInThisSeg) {
if (toMerge == null) {
toMerge = new ArrayList<>();
docIDBases = new ArrayList<>();
docMaps = new ArrayList<>();
}
docIDBases.add(lastDocIDBase);
final int curDocIDBase = lastDocIDBase;
docMaps.add(new MergeState.DocMap() {
@Override
public int get(int docID) {
return curDocIDBase + docID;
}
});
toMerge.add(w.finish(out));
valuesInThisSeg = TestUtil.nextInt(random(), numValues/10, numValues/2);
segCount = 0;
@ -620,8 +627,14 @@ public class TestBKD extends LuceneTestCase {
if (toMerge != null) {
if (segCount > 0) {
docIDBases.add(lastDocIDBase);
toMerge.add(w.finish(out));
final int curDocIDBase = lastDocIDBase;
docMaps.add(new MergeState.DocMap() {
@Override
public int get(int docID) {
return curDocIDBase + docID;
}
});
}
out.close();
in = dir.openInput("bkd", IOContext.DEFAULT);
@ -633,7 +646,7 @@ public class TestBKD extends LuceneTestCase {
readers.add(new BKDReader(in));
}
out = dir.createOutput("bkd2", IOContext.DEFAULT);
indexFP = w.merge(out, null, readers, docIDBases);
indexFP = w.merge(out, docMaps, readers);
out.close();
in.close();
in = dir.openInput("bkd2", IOContext.DEFAULT);

View File

@ -21,7 +21,6 @@ import java.util.Collections;
import java.util.Iterator;
import org.apache.lucene.index.BinaryDocValues;
import org.apache.lucene.index.PointValues;
import org.apache.lucene.index.DocValuesType;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
@ -29,11 +28,13 @@ import org.apache.lucene.index.Fields;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.PointValues;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.SortedNumericDocValues;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.index.StoredFieldVisitor;
import org.apache.lucene.index.Terms;
import org.apache.lucene.search.Sort;
import org.apache.lucene.util.Bits;
/**
@ -178,4 +179,8 @@ public class TermVectorLeafReader extends LeafReader {
public void document(int docID, StoredFieldVisitor visitor) throws IOException {
}
@Override
public Sort getIndexSort() {
return null;
}
}

View File

@ -40,6 +40,7 @@ import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.SimpleCollector;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.similarities.Similarity;
import org.apache.lucene.store.RAMDirectory;
import org.apache.lucene.util.*;
@ -1606,6 +1607,10 @@ public class MemoryIndex {
return info.getNormDocValues();
}
@Override
public Sort getIndexSort() {
return null;
}
}
/**

View File

@ -140,7 +140,7 @@ public class IndexSplitter {
SegmentInfo info = infoPerCommit.info;
// Same info just changing the dir:
SegmentInfo newInfo = new SegmentInfo(destFSDir, info.getVersion(), info.name, info.maxDoc(),
info.getUseCompoundFile(), info.getCodec(), info.getDiagnostics(), info.getId(), new HashMap<>());
info.getUseCompoundFile(), info.getCodec(), info.getDiagnostics(), info.getId(), new HashMap<>(), null);
destInfos.add(new SegmentCommitInfo(newInfo, infoPerCommit.getDelCount(),
infoPerCommit.getDelGen(), infoPerCommit.getFieldInfosGen(),
infoPerCommit.getDocValuesGen()));

View File

@ -24,6 +24,7 @@ import java.util.Map;
import org.apache.lucene.index.MultiDocValues.MultiSortedDocValues;
import org.apache.lucene.index.MultiDocValues.MultiSortedSetDocValues;
import org.apache.lucene.index.MultiDocValues.OrdinalMap;
import org.apache.lucene.search.Sort;
import org.apache.lucene.util.Bits;
/**
@ -272,4 +273,9 @@ public final class SlowCompositeReaderWrapper extends LeafReader {
ctx.reader().checkIntegrity();
}
}
@Override
public Sort getIndexSort() {
return null;
}
}

View File

@ -1,264 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.index;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.MergePolicy;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.MergeTrigger;
import org.apache.lucene.index.MultiReader;
import org.apache.lucene.index.SegmentCommitInfo;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentInfos;
import org.apache.lucene.index.SegmentReader;
import org.apache.lucene.index.SlowCompositeReaderWrapper;
import org.apache.lucene.search.Sort;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.InfoStream;
import org.apache.lucene.util.packed.PackedInts;
import org.apache.lucene.util.packed.PackedLongValues;
/** A {@link MergePolicy} that reorders documents according to a {@link Sort}
* before merging them. As a consequence, all segments resulting from a merge
* will be sorted while segments resulting from a flush will be in the order
* in which documents have been added.
* <p><b>NOTE</b>: Never use this policy if you rely on
* {@link IndexWriter#addDocuments(Iterable) IndexWriter.addDocuments}
* to have sequentially-assigned doc IDs, this policy will scatter doc IDs.
* <p><b>NOTE</b>: This policy should only be used with idempotent {@code Sort}s
* so that the order of segments is predictable. For example, using
* {@link Sort#INDEXORDER} in reverse (which is not idempotent) will make
* the order of documents in a segment depend on the number of times the segment
* has been merged.
* @lucene.experimental */
public final class SortingMergePolicy extends MergePolicyWrapper {
/**
* Put in the {@link SegmentInfo#getDiagnostics() diagnostics} to denote that
* this segment is sorted.
*/
public static final String SORTER_ID_PROP = "sorter";
class SortingOneMerge extends OneMerge {
List<CodecReader> unsortedReaders;
Sorter.DocMap docMap;
LeafReader sortedView;
final InfoStream infoStream;
SortingOneMerge(List<SegmentCommitInfo> segments, InfoStream infoStream) {
super(segments);
this.infoStream = infoStream;
}
@Override
public List<CodecReader> getMergeReaders() throws IOException {
if (unsortedReaders == null) {
unsortedReaders = super.getMergeReaders();
if (infoStream.isEnabled("SMP")) {
infoStream.message("SMP", "sorting " + unsortedReaders);
for (LeafReader leaf : unsortedReaders) {
String sortDescription = getSortDescription(leaf);
if (sortDescription == null) {
sortDescription = "not sorted";
}
infoStream.message("SMP", "seg=" + leaf + " " + sortDescription);
}
}
// wrap readers, to be optimal for merge;
List<LeafReader> wrapped = new ArrayList<>(unsortedReaders.size());
for (LeafReader leaf : unsortedReaders) {
if (leaf instanceof SegmentReader) {
leaf = new MergeReaderWrapper((SegmentReader)leaf);
}
wrapped.add(leaf);
}
final LeafReader atomicView;
if (wrapped.size() == 1) {
atomicView = wrapped.get(0);
} else {
final CompositeReader multiReader = new MultiReader(wrapped.toArray(new LeafReader[wrapped.size()]));
atomicView = new SlowCompositeReaderWrapper(multiReader, true);
}
docMap = sorter.sort(atomicView);
sortedView = SortingLeafReader.wrap(atomicView, docMap);
}
// a null doc map means that the readers are already sorted
if (docMap == null) {
if (infoStream.isEnabled("SMP")) {
infoStream.message("SMP", "readers already sorted, omitting sort");
}
return unsortedReaders;
} else {
if (infoStream.isEnabled("SMP")) {
infoStream.message("SMP", "sorting readers by " + sort);
}
return Collections.singletonList(SlowCodecReaderWrapper.wrap(sortedView));
}
}
@Override
public void setMergeInfo(SegmentCommitInfo info) {
Map<String,String> diagnostics = info.info.getDiagnostics();
diagnostics.put(SORTER_ID_PROP, sorter.getID());
super.setMergeInfo(info);
}
private PackedLongValues getDeletes(List<CodecReader> readers) {
PackedLongValues.Builder deletes = PackedLongValues.monotonicBuilder(PackedInts.COMPACT);
int deleteCount = 0;
for (LeafReader reader : readers) {
final int maxDoc = reader.maxDoc();
final Bits liveDocs = reader.getLiveDocs();
for (int i = 0; i < maxDoc; ++i) {
if (liveDocs != null && !liveDocs.get(i)) {
++deleteCount;
} else {
deletes.add(deleteCount);
}
}
}
return deletes.build();
}
@Override
public MergePolicy.DocMap getDocMap(final MergeState mergeState) {
if (unsortedReaders == null) {
throw new IllegalStateException();
}
if (docMap == null) {
return super.getDocMap(mergeState);
}
assert mergeState.docMaps.length == 1; // we returned a singleton reader
final PackedLongValues deletes = getDeletes(unsortedReaders);
return new MergePolicy.DocMap() {
@Override
public int map(int old) {
final int oldWithDeletes = old + (int) deletes.get(old);
final int newWithDeletes = docMap.oldToNew(oldWithDeletes);
return mergeState.docMaps[0].get(newWithDeletes);
}
};
}
@Override
public String toString() {
return "SortingMergePolicy.SortingOneMerge(segments=" + segString() + " sort=" + sort + ")";
}
}
class SortingMergeSpecification extends MergeSpecification {
final InfoStream infoStream;
SortingMergeSpecification(InfoStream infoStream) {
this.infoStream = infoStream;
}
@Override
public void add(OneMerge merge) {
super.add(new SortingOneMerge(merge.segments, infoStream));
}
@Override
public String segString(Directory dir) {
return "SortingMergeSpec(" + super.segString(dir) + ", sorter=" + sorter + ")";
}
}
/** Returns {@code true} if the given {@code reader} is sorted by the
* {@code sort} given. Typically the given {@code sort} would be the
* {@link SortingMergePolicy#getSort()} order of a {@link SortingMergePolicy}. */
public static boolean isSorted(LeafReader reader, Sort sort) {
String description = getSortDescription(reader);
if (description != null && description.equals(sort.toString())) {
return true;
}
return false;
}
private static String getSortDescription(LeafReader reader) {
if (reader instanceof SegmentReader) {
final SegmentReader segReader = (SegmentReader) reader;
final Map<String, String> diagnostics = segReader.getSegmentInfo().info.getDiagnostics();
if (diagnostics != null) {
return diagnostics.get(SORTER_ID_PROP);
}
} else if (reader instanceof FilterLeafReader) {
return getSortDescription(FilterLeafReader.unwrap(reader));
}
return null;
}
private MergeSpecification sortedMergeSpecification(MergeSpecification specification, InfoStream infoStream) {
if (specification == null) {
return null;
}
MergeSpecification sortingSpec = new SortingMergeSpecification(infoStream);
for (OneMerge merge : specification.merges) {
sortingSpec.add(merge);
}
return sortingSpec;
}
final Sorter sorter;
final Sort sort;
/** Create a new {@code MergePolicy} that sorts documents with the given {@code sort}. */
public SortingMergePolicy(MergePolicy in, Sort sort) {
super(in);
this.sorter = new Sorter(sort);
this.sort = sort;
}
/** Return the {@link Sort} order that is used to sort segments when merging. */
public Sort getSort() {
return sort;
}
@Override
public MergeSpecification findMerges(MergeTrigger mergeTrigger,
SegmentInfos segmentInfos, IndexWriter writer) throws IOException {
return sortedMergeSpecification(in.findMerges(mergeTrigger, segmentInfos, writer), writer.infoStream);
}
@Override
public MergeSpecification findForcedMerges(SegmentInfos segmentInfos,
int maxSegmentCount, Map<SegmentCommitInfo,Boolean> segmentsToMerge, IndexWriter writer)
throws IOException {
return sortedMergeSpecification(in.findForcedMerges(segmentInfos, maxSegmentCount, segmentsToMerge, writer), writer.infoStream);
}
@Override
public MergeSpecification findForcedDeletesMerges(SegmentInfos segmentInfos, IndexWriter writer)
throws IOException {
return sortedMergeSpecification(in.findForcedDeletesMerges(segmentInfos, writer), writer.infoStream);
}
@Override
public String toString() {
return "SortingMergePolicy(" + in + ", sorter=" + sorter + ")";
}
}

View File

@ -18,23 +18,5 @@
<html>
<body>
Misc index tools and index support.
SortingMergePolicy:
<p>Provides index sorting capablities. The application can use any
Sort specification, e.g. to sort by fields using DocValues or FieldCache, or to
reverse the order of the documents (by using SortField.Type.DOC in reverse).
Multi-level sorts can be specified the same way you would when searching, by
building Sort from multiple SortFields.
<p>{@link org.apache.lucene.index.SortingMergePolicy} can be used to
make Lucene sort segments before merging them. This will ensure that every
segment resulting from a merge will be sorted according to the provided
{@link org.apache.lucene.search.Sort}. This however makes merging and
thus indexing slower.
<p>Sorted segments allow for early query termination when the sort order
matches index order. This makes query execution faster since not all documents
need to be visited. Please note that this is an expert feature and should not
be used without a deep understanding of Lucene merging and document collection.
</body>
</html>

View File

@ -1,224 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.search;
import java.io.IOException;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.ReaderUtil;
import org.apache.lucene.index.SortingMergePolicy;
import org.apache.lucene.util.BitSet;
/**
* Helper class to sort readers that contain blocks of documents.
* <p>
* Note that this class is intended to used with {@link SortingMergePolicy},
* and for other purposes has some limitations:
* <ul>
* <li>Cannot yet be used with {@link IndexSearcher#searchAfter(ScoreDoc, Query, int, Sort) IndexSearcher.searchAfter}
* <li>Filling sort field values is not yet supported.
* </ul>
* @lucene.experimental
*/
// TODO: can/should we clean this thing up (e.g. return a proper sort value)
// and move to the join/ module?
public class BlockJoinComparatorSource extends FieldComparatorSource {
final Query parentsFilter;
final Sort parentSort;
final Sort childSort;
/**
* Create a new BlockJoinComparatorSource, sorting only blocks of documents
* with {@code parentSort} and not reordering children with a block.
*
* @param parentsFilter Filter identifying parent documents
* @param parentSort Sort for parent documents
*/
public BlockJoinComparatorSource(Query parentsFilter, Sort parentSort) {
this(parentsFilter, parentSort, new Sort(SortField.FIELD_DOC));
}
/**
* Create a new BlockJoinComparatorSource, specifying the sort order for both
* blocks of documents and children within a block.
*
* @param parentsFilter Filter identifying parent documents
* @param parentSort Sort for parent documents
* @param childSort Sort for child documents in the same block
*/
public BlockJoinComparatorSource(Query parentsFilter, Sort parentSort, Sort childSort) {
this.parentsFilter = parentsFilter;
this.parentSort = parentSort;
this.childSort = childSort;
}
@Override
@SuppressWarnings({"unchecked", "rawtypes"})
public FieldComparator<Integer> newComparator(String fieldname, int numHits, int sortPos, boolean reversed) throws IOException {
// we keep parallel slots: the parent ids and the child ids
final int parentSlots[] = new int[numHits];
final int childSlots[] = new int[numHits];
SortField parentFields[] = parentSort.getSort();
final int parentReverseMul[] = new int[parentFields.length];
final FieldComparator<?> parentComparators[] = new FieldComparator[parentFields.length];
for (int i = 0; i < parentFields.length; i++) {
parentReverseMul[i] = parentFields[i].getReverse() ? -1 : 1;
parentComparators[i] = parentFields[i].getComparator(1, i);
}
SortField childFields[] = childSort.getSort();
final int childReverseMul[] = new int[childFields.length];
final FieldComparator<?> childComparators[] = new FieldComparator[childFields.length];
for (int i = 0; i < childFields.length; i++) {
childReverseMul[i] = childFields[i].getReverse() ? -1 : 1;
childComparators[i] = childFields[i].getComparator(1, i);
}
// NOTE: we could return parent ID as value but really our sort "value" is more complex...
// So we throw UOE for now. At the moment you really should only use this at indexing time.
return new FieldComparator<Integer>() {
int bottomParent;
int bottomChild;
BitSet parentBits;
LeafFieldComparator[] parentLeafComparators;
LeafFieldComparator[] childLeafComparators;
@Override
public int compare(int slot1, int slot2) {
try {
return compare(childSlots[slot1], parentSlots[slot1], childSlots[slot2], parentSlots[slot2]);
} catch (IOException e) {
throw new RuntimeException(e);
}
}
@Override
public void setTopValue(Integer value) {
// we dont have enough information (the docid is needed)
throw new UnsupportedOperationException("this comparator cannot be used with deep paging");
}
@Override
public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
if (parentBits != null) {
throw new IllegalStateException("This comparator can only be used on a single segment");
}
IndexSearcher searcher = new IndexSearcher(ReaderUtil.getTopLevelContext(context));
searcher.setQueryCache(null);
final Weight weight = searcher.createNormalizedWeight(parentsFilter, false);
final Scorer parents = weight.scorer(context);
if (parents == null) {
throw new IllegalStateException("LeafReader " + context.reader() + " contains no parents!");
}
parentBits = BitSet.of(parents.iterator(), context.reader().maxDoc());
parentLeafComparators = new LeafFieldComparator[parentComparators.length];
for (int i = 0; i < parentComparators.length; i++) {
parentLeafComparators[i] = parentComparators[i].getLeafComparator(context);
}
childLeafComparators = new LeafFieldComparator[childComparators.length];
for (int i = 0; i < childComparators.length; i++) {
childLeafComparators[i] = childComparators[i].getLeafComparator(context);
}
return new LeafFieldComparator() {
@Override
public int compareBottom(int doc) throws IOException {
return compare(bottomChild, bottomParent, doc, parent(doc));
}
@Override
public int compareTop(int doc) throws IOException {
// we dont have enough information (the docid is needed)
throw new UnsupportedOperationException("this comparator cannot be used with deep paging");
}
@Override
public void copy(int slot, int doc) throws IOException {
childSlots[slot] = doc;
parentSlots[slot] = parent(doc);
}
@Override
public void setBottom(int slot) {
bottomParent = parentSlots[slot];
bottomChild = childSlots[slot];
}
@Override
public void setScorer(Scorer scorer) {
for (LeafFieldComparator comp : parentLeafComparators) {
comp.setScorer(scorer);
}
for (LeafFieldComparator comp : childLeafComparators) {
comp.setScorer(scorer);
}
}
};
}
@Override
public Integer value(int slot) {
// really our sort "value" is more complex...
throw new UnsupportedOperationException("filling sort field values is not yet supported");
}
int parent(int doc) {
return parentBits.nextSetBit(doc);
}
int compare(int docID1, int parent1, int docID2, int parent2) throws IOException {
if (parent1 == parent2) { // both are in the same block
if (docID1 == parent1 || docID2 == parent2) {
// keep parents at the end of blocks
return docID1 - docID2;
} else {
return compare(docID1, docID2, childLeafComparators, childReverseMul);
}
} else {
int cmp = compare(parent1, parent2, parentLeafComparators, parentReverseMul);
if (cmp == 0) {
return parent1 - parent2;
} else {
return cmp;
}
}
}
int compare(int docID1, int docID2, LeafFieldComparator comparators[], int reverseMul[]) throws IOException {
for (int i = 0; i < comparators.length; i++) {
// TODO: would be better if copy() didnt cause a term lookup in TermOrdVal & co,
// the segments are always the same here...
comparators[i].copy(0, docID1);
comparators[i].setBottom(0);
int comp = reverseMul[i] * comparators[i].compareBottom(docID2);
if (comp != 0) {
return comp;
}
}
return 0; // no need to docid tiebreak
}
};
}
@Override
public String toString() {
return "blockJoin(parentSort=" + parentSort + ",childSort=" + childSort + ")";
}
}

View File

@ -1,89 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.index;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.SlowCompositeReaderWrapper;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.TestUtil;
import org.junit.BeforeClass;
public class IndexSortingTest extends SorterTestBase {
private static final Sort[] SORT = new Sort[] {
new Sort(new SortField(NUMERIC_DV_FIELD, SortField.Type.LONG)),
new Sort(new SortField(null, SortField.Type.DOC, true))
};
@BeforeClass
public static void beforeClassSorterUtilTest() throws Exception {
// NOTE: index was created by by super's @BeforeClass
// only read the values of the undeleted documents, since after addIndexes,
// the deleted ones will be dropped from the index.
Bits liveDocs = unsortedReader.getLiveDocs();
List<Integer> values = new ArrayList<>();
for (int i = 0; i < unsortedReader.maxDoc(); i++) {
if (liveDocs == null || liveDocs.get(i)) {
values.add(Integer.valueOf(unsortedReader.document(i).get(ID_FIELD)));
}
}
int idx = random().nextInt(SORT.length);
Sort sorter = SORT[idx];
if (idx == 1) { // reverse doc sort
Collections.reverse(values);
} else {
Collections.sort(values);
if (random().nextBoolean()) {
sorter = new Sort(new SortField(NUMERIC_DV_FIELD, SortField.Type.LONG, true)); // descending
Collections.reverse(values);
}
}
sortedValues = values.toArray(new Integer[values.size()]);
if (VERBOSE) {
System.out.println("sortedValues: " + sortedValues);
System.out.println("Sorter: " + sorter);
}
Directory target = newDirectory();
IndexWriter writer = new IndexWriter(target, newIndexWriterConfig(null));
LeafReader reader = SortingLeafReader.wrap(unsortedReader, sorter);
writer.addIndexes(SlowCodecReaderWrapper.wrap(reader));
writer.close();
// NOTE: also closes unsortedReader
reader.close();
dir.close();
// CheckIndex the target directory
dir = target;
TestUtil.checkIndex(dir);
// set reader for tests
sortedReader = SlowCompositeReaderWrapper.wrap(DirectoryReader.open(dir));
assertFalse("index should not have deletions", sortedReader.hasDeletions());
}
}

View File

@ -1,405 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.index;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Random;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
import org.apache.lucene.document.BinaryDocValuesField;
import org.apache.lucene.document.BinaryPoint;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field.Store;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.document.SortedDocValuesField;
import org.apache.lucene.document.SortedNumericDocValuesField;
import org.apache.lucene.document.SortedSetDocValuesField;
import org.apache.lucene.document.StringField;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.PointValues.IntersectVisitor;
import org.apache.lucene.index.PointValues.Relation;
import org.apache.lucene.index.SortingLeafReader.SortingDocsEnum;
import org.apache.lucene.index.TermsEnum.SeekStatus;
import org.apache.lucene.search.CollectionStatistics;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.TermStatistics;
import org.apache.lucene.search.similarities.Similarity;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.NumericUtils;
import org.apache.lucene.util.TestUtil;
import org.junit.AfterClass;
import org.junit.BeforeClass;
public abstract class SorterTestBase extends LuceneTestCase {
static final class NormsSimilarity extends Similarity {
private final Similarity in;
public NormsSimilarity(Similarity in) {
this.in = in;
}
@Override
public long computeNorm(FieldInvertState state) {
if (state.getName().equals(NORMS_FIELD)) {
return Float.floatToIntBits(state.getBoost());
} else {
return in.computeNorm(state);
}
}
@Override
public SimWeight computeWeight(CollectionStatistics collectionStats, TermStatistics... termStats) {
return in.computeWeight(collectionStats, termStats);
}
@Override
public SimScorer simScorer(SimWeight weight, LeafReaderContext context) throws IOException {
return in.simScorer(weight, context);
}
}
static final class PositionsTokenStream extends TokenStream {
private final CharTermAttribute term;
private final PayloadAttribute payload;
private final OffsetAttribute offset;
private int pos, off;
public PositionsTokenStream() {
term = addAttribute(CharTermAttribute.class);
payload = addAttribute(PayloadAttribute.class);
offset = addAttribute(OffsetAttribute.class);
}
@Override
public boolean incrementToken() throws IOException {
if (pos == 0) {
return false;
}
clearAttributes();
term.append(DOC_POSITIONS_TERM);
payload.setPayload(new BytesRef(Integer.toString(pos)));
offset.setOffset(off, off);
--pos;
++off;
return true;
}
void setId(int id) {
pos = id / 10 + 1;
off = 0;
}
}
protected static final String ID_FIELD = "id";
protected static final String DOCS_ENUM_FIELD = "docs";
protected static final String DOCS_ENUM_TERM = "$all$";
protected static final String DOC_POSITIONS_FIELD = "positions";
protected static final String DOC_POSITIONS_TERM = "$all$";
protected static final String NUMERIC_DV_FIELD = "numeric";
protected static final String SORTED_NUMERIC_DV_FIELD = "sorted_numeric";
protected static final String NORMS_FIELD = "norm";
protected static final String BINARY_DV_FIELD = "binary";
protected static final String SORTED_DV_FIELD = "sorted";
protected static final String SORTED_SET_DV_FIELD = "sorted_set";
protected static final String TERM_VECTORS_FIELD = "term_vectors";
protected static final String DIMENSIONAL_FIELD = "numeric1d";
private static final FieldType TERM_VECTORS_TYPE = new FieldType(TextField.TYPE_NOT_STORED);
static {
TERM_VECTORS_TYPE.setStoreTermVectors(true);
TERM_VECTORS_TYPE.freeze();
}
private static final FieldType POSITIONS_TYPE = new FieldType(TextField.TYPE_NOT_STORED);
static {
POSITIONS_TYPE.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
POSITIONS_TYPE.freeze();
}
protected static Directory dir;
protected static LeafReader unsortedReader;
protected static LeafReader sortedReader;
protected static Integer[] sortedValues;
private static Document doc(final int id, PositionsTokenStream positions) {
final Document doc = new Document();
doc.add(new StringField(ID_FIELD, Integer.toString(id), Store.YES));
doc.add(new StringField(DOCS_ENUM_FIELD, DOCS_ENUM_TERM, Store.NO));
positions.setId(id);
doc.add(new Field(DOC_POSITIONS_FIELD, positions, POSITIONS_TYPE));
doc.add(new NumericDocValuesField(NUMERIC_DV_FIELD, id));
TextField norms = new TextField(NORMS_FIELD, Integer.toString(id), Store.NO);
norms.setBoost(Float.intBitsToFloat(id));
doc.add(norms);
doc.add(new BinaryDocValuesField(BINARY_DV_FIELD, new BytesRef(Integer.toString(id))));
doc.add(new SortedDocValuesField(SORTED_DV_FIELD, new BytesRef(Integer.toString(id))));
doc.add(new SortedSetDocValuesField(SORTED_SET_DV_FIELD, new BytesRef(Integer.toString(id))));
doc.add(new SortedSetDocValuesField(SORTED_SET_DV_FIELD, new BytesRef(Integer.toString(id + 1))));
doc.add(new SortedNumericDocValuesField(SORTED_NUMERIC_DV_FIELD, id));
doc.add(new SortedNumericDocValuesField(SORTED_NUMERIC_DV_FIELD, id + 1));
doc.add(new Field(TERM_VECTORS_FIELD, Integer.toString(id), TERM_VECTORS_TYPE));
byte[] bytes = new byte[4];
NumericUtils.intToSortableBytes(id, bytes, 0);
// TODO: index time sorting doesn't yet support points
//doc.add(new BinaryPoint(DIMENSIONAL_FIELD, bytes));
return doc;
}
/** Creates an unsorted index; subclasses then sort this index and open sortedReader. */
private static void createIndex(Directory dir, int numDocs, Random random) throws IOException {
List<Integer> ids = new ArrayList<>();
for (int i = 0; i < numDocs; i++) {
ids.add(Integer.valueOf(i * 10));
}
// shuffle them for indexing
Collections.shuffle(ids, random);
if (VERBOSE) {
System.out.println("Shuffled IDs for indexing: " + Arrays.toString(ids.toArray()));
}
PositionsTokenStream positions = new PositionsTokenStream();
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random));
conf.setMaxBufferedDocs(4); // create some segments
conf.setSimilarity(new NormsSimilarity(conf.getSimilarity())); // for testing norms field
RandomIndexWriter writer = new RandomIndexWriter(random, dir, conf);
writer.setDoRandomForceMerge(false);
for (int id : ids) {
writer.addDocument(doc(id, positions));
}
// delete some documents
writer.commit();
for (Integer id : ids) {
if (random.nextDouble() < 0.2) {
if (VERBOSE) {
System.out.println("delete doc_id " + id);
}
writer.deleteDocuments(new Term(ID_FIELD, id.toString()));
}
}
writer.close();
}
@BeforeClass
public static void beforeClassSorterTestBase() throws Exception {
dir = newDirectory();
int numDocs = atLeast(20);
createIndex(dir, numDocs, random());
unsortedReader = SlowCompositeReaderWrapper.wrap(DirectoryReader.open(dir));
}
@AfterClass
public static void afterClassSorterTestBase() throws Exception {
unsortedReader.close();
sortedReader.close();
dir.close();
unsortedReader = sortedReader = null;
dir = null;
}
public void testBinaryDocValuesField() throws Exception {
BinaryDocValues dv = sortedReader.getBinaryDocValues(BINARY_DV_FIELD);
for (int i = 0; i < sortedReader.maxDoc(); i++) {
final BytesRef bytes = dv.get(i);
assertEquals("incorrect binary DocValues for doc " + i, sortedValues[i].toString(), bytes.utf8ToString());
}
}
public void testDocsAndPositionsEnum() throws Exception {
TermsEnum termsEnum = sortedReader.terms(DOC_POSITIONS_FIELD).iterator();
assertEquals(SeekStatus.FOUND, termsEnum.seekCeil(new BytesRef(DOC_POSITIONS_TERM)));
PostingsEnum sortedPositions = termsEnum.postings(null, PostingsEnum.ALL);
int doc;
// test nextDoc()
while ((doc = sortedPositions.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
int freq = sortedPositions.freq();
assertEquals("incorrect freq for doc=" + doc, sortedValues[doc].intValue() / 10 + 1, freq);
for (int i = 0; i < freq; i++) {
assertEquals("incorrect position for doc=" + doc, i, sortedPositions.nextPosition());
assertEquals("incorrect startOffset for doc=" + doc, i, sortedPositions.startOffset());
assertEquals("incorrect endOffset for doc=" + doc, i, sortedPositions.endOffset());
assertEquals("incorrect payload for doc=" + doc, freq - i, Integer.parseInt(sortedPositions.getPayload().utf8ToString()));
}
}
// test advance()
final PostingsEnum reuse = sortedPositions;
sortedPositions = termsEnum.postings(reuse, PostingsEnum.ALL);
if (sortedPositions instanceof SortingDocsEnum) {
assertTrue(((SortingDocsEnum) sortedPositions).reused(reuse)); // make sure reuse worked
}
doc = 0;
while ((doc = sortedPositions.advance(doc + TestUtil.nextInt(random(), 1, 5))) != DocIdSetIterator.NO_MORE_DOCS) {
int freq = sortedPositions.freq();
assertEquals("incorrect freq for doc=" + doc, sortedValues[doc].intValue() / 10 + 1, freq);
for (int i = 0; i < freq; i++) {
assertEquals("incorrect position for doc=" + doc, i, sortedPositions.nextPosition());
assertEquals("incorrect startOffset for doc=" + doc, i, sortedPositions.startOffset());
assertEquals("incorrect endOffset for doc=" + doc, i, sortedPositions.endOffset());
assertEquals("incorrect payload for doc=" + doc, freq - i, Integer.parseInt(sortedPositions.getPayload().utf8ToString()));
}
}
}
Bits randomLiveDocs(int maxDoc) {
if (rarely()) {
if (random().nextBoolean()) {
return null;
} else {
return new Bits.MatchNoBits(maxDoc);
}
}
final FixedBitSet bits = new FixedBitSet(maxDoc);
final int bitsSet = TestUtil.nextInt(random(), 1, maxDoc - 1);
for (int i = 0; i < bitsSet; ++i) {
while (true) {
final int index = random().nextInt(maxDoc);
if (!bits.get(index)) {
bits.set(index);
break;
}
}
}
return bits;
}
public void testDocsEnum() throws Exception {
TermsEnum termsEnum = sortedReader.terms(DOCS_ENUM_FIELD).iterator();
assertEquals(SeekStatus.FOUND, termsEnum.seekCeil(new BytesRef(DOCS_ENUM_TERM)));
PostingsEnum docs = termsEnum.postings(null);
int doc;
while ((doc = docs.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
assertEquals("incorrect value; doc " + doc, sortedValues[doc].intValue(), Integer.parseInt(sortedReader.document(doc).get(ID_FIELD)));
}
PostingsEnum reuse = docs;
docs = termsEnum.postings(reuse);
if (docs instanceof SortingDocsEnum) {
assertTrue(((SortingDocsEnum) docs).reused(reuse)); // make sure reuse worked
}
doc = -1;
while ((doc = docs.advance(doc + 1)) != DocIdSetIterator.NO_MORE_DOCS) {
assertEquals("incorrect value; doc " + doc, sortedValues[doc].intValue(), Integer.parseInt(sortedReader.document(doc).get(ID_FIELD)));
}
}
public void testNormValues() throws Exception {
NumericDocValues dv = sortedReader.getNormValues(NORMS_FIELD);
int maxDoc = sortedReader.maxDoc();
for (int i = 0; i < maxDoc; i++) {
assertEquals("incorrect norm value for doc " + i, sortedValues[i].intValue(), dv.get(i));
}
}
public void testNumericDocValuesField() throws Exception {
NumericDocValues dv = sortedReader.getNumericDocValues(NUMERIC_DV_FIELD);
int maxDoc = sortedReader.maxDoc();
for (int i = 0; i < maxDoc; i++) {
assertEquals("incorrect numeric DocValues for doc " + i, sortedValues[i].intValue(), dv.get(i));
}
}
public void testSortedDocValuesField() throws Exception {
SortedDocValues dv = sortedReader.getSortedDocValues(SORTED_DV_FIELD);
int maxDoc = sortedReader.maxDoc();
for (int i = 0; i < maxDoc; i++) {
final BytesRef bytes = dv.get(i);
assertEquals("incorrect sorted DocValues for doc " + i, sortedValues[i].toString(), bytes.utf8ToString());
}
}
public void testSortedSetDocValuesField() throws Exception {
SortedSetDocValues dv = sortedReader.getSortedSetDocValues(SORTED_SET_DV_FIELD);
int maxDoc = sortedReader.maxDoc();
for (int i = 0; i < maxDoc; i++) {
dv.setDocument(i);
BytesRef bytes = dv.lookupOrd(dv.nextOrd());
int value = sortedValues[i].intValue();
assertEquals("incorrect sorted-set DocValues for doc " + i, Integer.valueOf(value).toString(), bytes.utf8ToString());
bytes = dv.lookupOrd(dv.nextOrd());
assertEquals("incorrect sorted-set DocValues for doc " + i, Integer.valueOf(value + 1).toString(), bytes.utf8ToString());
assertEquals(SortedSetDocValues.NO_MORE_ORDS, dv.nextOrd());
}
}
public void testSortedNumericDocValuesField() throws Exception {
SortedNumericDocValues dv = sortedReader.getSortedNumericDocValues(SORTED_NUMERIC_DV_FIELD);
int maxDoc = sortedReader.maxDoc();
for (int i = 0; i < maxDoc; i++) {
dv.setDocument(i);
assertEquals(2, dv.count());
int value = sortedValues[i].intValue();
assertEquals("incorrect sorted-numeric DocValues for doc " + i, value, dv.valueAt(0));
assertEquals("incorrect sorted-numeric DocValues for doc " + i, value + 1, dv.valueAt(1));
}
}
public void testTermVectors() throws Exception {
int maxDoc = sortedReader.maxDoc();
for (int i = 0; i < maxDoc; i++) {
Terms terms = sortedReader.getTermVector(i, TERM_VECTORS_FIELD);
assertNotNull("term vectors not found for doc " + i + " field [" + TERM_VECTORS_FIELD + "]", terms);
assertEquals("incorrect term vector for doc " + i, sortedValues[i].toString(), terms.iterator().next().utf8ToString());
}
}
// TODO: index sorting doesn't yet support points
/*
public void testPoints() throws Exception {
PointValues values = sortedReader.getPointValues();
values.intersect(DIMENSIONAL_FIELD,
new IntersectVisitor() {
@Override
public void visit(int docID) {
throw new IllegalStateException();
}
@Override
public void visit(int docID, byte[] packedValues) {
assertEquals(sortedValues[docID].intValue(), NumericUtils.bytesToInt(packedValues, 0));
}
@Override
public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
return Relation.CELL_CROSSES_QUERY;
}
});
}
*/
}

View File

@ -1,73 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.index;
import java.util.Arrays;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.TestUtil;
import org.junit.BeforeClass;
public class SortingLeafReaderTest extends SorterTestBase {
@BeforeClass
public static void beforeClassSortingLeafReaderTest() throws Exception {
// NOTE: index was created by by super's @BeforeClass
// sort the index by id (as integer, in NUMERIC_DV_FIELD)
Sort sort = new Sort(new SortField(NUMERIC_DV_FIELD, SortField.Type.INT));
final Sorter.DocMap docMap = new Sorter(sort).sort(unsortedReader);
// Sorter.compute also sorts the values
NumericDocValues dv = unsortedReader.getNumericDocValues(NUMERIC_DV_FIELD);
sortedValues = new Integer[unsortedReader.maxDoc()];
for (int i = 0; i < unsortedReader.maxDoc(); ++i) {
sortedValues[docMap.oldToNew(i)] = (int)dv.get(i);
}
if (VERBOSE) {
System.out.println("docMap: " + docMap);
System.out.println("sortedValues: " + Arrays.toString(sortedValues));
}
// sort the index by id (as integer, in NUMERIC_DV_FIELD)
sortedReader = SortingLeafReader.wrap(unsortedReader, sort);
if (VERBOSE) {
System.out.print("mapped-deleted-docs: ");
Bits mappedLiveDocs = sortedReader.getLiveDocs();
for (int i = 0; i < mappedLiveDocs.length(); i++) {
if (!mappedLiveDocs.get(i)) {
System.out.print(i + " ");
}
}
System.out.println();
}
TestUtil.checkReader(sortedReader);
}
public void testBadSort() throws Exception {
IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
SortingLeafReader.wrap(sortedReader, Sort.RELEVANCE);
});
assertEquals("Cannot sort an index with a Sort that refers to the relevance score", expected.getMessage());
}
}

View File

@ -1,128 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.index;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field.Store;
import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.document.StringField;
import org.apache.lucene.search.BlockJoinComparatorSource;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.Weight;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BitSet;
import org.apache.lucene.util.LuceneTestCase;
public class TestBlockJoinSorter extends LuceneTestCase {
public void test() throws IOException {
final int numParents = atLeast(200);
IndexWriterConfig cfg = newIndexWriterConfig(new MockAnalyzer(random()));
cfg.setMergePolicy(newLogMergePolicy());
final RandomIndexWriter writer = new RandomIndexWriter(random(), newDirectory(), cfg);
final Document parentDoc = new Document();
final NumericDocValuesField parentVal = new NumericDocValuesField("parent_val", 0L);
parentDoc.add(parentVal);
final StringField parent = new StringField("parent", "true", Store.YES);
parentDoc.add(parent);
for (int i = 0; i < numParents; ++i) {
List<Document> documents = new ArrayList<>();
final int numChildren = random().nextInt(10);
for (int j = 0; j < numChildren; ++j) {
final Document childDoc = new Document();
childDoc.add(new NumericDocValuesField("child_val", random().nextInt(5)));
documents.add(childDoc);
}
parentVal.setLongValue(random().nextInt(50));
documents.add(parentDoc);
writer.addDocuments(documents);
}
writer.forceMerge(1);
IndexReader indexReader = writer.getReader();
writer.close();
IndexSearcher searcher = newSearcher(indexReader);
indexReader = searcher.getIndexReader(); // newSearcher may have wrapped it
assertEquals(1, indexReader.leaves().size());
final LeafReader reader = indexReader.leaves().get(0).reader();
final Query parentsFilter = new TermQuery(new Term("parent", "true"));
final Weight weight = searcher.createNormalizedWeight(parentsFilter, false);
final Scorer parents = weight.scorer(indexReader.leaves().get(0));
final BitSet parentBits = BitSet.of(parents.iterator(), reader.maxDoc());
final NumericDocValues parentValues = reader.getNumericDocValues("parent_val");
final NumericDocValues childValues = reader.getNumericDocValues("child_val");
final Sort parentSort = new Sort(new SortField("parent_val", SortField.Type.LONG));
final Sort childSort = new Sort(new SortField("child_val", SortField.Type.LONG));
final Sort sort = new Sort(new SortField("custom", new BlockJoinComparatorSource(parentsFilter, parentSort, childSort)));
final Sorter sorter = new Sorter(sort);
final Sorter.DocMap docMap = sorter.sort(reader);
assertEquals(reader.maxDoc(), docMap.size());
int[] children = new int[1];
int numChildren = 0;
int previousParent = -1;
for (int i = 0; i < docMap.size(); ++i) {
final int oldID = docMap.newToOld(i);
if (parentBits.get(oldID)) {
// check that we have the right children
for (int j = 0; j < numChildren; ++j) {
assertEquals(oldID, parentBits.nextSetBit(children[j]));
}
// check that children are sorted
for (int j = 1; j < numChildren; ++j) {
final int doc1 = children[j-1];
final int doc2 = children[j];
if (childValues.get(doc1) == childValues.get(doc2)) {
assertTrue(doc1 < doc2); // sort is stable
} else {
assertTrue(childValues.get(doc1) < childValues.get(doc2));
}
}
// check that parents are sorted
if (previousParent != -1) {
if (parentValues.get(previousParent) == parentValues.get(oldID)) {
assertTrue(previousParent < oldID);
} else {
assertTrue(parentValues.get(previousParent) < parentValues.get(oldID));
}
}
// reset
previousParent = oldID;
numChildren = 0;
} else {
children = ArrayUtil.grow(children, numChildren+1);
children[numChildren++] = oldID;
}
}
indexReader.close();
writer.w.getDirectory().close();
}
}

View File

@ -1,201 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.index;
import java.io.IOException;
import java.lang.reflect.Method;
import java.lang.reflect.Modifier;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Random;
import java.util.Set;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field.Store;
import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.document.StringField;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.LogMergePolicy;
import org.apache.lucene.index.MergePolicy;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.SlowCompositeReaderWrapper;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.TieredMergePolicy;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
public class TestSortingMergePolicy extends BaseMergePolicyTestCase {
private List<String> terms;
private Directory dir1, dir2;
private Sort sort;
private boolean reversedSort;
private IndexReader reader;
private IndexReader sortedReader;
@Override
public void setUp() throws Exception {
super.setUp();
final Boolean reverse = (random().nextBoolean() ? null : new Boolean(random().nextBoolean()));
final SortField sort_field = (reverse == null
? new SortField("ndv", SortField.Type.LONG)
: new SortField("ndv", SortField.Type.LONG, reverse.booleanValue()));
sort = new Sort(sort_field);
reversedSort = (null != reverse && reverse.booleanValue());
createRandomIndexes();
}
private Document randomDocument() {
final Document doc = new Document();
doc.add(new NumericDocValuesField("ndv", random().nextLong()));
doc.add(new StringField("s", RandomPicks.randomFrom(random(), terms), Store.YES));
return doc;
}
public MergePolicy mergePolicy() {
return newSortingMergePolicy(sort);
}
public static SortingMergePolicy newSortingMergePolicy(Sort sort) {
// usually create a MP with a low merge factor so that many merges happen
MergePolicy mp;
int thingToDo = random().nextInt(3);
if (thingToDo == 0) {
TieredMergePolicy tmp = newTieredMergePolicy(random());
final int numSegs = TestUtil.nextInt(random(), 3, 5);
tmp.setSegmentsPerTier(numSegs);
tmp.setMaxMergeAtOnce(TestUtil.nextInt(random(), 2, numSegs));
mp = tmp;
} else if (thingToDo == 1) {
LogMergePolicy lmp = newLogMergePolicy(random());
lmp.setMergeFactor(TestUtil.nextInt(random(), 3, 5));
mp = lmp;
} else {
// just a regular random one from LTC (could be alcoholic etc)
mp = newMergePolicy();
}
// wrap it with a sorting mp
if (VERBOSE) {
System.out.println("TEST: return SortingMergePolicy(mp=" + mp + " sort=" + sort + ")");
}
return new SortingMergePolicy(mp, sort);
}
private void createRandomIndexes() throws IOException {
dir1 = newDirectory();
dir2 = newDirectory();
final int numDocs = atLeast(150);
final int numTerms = TestUtil.nextInt(random(), 1, numDocs / 5);
Set<String> randomTerms = new HashSet<>();
while (randomTerms.size() < numTerms) {
randomTerms.add(TestUtil.randomSimpleString(random()));
}
terms = new ArrayList<>(randomTerms);
final long seed = random().nextLong();
final IndexWriterConfig iwc1 = newIndexWriterConfig(new MockAnalyzer(new Random(seed)));
final IndexWriterConfig iwc2 = newIndexWriterConfig(new MockAnalyzer(new Random(seed)));
iwc2.setMergePolicy(mergePolicy());
final RandomIndexWriter iw1 = new RandomIndexWriter(new Random(seed), dir1, iwc1);
final RandomIndexWriter iw2 = new RandomIndexWriter(new Random(seed), dir2, iwc2);
for (int i = 0; i < numDocs; ++i) {
if (random().nextInt(5) == 0 && i != numDocs - 1) {
final String term = RandomPicks.randomFrom(random(), terms);
iw1.deleteDocuments(new Term("s", term));
iw2.deleteDocuments(new Term("s", term));
}
final Document doc = randomDocument();
iw1.addDocument(doc);
iw2.addDocument(doc);
if (random().nextInt(8) == 0) {
iw1.commit();
iw2.commit();
}
}
// Make sure we have something to merge
iw1.commit();
iw2.commit();
final Document doc = randomDocument();
// NOTE: don't use RIW.addDocument directly, since it sometimes commits
// which may trigger a merge, at which case forceMerge may not do anything.
// With field updates this is a problem, since the updates can go into the
// single segment in the index, and threefore the index won't be sorted.
// This hurts the assumption of the test later on, that the index is sorted
// by SortingMP.
iw1.w.addDocument(doc);
iw2.w.addDocument(doc);
// update NDV of docs belonging to one term (covers many documents)
final long value = random().nextLong();
final String term = RandomPicks.randomFrom(random(), terms);
iw1.w.updateNumericDocValue(new Term("s", term), "ndv", value);
iw2.w.updateNumericDocValue(new Term("s", term), "ndv", value);
iw1.forceMerge(1);
iw2.forceMerge(1);
iw1.close();
iw2.close();
reader = DirectoryReader.open(dir1);
sortedReader = DirectoryReader.open(dir2);
}
@Override
public void tearDown() throws Exception {
reader.close();
sortedReader.close();
dir1.close();
dir2.close();
super.tearDown();
}
private static void assertSorted(LeafReader reader, boolean reverse) throws IOException {
final NumericDocValues ndv = reader.getNumericDocValues("ndv");
for (int i = 1; i < reader.maxDoc(); ++i) {
final int lhs = (!reverse ? i-1 : i);
final int rhs = (!reverse ? i : i-1);
assertTrue("ndv(" + (i-1) + ")=" + ndv.get(i-1) + ",ndv(" + i + ")=" + ndv.get(i)+",reverse="+reverse, ndv.get(lhs) <= ndv.get(rhs));
}
}
public void testSortingMP() throws IOException {
final LeafReader sortedReader1 = SortingLeafReader.wrap(SlowCompositeReaderWrapper.wrap(reader), sort);
final LeafReader sortedReader2 = SlowCompositeReaderWrapper.wrap(sortedReader);
assertSorted(sortedReader1, reversedSort);
assertSorted(sortedReader2, reversedSort);
assertReaderEquals("", sortedReader1, sortedReader2);
}
public void testBadSort() throws Exception {
IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
new SortingMergePolicy(newMergePolicy(), Sort.RELEVANCE);
});
assertEquals("Cannot sort an index with a Sort that refers to the relevance score", expected.getMessage());
}
}

View File

@ -32,9 +32,9 @@ import org.apache.lucene.index.FieldInvertState;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.MultiDocValues;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.SlowCompositeReaderWrapper;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.BooleanClause.Occur;
@ -367,8 +367,7 @@ public class TestDiversifiedTopDocsCollector extends LuceneTestCase {
reader = writer.getReader();
writer.close();
searcher = newSearcher(reader);
LeafReader ar = SlowCompositeReaderWrapper.wrap(reader);
artistDocValues = ar.getSortedDocValues("artist");
artistDocValues = MultiDocValues.getSortedValues(reader, "artist");
// All searches sort by song popularity
final Similarity base = searcher.getSimilarity(true);

View File

@ -247,7 +247,7 @@ public class TestNearest extends LuceneTestCase {
private IndexWriterConfig getIndexWriterConfig() {
IndexWriterConfig iwc = newIndexWriterConfig();
iwc.setCodec(Codec.forName("Lucene60"));
iwc.setCodec(Codec.forName("Lucene62"));
return iwc;
}
}

View File

@ -85,14 +85,14 @@ import com.carrotsearch.randomizedtesting.generators.RandomInts;
public class TestGeo3DPoint extends LuceneTestCase {
private static Codec getCodec() {
if (Codec.getDefault().getName().equals("Lucene60")) {
if (Codec.getDefault().getName().equals("Lucene62")) {
int maxPointsInLeafNode = TestUtil.nextInt(random(), 16, 2048);
double maxMBSortInHeap = 3.0 + (3*random().nextDouble());
if (VERBOSE) {
System.out.println("TEST: using Lucene60PointsFormat with maxPointsInLeafNode=" + maxPointsInLeafNode + " and maxMBSortInHeap=" + maxMBSortInHeap);
}
return new FilterCodec("Lucene60", Codec.getDefault()) {
return new FilterCodec("Lucene62", Codec.getDefault()) {
@Override
public PointsFormat pointsFormat() {
return new PointsFormat() {

View File

@ -56,7 +56,6 @@ import org.apache.lucene.index.MultiDocValues;
import org.apache.lucene.index.ReaderUtil;
import org.apache.lucene.index.SegmentReader;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.index.SortingMergePolicy;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.BooleanClause.Occur;
import org.apache.lucene.search.BooleanClause;
@ -232,7 +231,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
// This way all merged segments will be sorted at
// merge time, allow for per-segment early termination
// when those segments are searched:
iwc.setMergePolicy(new SortingMergePolicy(iwc.getMergePolicy(), SORT));
iwc.setIndexSort(SORT);
return iwc;
}
@ -586,10 +585,9 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
// We sorted postings by weight during indexing, so we
// only retrieve the first num hits now:
final SortingMergePolicy sortingMergePolicy = (SortingMergePolicy) writer.getConfig().getMergePolicy();
Collector c2 = new EarlyTerminatingSortingCollector(c, SORT, num, sortingMergePolicy.getSort());
IndexSearcher searcher = searcherMgr.acquire();
Collector c2 = new EarlyTerminatingSortingCollector(c, SORT, num);
List<LookupResult> results = null;
IndexSearcher searcher = searcherMgr.acquire();
try {
//System.out.println("got searcher=" + searcher);
searcher.search(finalQuery, c2);

View File

@ -66,7 +66,7 @@ public class AnalyzingInfixSuggesterTest extends LuceneTestCase {
assertEquals("a penny saved is a penny earned", results.get(0).key);
assertEquals("a penny saved is a penny <b>ear</b>ned", results.get(0).highlightKey);
assertEquals(10, results.get(0).value);
assertEquals(new BytesRef("foobaz"), results.get(0).payload);
assertEquals("foobaz", results.get(0).payload.utf8ToString());
assertEquals("lend me your ear", results.get(1).key);
assertEquals("lend me your <b>ear</b>", results.get(1).highlightKey);

View File

@ -32,7 +32,7 @@ import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.lucene60.Lucene60Codec;
import org.apache.lucene.codecs.lucene62.Lucene62Codec;
import org.apache.lucene.document.IntPoint;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
@ -646,7 +646,7 @@ public class TestSuggestField extends LuceneTestCase {
static IndexWriterConfig iwcWithSuggestField(Analyzer analyzer, final Set<String> suggestFields) {
IndexWriterConfig iwc = newIndexWriterConfig(random(), analyzer);
iwc.setMergePolicy(newLogMergePolicy());
Codec filterCodec = new Lucene60Codec() {
Codec filterCodec = new Lucene62Codec() {
PostingsFormat postingsFormat = new Completion50PostingsFormat();
@Override

View File

@ -1242,7 +1242,7 @@ public abstract class BaseGeoPointTestCase extends LuceneTestCase {
// Else seeds may not reproduce:
iwc.setMergeScheduler(new SerialMergeScheduler());
int pointsInLeaf = 2 + random().nextInt(4);
iwc.setCodec(new FilterCodec("Lucene60", TestUtil.getDefaultCodec()) {
iwc.setCodec(new FilterCodec("Lucene62", TestUtil.getDefaultCodec()) {
@Override
public PointsFormat pointsFormat() {
return new PointsFormat() {

View File

@ -627,7 +627,7 @@ public abstract class BaseCompoundFormatTestCase extends BaseIndexFileFormatTest
/** Returns a new fake segment */
protected static SegmentInfo newSegmentInfo(Directory dir, String name) {
return new SegmentInfo(dir, Version.LATEST, name, 10000, false, Codec.getDefault(), Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
return new SegmentInfo(dir, Version.LATEST, name, 10000, false, Codec.getDefault(), Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
}
/** Creates a file of the specified size with random data. */

View File

@ -347,7 +347,7 @@ public abstract class BaseFieldInfoFormatTestCase extends BaseIndexFileFormatTes
/** Returns a new fake segment */
protected static SegmentInfo newSegmentInfo(Directory dir, String name) {
return new SegmentInfo(dir, Version.LATEST, name, 10000, false, Codec.getDefault(), Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
return new SegmentInfo(dir, Version.LATEST, name, 10000, false, Codec.getDefault(), Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
}
@Override

View File

@ -303,7 +303,7 @@ abstract class BaseIndexFileFormatTestCase extends LuceneTestCase {
Directory dir = newFSDirectory(createTempDir("justSoYouGetSomeChannelErrors"));
Codec codec = getCodec();
SegmentInfo segmentInfo = new SegmentInfo(dir, Version.LATEST, "_0", 1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
SegmentInfo segmentInfo = new SegmentInfo(dir, Version.LATEST, "_0", 1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
FieldInfo proto = oneDocReader.getFieldInfos().fieldInfo("field");
FieldInfo field = new FieldInfo(proto.name, proto.number, proto.hasVectors(), proto.omitsNorms(), proto.hasPayloads(),
proto.getIndexOptions(), proto.getDocValuesType(), proto.getDocValuesGen(), new HashMap<>(),

View File

@ -26,7 +26,8 @@ import java.util.Set;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.StoredField;
import org.apache.lucene.document.TextField;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.MockDirectoryWrapper;
@ -52,7 +53,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT
Codec codec = getCodec();
byte id[] = StringHelper.randomId();
SegmentInfo info = new SegmentInfo(dir, getVersions()[0], "_123", 1, false, codec,
Collections.<String,String>emptyMap(), id, new HashMap<>());
Collections.<String,String>emptyMap(), id, new HashMap<>(), null);
info.setFiles(Collections.<String>emptySet());
codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
SegmentInfo info2 = codec.segmentInfoFormat().read(dir, "_123", id, IOContext.DEFAULT);
@ -66,7 +67,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT
Codec codec = getCodec();
byte id[] = StringHelper.randomId();
SegmentInfo info = new SegmentInfo(dir, getVersions()[0], "_123", 1, false, codec,
Collections.<String,String>emptyMap(), id, new HashMap<>());
Collections.<String,String>emptyMap(), id, new HashMap<>(), null);
Set<String> originalFiles = Collections.singleton("_123.a");
info.setFiles(originalFiles);
codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
@ -95,7 +96,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT
diagnostics.put("key1", "value1");
diagnostics.put("key2", "value2");
SegmentInfo info = new SegmentInfo(dir, getVersions()[0], "_123", 1, false, codec,
diagnostics, id, new HashMap<>());
diagnostics, id, new HashMap<>(), null);
info.setFiles(Collections.<String>emptySet());
codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
SegmentInfo info2 = codec.segmentInfoFormat().read(dir, "_123", id, IOContext.DEFAULT);
@ -118,7 +119,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT
attributes.put("key1", "value1");
attributes.put("key2", "value2");
SegmentInfo info = new SegmentInfo(dir, getVersions()[0], "_123", 1, false, codec,
Collections.emptyMap(), id, attributes);
Collections.emptyMap(), id, attributes, null);
info.setFiles(Collections.<String>emptySet());
codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
SegmentInfo info2 = codec.segmentInfoFormat().read(dir, "_123", id, IOContext.DEFAULT);
@ -138,7 +139,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT
Directory dir = newDirectory();
byte id[] = StringHelper.randomId();
SegmentInfo info = new SegmentInfo(dir, getVersions()[0], "_123", 1, false, codec,
Collections.<String,String>emptyMap(), id, new HashMap<>());
Collections.<String,String>emptyMap(), id, new HashMap<>(), null);
info.setFiles(Collections.<String>emptySet());
codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
SegmentInfo info2 = codec.segmentInfoFormat().read(dir, "_123", id, IOContext.DEFAULT);
@ -153,7 +154,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT
Directory dir = newDirectory();
byte id[] = StringHelper.randomId();
SegmentInfo info = new SegmentInfo(dir, v, "_123", 1, false, codec,
Collections.<String,String>emptyMap(), id, new HashMap<>());
Collections.<String,String>emptyMap(), id, new HashMap<>(), null);
info.setFiles(Collections.<String>emptySet());
codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
SegmentInfo info2 = codec.segmentInfoFormat().read(dir, "_123", id, IOContext.DEFAULT);
@ -161,7 +162,57 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT
dir.close();
}
}
protected boolean supportsIndexSort() {
return true;
}
/** Test sort */
public void testSort() throws IOException {
assumeTrue("test requires a codec that can read/write index sort", supportsIndexSort());
final int iters = atLeast(5);
for (int i = 0; i < iters; ++i) {
Sort sort;
if (i == 0) {
sort = null;
} else {
final int numSortFields = TestUtil.nextInt(random(), 1, 3);
SortField[] sortFields = new SortField[numSortFields];
for (int j = 0; j < numSortFields; ++j) {
sortFields[j] = new SortField(
TestUtil.randomSimpleString(random()),
random().nextBoolean() ? SortField.Type.LONG : SortField.Type.STRING,
random().nextBoolean());
if (random().nextBoolean()) {
switch (sortFields[j].getType()) {
case LONG:
sortFields[j].setMissingValue(random().nextLong());
break;
case STRING:
sortFields[j].setMissingValue(random().nextBoolean() ? SortField.STRING_FIRST : SortField.STRING_LAST);
break;
default:
fail();
}
}
}
sort = new Sort(sortFields);
}
Directory dir = newDirectory();
Codec codec = getCodec();
byte id[] = StringHelper.randomId();
SegmentInfo info = new SegmentInfo(dir, getVersions()[0], "_123", 1, false, codec,
Collections.<String,String>emptyMap(), id, new HashMap<>(), sort);
info.setFiles(Collections.<String>emptySet());
codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
SegmentInfo info2 = codec.segmentInfoFormat().read(dir, "_123", id, IOContext.DEFAULT);
assertEquals(sort, info2.getIndexSort());
dir.close();
}
}
/**
* Test segment infos write that hits exception immediately on open.
* make sure we get our exception back, no file handle leaks, etc.
@ -183,7 +234,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT
Codec codec = getCodec();
byte id[] = StringHelper.randomId();
SegmentInfo info = new SegmentInfo(dir, getVersions()[0], "_123", 1, false, codec,
Collections.<String,String>emptyMap(), id, new HashMap<>());
Collections.<String,String>emptyMap(), id, new HashMap<>(), null);
info.setFiles(Collections.<String>emptySet());
fail.setDoFail();
@ -216,7 +267,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT
Codec codec = getCodec();
byte id[] = StringHelper.randomId();
SegmentInfo info = new SegmentInfo(dir, getVersions()[0], "_123", 1, false, codec,
Collections.<String,String>emptyMap(), id, new HashMap<>());
Collections.<String,String>emptyMap(), id, new HashMap<>(), null);
info.setFiles(Collections.<String>emptySet());
fail.setDoFail();
@ -249,7 +300,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT
Codec codec = getCodec();
byte id[] = StringHelper.randomId();
SegmentInfo info = new SegmentInfo(dir, getVersions()[0], "_123", 1, false, codec,
Collections.<String,String>emptyMap(), id, new HashMap<>());
Collections.<String,String>emptyMap(), id, new HashMap<>(), null);
info.setFiles(Collections.<String>emptySet());
codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
@ -283,7 +334,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT
Codec codec = getCodec();
byte id[] = StringHelper.randomId();
SegmentInfo info = new SegmentInfo(dir, getVersions()[0], "_123", 1, false, codec,
Collections.<String,String>emptyMap(), id, new HashMap<>());
Collections.<String,String>emptyMap(), id, new HashMap<>(), null);
info.setFiles(Collections.<String>emptySet());
codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
@ -332,7 +383,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT
TestUtil.randomUnicodeString(random()));
}
SegmentInfo info = new SegmentInfo(dir, version, name, docCount, isCompoundFile, codec, diagnostics, id, attributes);
SegmentInfo info = new SegmentInfo(dir, version, name, docCount, isCompoundFile, codec, diagnostics, id, attributes, null);
info.setFiles(files);
codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
SegmentInfo info2 = codec.segmentInfoFormat().read(dir, name, id, IOContext.DEFAULT);

View File

@ -138,7 +138,6 @@ public class MockRandomMergePolicy extends MergePolicy {
static class MockRandomOneMerge extends OneMerge {
final Random r;
ArrayList<CodecReader> readers;
MockRandomOneMerge(List<SegmentCommitInfo> segments, long seed) {
super(segments);
@ -146,34 +145,31 @@ public class MockRandomMergePolicy extends MergePolicy {
}
@Override
public List<CodecReader> getMergeReaders() throws IOException {
if (readers == null) {
readers = new ArrayList<CodecReader>(super.getMergeReaders());
for (int i = 0; i < readers.size(); i++) {
// wrap it (e.g. prevent bulk merge etc)
// TODO: cut this over to FilterCodecReader api, we can explicitly
// enable/disable bulk merge for portions of the index we want.
int thingToDo = r.nextInt(7);
if (thingToDo == 0) {
// simple no-op FilterReader
if (LuceneTestCase.VERBOSE) {
System.out.println("NOTE: MockRandomMergePolicy now swaps in a SlowCodecReaderWrapper for merging reader=" + readers.get(i));
}
readers.set(i, SlowCodecReaderWrapper.wrap(new FilterLeafReader(readers.get(i)) {}));
} else if (thingToDo == 1) {
// renumber fields
// NOTE: currently this only "blocks" bulk merges just by
// being a FilterReader. But it might find bugs elsewhere,
// and maybe the situation can be improved in the future.
if (LuceneTestCase.VERBOSE) {
System.out.println("NOTE: MockRandomMergePolicy now swaps in a MismatchedLeafReader for merging reader=" + readers.get(i));
}
readers.set(i, SlowCodecReaderWrapper.wrap(new MismatchedLeafReader(readers.get(i), r)));
}
// otherwise, reader is unchanged
public CodecReader wrapForMerge(CodecReader reader) throws IOException {
// wrap it (e.g. prevent bulk merge etc)
// TODO: cut this over to FilterCodecReader api, we can explicitly
// enable/disable bulk merge for portions of the index we want.
int thingToDo = r.nextInt(7);
if (thingToDo == 0) {
// simple no-op FilterReader
if (LuceneTestCase.VERBOSE) {
System.out.println("NOTE: MockRandomMergePolicy now swaps in a SlowCodecReaderWrapper for merging reader=" + reader);
}
return SlowCodecReaderWrapper.wrap(new FilterLeafReader(reader) {});
} else if (thingToDo == 1) {
// renumber fields
// NOTE: currently this only "blocks" bulk merges just by
// being a FilterReader. But it might find bugs elsewhere,
// and maybe the situation can be improved in the future.
if (LuceneTestCase.VERBOSE) {
System.out.println("NOTE: MockRandomMergePolicy now swaps in a MismatchedLeafReader for merging reader=" + reader);
}
return SlowCodecReaderWrapper.wrap(new MismatchedLeafReader(reader, r));
} else {
// otherwise, reader is unchanged
return reader;
}
return readers;
}
}
}

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