mirror of https://github.com/apache/lucene.git
LUCENE-6861: switch trunk to Luceen60 codec, implementing dimensional format
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1711732 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
d1a198a1f7
commit
d3869edb96
|
@ -43,6 +43,9 @@ New Features
|
|||
document and codec APIs, including a simple text implementation.
|
||||
(Mike McCandless)
|
||||
|
||||
* LUCENE-6861: Create Lucene60Codec, supporting dimensional values.
|
||||
(Mike McCandless)
|
||||
|
||||
API Changes
|
||||
|
||||
* LUCENE-3312: The API of oal.document was restructured to
|
||||
|
|
|
@ -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 5.4 file format.
|
||||
</body>
|
||||
</html>
|
|
@ -29,13 +29,13 @@ 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.lucene54.Lucene54Codec;
|
||||
import org.apache.lucene.codecs.lucene60.Lucene60Codec;
|
||||
import org.apache.lucene.index.ConcurrentMergeScheduler;
|
||||
import org.apache.lucene.index.IndexCommit;
|
||||
import org.apache.lucene.index.IndexDeletionPolicy;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.index.IndexWriterConfig;
|
||||
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
|
||||
import org.apache.lucene.index.IndexWriterConfig;
|
||||
import org.apache.lucene.index.LogMergePolicy;
|
||||
import org.apache.lucene.index.MergePolicy;
|
||||
import org.apache.lucene.index.MergeScheduler;
|
||||
|
@ -139,7 +139,7 @@ public class CreateIndexTask extends PerfTask {
|
|||
if (defaultCodec == null && postingsFormat != null) {
|
||||
try {
|
||||
final PostingsFormat postingsFormatChosen = PostingsFormat.forName(postingsFormat);
|
||||
iwConf.setCodec(new Lucene54Codec() {
|
||||
iwConf.setCodec(new Lucene60Codec() {
|
||||
@Override
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
return postingsFormatChosen;
|
||||
|
|
|
@ -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("Lucene54");
|
||||
static Codec defaultCodec = LOADER.lookup("Lucene60");
|
||||
}
|
||||
|
||||
private final String name;
|
||||
|
|
|
@ -0,0 +1,176 @@
|
|||
package org.apache.lucene.codecs.lucene60;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.util.Objects;
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.CompoundFormat;
|
||||
import org.apache.lucene.codecs.DimensionalFormat;
|
||||
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.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.Lucene50SegmentInfoFormat;
|
||||
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.perfield.PerFieldDocValuesFormat;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
|
||||
|
||||
/**
|
||||
* Implements the Lucene 6.0 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 Lucene60Codec extends Codec {
|
||||
private final TermVectorsFormat vectorsFormat = new Lucene50TermVectorsFormat();
|
||||
private final FieldInfosFormat fieldInfosFormat = new Lucene60FieldInfosFormat();
|
||||
private final SegmentInfoFormat segmentInfosFormat = new Lucene50SegmentInfoFormat();
|
||||
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 Lucene60Codec.this.getPostingsFormatForField(field);
|
||||
}
|
||||
};
|
||||
|
||||
private final DocValuesFormat docValuesFormat = new PerFieldDocValuesFormat() {
|
||||
@Override
|
||||
public DocValuesFormat getDocValuesFormatForField(String field) {
|
||||
return Lucene60Codec.this.getDocValuesFormatForField(field);
|
||||
}
|
||||
};
|
||||
|
||||
private final StoredFieldsFormat storedFieldsFormat;
|
||||
|
||||
/**
|
||||
* Instantiates a new codec.
|
||||
*/
|
||||
public Lucene60Codec() {
|
||||
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 Lucene60Codec(Mode mode) {
|
||||
super("Lucene60");
|
||||
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 DimensionalFormat dimensionalFormat() {
|
||||
return new Lucene60DimensionalFormat();
|
||||
}
|
||||
|
||||
/** 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 "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 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;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,107 @@
|
|||
package org.apache.lucene.codecs.lucene60;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.codecs.DimensionalFormat;
|
||||
import org.apache.lucene.codecs.DimensionalReader;
|
||||
import org.apache.lucene.codecs.DimensionalWriter;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
|
||||
/**
|
||||
* Lucene 6.0 dimensional format, which encodes dimensional values in a block KD-tree structure
|
||||
* for fast shape intersection filtering. See <a href="https://www.cs.duke.edu/~pankaj/publications/papers/bkd-sstd.pdf">this paper</a> for details.
|
||||
*
|
||||
* <p>This data structure is written as a series of blocks on disk, with an in-memory perfectly balanced
|
||||
* binary tree of split values referencing those blocks at the leaves.
|
||||
*
|
||||
* <p>The <code>.dim</code> file has both blocks and the index split
|
||||
* values, for each field. The file starts with {@link CodecUtil#writeIndexHeader}.
|
||||
*
|
||||
* <p>The blocks are written like this:
|
||||
*
|
||||
* <ul>
|
||||
* <li> count (vInt)
|
||||
* <li> delta-docID (vInt) <sup>count</sup> (delta coded docIDs, in sorted order)
|
||||
* <li> packedValue<sup>count</sup> (the <code>byte[]</code> value of each dimension packed into a single <code>byte[]</code>)
|
||||
* </ul>
|
||||
*
|
||||
* <p>After all blocks for a field are written, then the index is written:
|
||||
* <ul>
|
||||
* <li> numDims (vInt)
|
||||
* <li> maxPointsInLeafNode (vInt)
|
||||
* <li> bytesPerDim (vInt)
|
||||
* <li> count (vInt)
|
||||
* <li> byte[bytesPerDim]<sup>count</sup> (packed <code>byte[]</code> all split values)
|
||||
* <li> delta-blockFP (vLong)<sup>count</sup> (delta-coded file pointers to the on-disk leaf blocks))
|
||||
* </ul>
|
||||
*
|
||||
* <p>After all fields blocks + index data are written, {@link CodecUtil#writeFooter} writes the checksum.
|
||||
*
|
||||
* <p>The <code>.dii</code> file records the file pointer in the <code>.dim</code> file where each field's
|
||||
* index data was written. It starts with {@link CodecUtil#writeIndexHeader}, then has:
|
||||
*
|
||||
* <ul>
|
||||
* <li> fieldCount (vInt)
|
||||
* <li> (fieldNumber (vInt), fieldFilePointer (vLong))<sup>fieldCount</sup>
|
||||
* </ul>
|
||||
*
|
||||
* <p> After that, {@link CodecUtil#writeFooter} writes the checksum.
|
||||
*
|
||||
* <p>After all fields blocks + index data are written, {@link CodecUtil#writeFooter} writes the checksum.
|
||||
|
||||
* @lucene.experimental
|
||||
*/
|
||||
|
||||
public final class Lucene60DimensionalFormat extends DimensionalFormat {
|
||||
|
||||
static final String CODEC_NAME = "Lucene60DimensionalFormat";
|
||||
|
||||
/**
|
||||
* Filename extension for the leaf blocks
|
||||
*/
|
||||
public static final String DATA_EXTENSION = "dim";
|
||||
|
||||
/**
|
||||
* Filename extension for the index per field
|
||||
*/
|
||||
public static final String INDEX_EXTENSION = "dii";
|
||||
|
||||
static final int DATA_VERSION_START = 0;
|
||||
static final int DATA_VERSION_CURRENT = DATA_VERSION_START;
|
||||
|
||||
static final int INDEX_VERSION_START = 0;
|
||||
static final int INDEX_VERSION_CURRENT = INDEX_VERSION_START;
|
||||
|
||||
/** Sole constructor */
|
||||
public Lucene60DimensionalFormat() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public DimensionalWriter fieldsWriter(SegmentWriteState state) throws IOException {
|
||||
return new Lucene60DimensionalWriter(state);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DimensionalReader fieldsReader(SegmentReadState state) throws IOException {
|
||||
return new Lucene60DimensionalReader(state);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,132 @@
|
|||
package org.apache.lucene.codecs.lucene60;
|
||||
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.codecs.DimensionalReader;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.store.ChecksumIndexInput;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.Accountable;
|
||||
import org.apache.lucene.util.Accountables;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.bkd.BKDReader;
|
||||
|
||||
/** Reads dimensional values previously written with {@link Lucene60DimensionalWriter} */
|
||||
public class Lucene60DimensionalReader extends DimensionalReader implements Closeable {
|
||||
final IndexInput dataIn;
|
||||
final SegmentReadState readState;
|
||||
final Map<Integer,BKDReader> readers = new HashMap<>();
|
||||
|
||||
/** Sole constructor */
|
||||
public Lucene60DimensionalReader(SegmentReadState readState) throws IOException {
|
||||
this.readState = readState;
|
||||
String dataFileName = IndexFileNames.segmentFileName(readState.segmentInfo.name,
|
||||
readState.segmentSuffix,
|
||||
Lucene60DimensionalFormat.DATA_EXTENSION);
|
||||
dataIn = readState.directory.openInput(dataFileName, readState.context);
|
||||
String indexFileName = IndexFileNames.segmentFileName(readState.segmentInfo.name,
|
||||
readState.segmentSuffix,
|
||||
Lucene60DimensionalFormat.INDEX_EXTENSION);
|
||||
|
||||
boolean success = false;
|
||||
|
||||
// Read index file
|
||||
try (ChecksumIndexInput indexIn = readState.directory.openChecksumInput(indexFileName, readState.context)) {
|
||||
CodecUtil.checkIndexHeader(indexIn,
|
||||
Lucene60DimensionalFormat.CODEC_NAME,
|
||||
Lucene60DimensionalFormat.INDEX_VERSION_START,
|
||||
Lucene60DimensionalFormat.INDEX_VERSION_START,
|
||||
readState.segmentInfo.getId(),
|
||||
readState.segmentSuffix);
|
||||
int count = indexIn.readVInt();
|
||||
for(int i=0;i<count;i++) {
|
||||
int fieldNumber = indexIn.readVInt();
|
||||
long fp = indexIn.readVLong();
|
||||
dataIn.seek(fp);
|
||||
readers.put(fieldNumber, new BKDReader(dataIn));
|
||||
}
|
||||
CodecUtil.checkFooter(indexIn);
|
||||
success = true;
|
||||
} finally {
|
||||
if (success == false) {
|
||||
IOUtils.closeWhileHandlingException(dataIn);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void intersect(String field, IntersectVisitor visitor) throws IOException {
|
||||
FieldInfo fieldInfo = readState.fieldInfos.fieldInfo(field);
|
||||
if (fieldInfo == null) {
|
||||
throw new IllegalArgumentException("field=\"" + field + "\" is unrecognized");
|
||||
}
|
||||
if (fieldInfo.getDimensionCount() == 0) {
|
||||
throw new IllegalArgumentException("field=\"" + field + "\" did not index dimensional values");
|
||||
}
|
||||
|
||||
BKDReader reader = readers.get(fieldInfo.number);
|
||||
assert reader != null;
|
||||
|
||||
reader.intersect(visitor);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
long sizeInBytes = 0;
|
||||
for(BKDReader reader : readers.values()) {
|
||||
sizeInBytes += reader.ramBytesUsed();
|
||||
}
|
||||
return sizeInBytes;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<Accountable> getChildResources() {
|
||||
List<Accountable> resources = new ArrayList<>();
|
||||
for(Map.Entry<Integer,BKDReader> ent : readers.entrySet()) {
|
||||
resources.add(Accountables.namedAccountable(readState.fieldInfos.fieldInfo(ent.getKey()).name,
|
||||
ent.getValue()));
|
||||
}
|
||||
return Collections.unmodifiableList(resources);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void checkIntegrity() throws IOException {
|
||||
CodecUtil.checksumEntireFile(dataIn);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
dataIn.close();
|
||||
// Free up heap:
|
||||
readers.clear();
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,140 @@
|
|||
package org.apache.lucene.codecs.lucene60;
|
||||
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.codecs.DimensionalReader;
|
||||
import org.apache.lucene.codecs.DimensionalWriter;
|
||||
import org.apache.lucene.index.DimensionalValues.IntersectVisitor;
|
||||
import org.apache.lucene.index.DimensionalValues.Relation;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.bkd.BKDWriter;
|
||||
|
||||
/** Writes dimensional values */
|
||||
public class Lucene60DimensionalWriter extends DimensionalWriter implements Closeable {
|
||||
|
||||
final IndexOutput dataOut;
|
||||
final Map<String,Long> indexFPs = new HashMap<>();
|
||||
final SegmentWriteState writeState;
|
||||
final int maxPointsInLeafNode;
|
||||
final double maxMBSortInHeap;
|
||||
private boolean closed;
|
||||
|
||||
/** Full constructor */
|
||||
public Lucene60DimensionalWriter(SegmentWriteState writeState, int maxPointsInLeafNode, double maxMBSortInHeap) throws IOException {
|
||||
assert writeState.fieldInfos.hasDimensionalValues();
|
||||
this.writeState = writeState;
|
||||
this.maxPointsInLeafNode = maxPointsInLeafNode;
|
||||
this.maxMBSortInHeap = maxMBSortInHeap;
|
||||
String dataFileName = IndexFileNames.segmentFileName(writeState.segmentInfo.name,
|
||||
writeState.segmentSuffix,
|
||||
Lucene60DimensionalFormat.DATA_EXTENSION);
|
||||
dataOut = writeState.directory.createOutput(dataFileName, writeState.context);
|
||||
boolean success = false;
|
||||
try {
|
||||
CodecUtil.writeIndexHeader(dataOut,
|
||||
Lucene60DimensionalFormat.CODEC_NAME,
|
||||
Lucene60DimensionalFormat.DATA_VERSION_CURRENT,
|
||||
writeState.segmentInfo.getId(),
|
||||
writeState.segmentSuffix);
|
||||
success = true;
|
||||
} finally {
|
||||
if (success == false) {
|
||||
IOUtils.closeWhileHandlingException(dataOut);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** Uses the defaults values for {@code maxPointsInLeafNode} (1024) and {@code maxMBSortInHeap} (16.0) */
|
||||
public Lucene60DimensionalWriter(SegmentWriteState writeState) throws IOException {
|
||||
this(writeState, BKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE, BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeField(FieldInfo fieldInfo, DimensionalReader values) throws IOException {
|
||||
|
||||
try (BKDWriter writer = new BKDWriter(writeState.directory,
|
||||
writeState.segmentInfo.name,
|
||||
fieldInfo.getDimensionCount(),
|
||||
fieldInfo.getDimensionNumBytes(),
|
||||
maxPointsInLeafNode,
|
||||
maxMBSortInHeap)) {
|
||||
|
||||
values.intersect(fieldInfo.name, new IntersectVisitor() {
|
||||
@Override
|
||||
public void visit(int docID) {
|
||||
throw new IllegalStateException();
|
||||
}
|
||||
|
||||
public void visit(int docID, byte[] packedValue) throws IOException {
|
||||
writer.add(packedValue, docID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
|
||||
return Relation.QUERY_CROSSES_CELL;
|
||||
}
|
||||
});
|
||||
|
||||
indexFPs.put(fieldInfo.name, writer.finish(dataOut));
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (closed == false) {
|
||||
CodecUtil.writeFooter(dataOut);
|
||||
dataOut.close();
|
||||
closed = true;
|
||||
|
||||
String indexFileName = IndexFileNames.segmentFileName(writeState.segmentInfo.name,
|
||||
writeState.segmentSuffix,
|
||||
Lucene60DimensionalFormat.INDEX_EXTENSION);
|
||||
// Write index file
|
||||
try (IndexOutput indexOut = writeState.directory.createOutput(indexFileName, writeState.context)) {
|
||||
CodecUtil.writeIndexHeader(indexOut,
|
||||
Lucene60DimensionalFormat.CODEC_NAME,
|
||||
Lucene60DimensionalFormat.INDEX_VERSION_CURRENT,
|
||||
writeState.segmentInfo.getId(),
|
||||
writeState.segmentSuffix);
|
||||
int count = indexFPs.size();
|
||||
indexOut.writeVInt(count);
|
||||
for(Map.Entry<String,Long> ent : indexFPs.entrySet()) {
|
||||
FieldInfo fieldInfo = writeState.fieldInfos.fieldInfo(ent.getKey());
|
||||
if (fieldInfo == null) {
|
||||
throw new IllegalStateException("wrote field=\"" + ent.getKey() + "\" but that field doesn't exist in FieldInfos");
|
||||
}
|
||||
indexOut.writeVInt(fieldInfo.number);
|
||||
indexOut.writeVLong(ent.getValue());
|
||||
}
|
||||
CodecUtil.writeFooter(indexOut);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,311 @@
|
|||
package org.apache.lucene.codecs.lucene60;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.FieldInfosFormat;
|
||||
import org.apache.lucene.document.DimensionalField;
|
||||
import org.apache.lucene.index.CorruptIndexException;
|
||||
import org.apache.lucene.index.DocValuesType;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.IndexOptions;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.store.ChecksumIndexInput;
|
||||
import org.apache.lucene.store.DataOutput;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
|
||||
/**
|
||||
* Lucene 6.0 Field Infos format.
|
||||
* <p>Field names are stored in the field info file, with suffix <tt>.fnm</tt>.
|
||||
* <p>FieldInfos (.fnm) --> Header,FieldsCount, <FieldName,FieldNumber,
|
||||
* FieldBits,DocValuesBits,DocValuesGen,Attributes,DimensionCount,DimensionNumBytes> <sup>FieldsCount</sup>,Footer
|
||||
* <p>Data types:
|
||||
* <ul>
|
||||
* <li>Header --> {@link CodecUtil#checkIndexHeader IndexHeader}</li>
|
||||
* <li>FieldsCount --> {@link DataOutput#writeVInt VInt}</li>
|
||||
* <li>FieldName --> {@link DataOutput#writeString String}</li>
|
||||
* <li>FieldBits, IndexOptions, DocValuesBits --> {@link DataOutput#writeByte Byte}</li>
|
||||
* <li>FieldNumber, DimensionCount, DimensionNumBytes --> {@link DataOutput#writeInt VInt}</li>
|
||||
* <li>Attributes --> {@link DataOutput#writeMapOfStrings Map<String,String>}</li>
|
||||
* <li>DocValuesGen --> {@link DataOutput#writeLong(long) Int64}</li>
|
||||
* <li>Footer --> {@link CodecUtil#writeFooter CodecFooter}</li>
|
||||
* </ul>
|
||||
* Field Descriptions:
|
||||
* <ul>
|
||||
* <li>FieldsCount: the number of fields in this file.</li>
|
||||
* <li>FieldName: name of the field as a UTF-8 String.</li>
|
||||
* <li>FieldNumber: the field's number. Note that unlike previous versions of
|
||||
* Lucene, the fields are not numbered implicitly by their order in the
|
||||
* file, instead explicitly.</li>
|
||||
* <li>FieldBits: a byte containing field options.
|
||||
* <ul>
|
||||
* <li>The low order bit (0x1) is one for fields that have term vectors
|
||||
* stored, and zero for fields without term vectors.</li>
|
||||
* <li>If the second lowest order-bit is set (0x2), norms are omitted for the
|
||||
* indexed field.</li>
|
||||
* <li>If the third lowest-order bit is set (0x4), payloads are stored for the
|
||||
* indexed field.</li>
|
||||
* </ul>
|
||||
* </li>
|
||||
* <li>IndexOptions: a byte containing index options.
|
||||
* <ul>
|
||||
* <li>0: not indexed</li>
|
||||
* <li>1: indexed as DOCS_ONLY</li>
|
||||
* <li>2: indexed as DOCS_AND_FREQS</li>
|
||||
* <li>3: indexed as DOCS_AND_FREQS_AND_POSITIONS</li>
|
||||
* <li>4: indexed as DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS</li>
|
||||
* </ul>
|
||||
* </li>
|
||||
* <li>DocValuesBits: a byte containing per-document value types. The type
|
||||
* recorded as two four-bit integers, with the high-order bits representing
|
||||
* <code>norms</code> options, and the low-order bits representing
|
||||
* {@code DocValues} options. Each four-bit integer can be decoded as such:
|
||||
* <ul>
|
||||
* <li>0: no DocValues for this field.</li>
|
||||
* <li>1: NumericDocValues. ({@link DocValuesType#NUMERIC})</li>
|
||||
* <li>2: BinaryDocValues. ({@code DocValuesType#BINARY})</li>
|
||||
* <li>3: SortedDocValues. ({@code DocValuesType#SORTED})</li>
|
||||
* </ul>
|
||||
* </li>
|
||||
* <li>DocValuesGen is the generation count of the field's DocValues. If this is -1,
|
||||
* there are no DocValues updates to that field. Anything above zero means there
|
||||
* are updates stored by {@link DocValuesFormat}.</li>
|
||||
* <li>Attributes: a key-value map of codec-private attributes.</li>
|
||||
* <li>DimensionCount, DimensionNumBytes: these are non-zero only if the field is
|
||||
* indexed dimensionally using {@link DimensionalField}</li>
|
||||
* </ul>
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public final class Lucene60FieldInfosFormat extends FieldInfosFormat {
|
||||
|
||||
/** Sole constructor. */
|
||||
public Lucene60FieldInfosFormat() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldInfos read(Directory directory, SegmentInfo segmentInfo, String segmentSuffix, IOContext context) throws IOException {
|
||||
final String fileName = IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, EXTENSION);
|
||||
try (ChecksumIndexInput input = directory.openChecksumInput(fileName, context)) {
|
||||
Throwable priorE = null;
|
||||
FieldInfo infos[] = null;
|
||||
try {
|
||||
CodecUtil.checkIndexHeader(input,
|
||||
Lucene60FieldInfosFormat.CODEC_NAME,
|
||||
Lucene60FieldInfosFormat.FORMAT_START,
|
||||
Lucene60FieldInfosFormat.FORMAT_CURRENT,
|
||||
segmentInfo.getId(), segmentSuffix);
|
||||
|
||||
final int size = input.readVInt(); //read in the size
|
||||
infos = new FieldInfo[size];
|
||||
|
||||
// previous field's attribute map, we share when possible:
|
||||
Map<String,String> lastAttributes = Collections.emptyMap();
|
||||
|
||||
for (int i = 0; i < size; i++) {
|
||||
String name = input.readString();
|
||||
final int fieldNumber = input.readVInt();
|
||||
if (fieldNumber < 0) {
|
||||
throw new CorruptIndexException("invalid field number for field: " + name + ", fieldNumber=" + fieldNumber, input);
|
||||
}
|
||||
byte bits = input.readByte();
|
||||
boolean storeTermVector = (bits & STORE_TERMVECTOR) != 0;
|
||||
boolean omitNorms = (bits & OMIT_NORMS) != 0;
|
||||
boolean storePayloads = (bits & STORE_PAYLOADS) != 0;
|
||||
|
||||
final IndexOptions indexOptions = getIndexOptions(input, input.readByte());
|
||||
|
||||
// DV Types are packed in one byte
|
||||
final DocValuesType docValuesType = getDocValuesType(input, input.readByte());
|
||||
final long dvGen = input.readLong();
|
||||
Map<String,String> attributes = input.readMapOfStrings();
|
||||
// just use the last field's map if its the same
|
||||
if (attributes.equals(lastAttributes)) {
|
||||
attributes = lastAttributes;
|
||||
}
|
||||
lastAttributes = attributes;
|
||||
int dimensionCount = input.readVInt();
|
||||
int dimensionNumBytes;
|
||||
if (dimensionCount != 0) {
|
||||
dimensionNumBytes = input.readVInt();
|
||||
} else {
|
||||
dimensionNumBytes = 0;
|
||||
}
|
||||
|
||||
try {
|
||||
infos[i] = new FieldInfo(name, fieldNumber, storeTermVector, omitNorms, storePayloads,
|
||||
indexOptions, docValuesType, dvGen, attributes,
|
||||
dimensionCount, dimensionNumBytes);
|
||||
infos[i].checkConsistency();
|
||||
} catch (IllegalStateException e) {
|
||||
throw new CorruptIndexException("invalid fieldinfo for field: " + name + ", fieldNumber=" + fieldNumber, input, e);
|
||||
}
|
||||
}
|
||||
} catch (Throwable exception) {
|
||||
priorE = exception;
|
||||
} finally {
|
||||
CodecUtil.checkFooter(input, priorE);
|
||||
}
|
||||
return new FieldInfos(infos);
|
||||
}
|
||||
}
|
||||
|
||||
static {
|
||||
// We "mirror" DocValues enum values with the constants below; let's try to ensure if we add a new DocValuesType while this format is
|
||||
// still used for writing, we remember to fix this encoding:
|
||||
assert DocValuesType.values().length == 6;
|
||||
}
|
||||
|
||||
private static byte docValuesByte(DocValuesType type) {
|
||||
switch(type) {
|
||||
case NONE:
|
||||
return 0;
|
||||
case NUMERIC:
|
||||
return 1;
|
||||
case BINARY:
|
||||
return 2;
|
||||
case SORTED:
|
||||
return 3;
|
||||
case SORTED_SET:
|
||||
return 4;
|
||||
case SORTED_NUMERIC:
|
||||
return 5;
|
||||
default:
|
||||
// BUG
|
||||
throw new AssertionError("unhandled DocValuesType: " + type);
|
||||
}
|
||||
}
|
||||
|
||||
private static DocValuesType getDocValuesType(IndexInput input, byte b) throws IOException {
|
||||
switch(b) {
|
||||
case 0:
|
||||
return DocValuesType.NONE;
|
||||
case 1:
|
||||
return DocValuesType.NUMERIC;
|
||||
case 2:
|
||||
return DocValuesType.BINARY;
|
||||
case 3:
|
||||
return DocValuesType.SORTED;
|
||||
case 4:
|
||||
return DocValuesType.SORTED_SET;
|
||||
case 5:
|
||||
return DocValuesType.SORTED_NUMERIC;
|
||||
default:
|
||||
throw new CorruptIndexException("invalid docvalues byte: " + b, input);
|
||||
}
|
||||
}
|
||||
|
||||
static {
|
||||
// We "mirror" IndexOptions enum values with the constants below; let's try to ensure if we add a new IndexOption while this format is
|
||||
// still used for writing, we remember to fix this encoding:
|
||||
assert IndexOptions.values().length == 5;
|
||||
}
|
||||
|
||||
private static byte indexOptionsByte(IndexOptions indexOptions) {
|
||||
switch (indexOptions) {
|
||||
case NONE:
|
||||
return 0;
|
||||
case DOCS:
|
||||
return 1;
|
||||
case DOCS_AND_FREQS:
|
||||
return 2;
|
||||
case DOCS_AND_FREQS_AND_POSITIONS:
|
||||
return 3;
|
||||
case DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS:
|
||||
return 4;
|
||||
default:
|
||||
// BUG:
|
||||
throw new AssertionError("unhandled IndexOptions: " + indexOptions);
|
||||
}
|
||||
}
|
||||
|
||||
private static IndexOptions getIndexOptions(IndexInput input, byte b) throws IOException {
|
||||
switch (b) {
|
||||
case 0:
|
||||
return IndexOptions.NONE;
|
||||
case 1:
|
||||
return IndexOptions.DOCS;
|
||||
case 2:
|
||||
return IndexOptions.DOCS_AND_FREQS;
|
||||
case 3:
|
||||
return IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
|
||||
case 4:
|
||||
return IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS;
|
||||
default:
|
||||
// BUG
|
||||
throw new CorruptIndexException("invalid IndexOptions byte: " + b, input);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(Directory directory, SegmentInfo segmentInfo, String segmentSuffix, FieldInfos infos, IOContext context) throws IOException {
|
||||
final String fileName = IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, EXTENSION);
|
||||
try (IndexOutput output = directory.createOutput(fileName, context)) {
|
||||
CodecUtil.writeIndexHeader(output, Lucene60FieldInfosFormat.CODEC_NAME, Lucene60FieldInfosFormat.FORMAT_CURRENT, segmentInfo.getId(), segmentSuffix);
|
||||
output.writeVInt(infos.size());
|
||||
for (FieldInfo fi : infos) {
|
||||
fi.checkConsistency();
|
||||
|
||||
output.writeString(fi.name);
|
||||
output.writeVInt(fi.number);
|
||||
|
||||
byte bits = 0x0;
|
||||
if (fi.hasVectors()) bits |= STORE_TERMVECTOR;
|
||||
if (fi.omitsNorms()) bits |= OMIT_NORMS;
|
||||
if (fi.hasPayloads()) bits |= STORE_PAYLOADS;
|
||||
output.writeByte(bits);
|
||||
|
||||
output.writeByte(indexOptionsByte(fi.getIndexOptions()));
|
||||
|
||||
// pack the DV type and hasNorms in one byte
|
||||
output.writeByte(docValuesByte(fi.getDocValuesType()));
|
||||
output.writeLong(fi.getDocValuesGen());
|
||||
output.writeMapOfStrings(fi.attributes());
|
||||
int dimensionCount = fi.getDimensionCount();
|
||||
output.writeVInt(dimensionCount);
|
||||
if (dimensionCount != 0) {
|
||||
output.writeVInt(fi.getDimensionNumBytes());
|
||||
}
|
||||
}
|
||||
CodecUtil.writeFooter(output);
|
||||
}
|
||||
}
|
||||
|
||||
/** Extension of field infos */
|
||||
static final String EXTENSION = "fnm";
|
||||
|
||||
// Codec header
|
||||
static final String CODEC_NAME = "Lucene60FieldInfos";
|
||||
static final int FORMAT_START = 0;
|
||||
static final int FORMAT_CURRENT = FORMAT_START;
|
||||
|
||||
// Field flags
|
||||
static final byte STORE_TERMVECTOR = 0x1;
|
||||
static final byte OMIT_NORMS = 0x2;
|
||||
static final byte STORE_PAYLOADS = 0x4;
|
||||
}
|
|
@ -0,0 +1,412 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
/**
|
||||
* 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.Lucene60DimensionalFormat Dimensional values}.
|
||||
* Optional pair of files, recording dimesionally indexed fields, to enable fast
|
||||
* numeric range filtering and large numeric values like BigInteger and BigDecimal (1D)
|
||||
* and geo 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.Lucene60DimensionalFormat Dimensional values}</td>
|
||||
* <td>.dii, .dim</td>
|
||||
* <td>Holds dimensionally indexed fields, 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 & 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<String,String> 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>
|
||||
* </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>
|
||||
*/
|
||||
package org.apache.lucene.codecs.lucene60;
|
|
@ -33,6 +33,7 @@ import java.util.Locale;
|
|||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.DimensionalReader;
|
||||
import org.apache.lucene.codecs.DocValuesProducer;
|
||||
import org.apache.lucene.codecs.NormsProducer;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
|
@ -212,6 +213,9 @@ public class CheckIndex implements Closeable {
|
|||
|
||||
/** Status for testing of DocValues (null if DocValues could not be tested). */
|
||||
public DocValuesStatus docValuesStatus;
|
||||
|
||||
/** Status for testing of DimensionalValues (null if DimensionalValues could not be tested). */
|
||||
public DimensionalValuesStatus dimensionalValuesStatus;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -351,6 +355,24 @@ public class CheckIndex implements Closeable {
|
|||
/** Exception thrown during doc values test (null on success) */
|
||||
public Throwable error = null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Status from testing DimensionalValues
|
||||
*/
|
||||
public static final class DimensionalValuesStatus {
|
||||
|
||||
DimensionalValuesStatus() {
|
||||
}
|
||||
|
||||
/** Total number of dimensional values points tested. */
|
||||
public long totalValuePoints;
|
||||
|
||||
/** Total number of fields with dimensional values. */
|
||||
public int totalValueFields;
|
||||
|
||||
/** Exception thrown during doc values test (null on success) */
|
||||
public Throwable error = null;
|
||||
}
|
||||
}
|
||||
|
||||
/** Create a new CheckIndex on the directory. */
|
||||
|
@ -704,6 +726,9 @@ public class CheckIndex implements Closeable {
|
|||
// Test Docvalues
|
||||
segInfoStat.docValuesStatus = testDocValues(reader, infoStream, failFast);
|
||||
|
||||
// Test DimensionalValues
|
||||
segInfoStat.dimensionalValuesStatus = testDimensionalValues(reader, infoStream, failFast);
|
||||
|
||||
// Rethrow the first exception we encountered
|
||||
// This will cause stats for failed segments to be incremented properly
|
||||
if (segInfoStat.liveDocStatus.error != null) {
|
||||
|
@ -1659,7 +1684,102 @@ public class CheckIndex implements Closeable {
|
|||
|
||||
return status;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Test the dimensional values index.
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public static Status.DimensionalValuesStatus testDimensionalValues(CodecReader reader, PrintStream infoStream, boolean failFast) throws IOException {
|
||||
FieldInfos fieldInfos = reader.getFieldInfos();
|
||||
Status.DimensionalValuesStatus status = new Status.DimensionalValuesStatus();
|
||||
try {
|
||||
if (fieldInfos.hasDimensionalValues()) {
|
||||
DimensionalReader values = reader.getDimensionalReader();
|
||||
if (values == null) {
|
||||
throw new RuntimeException("there are fields with dimensional values, but reader.getDimensionalRader() is null");
|
||||
}
|
||||
for (FieldInfo fieldInfo : fieldInfos) {
|
||||
if (fieldInfo.getDimensionCount() > 0) {
|
||||
status.totalValueFields++;
|
||||
int dimCount = fieldInfo.getDimensionCount();
|
||||
int bytesPerDim = fieldInfo.getDimensionNumBytes();
|
||||
byte[] lastMinPackedValue = new byte[dimCount*bytesPerDim];
|
||||
BytesRef lastMinPacked = new BytesRef(lastMinPackedValue);
|
||||
byte[] lastMaxPackedValue = new byte[dimCount*bytesPerDim];
|
||||
BytesRef lastMaxPacked = new BytesRef(lastMaxPackedValue);
|
||||
BytesRef scratch = new BytesRef();
|
||||
lastMaxPacked.length = bytesPerDim;
|
||||
lastMinPacked.length = bytesPerDim;
|
||||
scratch.length = bytesPerDim;
|
||||
values.intersect(fieldInfo.name,
|
||||
new DimensionalValues.IntersectVisitor() {
|
||||
@Override
|
||||
public void visit(int docID) {
|
||||
throw new RuntimeException("codec called IntersectVisitor.visit without a packed value for docID=" + docID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void visit(int docID, byte[] packedValue) {
|
||||
checkPackedValue("packed value", packedValue, docID);
|
||||
scratch.bytes = packedValue;
|
||||
|
||||
for(int dim=0;dim<dimCount;dim++) {
|
||||
lastMaxPacked.offset = bytesPerDim * dim;
|
||||
lastMinPacked.offset = bytesPerDim * dim;
|
||||
scratch.offset = bytesPerDim * dim;
|
||||
|
||||
if (scratch.compareTo(lastMinPacked) < 0) {
|
||||
// This doc's point, in this dimension, is lower than the minimum value of the last cell checked:
|
||||
throw new RuntimeException("packed value " + Arrays.toString(packedValue) + " for docID=" + docID + " is out-of-bounds of the last cell min=" + Arrays.toString(lastMinPackedValue) + " max=" + Arrays.toString(lastMaxPackedValue) + " dim=" + dim);
|
||||
}
|
||||
if (scratch.compareTo(lastMaxPacked) > 0) {
|
||||
// This doc's point, in this dimension, is greater than the maximum value of the last cell checked:
|
||||
throw new RuntimeException("packed value " + Arrays.toString(packedValue) + " for docID=" + docID + " is out-of-bounds of the last cell min=" + Arrays.toString(lastMinPackedValue) + " max=" + Arrays.toString(lastMaxPackedValue) + " dim=" + dim);
|
||||
}
|
||||
}
|
||||
|
||||
status.totalValuePoints++;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DimensionalValues.Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
|
||||
checkPackedValue("min packed value", minPackedValue, -1);
|
||||
System.arraycopy(minPackedValue, 0, lastMinPackedValue, 0, minPackedValue.length);
|
||||
checkPackedValue("max packed value", maxPackedValue, -1);
|
||||
System.arraycopy(maxPackedValue, 0, lastMaxPackedValue, 0, maxPackedValue.length);
|
||||
|
||||
// We always pretend the query shape is so complex that it crosses every cell, so
|
||||
// that packedValue is passed for every document
|
||||
return DimensionalValues.Relation.QUERY_CROSSES_CELL;
|
||||
}
|
||||
|
||||
private void checkPackedValue(String desc, byte[] packedValue, int docID) {
|
||||
if (packedValue == null) {
|
||||
throw new RuntimeException(desc + " is null for docID=" + docID);
|
||||
}
|
||||
|
||||
if (packedValue.length != dimCount * bytesPerDim) {
|
||||
throw new RuntimeException(desc + " has incorrect length=" + packedValue.length + " vs expected=" + (dimCount * bytesPerDim) + " for docID=" + docID);
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
}
|
||||
} catch (Throwable e) {
|
||||
if (failFast) {
|
||||
IOUtils.reThrow(e);
|
||||
}
|
||||
msg(infoStream, "ERROR: " + e);
|
||||
status.error = e;
|
||||
if (infoStream != null) {
|
||||
e.printStackTrace(infoStream);
|
||||
}
|
||||
}
|
||||
|
||||
return status;
|
||||
}
|
||||
|
||||
/**
|
||||
* Test stored fields.
|
||||
* @lucene.experimental
|
||||
|
|
|
@ -35,7 +35,8 @@ public abstract class DimensionalValues {
|
|||
/** Return this if the cell and query do not overlap */
|
||||
QUERY_OUTSIDE_CELL,
|
||||
/** Return this if the cell partially overlapps the query */
|
||||
QUERY_CROSSES_CELL};
|
||||
QUERY_CROSSES_CELL
|
||||
};
|
||||
|
||||
/** We recurse the BKD tree, using a provided instance of this to guide the recursion.
|
||||
*
|
||||
|
@ -54,6 +55,8 @@ public abstract class DimensionalValues {
|
|||
Relation compare(byte[] minPackedValue, byte[] maxPackedValue);
|
||||
}
|
||||
|
||||
/** Finds all documents and points matching the provided visitor */
|
||||
/** Finds all documents and points matching the provided visitor.
|
||||
* This method does not enforce live docs, so it's up to the caller
|
||||
* to test whether each document is deleted, if necessary. */
|
||||
public abstract void intersect(String fieldName, IntersectVisitor visitor) throws IOException;
|
||||
}
|
||||
|
|
|
@ -82,7 +82,7 @@ public class FieldInfos implements Iterable<FieldInfo> {
|
|||
hasNorms |= info.hasNorms();
|
||||
hasDocValues |= info.getDocValuesType() != DocValuesType.NONE;
|
||||
hasPayloads |= info.hasPayloads();
|
||||
hasDimensionalValues |= info.getDimensionCount() != 0;
|
||||
hasDimensionalValues |= (info.getDimensionCount() != 0);
|
||||
}
|
||||
|
||||
this.hasVectors = hasVectors;
|
||||
|
|
|
@ -51,6 +51,10 @@ class MultiDimensionalValues extends DimensionalValues {
|
|||
}
|
||||
}
|
||||
|
||||
if (values.isEmpty()) {
|
||||
return null;
|
||||
}
|
||||
|
||||
return new MultiDimensionalValues(values, docBases);
|
||||
}
|
||||
|
||||
|
|
|
@ -46,7 +46,7 @@ public final class SlowCodecReaderWrapper {
|
|||
*/
|
||||
public static CodecReader wrap(final LeafReader reader) throws IOException {
|
||||
if (reader instanceof CodecReader) {
|
||||
return (CodecReader)reader;
|
||||
return (CodecReader) reader;
|
||||
} else {
|
||||
// simulate it slowly, over the leafReader api:
|
||||
reader.checkIntegrity();
|
||||
|
@ -130,6 +130,9 @@ public final class SlowCodecReaderWrapper {
|
|||
}
|
||||
|
||||
private static DimensionalReader dimensionalValuesToReader(DimensionalValues values) {
|
||||
if (values == null) {
|
||||
return null;
|
||||
}
|
||||
return new DimensionalReader() {
|
||||
@Override
|
||||
public void intersect(String fieldName, IntersectVisitor visitor) throws IOException {
|
||||
|
|
|
@ -79,8 +79,10 @@ public class BKDWriter implements Closeable {
|
|||
/** How many bytes each docs takes in the fixed-width offline format */
|
||||
private final int bytesPerDoc;
|
||||
|
||||
/** Default maximum number of point in each leaf block */
|
||||
public static final int DEFAULT_MAX_POINTS_IN_LEAF_NODE = 1024;
|
||||
|
||||
/** Default maximum heap to use, before spilling to (slower) disk */
|
||||
public static final float DEFAULT_MAX_MB_SORT_IN_HEAP = 16.0f;
|
||||
|
||||
/** Maximum number of dimensions */
|
||||
|
@ -370,6 +372,11 @@ public class BKDWriter implements Closeable {
|
|||
|
||||
// TODO: specialize the 1D case? it's much faster at indexing time (no partitioning on recruse...)
|
||||
|
||||
// Catch user silliness:
|
||||
if (heapPointWriter == null && tempInput == null) {
|
||||
throw new IllegalStateException("already finished");
|
||||
}
|
||||
|
||||
if (offlinePointWriter != null) {
|
||||
offlinePointWriter.close();
|
||||
}
|
||||
|
|
|
@ -13,4 +13,4 @@
|
|||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
|
||||
org.apache.lucene.codecs.lucene54.Lucene54Codec
|
||||
org.apache.lucene.codecs.lucene60.Lucene60Codec
|
||||
|
|
|
@ -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.lucene54.Lucene54Codec;
|
||||
import org.apache.lucene.codecs.lucene60.Lucene60Codec;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.StoredField;
|
||||
import org.apache.lucene.index.BaseStoredFieldsFormatTestCase;
|
||||
|
@ -34,7 +34,7 @@ import com.carrotsearch.randomizedtesting.generators.RandomPicks;
|
|||
public class TestLucene50StoredFieldsFormatHighCompression extends BaseStoredFieldsFormatTestCase {
|
||||
@Override
|
||||
protected Codec getCodec() {
|
||||
return new Lucene54Codec(Mode.BEST_COMPRESSION);
|
||||
return new Lucene60Codec(Mode.BEST_COMPRESSION);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -45,7 +45,7 @@ public class TestLucene50StoredFieldsFormatHighCompression extends BaseStoredFie
|
|||
Directory dir = newDirectory();
|
||||
for (int i = 0; i < 10; i++) {
|
||||
IndexWriterConfig iwc = newIndexWriterConfig();
|
||||
iwc.setCodec(new Lucene54Codec(RandomPicks.randomFrom(random(), Mode.values())));
|
||||
iwc.setCodec(new Lucene60Codec(RandomPicks.randomFrom(random(), Mode.values())));
|
||||
IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig());
|
||||
Document doc = new Document();
|
||||
doc.add(new StoredField("field1", "value1"));
|
||||
|
@ -72,7 +72,7 @@ public class TestLucene50StoredFieldsFormatHighCompression extends BaseStoredFie
|
|||
|
||||
public void testInvalidOptions() throws Exception {
|
||||
try {
|
||||
new Lucene54Codec(null);
|
||||
new Lucene60Codec(null);
|
||||
fail("didn't hit exception");
|
||||
} catch (NullPointerException expected) {
|
||||
// expected
|
||||
|
|
|
@ -18,18 +18,17 @@ package org.apache.lucene.codecs.lucene53;
|
|||
*/
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.lucene54.Lucene54Codec;
|
||||
import org.apache.lucene.codecs.lucene60.Lucene60Codec;
|
||||
import org.apache.lucene.index.BaseNormsFormatTestCase;
|
||||
|
||||
/**
|
||||
* Tests Lucene53NormsFormat
|
||||
*/
|
||||
public class TestLucene53NormsFormat extends BaseNormsFormatTestCase {
|
||||
private final Codec codec = new Lucene54Codec();
|
||||
private final Codec codec = new Lucene60Codec();
|
||||
|
||||
@Override
|
||||
protected Codec getCodec() {
|
||||
return codec;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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=Lucene53 -Dtestcase=Test2BTerms
|
||||
// ant test -Dtests.monster=true -Dtests.heapsize=8g -Dtests.codec=Lucene60 -Dtestcase=Test2BTerms
|
||||
//
|
||||
@SuppressCodecs({ "SimpleText", "Memory", "Direct" })
|
||||
@Monster("very slow, use 5g minimum heap")
|
||||
|
|
|
@ -24,7 +24,13 @@ import java.util.BitSet;
|
|||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.codecs.simpletext.SimpleTextCodec;
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.DimensionalFormat;
|
||||
import org.apache.lucene.codecs.DimensionalReader;
|
||||
import org.apache.lucene.codecs.DimensionalWriter;
|
||||
import org.apache.lucene.codecs.FilterCodec;
|
||||
import org.apache.lucene.codecs.lucene60.Lucene60DimensionalReader;
|
||||
import org.apache.lucene.codecs.lucene60.Lucene60DimensionalWriter;
|
||||
import org.apache.lucene.document.DimensionalField;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
|
@ -42,16 +48,13 @@ import org.apache.lucene.util.TestUtil;
|
|||
import org.apache.lucene.util.bkd.BKDUtil;
|
||||
import org.apache.lucene.util.bkd.BKDWriter;
|
||||
|
||||
// TODO: randomize the bkd settings w/ Lucene60DimensionalFormat
|
||||
|
||||
// TODO: factor out a BaseTestDimensionFormat
|
||||
|
||||
public class TestDimensionalValues extends LuceneTestCase {
|
||||
public void testBasic() throws Exception {
|
||||
Directory dir = getDirectory(20);
|
||||
// TODO: randomize codec once others support dimensional format
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
iwc.setCodec(new SimpleTextCodec());
|
||||
IndexWriterConfig iwc = newIndexWriterConfig();
|
||||
iwc.setMergePolicy(newLogMergePolicy());
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
byte[] point = new byte[4];
|
||||
|
@ -84,13 +87,13 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
assertEquals(docID, BKDUtil.bytesToInt(packedValue, 0));
|
||||
}
|
||||
});
|
||||
assertEquals(20, seen.cardinality());
|
||||
IOUtils.close(r, dir);
|
||||
}
|
||||
|
||||
public void testMerge() throws Exception {
|
||||
Directory dir = getDirectory(20);
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
iwc.setCodec(new SimpleTextCodec());
|
||||
IndexWriterConfig iwc = newIndexWriterConfig();
|
||||
iwc.setMergePolicy(newLogMergePolicy());
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
byte[] point = new byte[4];
|
||||
|
@ -126,6 +129,55 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
assertEquals(docID, BKDUtil.bytesToInt(packedValue, 0));
|
||||
}
|
||||
});
|
||||
assertEquals(20, seen.cardinality());
|
||||
IOUtils.close(r, dir);
|
||||
}
|
||||
|
||||
public void testAllDimensionalDocsDeletedInSegment() throws Exception {
|
||||
Directory dir = getDirectory(20);
|
||||
IndexWriterConfig iwc = newIndexWriterConfig();
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
byte[] point = new byte[4];
|
||||
for(int i=0;i<10;i++) {
|
||||
Document doc = new Document();
|
||||
BKDUtil.intToBytes(i, point, 0);
|
||||
doc.add(new DimensionalField("dim", point));
|
||||
doc.add(new NumericDocValuesField("id", i));
|
||||
doc.add(newStringField("x", "x", Field.Store.NO));
|
||||
w.addDocument(doc);
|
||||
}
|
||||
w.addDocument(new Document());
|
||||
w.deleteDocuments(new Term("x", "x"));
|
||||
if (random().nextBoolean()) {
|
||||
w.forceMerge(1);
|
||||
}
|
||||
w.close();
|
||||
DirectoryReader r = DirectoryReader.open(dir);
|
||||
assertEquals(1, r.numDocs());
|
||||
DimensionalValues values = MultiDimensionalValues.get(r);
|
||||
Bits liveDocs = MultiFields.getLiveDocs(r);
|
||||
NumericDocValues idValues = MultiDocValues.getNumericValues(r, "id");
|
||||
|
||||
if (values != null) {
|
||||
BitSet seen = new BitSet();
|
||||
values.intersect("dim",
|
||||
new IntersectVisitor() {
|
||||
@Override
|
||||
public Relation compare(byte[] minPacked, byte[] maxPacked) {
|
||||
return Relation.QUERY_CROSSES_CELL;
|
||||
}
|
||||
public void visit(int docID) {
|
||||
throw new IllegalStateException();
|
||||
}
|
||||
public void visit(int docID, byte[] packedValue) {
|
||||
if (liveDocs.get(docID)) {
|
||||
seen.set(docID);
|
||||
}
|
||||
assertEquals(idValues.get(docID), BKDUtil.bytesToInt(packedValue, 0));
|
||||
}
|
||||
});
|
||||
assertEquals(0, seen.cardinality());
|
||||
}
|
||||
IOUtils.close(r, dir);
|
||||
}
|
||||
|
||||
|
@ -146,7 +198,6 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
docValues[docID] = values;
|
||||
}
|
||||
|
||||
double maxMBHeap = 0.05;
|
||||
// Keep retrying until we 1) we allow a big enough heap, and 2) we hit a random IOExc from MDW:
|
||||
boolean done = false;
|
||||
while (done == false) {
|
||||
|
@ -157,7 +208,18 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
if (dir instanceof MockDirectoryWrapper) {
|
||||
dir.setEnableVirusScanner(false);
|
||||
}
|
||||
verify(dir, docValues, null, numDims, numBytesPerDim, 50, maxMBHeap);
|
||||
verify(dir, docValues, null, numDims, numBytesPerDim, true);
|
||||
} catch (IllegalStateException ise) {
|
||||
if (ise.getMessage().contains("this writer hit an unrecoverable error")) {
|
||||
Throwable cause = ise.getCause();
|
||||
if (cause != null && cause.getMessage().contains("a random IOException")) {
|
||||
done = true;
|
||||
} else {
|
||||
throw ise;
|
||||
}
|
||||
} else {
|
||||
throw ise;
|
||||
}
|
||||
} catch (AssertionError ae) {
|
||||
if (ae.getMessage().contains("does not exist; files=")) {
|
||||
// OK: likely we threw the random IOExc when IW was asserting the commit files exist
|
||||
|
@ -168,10 +230,9 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
} catch (IllegalArgumentException iae) {
|
||||
// This just means we got a too-small maxMB for the maxPointsInLeafNode; just retry w/ more heap
|
||||
assertTrue(iae.getMessage().contains("either increase maxMBSortInHeap or decrease maxPointsInLeafNode"));
|
||||
System.out.println(" more heap");
|
||||
maxMBHeap *= 1.25;
|
||||
} catch (IOException ioe) {
|
||||
if (ioe.getMessage().contains("a random IOException")) {
|
||||
String message = ioe.getMessage();
|
||||
if (message.contains("a random IOException") || message.contains("background merge hit exception")) {
|
||||
// BKDWriter should fully clean up after itself:
|
||||
done = true;
|
||||
} else {
|
||||
|
@ -266,7 +327,6 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
int numBytesPerDim = TestUtil.nextInt(random(), 2, 30);
|
||||
int numDims = TestUtil.nextInt(random(), 1, 5);
|
||||
IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random()));
|
||||
iwc.setCodec(new SimpleTextCodec());
|
||||
// We rely on docIDs not changing:
|
||||
iwc.setMergePolicy(newLogMergePolicy());
|
||||
RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
|
||||
|
@ -391,27 +451,23 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
doTestRandomBinary(10000);
|
||||
}
|
||||
|
||||
// TODO: enable this, but not using simple text:
|
||||
/*
|
||||
@Nightly
|
||||
public void testRandomBinaryBig() throws Exception {
|
||||
assumeFalse("too slow with SimpleText", Codec.getDefault().getName().equals("SimpleText"));
|
||||
doTestRandomBinary(200000);
|
||||
}
|
||||
*/
|
||||
|
||||
// Suddenly add dimensional values to an existing field:
|
||||
public void testUpgradeFieldToDimensional() throws Exception {
|
||||
Directory dir = getDirectory(1);
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
iwc.setCodec(new SimpleTextCodec());
|
||||
IndexWriterConfig iwc = newIndexWriterConfig();
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(newStringField("dim", "foo", Field.Store.NO));
|
||||
w.addDocument(doc);
|
||||
w.close();
|
||||
|
||||
iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
iwc.setCodec(new SimpleTextCodec());
|
||||
iwc = newIndexWriterConfig();
|
||||
w = new IndexWriter(dir, iwc);
|
||||
doc.add(new DimensionalField("dim", new byte[4]));
|
||||
w.close();
|
||||
|
@ -423,7 +479,6 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
public void testIllegalDimChangeOneDoc() throws Exception {
|
||||
Directory dir = getDirectory(1);
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
iwc.setCodec(new SimpleTextCodec());
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new DimensionalField("dim", new byte[4]));
|
||||
|
@ -441,7 +496,6 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
public void testIllegalDimChangeTwoDocs() throws Exception {
|
||||
Directory dir = getDirectory(1);
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
iwc.setCodec(new SimpleTextCodec());
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new DimensionalField("dim", new byte[4]));
|
||||
|
@ -461,7 +515,6 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
public void testIllegalDimChangeTwoSegments() throws Exception {
|
||||
Directory dir = getDirectory(1);
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
iwc.setCodec(new SimpleTextCodec());
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new DimensionalField("dim", new byte[4]));
|
||||
|
@ -482,14 +535,12 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
public void testIllegalDimChangeTwoWriters() throws Exception {
|
||||
Directory dir = getDirectory(1);
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
iwc.setCodec(new SimpleTextCodec());
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new DimensionalField("dim", new byte[4]));
|
||||
w.addDocument(doc);
|
||||
w.close();
|
||||
iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
iwc.setCodec(new SimpleTextCodec());
|
||||
w = new IndexWriter(dir, iwc);
|
||||
doc = new Document();
|
||||
doc.add(new DimensionalField("dim", new byte[4], new byte[4]));
|
||||
|
@ -506,7 +557,6 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
public void testIllegalDimChangeViaAddIndexesDirectory() throws Exception {
|
||||
Directory dir = getDirectory(1);
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
iwc.setCodec(new SimpleTextCodec());
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new DimensionalField("dim", new byte[4]));
|
||||
|
@ -515,7 +565,6 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
|
||||
Directory dir2 = getDirectory(1);
|
||||
iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
iwc.setCodec(new SimpleTextCodec());
|
||||
w = new IndexWriter(dir2, iwc);
|
||||
doc = new Document();
|
||||
doc.add(new DimensionalField("dim", new byte[4], new byte[4]));
|
||||
|
@ -531,7 +580,6 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
public void testIllegalDimChangeViaAddIndexesCodecReader() throws Exception {
|
||||
Directory dir = getDirectory(1);
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
iwc.setCodec(new SimpleTextCodec());
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new DimensionalField("dim", new byte[4]));
|
||||
|
@ -540,7 +588,6 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
|
||||
Directory dir2 = getDirectory(1);
|
||||
iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
iwc.setCodec(new SimpleTextCodec());
|
||||
w = new IndexWriter(dir2, iwc);
|
||||
doc = new Document();
|
||||
doc.add(new DimensionalField("dim", new byte[4], new byte[4]));
|
||||
|
@ -557,7 +604,6 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
public void testIllegalDimChangeViaAddIndexesSlowCodecReader() throws Exception {
|
||||
Directory dir = getDirectory(1);
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
iwc.setCodec(new SimpleTextCodec());
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new DimensionalField("dim", new byte[4]));
|
||||
|
@ -566,7 +612,6 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
|
||||
Directory dir2 = getDirectory(1);
|
||||
iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
iwc.setCodec(new SimpleTextCodec());
|
||||
w = new IndexWriter(dir2, iwc);
|
||||
doc = new Document();
|
||||
doc.add(new DimensionalField("dim", new byte[4], new byte[4]));
|
||||
|
@ -583,7 +628,6 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
public void testIllegalTooManyDimensions() throws Exception {
|
||||
Directory dir = getDirectory(1);
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
iwc.setCodec(new SimpleTextCodec());
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
byte[][] values = new byte[BKDWriter.MAX_DIMS+1][];
|
||||
|
@ -604,7 +648,6 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
public void testIllegalNumBytesChangeOneDoc() throws Exception {
|
||||
Directory dir = getDirectory(1);
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
iwc.setCodec(new SimpleTextCodec());
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new DimensionalField("dim", new byte[4]));
|
||||
|
@ -622,7 +665,6 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
public void testIllegalNumBytesChangeTwoDocs() throws Exception {
|
||||
Directory dir = getDirectory(1);
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
iwc.setCodec(new SimpleTextCodec());
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new DimensionalField("dim", new byte[4]));
|
||||
|
@ -642,7 +684,6 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
public void testIllegalNumBytesChangeTwoSegments() throws Exception {
|
||||
Directory dir = getDirectory(1);
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
iwc.setCodec(new SimpleTextCodec());
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new DimensionalField("dim", new byte[4]));
|
||||
|
@ -663,14 +704,12 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
public void testIllegalNumBytesChangeTwoWriters() throws Exception {
|
||||
Directory dir = getDirectory(1);
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
iwc.setCodec(new SimpleTextCodec());
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new DimensionalField("dim", new byte[4]));
|
||||
w.addDocument(doc);
|
||||
w.close();
|
||||
iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
iwc.setCodec(new SimpleTextCodec());
|
||||
w = new IndexWriter(dir, iwc);
|
||||
doc = new Document();
|
||||
doc.add(new DimensionalField("dim", new byte[6]));
|
||||
|
@ -687,7 +726,6 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
public void testIllegalNumBytesChangeViaAddIndexesDirectory() throws Exception {
|
||||
Directory dir = getDirectory(1);
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
iwc.setCodec(new SimpleTextCodec());
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new DimensionalField("dim", new byte[4]));
|
||||
|
@ -696,7 +734,6 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
|
||||
Directory dir2 = getDirectory(1);
|
||||
iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
iwc.setCodec(new SimpleTextCodec());
|
||||
w = new IndexWriter(dir2, iwc);
|
||||
doc = new Document();
|
||||
doc.add(new DimensionalField("dim", new byte[6]));
|
||||
|
@ -712,7 +749,6 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
public void testIllegalNumBytesChangeViaAddIndexesCodecReader() throws Exception {
|
||||
Directory dir = getDirectory(1);
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
iwc.setCodec(new SimpleTextCodec());
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new DimensionalField("dim", new byte[4]));
|
||||
|
@ -721,7 +757,6 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
|
||||
Directory dir2 = getDirectory(1);
|
||||
iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
iwc.setCodec(new SimpleTextCodec());
|
||||
w = new IndexWriter(dir2, iwc);
|
||||
doc = new Document();
|
||||
doc.add(new DimensionalField("dim", new byte[6]));
|
||||
|
@ -738,7 +773,6 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
public void testIllegalNumBytesChangeViaAddIndexesSlowCodecReader() throws Exception {
|
||||
Directory dir = getDirectory(1);
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
iwc.setCodec(new SimpleTextCodec());
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new DimensionalField("dim", new byte[4]));
|
||||
|
@ -747,7 +781,6 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
|
||||
Directory dir2 = getDirectory(1);
|
||||
iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
iwc.setCodec(new SimpleTextCodec());
|
||||
w = new IndexWriter(dir2, iwc);
|
||||
doc = new Document();
|
||||
doc.add(new DimensionalField("dim", new byte[6]));
|
||||
|
@ -781,14 +814,41 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
verify(docValues, null, numDims, numBytesPerDim);
|
||||
}
|
||||
|
||||
private Codec getCodec() {
|
||||
if (Codec.getDefault().getName().equals("Lucene60")) {
|
||||
int maxPointsInLeafNode = TestUtil.nextInt(random(), 50, 500);
|
||||
double maxMBSortInHeap = 0.1 + (3*random().nextDouble());
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: using Lucene60DimensionalFormat with maxPointsInLeafNode=" + maxPointsInLeafNode + " and maxMBSortInHeap=" + maxMBSortInHeap);
|
||||
}
|
||||
|
||||
return new FilterCodec("Lucene60", Codec.getDefault()) {
|
||||
@Override
|
||||
public DimensionalFormat dimensionalFormat() {
|
||||
return new DimensionalFormat() {
|
||||
@Override
|
||||
public DimensionalWriter fieldsWriter(SegmentWriteState writeState) throws IOException {
|
||||
return new Lucene60DimensionalWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DimensionalReader fieldsReader(SegmentReadState readState) throws IOException {
|
||||
return new Lucene60DimensionalReader(readState);
|
||||
}
|
||||
};
|
||||
}
|
||||
};
|
||||
} else {
|
||||
return Codec.getDefault();
|
||||
}
|
||||
}
|
||||
|
||||
/** docIDs can be null, for the single valued case, else it maps value to docID, but all values for one doc must be adjacent */
|
||||
private void verify(byte[][][] docValues, int[] docIDs, int numDims, int numBytesPerDim) throws Exception {
|
||||
try (Directory dir = getDirectory(docValues.length)) {
|
||||
while (true) {
|
||||
int maxPointsInLeafNode = TestUtil.nextInt(random(), 50, 100);
|
||||
double maxMB = (float) 0.1 + (3*random().nextDouble());
|
||||
try {
|
||||
verify(dir, docValues, docIDs, numDims, numBytesPerDim, maxPointsInLeafNode, maxMB);
|
||||
verify(dir, docValues, docIDs, numDims, numBytesPerDim, false);
|
||||
return;
|
||||
} catch (IllegalArgumentException iae) {
|
||||
// This just means we got a too-small maxMB for the maxPointsInLeafNode; just retry
|
||||
|
@ -798,21 +858,61 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
private void verify(Directory dir, byte[][][] docValues, int[] ids, int numDims, int numBytesPerDim, int maxPointsInLeafNode, double maxMB) throws Exception {
|
||||
private void verify(Directory dir, byte[][][] docValues, int[] ids, int numDims, int numBytesPerDim, boolean expectExceptions) throws Exception {
|
||||
int numValues = docValues.length;
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: numValues=" + numValues + " numDims=" + numDims + " numBytesPerDim=" + numBytesPerDim + " maxPointsInLeafNode=" + maxPointsInLeafNode + " maxMB=" + maxMB);
|
||||
System.out.println("TEST: numValues=" + numValues + " numDims=" + numDims + " numBytesPerDim=" + numBytesPerDim);
|
||||
}
|
||||
//System.out.println("DIR: " + ((FSDirectory) dir).getDirectory());
|
||||
|
||||
//IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
IndexWriterConfig iwc = newIndexWriterConfig();
|
||||
//iwc.setUseCompoundFile(false);
|
||||
//iwc.getMergePolicy().setNoCFSRatio(0.0);
|
||||
iwc.setCodec(new SimpleTextCodec());
|
||||
// RandomIndexWriter is too slow:
|
||||
boolean useRealWriter = docValues.length > 10000;
|
||||
|
||||
IndexWriterConfig iwc;
|
||||
if (useRealWriter) {
|
||||
iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
} else {
|
||||
iwc = newIndexWriterConfig();
|
||||
}
|
||||
iwc.setCodec(getCodec());
|
||||
|
||||
if (expectExceptions) {
|
||||
MergeScheduler ms = iwc.getMergeScheduler();
|
||||
if (ms instanceof ConcurrentMergeScheduler) {
|
||||
((ConcurrentMergeScheduler) ms).setSuppressExceptions();
|
||||
}
|
||||
}
|
||||
RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
|
||||
DirectoryReader r = null;
|
||||
|
||||
// 20% of the time we add into a separate directory, then at some point use
|
||||
// addIndexes to bring the indexed dimensional values to the main directory:
|
||||
Directory saveDir;
|
||||
RandomIndexWriter saveW;
|
||||
int addIndexesAt;
|
||||
if (random().nextInt(5) == 1) {
|
||||
saveDir = dir;
|
||||
saveW = w;
|
||||
dir = getDirectory(numValues);
|
||||
if (useRealWriter) {
|
||||
iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
} else {
|
||||
iwc = newIndexWriterConfig();
|
||||
}
|
||||
iwc.setCodec(getCodec());
|
||||
if (expectExceptions) {
|
||||
MergeScheduler ms = iwc.getMergeScheduler();
|
||||
if (ms instanceof ConcurrentMergeScheduler) {
|
||||
((ConcurrentMergeScheduler) ms).setSuppressExceptions();
|
||||
}
|
||||
}
|
||||
w = new RandomIndexWriter(random(), dir, iwc);
|
||||
addIndexesAt = TestUtil.nextInt(random(), 1, numValues-1);
|
||||
} else {
|
||||
saveW = null;
|
||||
saveDir = null;
|
||||
addIndexesAt = 0;
|
||||
}
|
||||
|
||||
try {
|
||||
|
||||
Document doc = null;
|
||||
|
@ -826,7 +926,11 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
}
|
||||
if (id != lastID) {
|
||||
if (doc != null) {
|
||||
w.addDocument(doc);
|
||||
if (useRealWriter) {
|
||||
w.w.addDocument(doc);
|
||||
} else {
|
||||
w.addDocument(doc);
|
||||
}
|
||||
}
|
||||
doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", id));
|
||||
|
@ -836,7 +940,11 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
|
||||
if (random().nextInt(30) == 17) {
|
||||
// randomly index some documents without this field
|
||||
w.addDocument(new Document());
|
||||
if (useRealWriter) {
|
||||
w.w.addDocument(new Document());
|
||||
} else {
|
||||
w.addDocument(new Document());
|
||||
}
|
||||
if (VERBOSE) {
|
||||
System.out.println("add empty doc");
|
||||
}
|
||||
|
@ -847,10 +955,22 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
Document xdoc = new Document();
|
||||
xdoc.add(new DimensionalField("field", docValues[ord]));
|
||||
xdoc.add(new StringField("nukeme", "yes", Field.Store.NO));
|
||||
w.addDocument(xdoc);
|
||||
if (useRealWriter) {
|
||||
w.w.addDocument(xdoc);
|
||||
} else {
|
||||
w.addDocument(xdoc);
|
||||
}
|
||||
if (VERBOSE) {
|
||||
System.out.println("add doc doc-to-delete");
|
||||
}
|
||||
|
||||
if (random().nextInt(5) == 1) {
|
||||
if (useRealWriter) {
|
||||
w.w.deleteDocuments(new Term("nukeme", "yes"));
|
||||
} else {
|
||||
w.deleteDocuments(new Term("nukeme", "yes"));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (VERBOSE) {
|
||||
|
@ -859,6 +979,14 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
System.out.println(" dim=" + dim + " value=" + new BytesRef(docValues[ord][dim]));
|
||||
}
|
||||
}
|
||||
|
||||
if (saveW != null && ord >= addIndexesAt) {
|
||||
switchIndex(w, dir, saveW);
|
||||
w = saveW;
|
||||
dir = saveDir;
|
||||
saveW = null;
|
||||
saveDir = null;
|
||||
}
|
||||
}
|
||||
w.addDocument(doc);
|
||||
w.deleteDocuments(new Term("nukeme", "yes"));
|
||||
|
@ -873,7 +1001,9 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
r = w.getReader();
|
||||
w.close();
|
||||
|
||||
//System.out.println("TEST: r=" + r);
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: reader=" + r);
|
||||
}
|
||||
|
||||
DimensionalValues dimValues = MultiDimensionalValues.get(r);
|
||||
if (VERBOSE) {
|
||||
|
@ -927,7 +1057,7 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
if (liveDocs != null && liveDocs.get(docID) == false) {
|
||||
return;
|
||||
}
|
||||
//System.out.println("visit check docID=" + docID);
|
||||
//System.out.println("visit check docID=" + docID + " id=" + idValues.get(docID));
|
||||
for(int dim=0;dim<numDims;dim++) {
|
||||
//System.out.println(" dim=" + dim + " value=" + new BytesRef(packedValue, dim*numBytesPerDim, numBytesPerDim));
|
||||
if (BKDUtil.compare(numBytesPerDim, packedValue, dim, queryMin[dim], 0) < 0 ||
|
||||
|
@ -1010,10 +1140,43 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
} finally {
|
||||
IOUtils.closeWhileHandlingException(r, w);
|
||||
IOUtils.closeWhileHandlingException(r, w, saveW, saveDir == null ? null : dir);
|
||||
}
|
||||
}
|
||||
|
||||
private void switchIndex(RandomIndexWriter w, Directory dir, RandomIndexWriter saveW) throws IOException {
|
||||
if (random().nextBoolean()) {
|
||||
// Add via readers:
|
||||
try (DirectoryReader r = w.getReader()) {
|
||||
if (random().nextBoolean()) {
|
||||
// Add via CodecReaders:
|
||||
List<CodecReader> subs = new ArrayList<>();
|
||||
for (LeafReaderContext context : r.leaves()) {
|
||||
subs.add((CodecReader) context.reader());
|
||||
}
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: now use addIndexes(CodecReader[]) to switch writers");
|
||||
}
|
||||
saveW.addIndexes(subs.toArray(new CodecReader[subs.size()]));
|
||||
} else {
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: now use TestUtil.addIndexesSlowly(DirectoryReader[]) to switch writers");
|
||||
}
|
||||
TestUtil.addIndexesSlowly(saveW.w, r);
|
||||
}
|
||||
}
|
||||
} else {
|
||||
// Add via directory:
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: now use addIndexes(Directory[]) to switch writers");
|
||||
}
|
||||
w.close();
|
||||
saveW.addIndexes(new Directory[] {dir});
|
||||
}
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
private BigInteger randomBigInt(int numBytes) {
|
||||
BigInteger x = new BigInteger(numBytes*8-1, random());
|
||||
if (random().nextBoolean()) {
|
||||
|
|
|
@ -270,12 +270,12 @@ public class SortingLeafReader extends FilterLeafReader {
|
|||
new IntersectVisitor() {
|
||||
@Override
|
||||
public void visit(int docID) throws IOException {
|
||||
visitor.visit(docMap.newToOld(docID));
|
||||
visitor.visit(docMap.oldToNew(docID));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void visit(int docID, byte[] packedValue) throws IOException {
|
||||
visitor.visit(docMap.newToOld(docID), packedValue);
|
||||
visitor.visit(docMap.oldToNew(docID), packedValue);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -30,9 +30,10 @@ 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.DimensionalField;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
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;
|
||||
|
@ -40,6 +41,8 @@ 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.DimensionalValues.IntersectVisitor;
|
||||
import org.apache.lucene.index.DimensionalValues.Relation;
|
||||
import org.apache.lucene.index.SortingLeafReader.SortingDocsEnum;
|
||||
import org.apache.lucene.index.TermsEnum.SeekStatus;
|
||||
import org.apache.lucene.search.CollectionStatistics;
|
||||
|
@ -52,9 +55,9 @@ import org.apache.lucene.util.BytesRef;
|
|||
import org.apache.lucene.util.FixedBitSet;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
import org.apache.lucene.util.bkd.BKDUtil;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
public abstract class SorterTestBase extends LuceneTestCase {
|
||||
|
||||
|
@ -134,6 +137,7 @@ public abstract class SorterTestBase extends LuceneTestCase {
|
|||
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 {
|
||||
|
@ -169,6 +173,9 @@ public abstract class SorterTestBase extends LuceneTestCase {
|
|||
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];
|
||||
BKDUtil.intToBytes(id, bytes, 0);
|
||||
doc.add(new DimensionalField(DIMENSIONAL_FIELD, bytes));
|
||||
return doc;
|
||||
}
|
||||
|
||||
|
@ -224,7 +231,6 @@ public abstract class SorterTestBase extends LuceneTestCase {
|
|||
dir = null;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBinaryDocValuesField() throws Exception {
|
||||
BinaryDocValues dv = sortedReader.getBinaryDocValues(BINARY_DV_FIELD);
|
||||
for (int i = 0; i < sortedReader.maxDoc(); i++) {
|
||||
|
@ -233,7 +239,6 @@ public abstract class SorterTestBase extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDocsAndPositionsEnum() throws Exception {
|
||||
TermsEnum termsEnum = sortedReader.terms(DOC_POSITIONS_FIELD).iterator();
|
||||
assertEquals(SeekStatus.FOUND, termsEnum.seekCeil(new BytesRef(DOC_POSITIONS_TERM)));
|
||||
|
@ -293,14 +298,12 @@ public abstract class SorterTestBase extends LuceneTestCase {
|
|||
return bits;
|
||||
}
|
||||
|
||||
@Test
|
||||
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;
|
||||
int prev = -1;
|
||||
while ((doc = docs.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
|
||||
assertEquals("incorrect value; doc " + doc, sortedValues[doc].intValue(), Integer.parseInt(sortedReader.document(doc).get(ID_FIELD)));
|
||||
}
|
||||
|
@ -311,13 +314,11 @@ public abstract class SorterTestBase extends LuceneTestCase {
|
|||
assertTrue(((SortingDocsEnum) docs).reused(reuse)); // make sure reuse worked
|
||||
}
|
||||
doc = -1;
|
||||
prev = -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)));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNormValues() throws Exception {
|
||||
NumericDocValues dv = sortedReader.getNormValues(NORMS_FIELD);
|
||||
int maxDoc = sortedReader.maxDoc();
|
||||
|
@ -326,7 +327,6 @@ public abstract class SorterTestBase extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNumericDocValuesField() throws Exception {
|
||||
NumericDocValues dv = sortedReader.getNumericDocValues(NUMERIC_DV_FIELD);
|
||||
int maxDoc = sortedReader.maxDoc();
|
||||
|
@ -335,7 +335,6 @@ public abstract class SorterTestBase extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSortedDocValuesField() throws Exception {
|
||||
SortedDocValues dv = sortedReader.getSortedDocValues(SORTED_DV_FIELD);
|
||||
int maxDoc = sortedReader.maxDoc();
|
||||
|
@ -345,7 +344,6 @@ public abstract class SorterTestBase extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSortedSetDocValuesField() throws Exception {
|
||||
SortedSetDocValues dv = sortedReader.getSortedSetDocValues(SORTED_SET_DV_FIELD);
|
||||
int maxDoc = sortedReader.maxDoc();
|
||||
|
@ -360,7 +358,6 @@ public abstract class SorterTestBase extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSortedNumericDocValuesField() throws Exception {
|
||||
SortedNumericDocValues dv = sortedReader.getSortedNumericDocValues(SORTED_NUMERIC_DV_FIELD);
|
||||
int maxDoc = sortedReader.maxDoc();
|
||||
|
@ -373,7 +370,6 @@ public abstract class SorterTestBase extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTermVectors() throws Exception {
|
||||
int maxDoc = sortedReader.maxDoc();
|
||||
for (int i = 0; i < maxDoc; i++) {
|
||||
|
@ -383,5 +379,24 @@ public abstract class SorterTestBase extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
// TODO: need DimensionalValues test here, once default codec supports DimensionalValues
|
||||
public void testDimensionalValues() throws Exception {
|
||||
DimensionalValues values = sortedReader.getDimensionalValues();
|
||||
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(), BKDUtil.bytesToInt(packedValues, 0));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
|
||||
return Relation.QUERY_CROSSES_CELL;
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,7 +19,7 @@ package org.apache.lucene.bkdtree;
|
|||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.lucene54.Lucene54Codec;
|
||||
import org.apache.lucene.codecs.lucene60.Lucene60Codec;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexWriterConfig;
|
||||
|
@ -69,7 +69,7 @@ public class TestBKDTree extends BaseGeoPointTestCase {
|
|||
@Override
|
||||
protected void initIndexWriterConfig(final String fieldName, IndexWriterConfig iwc) {
|
||||
final DocValuesFormat dvFormat = getDocValuesFormat();
|
||||
Codec codec = new Lucene54Codec() {
|
||||
Codec codec = new Lucene60Codec() {
|
||||
@Override
|
||||
public DocValuesFormat getDocValuesFormatForField(String field) {
|
||||
if (field.equals(fieldName)) {
|
||||
|
|
|
@ -27,7 +27,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
|
|||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.lucene54.Lucene54Codec;
|
||||
import org.apache.lucene.codecs.lucene60.Lucene60Codec;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.NumericDocValuesField;
|
||||
|
@ -356,7 +356,7 @@ public class TestRangeTree extends LuceneTestCase {
|
|||
iwc.setMaxBufferedDocs(values.length/100);
|
||||
}
|
||||
final DocValuesFormat dvFormat = getDocValuesFormat();
|
||||
Codec codec = new Lucene54Codec() {
|
||||
Codec codec = new Lucene60Codec() {
|
||||
@Override
|
||||
public DocValuesFormat getDocValuesFormatForField(String field) {
|
||||
if (field.equals("sn_value") || field.equals("ss_value")) {
|
||||
|
|
|
@ -29,7 +29,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
|
|||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.lucene54.Lucene54Codec;
|
||||
import org.apache.lucene.codecs.lucene60.Lucene60Codec;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.NumericDocValuesField;
|
||||
|
@ -910,7 +910,7 @@ public class TestGeo3DPointField extends LuceneTestCase {
|
|||
iwc.setMaxBufferedDocs(lats.length/100);
|
||||
}
|
||||
final DocValuesFormat dvFormat = new Geo3DDocValuesFormat(planetModel, maxPointsInLeaf, maxPointsSortInHeap);
|
||||
Codec codec = new Lucene54Codec() {
|
||||
Codec codec = new Lucene60Codec() {
|
||||
@Override
|
||||
public DocValuesFormat getDocValuesFormatForField(String field) {
|
||||
if (field.equals("point")) {
|
||||
|
|
|
@ -33,7 +33,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.lucene54.Lucene54Codec;
|
||||
import org.apache.lucene.codecs.lucene60.Lucene60Codec;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.IntField;
|
||||
|
@ -657,7 +657,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 Lucene54Codec() {
|
||||
Codec filterCodec = new Lucene60Codec() {
|
||||
PostingsFormat postingsFormat = new Completion50PostingsFormat();
|
||||
|
||||
@Override
|
||||
|
|
|
@ -110,6 +110,9 @@ public final class AssertingDimensionalFormat extends DimensionalFormat {
|
|||
|
||||
@Override
|
||||
public void writeField(FieldInfo fieldInfo, DimensionalReader values) throws IOException {
|
||||
if (fieldInfo.getDimensionCount() == 0) {
|
||||
throw new IllegalArgumentException("writing field=\"" + fieldInfo.name + "\" but dimensionalCount is 0");
|
||||
}
|
||||
in.writeField(fieldInfo, values);
|
||||
}
|
||||
|
||||
|
|
|
@ -67,8 +67,8 @@ public class MismatchedLeafReader extends FilterLeafReader {
|
|||
oldInfo.getDocValuesType(), // docValuesType
|
||||
oldInfo.getDocValuesGen(), // dvGen
|
||||
oldInfo.attributes(), // attributes
|
||||
oldInfo.getDimensionCount(),
|
||||
oldInfo.getDimensionNumBytes());
|
||||
oldInfo.getDimensionCount(), // dimension count
|
||||
oldInfo.getDimensionNumBytes()); // dimension numBytes
|
||||
shuffled.set(i, newInfo);
|
||||
}
|
||||
|
||||
|
|
|
@ -381,19 +381,28 @@ public class RandomIndexWriter implements Closeable {
|
|||
*/
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (w.isClosed() == false) {
|
||||
LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
|
||||
}
|
||||
// if someone isn't using getReader() API, we want to be sure to
|
||||
// forceMerge since presumably they might open a reader on the dir.
|
||||
if (getReaderCalled == false && r.nextInt(8) == 2 && w.isClosed() == false) {
|
||||
doRandomForceMerge();
|
||||
if (w.getConfig().getCommitOnClose() == false) {
|
||||
// index may have changed, must commit the changes, or otherwise they are discarded by the call to close()
|
||||
w.commit();
|
||||
boolean success = false;
|
||||
try {
|
||||
if (w.isClosed() == false) {
|
||||
LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
|
||||
}
|
||||
// if someone isn't using getReader() API, we want to be sure to
|
||||
// forceMerge since presumably they might open a reader on the dir.
|
||||
if (getReaderCalled == false && r.nextInt(8) == 2 && w.isClosed() == false) {
|
||||
doRandomForceMerge();
|
||||
if (w.getConfig().getCommitOnClose() == false) {
|
||||
// index may have changed, must commit the changes, or otherwise they are discarded by the call to close()
|
||||
w.commit();
|
||||
}
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
if (success) {
|
||||
IOUtils.close(w, analyzer);
|
||||
} else {
|
||||
IOUtils.closeWhileHandlingException(w, analyzer);
|
||||
}
|
||||
}
|
||||
IOUtils.close(w, analyzer);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -438,10 +438,12 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
|
|||
|
||||
void maybeThrowIOException(String message) throws IOException {
|
||||
if (randomState.nextDouble() < randomIOExceptionRate) {
|
||||
IOException ioe = new IOException("a random IOException" + (message == null ? "" : " (" + message + ")"));
|
||||
if (LuceneTestCase.VERBOSE) {
|
||||
System.out.println(Thread.currentThread().getName() + ": MockDirectoryWrapper: now throw random exception" + (message == null ? "" : " (" + message + ")"));
|
||||
ioe.printStackTrace(System.out);
|
||||
}
|
||||
throw new IOException("a random IOException" + (message == null ? "" : " (" + message + ")"));
|
||||
throw ioe;
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -33,7 +33,7 @@ import org.apache.lucene.codecs.asserting.AssertingPostingsFormat;
|
|||
import org.apache.lucene.codecs.cheapbastard.CheapBastardCodec;
|
||||
import org.apache.lucene.codecs.compressing.CompressingCodec;
|
||||
import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.lucene54.Lucene54Codec;
|
||||
import org.apache.lucene.codecs.lucene60.Lucene60Codec;
|
||||
import org.apache.lucene.codecs.mockrandom.MockRandomPostingsFormat;
|
||||
import org.apache.lucene.codecs.simpletext.SimpleTextCodec;
|
||||
import org.apache.lucene.index.RandomCodec;
|
||||
|
@ -182,8 +182,8 @@ final class TestRuleSetupAndRestoreClassEnv extends AbstractBeforeAfterRule {
|
|||
codec = new AssertingCodec();
|
||||
} else if ("Compressing".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 6 && !shouldAvoidCodec("Compressing"))) {
|
||||
codec = CompressingCodec.randomInstance(random);
|
||||
} else if ("Lucene54".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 5 && !shouldAvoidCodec("Lucene54"))) {
|
||||
codec = new Lucene54Codec(RandomPicks.randomFrom(random, Lucene50StoredFieldsFormat.Mode.values()));
|
||||
} else if ("Lucene60".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 5 && !shouldAvoidCodec("Lucene60"))) {
|
||||
codec = new Lucene60Codec(RandomPicks.randomFrom(random, Lucene50StoredFieldsFormat.Mode.values()));
|
||||
} else if (!"random".equals(TEST_CODEC)) {
|
||||
codec = Codec.forName(TEST_CODEC);
|
||||
} else if ("random".equals(TEST_POSTINGSFORMAT)) {
|
||||
|
|
|
@ -45,9 +45,6 @@ import java.util.regex.PatternSyntaxException;
|
|||
import java.util.zip.ZipEntry;
|
||||
import java.util.zip.ZipInputStream;
|
||||
|
||||
import com.carrotsearch.randomizedtesting.generators.RandomInts;
|
||||
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
|
@ -55,8 +52,8 @@ import org.apache.lucene.codecs.asserting.AssertingCodec;
|
|||
import org.apache.lucene.codecs.blockterms.LuceneFixedGap;
|
||||
import org.apache.lucene.codecs.blocktreeords.BlockTreeOrdsPostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene54.Lucene54Codec;
|
||||
import org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat;
|
||||
import org.apache.lucene.codecs.lucene60.Lucene60Codec;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
|
||||
import org.apache.lucene.document.BinaryDocValuesField;
|
||||
|
@ -70,13 +67,13 @@ import org.apache.lucene.document.LongField;
|
|||
import org.apache.lucene.document.NumericDocValuesField;
|
||||
import org.apache.lucene.document.SortedDocValuesField;
|
||||
import org.apache.lucene.index.CheckIndex;
|
||||
import org.apache.lucene.index.CodecReader;
|
||||
import org.apache.lucene.index.ConcurrentMergeScheduler;
|
||||
import org.apache.lucene.index.DirectoryReader;
|
||||
import org.apache.lucene.index.DocValuesType;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.PostingsEnum;
|
||||
import org.apache.lucene.index.FilterLeafReader;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.index.IndexableField;
|
||||
|
@ -85,8 +82,8 @@ import org.apache.lucene.index.LeafReaderContext;
|
|||
import org.apache.lucene.index.LogMergePolicy;
|
||||
import org.apache.lucene.index.MergePolicy;
|
||||
import org.apache.lucene.index.MergeScheduler;
|
||||
import org.apache.lucene.index.CodecReader;
|
||||
import org.apache.lucene.index.MultiFields;
|
||||
import org.apache.lucene.index.PostingsEnum;
|
||||
import org.apache.lucene.index.SegmentReader;
|
||||
import org.apache.lucene.index.SlowCodecReaderWrapper;
|
||||
import org.apache.lucene.index.Terms;
|
||||
|
@ -101,6 +98,9 @@ import org.apache.lucene.store.NoLockFactory;
|
|||
import org.apache.lucene.store.RAMDirectory;
|
||||
import org.junit.Assert;
|
||||
|
||||
import com.carrotsearch.randomizedtesting.generators.RandomInts;
|
||||
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
|
||||
|
||||
|
||||
/**
|
||||
* General utility methods for Lucene unit tests.
|
||||
|
@ -879,7 +879,7 @@ public final class TestUtil {
|
|||
* This may be different than {@link Codec#getDefault()} because that is randomized.
|
||||
*/
|
||||
public static Codec getDefaultCodec() {
|
||||
return new Lucene54Codec();
|
||||
return new Lucene60Codec();
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -3,7 +3,7 @@ package org.apache.solr.core;
|
|||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene54.Lucene54Codec;
|
||||
import org.apache.lucene.codecs.lucene60.Lucene60Codec;
|
||||
import org.apache.solr.common.util.NamedList;
|
||||
import org.apache.solr.schema.SchemaField;
|
||||
import org.apache.solr.util.plugin.SolrCoreAware;
|
||||
|
@ -51,7 +51,7 @@ public class SchemaCodecFactory extends CodecFactory implements SolrCoreAware {
|
|||
@Override
|
||||
public void init(NamedList args) {
|
||||
super.init(args);
|
||||
codec = new Lucene54Codec() {
|
||||
codec = new Lucene60Codec() {
|
||||
@Override
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
final SchemaField schemaField = core.getLatestSchema().getFieldOrNull(field);
|
||||
|
|
Loading…
Reference in New Issue