LUCENE-9148: Move the BKD index to its own file. (#1475)

This commit is contained in:
Adrien Grand 2020-06-09 09:59:14 +02:00 committed by GitHub
parent 684c2e6afe
commit 54c5dd7d6d
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
27 changed files with 1228 additions and 220 deletions

View File

@ -227,6 +227,8 @@ Optimizations
* LUCENE-9087: Build always trees with full leaves and lower the default value for maxPointsPerLeafNode to 512.
(Ignacio Vera)
* LUCENE-9148: Points now write their index in a separate file. (Adrien Grand)
Bug Fixes
---------------------
* LUCENE-9259: Fix wrong NGramFilterFactory argument name for preserveOriginal option (Paul Pazderski)

View File

@ -73,7 +73,7 @@ import org.apache.lucene.index.SegmentWriteState;
* @lucene.experimental
*/
public final class Lucene60PointsFormat extends PointsFormat {
public class Lucene60PointsFormat extends PointsFormat {
static final String DATA_CODEC_NAME = "Lucene60PointsFormatData";
static final String META_CODEC_NAME = "Lucene60PointsFormatMeta";
@ -100,7 +100,7 @@ public final class Lucene60PointsFormat extends PointsFormat {
@Override
public PointsWriter fieldsWriter(SegmentWriteState state) throws IOException {
return new Lucene60PointsWriter(state);
throw new UnsupportedOperationException("Old codecs may only be used for reading");
}
@Override

View File

@ -39,7 +39,7 @@ import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.bkd.BKDReader;
/** Reads point values previously written with {@link Lucene60PointsWriter} */
/** Reads point values previously written with Lucene60PointsWriter */
public class Lucene60PointsReader extends PointsReader implements Closeable {
final IndexInput dataIn;
final SegmentReadState readState;
@ -63,7 +63,7 @@ public class Lucene60PointsReader extends PointsReader implements Closeable {
CodecUtil.checkIndexHeader(indexIn,
Lucene60PointsFormat.META_CODEC_NAME,
Lucene60PointsFormat.INDEX_VERSION_START,
Lucene60PointsFormat.INDEX_VERSION_START,
Lucene60PointsFormat.INDEX_VERSION_CURRENT,
readState.segmentInfo.getId(),
readState.segmentSuffix);
int count = indexIn.readVInt();
@ -103,7 +103,7 @@ public class Lucene60PointsReader extends PointsReader implements Closeable {
int fieldNumber = ent.getKey();
long fp = ent.getValue();
dataIn.seek(fp);
BKDReader reader = new BKDReader(dataIn);
BKDReader reader = new BKDReader(dataIn, dataIn, dataIn);
readers.put(fieldNumber, reader);
}

View File

@ -0,0 +1,25 @@
<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
<!--
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to You under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
-->
<html>
<head>
<meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
</head>
<body>
Lucene 6.0 file format.
</body>
</html>

View File

@ -117,7 +117,7 @@ public class Lucene84Codec extends Codec {
}
@Override
public final SegmentInfoFormat segmentInfoFormat() {
public SegmentInfoFormat segmentInfoFormat() {
return segmentInfosFormat;
}
@ -132,7 +132,7 @@ public class Lucene84Codec extends Codec {
}
@Override
public final PointsFormat pointsFormat() {
public PointsFormat pointsFormat() {
return new Lucene60PointsFormat();
}

View File

@ -0,0 +1,25 @@
<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
<!--
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to You under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
-->
<html>
<head>
<meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
</head>
<body>
Lucene 8.4 file format.
</body>
</html>

View File

@ -101,9 +101,10 @@ public class Lucene60PointsWriter extends PointsWriter implements Closeable {
values.size())) {
if (values instanceof MutablePointValues) {
final long fp = writer.writeField(dataOut, fieldInfo.name, (MutablePointValues) values);
if (fp != -1) {
indexFPs.put(fieldInfo.name, fp);
Runnable finalizer = writer.writeField(dataOut, dataOut, dataOut, fieldInfo.name, (MutablePointValues) values);
if (finalizer != null) {
indexFPs.put(fieldInfo.name, dataOut.getFilePointer());
finalizer.run();
}
return;
}
@ -125,8 +126,10 @@ public class Lucene60PointsWriter extends PointsWriter implements Closeable {
});
// We could have 0 points on merge since all docs with dimensional fields may be deleted:
if (writer.getPointCount() > 0) {
indexFPs.put(fieldInfo.name, writer.finish(dataOut));
Runnable finalizer = writer.finish(dataOut, dataOut, dataOut);
if (finalizer != null) {
indexFPs.put(fieldInfo.name, dataOut.getFilePointer());
finalizer.run();
}
}
}
@ -210,9 +213,10 @@ public class Lucene60PointsWriter extends PointsWriter implements Closeable {
}
}
long fp = writer.merge(dataOut, docMaps, bkdReaders);
if (fp != -1) {
indexFPs.put(fieldInfo.name, fp);
Runnable finalizer = writer.merge(dataOut, dataOut, dataOut, docMaps, bkdReaders);
if (finalizer != null) {
indexFPs.put(fieldInfo.name, dataOut.getFilePointer());
finalizer.run();
}
}
} else {

View File

@ -14,9 +14,22 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene60;
/**
* Components from the Lucene 8.4 index format. See {@link org.apache.lucene.codecs.lucene86}
* for an overview of the current index format.
*/
package org.apache.lucene.codecs.lucene84;
import java.io.IOException;
import org.apache.lucene.codecs.PointsWriter;
import org.apache.lucene.index.SegmentWriteState;
/** RW variant of Lucene60PointsFormat */
public class Lucene60RWPointsFormat extends Lucene60PointsFormat {
/** Sole constructor. */
public Lucene60RWPointsFormat() {}
@Override
public PointsWriter fieldsWriter(SegmentWriteState state) throws IOException {
return new Lucene60PointsWriter(state);
}
}

View File

@ -21,10 +21,7 @@ import java.io.IOException;
import java.util.Arrays;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.PointsFormat;
import org.apache.lucene.codecs.PointsReader;
import org.apache.lucene.codecs.PointsWriter;
import org.apache.lucene.codecs.lucene84.Lucene84RWCodec;
import org.apache.lucene.document.BinaryPoint;
import org.apache.lucene.document.Document;
import org.apache.lucene.index.BasePointsFormatTestCase;
@ -35,8 +32,6 @@ import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.MockRandomMergePolicy;
import org.apache.lucene.index.PointValues;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.PointValues.IntersectVisitor;
import org.apache.lucene.index.PointValues.Relation;
import org.apache.lucene.store.Directory;
@ -51,51 +46,15 @@ public class TestLucene60PointsFormat extends BasePointsFormatTestCase {
private final int maxPointsInLeafNode;
public TestLucene60PointsFormat() {
// standard issue
Codec defaultCodec = TestUtil.getDefaultCodec();
if (random().nextBoolean()) {
// randomize parameters
maxPointsInLeafNode = TestUtil.nextInt(random(), 50, 500);
double maxMBSortInHeap = 3.0 + (3*random().nextDouble());
if (VERBOSE) {
System.out.println("TEST: using Lucene60PointsFormat with maxPointsInLeafNode=" + maxPointsInLeafNode + " and maxMBSortInHeap=" + maxMBSortInHeap);
}
// sneaky impersonation!
codec = new FilterCodec(defaultCodec.getName(), defaultCodec) {
@Override
public PointsFormat pointsFormat() {
return new PointsFormat() {
@Override
public PointsWriter fieldsWriter(SegmentWriteState writeState) throws IOException {
return new Lucene60PointsWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap);
}
@Override
public PointsReader fieldsReader(SegmentReadState readState) throws IOException {
return new Lucene60PointsReader(readState);
}
};
}
};
} else {
// standard issue
codec = defaultCodec;
codec = new Lucene84RWCodec();
maxPointsInLeafNode = BKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE;
}
}
@Override
protected Codec getCodec() {
return codec;
}
@Override
public void testMergeStability() throws Exception {
assumeFalse("TODO: mess with the parameters and test gets angry!", codec instanceof FilterCodec);
super.testMergeStability();
}
public void testEstimatePointCount() throws IOException {
Directory dir = newDirectory();
IndexWriterConfig iwc = newIndexWriterConfig();

View File

@ -0,0 +1,39 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene84;
import org.apache.lucene.codecs.PointsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.lucene60.Lucene60RWPointsFormat;
import org.apache.lucene.codecs.lucene70.Lucene70RWSegmentInfoFormat;
/**
* RW impersonation of {@link Lucene84Codec}.
*/
public class Lucene84RWCodec extends Lucene84Codec {
@Override
public PointsFormat pointsFormat() {
return new Lucene60RWPointsFormat();
}
@Override
public SegmentInfoFormat segmentInfoFormat() {
return new Lucene70RWSegmentInfoFormat();
}
}

View File

@ -16,7 +16,7 @@
*/
/**
* Components from the Lucene 6.0 index format. See {@link org.apache.lucene.codecs.lucene80}
* Components from the Lucene 6.0 index format. See {@link org.apache.lucene.codecs.lucene86}
* for an overview of the current index format.
*/
package org.apache.lucene.codecs.lucene60;

View File

@ -36,7 +36,6 @@ import org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat;
import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat;
import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat;
import org.apache.lucene.codecs.lucene60.Lucene60FieldInfosFormat;
import org.apache.lucene.codecs.lucene60.Lucene60PointsFormat;
import org.apache.lucene.codecs.lucene80.Lucene80NormsFormat;
import org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
@ -59,6 +58,7 @@ public class Lucene86Codec extends Codec {
private final SegmentInfoFormat segmentInfosFormat = new Lucene86SegmentInfoFormat();
private final LiveDocsFormat liveDocsFormat = new Lucene50LiveDocsFormat();
private final CompoundFormat compoundFormat = new Lucene50CompoundFormat();
private final PointsFormat pointsFormat = new Lucene86PointsFormat();
private final PostingsFormat defaultFormat;
private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
@ -133,7 +133,7 @@ public class Lucene86Codec extends Codec {
@Override
public final PointsFormat pointsFormat() {
return new Lucene60PointsFormat();
return pointsFormat;
}
/** Returns the postings format that should be used for writing

View File

@ -0,0 +1,80 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene86;
import java.io.IOException;
import org.apache.lucene.codecs.PointsFormat;
import org.apache.lucene.codecs.PointsReader;
import org.apache.lucene.codecs.PointsWriter;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
/**
* Lucene 8.6 point format, which encodes dimensional values in a block KD-tree structure
* for fast 1D range and N dimensional shape intersection filtering.
* See <a href="https://www.cs.duke.edu/~pankaj/publications/papers/bkd-sstd.pdf">this paper</a> for details.
*
* <p>Data is stored across three files
* <ul>
* <li>A .kdm file that records metadata about the fields, such as numbers of
* dimensions or numbers of bytes per dimension.
* <li>A .kdi file that stores inner nodes of the tree.
* <li>A .kdm file that stores leaf nodes, where most of the data lives.
* </ul>
*
* @lucene.experimental
*/
public final class Lucene86PointsFormat extends PointsFormat {
static final String DATA_CODEC_NAME = "Lucene86PointsFormatData";
static final String INDEX_CODEC_NAME = "Lucene86PointsFormatIndex";
static final String META_CODEC_NAME = "Lucene86PointsFormatMeta";
/**
* Filename extension for the leaf blocks
*/
public static final String DATA_EXTENSION = "kdd";
/**
* Filename extension for the index per field
*/
public static final String INDEX_EXTENSION = "kdi";
/**
* Filename extension for the meta per field
*/
public static final String META_EXTENSION = "kdm";
static final int VERSION_START = 0;
static final int VERSION_CURRENT = VERSION_START;
/** Sole constructor */
public Lucene86PointsFormat() {
}
@Override
public PointsWriter fieldsWriter(SegmentWriteState state) throws IOException {
return new Lucene86PointsWriter(state);
}
@Override
public PointsReader fieldsReader(SegmentReadState state) throws IOException {
return new Lucene86PointsReader(state);
}
}

View File

@ -0,0 +1,167 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene86;
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.PointsReader;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.PointValues;
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 point values previously written with {@link Lucene86PointsWriter} */
public class Lucene86PointsReader extends PointsReader implements Closeable {
final IndexInput indexIn, dataIn;
final SegmentReadState readState;
final Map<Integer,BKDReader> readers = new HashMap<>();
/** Sole constructor */
public Lucene86PointsReader(SegmentReadState readState) throws IOException {
this.readState = readState;
String metaFileName = IndexFileNames.segmentFileName(readState.segmentInfo.name,
readState.segmentSuffix,
Lucene86PointsFormat.META_EXTENSION);
String indexFileName = IndexFileNames.segmentFileName(readState.segmentInfo.name,
readState.segmentSuffix,
Lucene86PointsFormat.INDEX_EXTENSION);
String dataFileName = IndexFileNames.segmentFileName(readState.segmentInfo.name,
readState.segmentSuffix,
Lucene86PointsFormat.DATA_EXTENSION);
boolean success = false;
try {
indexIn = readState.directory.openInput(indexFileName, readState.context);
CodecUtil.checkIndexHeader(indexIn,
Lucene86PointsFormat.INDEX_CODEC_NAME,
Lucene86PointsFormat.VERSION_START,
Lucene86PointsFormat.VERSION_CURRENT,
readState.segmentInfo.getId(),
readState.segmentSuffix);
CodecUtil.retrieveChecksum(indexIn);
dataIn = readState.directory.openInput(dataFileName, readState.context);
CodecUtil.checkIndexHeader(dataIn,
Lucene86PointsFormat.DATA_CODEC_NAME,
Lucene86PointsFormat.VERSION_START,
Lucene86PointsFormat.VERSION_CURRENT,
readState.segmentInfo.getId(),
readState.segmentSuffix);
CodecUtil.retrieveChecksum(dataIn);
try (ChecksumIndexInput metaIn = readState.directory.openChecksumInput(metaFileName, readState.context)) {
Throwable priorE = null;
try {
CodecUtil.checkIndexHeader(metaIn,
Lucene86PointsFormat.META_CODEC_NAME,
Lucene86PointsFormat.VERSION_START,
Lucene86PointsFormat.VERSION_CURRENT,
readState.segmentInfo.getId(),
readState.segmentSuffix);
while (true) {
int fieldNumber = metaIn.readInt();
if (fieldNumber == -1) {
break;
} else if (fieldNumber < 0) {
throw new CorruptIndexException("Illegal field number: " + fieldNumber, metaIn);
}
BKDReader reader = new BKDReader(metaIn, indexIn, dataIn);
readers.put(fieldNumber, reader);
}
} catch (Throwable t) {
priorE = t;
} finally {
CodecUtil.checkFooter(metaIn, priorE);
}
}
success = true;
} finally {
if (success == false) {
IOUtils.closeWhileHandlingException(this);
}
}
}
/** Returns the underlying {@link BKDReader}.
*
* @lucene.internal */
@Override
public PointValues getValues(String fieldName) {
FieldInfo fieldInfo = readState.fieldInfos.fieldInfo(fieldName);
if (fieldInfo == null) {
throw new IllegalArgumentException("field=\"" + fieldName + "\" is unrecognized");
}
if (fieldInfo.getPointDimensionCount() == 0) {
throw new IllegalArgumentException("field=\"" + fieldName + "\" did not index point values");
}
return readers.get(fieldInfo.number);
}
@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(indexIn);
CodecUtil.checksumEntireFile(dataIn);
}
@Override
public void close() throws IOException {
IOUtils.close(indexIn, dataIn);
// Free up heap:
readers.clear();
}
}

View File

@ -0,0 +1,263 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene86;
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.MutablePointValues;
import org.apache.lucene.codecs.PointsReader;
import org.apache.lucene.codecs.PointsWriter;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.PointValues;
import org.apache.lucene.index.PointValues.IntersectVisitor;
import org.apache.lucene.index.PointValues.Relation;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.bkd.BKDReader;
import org.apache.lucene.util.bkd.BKDWriter;
/** Writes dimensional values */
public class Lucene86PointsWriter extends PointsWriter implements Closeable {
/** Outputs used to write the BKD tree data files. */
protected final IndexOutput metaOut, indexOut, dataOut;
final SegmentWriteState writeState;
final int maxPointsInLeafNode;
final double maxMBSortInHeap;
private boolean finished;
/** Full constructor */
public Lucene86PointsWriter(SegmentWriteState writeState, int maxPointsInLeafNode, double maxMBSortInHeap) throws IOException {
assert writeState.fieldInfos.hasPointValues();
this.writeState = writeState;
this.maxPointsInLeafNode = maxPointsInLeafNode;
this.maxMBSortInHeap = maxMBSortInHeap;
String dataFileName = IndexFileNames.segmentFileName(writeState.segmentInfo.name,
writeState.segmentSuffix,
Lucene86PointsFormat.DATA_EXTENSION);
dataOut = writeState.directory.createOutput(dataFileName, writeState.context);
boolean success = false;
try {
CodecUtil.writeIndexHeader(dataOut,
Lucene86PointsFormat.DATA_CODEC_NAME,
Lucene86PointsFormat.VERSION_CURRENT,
writeState.segmentInfo.getId(),
writeState.segmentSuffix);
String metaFileName = IndexFileNames.segmentFileName(writeState.segmentInfo.name,
writeState.segmentSuffix,
Lucene86PointsFormat.META_EXTENSION);
metaOut = writeState.directory.createOutput(metaFileName, writeState.context);
CodecUtil.writeIndexHeader(metaOut,
Lucene86PointsFormat.META_CODEC_NAME,
Lucene86PointsFormat.VERSION_CURRENT,
writeState.segmentInfo.getId(),
writeState.segmentSuffix);
String indexFileName = IndexFileNames.segmentFileName(writeState.segmentInfo.name,
writeState.segmentSuffix,
Lucene86PointsFormat.INDEX_EXTENSION);
indexOut = writeState.directory.createOutput(indexFileName, writeState.context);
CodecUtil.writeIndexHeader(indexOut,
Lucene86PointsFormat.INDEX_CODEC_NAME,
Lucene86PointsFormat.VERSION_CURRENT,
writeState.segmentInfo.getId(),
writeState.segmentSuffix);
success = true;
} finally {
if (success == false) {
IOUtils.closeWhileHandlingException(this);
}
}
}
/** Uses the defaults values for {@code maxPointsInLeafNode} (1024) and {@code maxMBSortInHeap} (16.0) */
public Lucene86PointsWriter(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, PointsReader reader) throws IOException {
PointValues values = reader.getValues(fieldInfo.name);
try (BKDWriter writer = new BKDWriter(writeState.segmentInfo.maxDoc(),
writeState.directory,
writeState.segmentInfo.name,
fieldInfo.getPointDimensionCount(),
fieldInfo.getPointIndexDimensionCount(),
fieldInfo.getPointNumBytes(),
maxPointsInLeafNode,
maxMBSortInHeap,
values.size())) {
if (values instanceof MutablePointValues) {
Runnable finalizer = writer.writeField(metaOut, indexOut, dataOut, fieldInfo.name, (MutablePointValues) values);
if (finalizer != null) {
metaOut.writeInt(fieldInfo.number);
finalizer.run();
}
return;
}
values.intersect(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.CELL_CROSSES_QUERY;
}
});
// We could have 0 points on merge since all docs with dimensional fields may be deleted:
Runnable finalizer = writer.finish(metaOut, indexOut, dataOut);
if (finalizer != null) {
metaOut.writeInt(fieldInfo.number);
finalizer.run();
}
}
}
@Override
public void merge(MergeState mergeState) throws IOException {
/**
* If indexSort is activated and some of the leaves are not sorted the next test will catch that and the non-optimized merge will run.
* If the readers are all sorted then it's safe to perform a bulk merge of the points.
**/
for(PointsReader reader : mergeState.pointsReaders) {
if (reader instanceof Lucene86PointsReader == false) {
// We can only bulk merge when all to-be-merged segments use our format:
super.merge(mergeState);
return;
}
}
for (PointsReader reader : mergeState.pointsReaders) {
if (reader != null) {
reader.checkIntegrity();
}
}
for (FieldInfo fieldInfo : mergeState.mergeFieldInfos) {
if (fieldInfo.getPointDimensionCount() != 0) {
if (fieldInfo.getPointDimensionCount() == 1) {
// Worst case total maximum size (if none of the points are deleted):
long totMaxSize = 0;
for(int i=0;i<mergeState.pointsReaders.length;i++) {
PointsReader reader = mergeState.pointsReaders[i];
if (reader != null) {
FieldInfos readerFieldInfos = mergeState.fieldInfos[i];
FieldInfo readerFieldInfo = readerFieldInfos.fieldInfo(fieldInfo.name);
if (readerFieldInfo != null && readerFieldInfo.getPointDimensionCount() > 0) {
PointValues values = reader.getValues(fieldInfo.name);
if (values != null) {
totMaxSize += values.size();
}
}
}
}
//System.out.println("MERGE: field=" + fieldInfo.name);
// Optimize the 1D case to use BKDWriter.merge, which does a single merge sort of the
// already sorted incoming segments, instead of trying to sort all points again as if
// we were simply reindexing them:
try (BKDWriter writer = new BKDWriter(writeState.segmentInfo.maxDoc(),
writeState.directory,
writeState.segmentInfo.name,
fieldInfo.getPointDimensionCount(),
fieldInfo.getPointIndexDimensionCount(),
fieldInfo.getPointNumBytes(),
maxPointsInLeafNode,
maxMBSortInHeap,
totMaxSize)) {
List<BKDReader> bkdReaders = new ArrayList<>();
List<MergeState.DocMap> docMaps = new ArrayList<>();
for(int i=0;i<mergeState.pointsReaders.length;i++) {
PointsReader reader = mergeState.pointsReaders[i];
if (reader != null) {
// we confirmed this up above
assert reader instanceof Lucene86PointsReader;
Lucene86PointsReader reader60 = (Lucene86PointsReader) reader;
// NOTE: we cannot just use the merged fieldInfo.number (instead of resolving to this
// reader's FieldInfo as we do below) because field numbers can easily be different
// when addIndexes(Directory...) copies over segments from another index:
FieldInfos readerFieldInfos = mergeState.fieldInfos[i];
FieldInfo readerFieldInfo = readerFieldInfos.fieldInfo(fieldInfo.name);
if (readerFieldInfo != null && readerFieldInfo.getPointDimensionCount() > 0) {
BKDReader bkdReader = reader60.readers.get(readerFieldInfo.number);
if (bkdReader != null) {
bkdReaders.add(bkdReader);
docMaps.add(mergeState.docMaps[i]);
}
}
}
}
Runnable finalizer = writer.merge(metaOut, indexOut, dataOut, docMaps, bkdReaders);
if (finalizer != null) {
metaOut.writeInt(fieldInfo.number);
finalizer.run();
}
}
} else {
mergeOneField(mergeState, fieldInfo);
}
}
}
finish();
}
@Override
public void finish() throws IOException {
if (finished) {
throw new IllegalStateException("already finished");
}
finished = true;
metaOut.writeInt(-1);
CodecUtil.writeFooter(metaOut);
CodecUtil.writeFooter(indexOut);
CodecUtil.writeFooter(dataOut);
}
@Override
public void close() throws IOException {
IOUtils.close(metaOut, indexOut, dataOut);
}
}

View File

@ -186,7 +186,7 @@
* An optional file indicating which documents are live.
* </li>
* <li>
* {@link org.apache.lucene.codecs.lucene60.Lucene60PointsFormat Point values}.
* {@link org.apache.lucene.codecs.lucene86.Lucene86PointsFormat Point values}.
* Optional pair of files, recording dimensionally indexed fields, to enable fast
* numeric range filtering and large numeric values like BigInteger and BigDecimal (1D)
* and geographic shape intersection (2D, 3D).
@ -310,7 +310,7 @@
* <td>Info about what documents are live</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene60.Lucene60PointsFormat Point values}</td>
* <td>{@link org.apache.lucene.codecs.lucene86.Lucene86PointsFormat Point values}</td>
* <td>.dii, .dim</td>
* <td>Holds indexed points, if any</td>
* </tr>

View File

@ -39,7 +39,6 @@ import org.apache.lucene.util.MathUtil;
public final class BKDReader extends PointValues implements Accountable {
private static abstract class BKDInput extends DataInput implements Cloneable {
abstract long getMinLeafBlockFP();
abstract long ramBytesUsed();
abstract int getPosition();
@ -54,26 +53,14 @@ public final class BKDReader extends PointValues implements Accountable {
private static class BKDOffHeapInput extends BKDInput implements Cloneable {
private final IndexInput packedIndex;
private final long minLeafBlockFP;
BKDOffHeapInput(IndexInput packedIndex) throws IOException {
BKDOffHeapInput(IndexInput packedIndex) {
this.packedIndex = packedIndex;
this.minLeafBlockFP = packedIndex.clone().readVLong();
}
private BKDOffHeapInput(IndexInput packedIndex, long minLeadBlockFP) {
this.packedIndex = packedIndex;
this.minLeafBlockFP = minLeadBlockFP;
}
@Override
public BKDOffHeapInput clone() {
return new BKDOffHeapInput(packedIndex.clone(), minLeafBlockFP);
}
@Override
long getMinLeafBlockFP() {
return minLeafBlockFP;
return new BKDOffHeapInput(packedIndex.clone());
}
@Override
@ -105,28 +92,20 @@ public final class BKDReader extends PointValues implements Accountable {
private static class BKDOnHeapInput extends BKDInput implements Cloneable {
private final ByteArrayDataInput packedIndex;
private final long minLeafBlockFP;
BKDOnHeapInput(IndexInput packedIndex, int numBytes) throws IOException {
byte[] packedBytes = new byte[numBytes];
packedIndex.readBytes(packedBytes, 0, numBytes);
this.packedIndex = new ByteArrayDataInput(packedBytes);
this.minLeafBlockFP = this.packedIndex.clone().readVLong();
}
private BKDOnHeapInput(ByteArrayDataInput packedIndex, long minLeadBlockFP) {
private BKDOnHeapInput(ByteArrayDataInput packedIndex) {
this.packedIndex = packedIndex;
this.minLeafBlockFP = minLeadBlockFP;
}
@Override
public BKDOnHeapInput clone() {
return new BKDOnHeapInput((ByteArrayDataInput)packedIndex.clone(), minLeafBlockFP);
}
@Override
long getMinLeafBlockFP() {
return minLeafBlockFP;
return new BKDOnHeapInput((ByteArrayDataInput)packedIndex.clone());
}
@Override
@ -170,64 +149,74 @@ public final class BKDReader extends PointValues implements Accountable {
final int version;
protected final int packedBytesLength;
protected final int packedIndexBytesLength;
final long minLeafBlockFP;
final BKDInput packedIndex;
/** Caller must pre-seek the provided {@link IndexInput} to the index location that {@link BKDWriter#finish} returned */
public BKDReader(IndexInput in) throws IOException {
this(in, in instanceof ByteBufferIndexInput);
public BKDReader(IndexInput metaIn, IndexInput indexIn, IndexInput dataIn) throws IOException {
this(metaIn, indexIn, dataIn, indexIn instanceof ByteBufferIndexInput);
}
/**
* Caller must pre-seek the provided {@link IndexInput} to the index location that {@link BKDWriter#finish} returned
* and specify {@code true} to store BKD off-heap ({@code false} otherwise)
*/
public BKDReader(IndexInput in, boolean offHeap) throws IOException {
version = CodecUtil.checkHeader(in, BKDWriter.CODEC_NAME, BKDWriter.VERSION_START, BKDWriter.VERSION_CURRENT);
numDataDims = in.readVInt();
public BKDReader(IndexInput metaIn, IndexInput indexIn, IndexInput dataIn, boolean offHeap) throws IOException {
version = CodecUtil.checkHeader(metaIn, BKDWriter.CODEC_NAME, BKDWriter.VERSION_START, BKDWriter.VERSION_CURRENT);
numDataDims = metaIn.readVInt();
if (version >= BKDWriter.VERSION_SELECTIVE_INDEXING) {
numIndexDims = in.readVInt();
numIndexDims = metaIn.readVInt();
} else {
numIndexDims = numDataDims;
}
maxPointsInLeafNode = in.readVInt();
bytesPerDim = in.readVInt();
maxPointsInLeafNode = metaIn.readVInt();
bytesPerDim = metaIn.readVInt();
packedBytesLength = numDataDims * bytesPerDim;
packedIndexBytesLength = numIndexDims * bytesPerDim;
// Read index:
numLeaves = in.readVInt();
numLeaves = metaIn.readVInt();
assert numLeaves > 0;
leafNodeOffset = numLeaves;
minPackedValue = new byte[packedIndexBytesLength];
maxPackedValue = new byte[packedIndexBytesLength];
in.readBytes(minPackedValue, 0, packedIndexBytesLength);
in.readBytes(maxPackedValue, 0, packedIndexBytesLength);
metaIn.readBytes(minPackedValue, 0, packedIndexBytesLength);
metaIn.readBytes(maxPackedValue, 0, packedIndexBytesLength);
for(int dim=0;dim<numIndexDims;dim++) {
if (Arrays.compareUnsigned(minPackedValue, dim * bytesPerDim, dim * bytesPerDim + bytesPerDim, maxPackedValue, dim * bytesPerDim, dim * bytesPerDim + bytesPerDim) > 0) {
throw new CorruptIndexException("minPackedValue " + new BytesRef(minPackedValue) + " is > maxPackedValue " + new BytesRef(maxPackedValue) + " for dim=" + dim, in);
throw new CorruptIndexException("minPackedValue " + new BytesRef(minPackedValue) + " is > maxPackedValue " + new BytesRef(maxPackedValue) + " for dim=" + dim, metaIn);
}
}
pointCount = in.readVLong();
docCount = in.readVInt();
pointCount = metaIn.readVLong();
docCount = metaIn.readVInt();
int numBytes = in.readVInt();
IndexInput slice = in.slice("packedIndex", in.getFilePointer(), numBytes);
int numIndexBytes = metaIn.readVInt();
long indexStartPointer;
if (version >= BKDWriter.VERSION_META_FILE) {
minLeafBlockFP = metaIn.readLong();
indexStartPointer = metaIn.readLong();
} else {
indexStartPointer = indexIn.getFilePointer();
minLeafBlockFP = indexIn.readVLong();
indexIn.seek(indexStartPointer);
}
IndexInput slice = indexIn.slice("packedIndex", indexStartPointer, numIndexBytes);
if (offHeap) {
packedIndex = new BKDOffHeapInput(slice);
} else {
packedIndex = new BKDOnHeapInput(slice, numBytes);
packedIndex = new BKDOnHeapInput(slice, numIndexBytes);
}
this.in = in;
this.in = dataIn;
}
long getMinLeafBlockFP() {
return packedIndex.getMinLeafBlockFP();
return minLeafBlockFP;
}
/** Used to walk the in-heap index. The format takes advantage of the limited

View File

@ -18,6 +18,7 @@ package org.apache.lucene.util.bkd;
import java.io.Closeable;
import java.io.IOException;
import java.io.UncheckedIOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
@ -82,7 +83,8 @@ public class BKDWriter implements Closeable {
public static final int VERSION_LEAF_STORES_BOUNDS = 5;
public static final int VERSION_SELECTIVE_INDEXING = 6;
public static final int VERSION_LOW_CARDINALITY_LEAVES = 7;
public static final int VERSION_CURRENT = VERSION_LOW_CARDINALITY_LEAVES;
public static final int VERSION_META_FILE = 9;
public static final int VERSION_CURRENT = VERSION_META_FILE;
/** How many bytes each docs takes in the fixed-width offline format */
private final int bytesPerDoc;
@ -253,11 +255,6 @@ public class BKDWriter implements Closeable {
docsSeen.set(docID);
}
/** How many points have been added so far */
public long getPointCount() {
return pointCount;
}
private static class MergeReader {
final BKDReader bkd;
final BKDReader.IntersectState state;
@ -393,11 +390,11 @@ public class BKDWriter implements Closeable {
* there is opportunity for reordering points before writing them to
* disk. This method does not use transient disk in order to reorder points.
*/
public long writeField(IndexOutput out, String fieldName, MutablePointValues reader) throws IOException {
public Runnable writeField(IndexOutput metaOut, IndexOutput indexOut, IndexOutput dataOut, String fieldName, MutablePointValues reader) throws IOException {
if (numDataDims == 1) {
return writeField1Dim(out, fieldName, reader);
return writeField1Dim(metaOut, indexOut, dataOut, fieldName, reader);
} else {
return writeFieldNDims(out, fieldName, reader);
return writeFieldNDims(metaOut, indexOut, dataOut, fieldName, reader);
}
}
@ -424,7 +421,7 @@ public class BKDWriter implements Closeable {
/* In the 2+D case, we recursively pick the split dimension, compute the
* median value and partition other values around it. */
private long writeFieldNDims(IndexOutput out, String fieldName, MutablePointValues values) throws IOException {
private Runnable writeFieldNDims(IndexOutput metaOut, IndexOutput indexOut, IndexOutput dataOut, String fieldName, MutablePointValues values) throws IOException {
if (pointCount != 0) {
throw new IllegalStateException("cannot mix add and writeField");
}
@ -454,8 +451,9 @@ public class BKDWriter implements Closeable {
docsSeen.set(values.getDocID(i));
}
final long dataStartFP = dataOut.getFilePointer();
final int[] parentSplits = new int[numIndexDims];
build(0, numLeaves, values, 0, Math.toIntExact(pointCount), out,
build(0, numLeaves, values, 0, Math.toIntExact(pointCount), dataOut,
minPackedValue.clone(), maxPackedValue.clone(), parentSplits,
splitPackedValues, splitDimensionValues, leafBlockFPs,
new int[maxPointsInLeafNode]);
@ -487,17 +485,21 @@ public class BKDWriter implements Closeable {
}
};
long indexFP = out.getFilePointer();
writeIndex(out, maxPointsInLeafNode, leafNodes);
return indexFP;
return () -> {
try {
writeIndex(metaOut, indexOut, maxPointsInLeafNode, leafNodes, dataStartFP);
} catch (IOException e) {
throw new UncheckedIOException(e);
}
};
}
/* In the 1D case, we can simply sort points in ascending order and use the
* same writing logic as we use at merge time. */
private long writeField1Dim(IndexOutput out, String fieldName, MutablePointValues reader) throws IOException {
private Runnable writeField1Dim(IndexOutput metaOut, IndexOutput indexOut, IndexOutput dataOut, String fieldName, MutablePointValues reader) throws IOException {
MutablePointsReaderUtils.sort(maxDoc, packedIndexBytesLength, reader, 0, Math.toIntExact(reader.size()));
final OneDimensionBKDWriter oneDimWriter = new OneDimensionBKDWriter(out);
final OneDimensionBKDWriter oneDimWriter = new OneDimensionBKDWriter(metaOut, indexOut, dataOut);
reader.intersect(new IntersectVisitor() {
@ -523,7 +525,7 @@ public class BKDWriter implements Closeable {
/** More efficient bulk-add for incoming {@link BKDReader}s. This does a merge sort of the already
* sorted values and currently only works when numDims==1. This returns -1 if all documents containing
* dimensional values were deleted. */
public long merge(IndexOutput out, List<MergeState.DocMap> docMaps, List<BKDReader> readers) throws IOException {
public Runnable merge(IndexOutput metaOut, IndexOutput indexOut, IndexOutput dataOut, List<MergeState.DocMap> docMaps, List<BKDReader> readers) throws IOException {
assert docMaps == null || readers.size() == docMaps.size();
BKDMergeQueue queue = new BKDMergeQueue(bytesPerDim, readers.size());
@ -542,7 +544,7 @@ public class BKDWriter implements Closeable {
}
}
OneDimensionBKDWriter oneDimWriter = new OneDimensionBKDWriter(out);
OneDimensionBKDWriter oneDimWriter = new OneDimensionBKDWriter(metaOut, indexOut, dataOut);
while (queue.size() != 0) {
MergeReader reader = queue.top();
@ -566,7 +568,8 @@ public class BKDWriter implements Closeable {
private class OneDimensionBKDWriter {
final IndexOutput out;
final IndexOutput metaOut, indexOut, dataOut;
final long dataStartFP;
final List<Long> leafBlockFPs = new ArrayList<>();
final List<byte[]> leafBlockStartValues = new ArrayList<>();
final byte[] leafValues = new byte[maxPointsInLeafNode * packedBytesLength];
@ -575,7 +578,7 @@ public class BKDWriter implements Closeable {
private int leafCount;
private int leafCardinality;
OneDimensionBKDWriter(IndexOutput out) {
OneDimensionBKDWriter(IndexOutput metaOut, IndexOutput indexOut, IndexOutput dataOut) {
if (numIndexDims != 1) {
throw new UnsupportedOperationException("numIndexDims must be 1 but got " + numIndexDims);
}
@ -591,7 +594,10 @@ public class BKDWriter implements Closeable {
// Mark that we already finished:
finished = true;
this.out = out;
this.metaOut = metaOut;
this.indexOut = indexOut;
this.dataOut = dataOut;
this.dataStartFP = dataOut.getFilePointer();
lastPackedValue = new byte[packedBytesLength];
}
@ -627,7 +633,7 @@ public class BKDWriter implements Closeable {
assert (lastDocID = docID) >= 0; // only assign when asserts are enabled
}
public long finish() throws IOException {
public Runnable finish() throws IOException {
if (leafCount > 0) {
writeLeafBlock(leafCardinality);
leafCardinality = 0;
@ -635,13 +641,11 @@ public class BKDWriter implements Closeable {
}
if (valueCount == 0) {
return -1;
return null;
}
pointCount = valueCount;
long indexFP = out.getFilePointer();
scratchBytesRef1.length = bytesPerDim;
scratchBytesRef1.offset = 0;
assert leafBlockStartValues.size() + 1 == leafBlockFPs.size();
@ -667,8 +671,13 @@ public class BKDWriter implements Closeable {
return leafBlockFPs.size();
}
};
writeIndex(out, maxPointsInLeafNode, leafNodes);
return indexFP;
return () -> {
try {
writeIndex(metaOut, indexOut, maxPointsInLeafNode, leafNodes, dataStartFP);
} catch (IOException e) {
throw new UncheckedIOException(e);
}
};
}
private void writeLeafBlock(int leafCardinality) throws IOException {
@ -684,7 +693,7 @@ public class BKDWriter implements Closeable {
// Save the first (minimum) value in each leaf block except the first, to build the split value index in the end:
leafBlockStartValues.add(ArrayUtil.copyOfSubArray(leafValues, 0, packedBytesLength));
}
leafBlockFPs.add(out.getFilePointer());
leafBlockFPs.add(dataOut.getFilePointer());
checkMaxLeafNodeCount(leafBlockFPs.size());
// Find per-dim common prefix:
@ -714,7 +723,7 @@ public class BKDWriter implements Closeable {
ArrayUtil.copyOfSubArray(leafValues, (leafCount - 1) * packedBytesLength, leafCount * packedBytesLength),
packedValues, leafDocs, 0);
writeLeafBlockPackedValues(scratchOut, commonPrefixLengths, leafCount, 0, packedValues, leafCardinality);
scratchOut.copyTo(out);
scratchOut.copyTo(dataOut);
scratchOut.reset();
}
}
@ -763,8 +772,9 @@ public class BKDWriter implements Closeable {
}
}
/** Writes the BKD tree to the provided {@link IndexOutput} and returns the file offset where index was written. */
public long finish(IndexOutput out) throws IOException {
/** Writes the BKD tree to the provided {@link IndexOutput}s and returns a {@link Runnable} that
* writes the index of the tree if at least one point has been added, or {@code null} otherwise. */
public Runnable finish(IndexOutput metaOut, IndexOutput indexOut, IndexOutput dataOut) throws IOException {
// System.out.println("\nBKDTreeWriter.finish pointCount=" + pointCount + " out=" + out + " heapWriter=" + heapPointWriter);
// TODO: specialize the 1D case? it's much faster at indexing time (no partitioning on recurse...)
@ -775,7 +785,7 @@ public class BKDWriter implements Closeable {
}
if (pointCount == 0) {
throw new IllegalStateException("must index at least one point");
return null;
}
//mark as finished
@ -808,12 +818,13 @@ public class BKDWriter implements Closeable {
//We re-use the selector so we do not need to create an object every time.
BKDRadixSelector radixSelector = new BKDRadixSelector(numDataDims, numIndexDims, bytesPerDim, maxPointsSortInHeap, tempDir, tempFileNamePrefix);
final long dataStartFP = dataOut.getFilePointer();
boolean success = false;
try {
final int[] parentSplits = new int[numIndexDims];
build(0, numLeaves, points,
out, radixSelector,
dataOut, radixSelector,
minPackedValue.clone(), maxPackedValue.clone(),
parentSplits,
splitPackedValues,
@ -859,10 +870,14 @@ public class BKDWriter implements Closeable {
}
};
return () -> {
// Write index:
long indexFP = out.getFilePointer();
writeIndex(out, maxPointsInLeafNode, leafNodes);
return indexFP;
try {
writeIndex(metaOut, indexOut, maxPointsInLeafNode, leafNodes, dataStartFP);
} catch (IOException e) {
throw new UncheckedIOException(e);
}
};
}
/** Packs the two arrays, representing a semi-balanced binary tree, into a compact byte[] structure. */
@ -1019,28 +1034,32 @@ public class BKDWriter implements Closeable {
}
}
private void writeIndex(IndexOutput out, int countPerLeaf, BKDTreeLeafNodes leafNodes) throws IOException {
private void writeIndex(IndexOutput metaOut, IndexOutput indexOut, int countPerLeaf, BKDTreeLeafNodes leafNodes, long dataStartFP) throws IOException {
byte[] packedIndex = packIndex(leafNodes);
writeIndex(out, countPerLeaf, leafNodes.numLeaves(), packedIndex);
writeIndex(metaOut, indexOut, countPerLeaf, leafNodes.numLeaves(), packedIndex, dataStartFP);
}
private void writeIndex(IndexOutput out, int countPerLeaf, int numLeaves, byte[] packedIndex) throws IOException {
CodecUtil.writeHeader(out, CODEC_NAME, VERSION_CURRENT);
out.writeVInt(numDataDims);
out.writeVInt(numIndexDims);
out.writeVInt(countPerLeaf);
out.writeVInt(bytesPerDim);
private void writeIndex(IndexOutput metaOut, IndexOutput indexOut, int countPerLeaf, int numLeaves, byte[] packedIndex, long dataStartFP) throws IOException {
CodecUtil.writeHeader(metaOut, CODEC_NAME, VERSION_CURRENT);
metaOut.writeVInt(numDataDims);
metaOut.writeVInt(numIndexDims);
metaOut.writeVInt(countPerLeaf);
metaOut.writeVInt(bytesPerDim);
assert numLeaves > 0;
out.writeVInt(numLeaves);
out.writeBytes(minPackedValue, 0, packedIndexBytesLength);
out.writeBytes(maxPackedValue, 0, packedIndexBytesLength);
metaOut.writeVInt(numLeaves);
metaOut.writeBytes(minPackedValue, 0, packedIndexBytesLength);
metaOut.writeBytes(maxPackedValue, 0, packedIndexBytesLength);
out.writeVLong(pointCount);
out.writeVInt(docsSeen.cardinality());
out.writeVInt(packedIndex.length);
out.writeBytes(packedIndex, 0, packedIndex.length);
metaOut.writeVLong(pointCount);
metaOut.writeVInt(docsSeen.cardinality());
metaOut.writeVInt(packedIndex.length);
metaOut.writeLong(dataStartFP);
// If metaOut and indexOut are the same file, we account for the fact that
// writing a long makes the index start 8 bytes later.
metaOut.writeLong(indexOut.getFilePointer() + (metaOut == indexOut ? Long.BYTES : 0));
indexOut.writeBytes(packedIndex, 0, packedIndex.length);
}
private void writeLeafBlockDocs(DataOutput out, int[] docIDs, int start, int count) throws IOException {

View File

@ -0,0 +1,393 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene86;
import java.io.IOException;
import java.util.Arrays;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.PointsFormat;
import org.apache.lucene.codecs.PointsReader;
import org.apache.lucene.codecs.PointsWriter;
import org.apache.lucene.document.BinaryPoint;
import org.apache.lucene.document.Document;
import org.apache.lucene.index.BasePointsFormatTestCase;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.MockRandomMergePolicy;
import org.apache.lucene.index.PointValues;
import org.apache.lucene.index.PointValues.IntersectVisitor;
import org.apache.lucene.index.PointValues.Relation;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.TestUtil;
import org.apache.lucene.util.bkd.BKDWriter;
public class TestLucene86PointsFormat extends BasePointsFormatTestCase {
private final Codec codec;
private final int maxPointsInLeafNode;
public TestLucene86PointsFormat() {
// standard issue
Codec defaultCodec = new Lucene86Codec();
if (random().nextBoolean()) {
// randomize parameters
maxPointsInLeafNode = TestUtil.nextInt(random(), 50, 500);
double maxMBSortInHeap = 3.0 + (3*random().nextDouble());
if (VERBOSE) {
System.out.println("TEST: using Lucene60PointsFormat with maxPointsInLeafNode=" + maxPointsInLeafNode + " and maxMBSortInHeap=" + maxMBSortInHeap);
}
// sneaky impersonation!
codec = new FilterCodec(defaultCodec.getName(), defaultCodec) {
@Override
public PointsFormat pointsFormat() {
return new PointsFormat() {
@Override
public PointsWriter fieldsWriter(SegmentWriteState writeState) throws IOException {
return new Lucene86PointsWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap);
}
@Override
public PointsReader fieldsReader(SegmentReadState readState) throws IOException {
return new Lucene86PointsReader(readState);
}
};
}
};
} else {
// standard issue
codec = defaultCodec;
maxPointsInLeafNode = BKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE;
}
}
@Override
protected Codec getCodec() {
return codec;
}
@Override
public void testMergeStability() throws Exception {
assumeFalse("TODO: mess with the parameters and test gets angry!", codec instanceof FilterCodec);
super.testMergeStability();
}
public void testEstimatePointCount() throws IOException {
Directory dir = newDirectory();
IndexWriterConfig iwc = newIndexWriterConfig();
// Avoid mockRandomMP since it may cause non-optimal merges that make the
// number of points per leaf hard to predict
while (iwc.getMergePolicy() instanceof MockRandomMergePolicy) {
iwc.setMergePolicy(newMergePolicy());
}
IndexWriter w = new IndexWriter(dir, iwc);
byte[] pointValue = new byte[3];
byte[] uniquePointValue = new byte[3];
random().nextBytes(uniquePointValue);
final int numDocs = TEST_NIGHTLY ? atLeast(10000) : atLeast(500); // at night, make sure we have several leaves
final boolean multiValues = random().nextBoolean();
for (int i = 0; i < numDocs; ++i) {
Document doc = new Document();
if (i == numDocs / 2) {
doc.add(new BinaryPoint("f", uniquePointValue));
} else {
final int numValues = (multiValues) ? TestUtil.nextInt(random(), 2, 100) : 1;
for (int j = 0; j < numValues; j ++) {
do {
random().nextBytes(pointValue);
} while (Arrays.equals(pointValue, uniquePointValue));
doc.add(new BinaryPoint("f", pointValue));
}
}
w.addDocument(doc);
}
w.forceMerge(1);
final IndexReader r = DirectoryReader.open(w);
w.close();
final LeafReader lr = getOnlyLeafReader(r);
PointValues points = lr.getPointValues("f");
// If all points match, then the point count is numLeaves * maxPointsInLeafNode
final int numLeaves = (int) Math.ceil((double) points.size() / maxPointsInLeafNode);
IntersectVisitor allPointsVisitor = new IntersectVisitor() {
@Override
public void visit(int docID, byte[] packedValue) throws IOException {}
@Override
public void visit(int docID) throws IOException {}
@Override
public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
return Relation.CELL_INSIDE_QUERY;
}
};
assertEquals(numLeaves * maxPointsInLeafNode, points.estimatePointCount(allPointsVisitor));
assertEquals(numDocs, points.estimateDocCount(allPointsVisitor));
IntersectVisitor noPointsVisitor = new IntersectVisitor() {
@Override
public void visit(int docID, byte[] packedValue) throws IOException {}
@Override
public void visit(int docID) throws IOException {}
@Override
public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
return Relation.CELL_OUTSIDE_QUERY;
}
};
// Return 0 if no points match
assertEquals(0, points.estimatePointCount(noPointsVisitor));
assertEquals(0, points.estimateDocCount(noPointsVisitor));
IntersectVisitor onePointMatchVisitor = new IntersectVisitor() {
@Override
public void visit(int docID, byte[] packedValue) throws IOException {}
@Override
public void visit(int docID) throws IOException {}
@Override
public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
if (Arrays.compareUnsigned(uniquePointValue, 0, 3, maxPackedValue, 0, 3) > 0 ||
Arrays.compareUnsigned(uniquePointValue, 0, 3, minPackedValue, 0, 3) < 0) {
return Relation.CELL_OUTSIDE_QUERY;
}
return Relation.CELL_CROSSES_QUERY;
}
};
// If only one point matches, then the point count is (maxPointsInLeafNode + 1) / 2
// in general, or maybe 2x that if the point is a split value
final long pointCount = points.estimatePointCount(onePointMatchVisitor);
assertTrue(""+pointCount,
pointCount == (maxPointsInLeafNode + 1) / 2 || // common case
pointCount == 2*((maxPointsInLeafNode + 1) / 2)); // if the point is a split value
final long docCount = points.estimateDocCount(onePointMatchVisitor);
if (multiValues) {
assertEquals(docCount, (long) (docCount * (1d - Math.pow( (numDocs - pointCount) / points.size() , points.size() / docCount))));
} else {
assertEquals(Math.min(pointCount, numDocs), docCount);
}
r.close();
dir.close();
}
// The tree is always balanced in the N dims case, and leaves are
// not all full so things are a bit different
public void testEstimatePointCount2Dims() throws IOException {
Directory dir = newDirectory();
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig());
byte[][] pointValue = new byte[2][];
pointValue[0] = new byte[3];
pointValue[1] = new byte[3];
byte[][] uniquePointValue = new byte[2][];
uniquePointValue[0] = new byte[3];
uniquePointValue[1] = new byte[3];
random().nextBytes(uniquePointValue[0]);
random().nextBytes(uniquePointValue[1]);
final int numDocs = TEST_NIGHTLY? atLeast(10000) : atLeast(1000); // in nightly, make sure we have several leaves
final boolean multiValues = random().nextBoolean();
for (int i = 0; i < numDocs; ++i) {
Document doc = new Document();
if (i == numDocs / 2) {
doc.add(new BinaryPoint("f", uniquePointValue));
} else {
final int numValues = (multiValues) ? TestUtil.nextInt(random(), 2, 100) : 1;
for (int j = 0; j < numValues; j ++) {
do {
random().nextBytes(pointValue[0]);
random().nextBytes(pointValue[1]);
} while (Arrays.equals(pointValue[0], uniquePointValue[0]) || Arrays.equals(pointValue[1], uniquePointValue[1]));
doc.add(new BinaryPoint("f", pointValue));
}
}
w.addDocument(doc);
}
w.forceMerge(1);
final IndexReader r = DirectoryReader.open(w);
w.close();
final LeafReader lr = getOnlyLeafReader(r);
PointValues points = lr.getPointValues("f");
IntersectVisitor allPointsVisitor = new IntersectVisitor() {
@Override
public void visit(int docID, byte[] packedValue) throws IOException {}
@Override
public void visit(int docID) throws IOException {}
@Override
public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
return Relation.CELL_INSIDE_QUERY;
}
};
// If all points match, then the point count is numLeaves * maxPointsInLeafNode
final int numLeaves = (int) Math.ceil((double) points.size() / maxPointsInLeafNode);
assertEquals(numLeaves * maxPointsInLeafNode, points.estimatePointCount(allPointsVisitor));
assertEquals(numDocs, points.estimateDocCount(allPointsVisitor));
IntersectVisitor noPointsVisitor = new IntersectVisitor() {
@Override
public void visit(int docID, byte[] packedValue) throws IOException {}
@Override
public void visit(int docID) throws IOException {}
@Override
public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
return Relation.CELL_OUTSIDE_QUERY;
}
};
// Return 0 if no points match
assertEquals(0, points.estimatePointCount(noPointsVisitor));
assertEquals(0, points.estimateDocCount(noPointsVisitor));
IntersectVisitor onePointMatchVisitor = new IntersectVisitor() {
@Override
public void visit(int docID, byte[] packedValue) throws IOException {}
@Override
public void visit(int docID) throws IOException {}
@Override
public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
for (int dim = 0; dim < 2; ++dim) {
if (Arrays.compareUnsigned(uniquePointValue[dim], 0, 3, maxPackedValue, dim * 3, dim * 3 + 3) > 0 ||
Arrays.compareUnsigned(uniquePointValue[dim], 0, 3, minPackedValue, dim * 3, dim * 3 + 3) < 0) {
return Relation.CELL_OUTSIDE_QUERY;
}
}
return Relation.CELL_CROSSES_QUERY;
}
};
final long pointCount = points.estimatePointCount(onePointMatchVisitor);
// The number of matches needs to be multiple of count per leaf
final long countPerLeaf = (maxPointsInLeafNode + 1) / 2;
assertTrue(""+pointCount, pointCount % countPerLeaf == 0);
// in extreme cases, a point can be be shared by 4 leaves
assertTrue(""+pointCount, pointCount / countPerLeaf <= 4 && pointCount / countPerLeaf >= 1);
final long docCount = points.estimateDocCount(onePointMatchVisitor);
if (multiValues) {
assertEquals(docCount, (long) (docCount * (1d - Math.pow( (numDocs - pointCount) / points.size() , points.size() / docCount))));
} else {
assertEquals(Math.min(pointCount, numDocs), docCount);
}
r.close();
dir.close();
}
public void testDocCountEdgeCases() {
PointValues values = getPointValues(Long.MAX_VALUE, 1, Long.MAX_VALUE);
long docs = values.estimateDocCount(null);
assertEquals(1, docs);
values = getPointValues(Long.MAX_VALUE, 1, 1);
docs = values.estimateDocCount(null);
assertEquals(1, docs);
values = getPointValues(Long.MAX_VALUE, Integer.MAX_VALUE, Long.MAX_VALUE);
docs = values.estimateDocCount(null);
assertEquals(Integer.MAX_VALUE, docs);
values = getPointValues(Long.MAX_VALUE, Integer.MAX_VALUE, Long.MAX_VALUE / 2);
docs = values.estimateDocCount(null);
assertEquals(Integer.MAX_VALUE, docs);
values = getPointValues(Long.MAX_VALUE, Integer.MAX_VALUE, 1);
docs = values.estimateDocCount(null);
assertEquals(1, docs);
}
public void testRandomDocCount() {
for (int i = 0; i < 100; i++) {
long size = TestUtil.nextLong(random(), 1, Long.MAX_VALUE);
int maxDoc = (size > Integer.MAX_VALUE) ? Integer.MAX_VALUE : Math.toIntExact(size);
int docCount = TestUtil.nextInt(random(), 1, maxDoc);
long estimatedPointCount = TestUtil.nextLong(random(), 0, size);
PointValues values = getPointValues(size, docCount, estimatedPointCount);
long docs = values.estimateDocCount(null);
assertTrue(docs <= estimatedPointCount);
assertTrue(docs <= maxDoc);
assertTrue(docs >= estimatedPointCount / (size/docCount));
}
}
private PointValues getPointValues(long size, int docCount, long estimatedPointCount) {
return new PointValues() {
@Override
public void intersect(IntersectVisitor visitor) {
throw new UnsupportedOperationException();
}
@Override
public long estimatePointCount(IntersectVisitor visitor) {
return estimatedPointCount;
}
@Override
public byte[] getMinPackedValue() throws IOException {
throw new UnsupportedOperationException();
}
@Override
public byte[] getMaxPackedValue() throws IOException {
throw new UnsupportedOperationException();
}
@Override
public int getNumDimensions() throws IOException {
throw new UnsupportedOperationException();
}
@Override
public int getNumIndexDimensions() throws IOException {
throw new UnsupportedOperationException();
}
@Override
public int getBytesPerDimension() throws IOException {
throw new UnsupportedOperationException();
}
@Override
public long size() {
return size;
}
@Override
public int getDocCount() {
return docCount;
}
};
}
}

View File

@ -38,8 +38,8 @@ import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.PointsFormat;
import org.apache.lucene.codecs.PointsReader;
import org.apache.lucene.codecs.PointsWriter;
import org.apache.lucene.codecs.lucene60.Lucene60PointsReader;
import org.apache.lucene.codecs.lucene60.Lucene60PointsWriter;
import org.apache.lucene.codecs.lucene86.Lucene86PointsReader;
import org.apache.lucene.codecs.lucene86.Lucene86PointsWriter;
import org.apache.lucene.document.BinaryPoint;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.DoublePoint;
@ -1173,12 +1173,12 @@ public class TestPointQueries extends LuceneTestCase {
return new PointsFormat() {
@Override
public PointsWriter fieldsWriter(SegmentWriteState writeState) throws IOException {
return new Lucene60PointsWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap);
return new Lucene86PointsWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap);
}
@Override
public PointsReader fieldsReader(SegmentReadState readState) throws IOException {
return new Lucene60PointsReader(readState);
return new Lucene86PointsReader(readState);
}
};
}

View File

@ -58,12 +58,14 @@ public class Test2BBKDPoints extends LuceneTestCase {
}
}
IndexOutput out = dir.createOutput("1d.bkd", IOContext.DEFAULT);
long indexFP = w.finish(out);
Runnable finalizer = w.finish(out, out, out);
long indexFP = out.getFilePointer();
finalizer.run();
out.close();
IndexInput in = dir.openInput("1d.bkd", IOContext.DEFAULT);
in.seek(indexFP);
BKDReader r = new BKDReader(in);
BKDReader r = new BKDReader(in, in, in);
CheckIndex.VerifyPointsVisitor visitor = new CheckIndex.VerifyPointsVisitor("1d", numDocs, r);
r.intersect(visitor);
assertEquals(r.size(), visitor.getPointCountSeen());
@ -98,12 +100,14 @@ public class Test2BBKDPoints extends LuceneTestCase {
}
}
IndexOutput out = dir.createOutput("2d.bkd", IOContext.DEFAULT);
long indexFP = w.finish(out);
Runnable finalizer = w.finish(out, out, out);
long indexFP = out.getFilePointer();
finalizer.run();
out.close();
IndexInput in = dir.openInput("2d.bkd", IOContext.DEFAULT);
in.seek(indexFP);
BKDReader r = new BKDReader(in);
BKDReader r = new BKDReader(in, in, in);
CheckIndex.VerifyPointsVisitor visitor = new CheckIndex.VerifyPointsVisitor("2d", numDocs, r);
r.intersect(visitor);
assertEquals(r.size(), visitor.getPointCountSeen());

View File

@ -61,12 +61,14 @@ public class TestBKD extends LuceneTestCase {
long indexFP;
try (IndexOutput out = dir.createOutput("bkd", IOContext.DEFAULT)) {
indexFP = w.finish(out);
Runnable finalizer = w.finish(out, out, out);
indexFP = out.getFilePointer();
finalizer.run();
}
try (IndexInput in = dir.openInput("bkd", IOContext.DEFAULT)) {
in.seek(indexFP);
BKDReader r = new BKDReader(in, randomBoolean());
BKDReader r = new BKDReader(in, in, in, false);//randomBoolean());
// Simple 1D range query:
final int queryMin = 42;
@ -163,12 +165,14 @@ public class TestBKD extends LuceneTestCase {
long indexFP;
try (IndexOutput out = dir.createOutput("bkd", IOContext.DEFAULT)) {
indexFP = w.finish(out);
Runnable finalizer = w.finish(out, out, out);
indexFP = out.getFilePointer();
finalizer.run();
}
try (IndexInput in = dir.openInput("bkd", IOContext.DEFAULT)) {
in.seek(indexFP);
BKDReader r = new BKDReader(in, randomBoolean());
BKDReader r = new BKDReader(in, in, in, randomBoolean());
byte[] minPackedValue = r.getMinPackedValue();
byte[] maxPackedValue = r.getMaxPackedValue();
@ -291,12 +295,14 @@ public class TestBKD extends LuceneTestCase {
long indexFP;
try (IndexOutput out = dir.createOutput("bkd", IOContext.DEFAULT)) {
indexFP = w.finish(out);
Runnable finalizer = w.finish(out, out, out);
indexFP = out.getFilePointer();
finalizer.run();
}
try (IndexInput in = dir.openInput("bkd", IOContext.DEFAULT)) {
in.seek(indexFP);
BKDReader r = new BKDReader(in, randomBoolean());
BKDReader r = new BKDReader(in, in, in, randomBoolean());
int iters = atLeast(100);
for(int iter=0;iter<iters;iter++) {
@ -756,7 +762,9 @@ public class TestBKD extends LuceneTestCase {
return curDocIDBase + docID;
}
});
toMerge.add(w.finish(out));
Runnable finalizer = w.finish(out, out, out);
toMerge.add(out.getFilePointer());
finalizer.run();
valuesInThisSeg = TestUtil.nextInt(random(), numValues/10, numValues/2);
segCount = 0;
@ -772,7 +780,9 @@ public class TestBKD extends LuceneTestCase {
if (toMerge != null) {
if (segCount > 0) {
toMerge.add(w.finish(out));
Runnable finalizer = w.finish(out, out, out);
toMerge.add(out.getFilePointer());
finalizer.run();
final int curDocIDBase = lastDocIDBase;
docMaps.add(new MergeState.DocMap() {
@Override
@ -788,21 +798,25 @@ public class TestBKD extends LuceneTestCase {
List<BKDReader> readers = new ArrayList<>();
for(long fp : toMerge) {
in.seek(fp);
readers.add(new BKDReader(in, randomBoolean()));
readers.add(new BKDReader(in, in, in, randomBoolean()));
}
out = dir.createOutput("bkd2", IOContext.DEFAULT);
indexFP = w.merge(out, docMaps, readers);
Runnable finalizer = w.merge(out, out, out, docMaps, readers);
indexFP = out.getFilePointer();
finalizer.run();
out.close();
in.close();
in = dir.openInput("bkd2", IOContext.DEFAULT);
} else {
indexFP = w.finish(out);
Runnable finalizer = w.finish(out, out, out);
indexFP = out.getFilePointer();
finalizer.run();
out.close();
in = dir.openInput("bkd", IOContext.DEFAULT);
}
in.seek(indexFP);
BKDReader r = new BKDReader(in, randomBoolean());
BKDReader r = new BKDReader(in, in, in, randomBoolean());
int iters = atLeast(100);
for(int iter=0;iter<iters;iter++) {
@ -1071,12 +1085,14 @@ public class TestBKD extends LuceneTestCase {
}
IndexOutput out = dir.createOutput("bkd", IOContext.DEFAULT);
long fp = w.finish(out);
Runnable finalizer = w.finish(out, out, out);
long fp = out.getFilePointer();
finalizer.run();
out.close();
IndexInput in = dir.openInput("bkd", IOContext.DEFAULT);
in.seek(fp);
BKDReader r = new BKDReader(in, randomBoolean());
BKDReader r = new BKDReader(in, in, in, randomBoolean());
r.intersect(new IntersectVisitor() {
int lastDocID = -1;
@ -1127,13 +1143,15 @@ public class TestBKD extends LuceneTestCase {
}
final long indexFP;
try (IndexOutput out = dir.createOutput("bkd", IOContext.DEFAULT)) {
indexFP = w.finish(out);
Runnable finalizer = w.finish(out, out, out);
indexFP = out.getFilePointer();
finalizer.run();
w.close();
}
IndexInput pointsIn = dir.openInput("bkd", IOContext.DEFAULT);
pointsIn.seek(indexFP);
BKDReader points = new BKDReader(pointsIn);
BKDReader points = new BKDReader(pointsIn, pointsIn, pointsIn);
points.intersect(new IntersectVisitor() {
@ -1185,12 +1203,14 @@ public class TestBKD extends LuceneTestCase {
}
IndexOutput out = dir.createOutput("bkd", IOContext.DEFAULT);
long fp = w.finish(out);
Runnable finalizer = w.finish(out, out, out);
long fp = out.getFilePointer();
finalizer.run();
out.close();
IndexInput in = dir.openInput("bkd", IOContext.DEFAULT);
in.seek(fp);
BKDReader r = new BKDReader(in, randomBoolean());
BKDReader r = new BKDReader(in, in, in, randomBoolean());
int[] count = new int[1];
r.intersect(new IntersectVisitor() {
@ -1241,12 +1261,14 @@ public class TestBKD extends LuceneTestCase {
}
IndexOutput out = dir.createOutput("bkd", IOContext.DEFAULT);
long fp = w.finish(out);
Runnable finalizer = w.finish(out, out, out);
long fp = out.getFilePointer();
finalizer.run();
out.close();
IndexInput in = dir.openInput("bkd", IOContext.DEFAULT);
in.seek(fp);
BKDReader r = new BKDReader(in, randomBoolean());
BKDReader r = new BKDReader(in, in, in, randomBoolean());
int[] count = new int[1];
r.intersect(new IntersectVisitor() {
@ -1300,13 +1322,15 @@ public class TestBKD extends LuceneTestCase {
}
final long indexFP;
try (IndexOutput out = dir.createOutput("bkd", IOContext.DEFAULT)) {
indexFP = w.finish(out);
Runnable finalizer = w.finish(out, out, out);
indexFP = out.getFilePointer();
finalizer.run();
w.close();
}
IndexInput pointsIn = dir.openInput("bkd", IOContext.DEFAULT);
pointsIn.seek(indexFP);
BKDReader points = new BKDReader(pointsIn);
BKDReader points = new BKDReader(pointsIn, pointsIn, pointsIn);
// If all points match, then the point count is numLeaves * maxPointsInLeafNode
int numLeaves = numValues / maxPointsInLeafNode;
@ -1450,7 +1474,7 @@ public class TestBKD extends LuceneTestCase {
BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP, numValues);
expectThrows(IllegalStateException.class, () -> {
try (IndexOutput out = dir.createOutput("bkd", IOContext.DEFAULT)) {
w.writeField(out, "test_field_name", reader);
w.writeField(out, out, out, "test_field_name", reader);
} finally {
w.close();
dir.close();
@ -1561,7 +1585,7 @@ public class TestBKD extends LuceneTestCase {
}
};
try (IndexOutput out = dir.createOutput("bkd", IOContext.DEFAULT)) {
IllegalStateException ex = expectThrows(IllegalStateException.class, () -> { w.writeField(out, "", val);});
IllegalStateException ex = expectThrows(IllegalStateException.class, () -> { w.writeField(out, out, out, "", val);});
assertEquals("totalPointCount=10 was passed when we were created, but we just hit 11 values", ex.getMessage());
w.close();
}

View File

@ -20,7 +20,7 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.lucene.codecs.lucene60.Lucene60PointsFormat;
import org.apache.lucene.codecs.lucene86.Lucene86PointsFormat;
import org.apache.lucene.document.LatLonDocValuesField;
import org.apache.lucene.document.LatLonPoint;
import org.apache.lucene.geo.GeoUtils;
@ -49,7 +49,7 @@ public class LatLonPointPrototypeQueries {
* <p>
* This is functionally equivalent to running {@link MatchAllDocsQuery} with a {@link LatLonDocValuesField#newDistanceSort},
* but is far more efficient since it takes advantage of properties the indexed BKD tree. Currently this
* only works with {@link Lucene60PointsFormat} (used by the default codec). Multi-valued fields are
* only works with {@link Lucene86PointsFormat} (used by the default codec). Multi-valued fields are
* currently not de-duplicated, so if a document had multiple instances of the specified field that
* make it into the top n, that document will appear more than once.
* <p>

View File

@ -32,8 +32,8 @@ import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.PointsFormat;
import org.apache.lucene.codecs.PointsReader;
import org.apache.lucene.codecs.PointsWriter;
import org.apache.lucene.codecs.lucene60.Lucene60PointsReader;
import org.apache.lucene.codecs.lucene60.Lucene60PointsWriter;
import org.apache.lucene.codecs.lucene86.Lucene86PointsReader;
import org.apache.lucene.codecs.lucene86.Lucene86PointsWriter;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.NumericDocValuesField;
@ -104,12 +104,12 @@ public class TestGeo3DPoint extends LuceneTestCase {
return new PointsFormat() {
@Override
public PointsWriter fieldsWriter(SegmentWriteState writeState) throws IOException {
return new Lucene60PointsWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap);
return new Lucene86PointsWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap);
}
@Override
public PointsReader fieldsReader(SegmentReadState readState) throws IOException {
return new Lucene60PointsReader(readState);
return new Lucene86PointsReader(readState);
}
};
}

View File

@ -30,8 +30,8 @@ import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.PointsFormat;
import org.apache.lucene.codecs.PointsReader;
import org.apache.lucene.codecs.PointsWriter;
import org.apache.lucene.codecs.lucene60.Lucene60PointsReader;
import org.apache.lucene.codecs.lucene60.Lucene60PointsWriter;
import org.apache.lucene.codecs.lucene86.Lucene86PointsReader;
import org.apache.lucene.codecs.lucene86.Lucene86PointsWriter;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.NumericDocValuesField;
@ -1282,12 +1282,12 @@ public abstract class BaseGeoPointTestCase extends LuceneTestCase {
return new PointsFormat() {
@Override
public PointsWriter fieldsWriter(SegmentWriteState writeState) throws IOException {
return new Lucene60PointsWriter(writeState, pointsInLeaf, BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP);
return new Lucene86PointsWriter(writeState, pointsInLeaf, BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP);
}
@Override
public PointsReader fieldsReader(SegmentReadState readState) throws IOException {
return new Lucene60PointsReader(readState);
return new Lucene86PointsReader(readState);
}
};
}

View File

@ -30,8 +30,8 @@ import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.PointsFormat;
import org.apache.lucene.codecs.PointsReader;
import org.apache.lucene.codecs.PointsWriter;
import org.apache.lucene.codecs.lucene60.Lucene60PointsReader;
import org.apache.lucene.codecs.lucene60.Lucene60PointsWriter;
import org.apache.lucene.codecs.lucene86.Lucene86PointsReader;
import org.apache.lucene.codecs.lucene86.Lucene86PointsWriter;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.NumericDocValuesField;
@ -1196,12 +1196,12 @@ public abstract class BaseXYPointTestCase extends LuceneTestCase {
return new PointsFormat() {
@Override
public PointsWriter fieldsWriter(SegmentWriteState writeState) throws IOException {
return new Lucene60PointsWriter(writeState, pointsInLeaf, BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP);
return new Lucene86PointsWriter(writeState, pointsInLeaf, BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP);
}
@Override
public PointsReader fieldsReader(SegmentReadState readState) throws IOException {
return new Lucene60PointsReader(readState);
return new Lucene86PointsReader(readState);
}
};
}

View File

@ -40,8 +40,8 @@ import org.apache.lucene.codecs.blockterms.LuceneVarGapDocFreqInterval;
import org.apache.lucene.codecs.blockterms.LuceneVarGapFixedInterval;
import org.apache.lucene.codecs.blocktreeords.BlockTreeOrdsPostingsFormat;
import org.apache.lucene.codecs.bloom.TestBloomFilteredLucenePostings;
import org.apache.lucene.codecs.lucene60.Lucene60PointsReader;
import org.apache.lucene.codecs.lucene60.Lucene60PointsWriter;
import org.apache.lucene.codecs.lucene86.Lucene86PointsReader;
import org.apache.lucene.codecs.lucene86.Lucene86PointsWriter;
import org.apache.lucene.codecs.memory.DirectPostingsFormat;
import org.apache.lucene.codecs.memory.FSTPostingsFormat;
import org.apache.lucene.codecs.mockrandom.MockRandomPostingsFormat;
@ -97,9 +97,9 @@ public class RandomCodec extends AssertingCodec {
@Override
public PointsWriter fieldsWriter(SegmentWriteState writeState) throws IOException {
// Randomize how BKDWriter chooses its splis:
// Randomize how BKDWriter chooses its splits:
return new Lucene60PointsWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap) {
return new Lucene86PointsWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap) {
@Override
public void writeField(FieldInfo fieldInfo, PointsReader reader) throws IOException {
@ -132,8 +132,10 @@ public class RandomCodec extends AssertingCodec {
});
// We could have 0 points on merge since all docs with dimensional fields may be deleted:
if (writer.getPointCount() > 0) {
indexFPs.put(fieldInfo.name, writer.finish(dataOut));
Runnable finalizer = writer.finish(metaOut, indexOut, dataOut);
if (finalizer != null) {
metaOut.writeInt(fieldInfo.number);
finalizer.run();
}
}
}
@ -142,7 +144,7 @@ public class RandomCodec extends AssertingCodec {
@Override
public PointsReader fieldsReader(SegmentReadState readState) throws IOException {
return new Lucene60PointsReader(readState);
return new Lucene86PointsReader(readState);
}
});
}