mirror of https://github.com/apache/lucene.git
LUCENE-6975: rename dimensional values to points; add ExactPointQuery to match documents containing exactly an indexed point
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1725998 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
977980fcdc
commit
29cb337377
|
@ -39,25 +39,28 @@ New Features
|
|||
|
||||
* LUCENE-6825: Add low-level support for block-KD trees (Mike McCandless)
|
||||
|
||||
* LUCENE-6852: Add support for dimensionally indexed values to index,
|
||||
document and codec APIs, including a simple text implementation.
|
||||
(Mike McCandless)
|
||||
* LUCENE-6852, LUCENE-6975: Add support for points (dimensionally
|
||||
indexed values) to index, document and codec APIs, including a
|
||||
simple text implementation. (Mike McCandless)
|
||||
|
||||
* LUCENE-6861: Create Lucene60Codec, supporting dimensional values.
|
||||
* LUCENE-6861: Create Lucene60Codec, supporting points.
|
||||
(Mike McCandless)
|
||||
|
||||
* LUCENE-6879: Allow to define custom CharTokenizer instances without
|
||||
subclassing using Java 8 lambdas or method references. (Uwe Schindler)
|
||||
|
||||
* LUCENE-6881: Cutover all BKD implementations to dimensional values
|
||||
* LUCENE-6881: Cutover all BKD implementations to points
|
||||
(Mike McCandless)
|
||||
|
||||
* LUCENE-6837: Add N-best output support to JapaneseTokenizer.
|
||||
(Hiroharu Konno via Christian Moen)
|
||||
|
||||
* LUCENE-6962: Add per-dimension min/max to dimensional values
|
||||
* LUCENE-6962: Add per-dimension min/max to points
|
||||
(Mike McCandless)
|
||||
|
||||
* LUCENE-6975: Add ExactPointQuery, to match a single N-dimensional
|
||||
point (Robert Muir, Mike McCandless)
|
||||
|
||||
API Changes
|
||||
|
||||
* LUCENE-6067: Accountable.getChildResources has a default
|
||||
|
@ -82,18 +85,18 @@ API Changes
|
|||
McCandless)
|
||||
|
||||
* LUCENE-6917: Deprecate and rename NumericXXX classes to
|
||||
LegacyNumericXXX in favor of dimensional values (Mike McCandless)
|
||||
LegacyNumericXXX in favor of points (Mike McCandless)
|
||||
|
||||
* LUCENE-6947: SortField.missingValue is now protected. You can read its value
|
||||
using the new SortField.getMissingValue getter. (Adrien Grand)
|
||||
|
||||
Optimizations
|
||||
|
||||
* LUCENE-6891: Use prefix coding when writing dimensional values in
|
||||
* LUCENE-6891: Use prefix coding when writing points in
|
||||
each leaf block in the default codec, to reduce the index
|
||||
size (Mike McCandless)
|
||||
|
||||
* LUCENE-6901: Optimize dimensional values indexing: use faster
|
||||
* LUCENE-6901: Optimize points indexing: use faster
|
||||
IntroSorter instead of InPlaceMergeSorter, and specialize 1D
|
||||
merging to merge sort the already sorted segments instead of
|
||||
re-indexing (Mike McCandless)
|
||||
|
|
|
@ -21,7 +21,7 @@ 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.PointFormat;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.FieldInfosFormat;
|
||||
import org.apache.lucene.codecs.FilterCodec;
|
||||
|
@ -154,8 +154,8 @@ public class Lucene50Codec extends Codec {
|
|||
}
|
||||
|
||||
@Override
|
||||
public final DimensionalFormat dimensionalFormat() {
|
||||
return DimensionalFormat.EMPTY;
|
||||
public final PointFormat pointFormat() {
|
||||
return PointFormat.EMPTY;
|
||||
}
|
||||
|
||||
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene50");
|
||||
|
|
|
@ -21,7 +21,7 @@ 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.PointFormat;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.FieldInfosFormat;
|
||||
import org.apache.lucene.codecs.FilterCodec;
|
||||
|
@ -160,8 +160,8 @@ public class Lucene53Codec extends Codec {
|
|||
}
|
||||
|
||||
@Override
|
||||
public final DimensionalFormat dimensionalFormat() {
|
||||
return DimensionalFormat.EMPTY;
|
||||
public final PointFormat pointFormat() {
|
||||
return PointFormat.EMPTY;
|
||||
}
|
||||
|
||||
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene50");
|
||||
|
|
|
@ -21,7 +21,7 @@ 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.PointFormat;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.FieldInfosFormat;
|
||||
import org.apache.lucene.codecs.FilterCodec;
|
||||
|
@ -160,8 +160,8 @@ public class Lucene54Codec extends Codec {
|
|||
}
|
||||
|
||||
@Override
|
||||
public final DimensionalFormat dimensionalFormat() {
|
||||
return DimensionalFormat.EMPTY;
|
||||
public final PointFormat pointFormat() {
|
||||
return PointFormat.EMPTY;
|
||||
}
|
||||
|
||||
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene50");
|
||||
|
|
|
@ -20,16 +20,16 @@ package org.apache.lucene.codecs.simpletext;
|
|||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
|
||||
import org.apache.lucene.index.DimensionalValues.IntersectVisitor;
|
||||
import org.apache.lucene.index.PointValues.IntersectVisitor;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefBuilder;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
import org.apache.lucene.util.bkd.BKDReader;
|
||||
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextDimensionalWriter.BLOCK_COUNT;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextDimensionalWriter.BLOCK_DOC_ID;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextDimensionalWriter.BLOCK_VALUE;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.BLOCK_COUNT;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.BLOCK_DOC_ID;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.BLOCK_VALUE;
|
||||
|
||||
class SimpleTextBKDReader extends BKDReader {
|
||||
|
||||
|
|
|
@ -19,7 +19,7 @@ package org.apache.lucene.codecs.simpletext;
|
|||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.CompoundFormat;
|
||||
import org.apache.lucene.codecs.DimensionalFormat;
|
||||
import org.apache.lucene.codecs.PointFormat;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.FieldInfosFormat;
|
||||
import org.apache.lucene.codecs.LiveDocsFormat;
|
||||
|
@ -45,7 +45,7 @@ public final class SimpleTextCodec extends Codec {
|
|||
private final LiveDocsFormat liveDocs = new SimpleTextLiveDocsFormat();
|
||||
private final DocValuesFormat dvFormat = new SimpleTextDocValuesFormat();
|
||||
private final CompoundFormat compoundFormat = new SimpleTextCompoundFormat();
|
||||
private final DimensionalFormat dimensionalFormat = new SimpleTextDimensionalFormat();
|
||||
private final PointFormat pointFormat = new SimpleTextPointFormat();
|
||||
|
||||
public SimpleTextCodec() {
|
||||
super("SimpleText");
|
||||
|
@ -97,7 +97,7 @@ public final class SimpleTextCodec extends Codec {
|
|||
}
|
||||
|
||||
@Override
|
||||
public DimensionalFormat dimensionalFormat() {
|
||||
return dimensionalFormat;
|
||||
public PointFormat pointFormat() {
|
||||
return pointFormat;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -232,11 +232,11 @@ public class SimpleTextFieldInfosFormat extends FieldInfosFormat {
|
|||
}
|
||||
|
||||
SimpleTextUtil.write(out, DIM_COUNT);
|
||||
SimpleTextUtil.write(out, Integer.toString(fi.getDimensionCount()), scratch);
|
||||
SimpleTextUtil.write(out, Integer.toString(fi.getPointDimensionCount()), scratch);
|
||||
SimpleTextUtil.writeNewline(out);
|
||||
|
||||
SimpleTextUtil.write(out, DIM_NUM_BYTES);
|
||||
SimpleTextUtil.write(out, Integer.toString(fi.getDimensionNumBytes()), scratch);
|
||||
SimpleTextUtil.write(out, Integer.toString(fi.getPointNumBytes()), scratch);
|
||||
SimpleTextUtil.writeNewline(out);
|
||||
}
|
||||
SimpleTextUtil.writeChecksum(out, scratch);
|
||||
|
|
|
@ -19,9 +19,9 @@ package org.apache.lucene.codecs.simpletext;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.codecs.DimensionalFormat;
|
||||
import org.apache.lucene.codecs.DimensionalReader;
|
||||
import org.apache.lucene.codecs.DimensionalWriter;
|
||||
import org.apache.lucene.codecs.PointFormat;
|
||||
import org.apache.lucene.codecs.PointReader;
|
||||
import org.apache.lucene.codecs.PointWriter;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
|
||||
|
@ -33,21 +33,21 @@ import org.apache.lucene.index.SegmentWriteState;
|
|||
* any text editor, and even edit it to alter your index.
|
||||
*
|
||||
* @lucene.experimental */
|
||||
public final class SimpleTextDimensionalFormat extends DimensionalFormat {
|
||||
public final class SimpleTextPointFormat extends PointFormat {
|
||||
|
||||
@Override
|
||||
public DimensionalWriter fieldsWriter(SegmentWriteState state) throws IOException {
|
||||
return new SimpleTextDimensionalWriter(state);
|
||||
public PointWriter fieldsWriter(SegmentWriteState state) throws IOException {
|
||||
return new SimpleTextPointWriter(state);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DimensionalReader fieldsReader(SegmentReadState state) throws IOException {
|
||||
return new SimpleTextDimensionalReader(state);
|
||||
public PointReader fieldsReader(SegmentReadState state) throws IOException {
|
||||
return new SimpleTextPointReader(state);
|
||||
}
|
||||
|
||||
/** Extension of dimensional data file */
|
||||
static final String DIMENSIONAL_EXTENSION = "dim";
|
||||
/** Extension of points data file */
|
||||
static final String POINT_EXTENSION = "dim";
|
||||
|
||||
/** Extension of dimensional index file */
|
||||
static final String DIMENSIONAL_INDEX_EXTENSION = "dii";
|
||||
/** Extension of points index file */
|
||||
static final String POINT_INDEX_EXTENSION = "dii";
|
||||
}
|
|
@ -22,7 +22,7 @@ import java.nio.charset.StandardCharsets;
|
|||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.codecs.DimensionalReader;
|
||||
import org.apache.lucene.codecs.PointReader;
|
||||
import org.apache.lucene.index.CorruptIndexException;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
|
@ -36,32 +36,32 @@ import org.apache.lucene.util.BytesRefBuilder;
|
|||
import org.apache.lucene.util.StringHelper;
|
||||
import org.apache.lucene.util.bkd.BKDReader;
|
||||
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextDimensionalWriter.BLOCK_FP;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextDimensionalWriter.BYTES_PER_DIM;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextDimensionalWriter.FIELD_COUNT;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextDimensionalWriter.FIELD_FP;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextDimensionalWriter.FIELD_FP_NAME;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextDimensionalWriter.INDEX_COUNT;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextDimensionalWriter.MAX_LEAF_POINTS;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextDimensionalWriter.MAX_VALUE;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextDimensionalWriter.MIN_VALUE;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextDimensionalWriter.NUM_DIMS;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextDimensionalWriter.SPLIT_COUNT;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextDimensionalWriter.SPLIT_DIM;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextDimensionalWriter.SPLIT_VALUE;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.BLOCK_FP;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.BYTES_PER_DIM;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.FIELD_COUNT;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.FIELD_FP;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.FIELD_FP_NAME;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.INDEX_COUNT;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.MAX_LEAF_POINTS;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.MAX_VALUE;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.MIN_VALUE;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.NUM_DIMS;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.SPLIT_COUNT;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.SPLIT_DIM;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.SPLIT_VALUE;
|
||||
|
||||
class SimpleTextDimensionalReader extends DimensionalReader {
|
||||
class SimpleTextPointReader extends PointReader {
|
||||
|
||||
private final IndexInput dataIn;
|
||||
final SegmentReadState readState;
|
||||
final Map<String,BKDReader> readers = new HashMap<>();
|
||||
final BytesRefBuilder scratch = new BytesRefBuilder();
|
||||
|
||||
public SimpleTextDimensionalReader(SegmentReadState readState) throws IOException {
|
||||
public SimpleTextPointReader(SegmentReadState readState) throws IOException {
|
||||
// Initialize readers now:
|
||||
String fileName = IndexFileNames.segmentFileName(readState.segmentInfo.name, readState.segmentSuffix, SimpleTextDimensionalFormat.DIMENSIONAL_EXTENSION);
|
||||
String fileName = IndexFileNames.segmentFileName(readState.segmentInfo.name, readState.segmentSuffix, SimpleTextPointFormat.POINT_EXTENSION);
|
||||
dataIn = readState.directory.openInput(fileName, IOContext.DEFAULT);
|
||||
String indexFileName = IndexFileNames.segmentFileName(readState.segmentInfo.name, readState.segmentSuffix, SimpleTextDimensionalFormat.DIMENSIONAL_INDEX_EXTENSION);
|
||||
String indexFileName = IndexFileNames.segmentFileName(readState.segmentInfo.name, readState.segmentSuffix, SimpleTextPointFormat.POINT_INDEX_EXTENSION);
|
||||
try (ChecksumIndexInput in = readState.directory.openChecksumInput(indexFileName, IOContext.DEFAULT)) {
|
||||
readLine(in);
|
||||
int count = parseInt(FIELD_COUNT);
|
||||
|
@ -78,7 +78,7 @@ class SimpleTextDimensionalReader extends DimensionalReader {
|
|||
}
|
||||
|
||||
private BKDReader initReader(long fp) throws IOException {
|
||||
// NOTE: matches what writeIndex does in SimpleTextDimensionalWriter
|
||||
// NOTE: matches what writeIndex does in SimpleTextPointWriter
|
||||
dataIn.seek(fp);
|
||||
readLine(dataIn);
|
||||
int numDims = parseInt(NUM_DIMS);
|
||||
|
@ -151,8 +151,8 @@ class SimpleTextDimensionalReader extends DimensionalReader {
|
|||
if (fieldInfo == null) {
|
||||
throw new IllegalArgumentException("field=\"" + fieldName + "\" is unrecognized");
|
||||
}
|
||||
if (fieldInfo.getDimensionCount() == 0) {
|
||||
throw new IllegalArgumentException("field=\"" + fieldName + "\" did not index dimensional values");
|
||||
if (fieldInfo.getPointDimensionCount() == 0) {
|
||||
throw new IllegalArgumentException("field=\"" + fieldName + "\" did not index points");
|
||||
}
|
||||
return readers.get(fieldName);
|
||||
}
|
||||
|
@ -162,8 +162,8 @@ class SimpleTextDimensionalReader extends DimensionalReader {
|
|||
public void intersect(String fieldName, IntersectVisitor visitor) throws IOException {
|
||||
BKDReader bkdReader = getBKDReader(fieldName);
|
||||
if (bkdReader == null) {
|
||||
// Schema ghost corner case! This field did index dimensional values in the past, but
|
||||
// now all docs having this dimensional field were deleted in this segment:
|
||||
// Schema ghost corner case! This field did index points in the past, but
|
||||
// now all docs having this field were deleted in this segment:
|
||||
return;
|
||||
}
|
||||
bkdReader.intersect(visitor);
|
||||
|
@ -203,15 +203,15 @@ class SimpleTextDimensionalReader extends DimensionalReader {
|
|||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "SimpleTextDimensionalReader(segment=" + readState.segmentInfo.name + " maxDoc=" + readState.segmentInfo.maxDoc() + ")";
|
||||
return "SimpleTextPointReader(segment=" + readState.segmentInfo.name + " maxDoc=" + readState.segmentInfo.maxDoc() + ")";
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getMinPackedValue(String fieldName) {
|
||||
BKDReader bkdReader = getBKDReader(fieldName);
|
||||
if (bkdReader == null) {
|
||||
// Schema ghost corner case! This field did index dimensional values in the past, but
|
||||
// now all docs having this dimensional field were deleted in this segment:
|
||||
// Schema ghost corner case! This field did index points in the past, but
|
||||
// now all docs having this field were deleted in this segment:
|
||||
return null;
|
||||
}
|
||||
return bkdReader.getMinPackedValue();
|
||||
|
@ -221,8 +221,8 @@ class SimpleTextDimensionalReader extends DimensionalReader {
|
|||
public byte[] getMaxPackedValue(String fieldName) {
|
||||
BKDReader bkdReader = getBKDReader(fieldName);
|
||||
if (bkdReader == null) {
|
||||
// Schema ghost corner case! This field did index dimensional values in the past, but
|
||||
// now all docs having this dimensional field were deleted in this segment:
|
||||
// Schema ghost corner case! This field did index points in the past, but
|
||||
// now all docs having this field were deleted in this segment:
|
||||
return null;
|
||||
}
|
||||
return bkdReader.getMaxPackedValue();
|
||||
|
@ -232,8 +232,8 @@ class SimpleTextDimensionalReader extends DimensionalReader {
|
|||
public int getNumDimensions(String fieldName) {
|
||||
BKDReader bkdReader = getBKDReader(fieldName);
|
||||
if (bkdReader == null) {
|
||||
// Schema ghost corner case! This field did index dimensional values in the past, but
|
||||
// now all docs having this dimensional field were deleted in this segment:
|
||||
// Schema ghost corner case! This field did index points in the past, but
|
||||
// now all docs having this field were deleted in this segment:
|
||||
return 0;
|
||||
}
|
||||
return bkdReader.getNumDimensions();
|
||||
|
@ -243,8 +243,8 @@ class SimpleTextDimensionalReader extends DimensionalReader {
|
|||
public int getBytesPerDimension(String fieldName) {
|
||||
BKDReader bkdReader = getBKDReader(fieldName);
|
||||
if (bkdReader == null) {
|
||||
// Schema ghost corner case! This field did index dimensional values in the past, but
|
||||
// now all docs having this dimensional field were deleted in this segment:
|
||||
// Schema ghost corner case! This field did index points in the past, but
|
||||
// now all docs having this field were deleted in this segment:
|
||||
return 0;
|
||||
}
|
||||
return bkdReader.getBytesPerDimension();
|
|
@ -21,10 +21,10 @@ import java.io.IOException;
|
|||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
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.codecs.PointReader;
|
||||
import org.apache.lucene.codecs.PointWriter;
|
||||
import org.apache.lucene.index.PointValues.IntersectVisitor;
|
||||
import org.apache.lucene.index.PointValues.Relation;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
|
@ -33,7 +33,7 @@ import org.apache.lucene.util.BytesRef;
|
|||
import org.apache.lucene.util.BytesRefBuilder;
|
||||
import org.apache.lucene.util.bkd.BKDWriter;
|
||||
|
||||
class SimpleTextDimensionalWriter extends DimensionalWriter {
|
||||
class SimpleTextPointWriter extends PointWriter {
|
||||
|
||||
final static BytesRef NUM_DIMS = new BytesRef("num dims ");
|
||||
final static BytesRef BYTES_PER_DIM = new BytesRef("bytes per dim ");
|
||||
|
@ -57,20 +57,20 @@ class SimpleTextDimensionalWriter extends DimensionalWriter {
|
|||
final SegmentWriteState writeState;
|
||||
final Map<String,Long> indexFPs = new HashMap<>();
|
||||
|
||||
public SimpleTextDimensionalWriter(SegmentWriteState writeState) throws IOException {
|
||||
String fileName = IndexFileNames.segmentFileName(writeState.segmentInfo.name, writeState.segmentSuffix, SimpleTextDimensionalFormat.DIMENSIONAL_EXTENSION);
|
||||
public SimpleTextPointWriter(SegmentWriteState writeState) throws IOException {
|
||||
String fileName = IndexFileNames.segmentFileName(writeState.segmentInfo.name, writeState.segmentSuffix, SimpleTextPointFormat.POINT_EXTENSION);
|
||||
dataOut = writeState.directory.createOutput(fileName, writeState.context);
|
||||
this.writeState = writeState;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeField(FieldInfo fieldInfo, DimensionalReader values) throws IOException {
|
||||
public void writeField(FieldInfo fieldInfo, PointReader values) throws IOException {
|
||||
|
||||
// We use the normal BKDWriter, but subclass to customize how it writes the index and blocks to disk:
|
||||
BKDWriter writer = new BKDWriter(writeState.directory,
|
||||
writeState.segmentInfo.name,
|
||||
fieldInfo.getDimensionCount(),
|
||||
fieldInfo.getDimensionNumBytes(),
|
||||
fieldInfo.getPointDimensionCount(),
|
||||
fieldInfo.getPointNumBytes(),
|
||||
BKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE,
|
||||
BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP) {
|
||||
|
||||
|
@ -108,8 +108,8 @@ class SimpleTextDimensionalWriter extends DimensionalWriter {
|
|||
newline(out);
|
||||
}
|
||||
|
||||
assert (splitPackedValues.length % (1 + fieldInfo.getDimensionNumBytes())) == 0;
|
||||
int count = splitPackedValues.length / (1 + fieldInfo.getDimensionNumBytes());
|
||||
assert (splitPackedValues.length % (1 + fieldInfo.getPointNumBytes())) == 0;
|
||||
int count = splitPackedValues.length / (1 + fieldInfo.getPointNumBytes());
|
||||
assert count == leafBlockFPs.length;
|
||||
|
||||
write(out, SPLIT_COUNT);
|
||||
|
@ -118,10 +118,10 @@ class SimpleTextDimensionalWriter extends DimensionalWriter {
|
|||
|
||||
for(int i=0;i<count;i++) {
|
||||
write(out, SPLIT_DIM);
|
||||
writeInt(out, splitPackedValues[i * (1 + fieldInfo.getDimensionNumBytes())] & 0xff);
|
||||
writeInt(out, splitPackedValues[i * (1 + fieldInfo.getPointNumBytes())] & 0xff);
|
||||
newline(out);
|
||||
write(out, SPLIT_VALUE);
|
||||
br = new BytesRef(splitPackedValues, 1+(i * (1+fieldInfo.getDimensionNumBytes())), fieldInfo.getDimensionNumBytes());
|
||||
br = new BytesRef(splitPackedValues, 1+(i * (1+fieldInfo.getPointNumBytes())), fieldInfo.getPointNumBytes());
|
||||
write(out, br.toString());
|
||||
newline(out);
|
||||
}
|
||||
|
@ -170,7 +170,7 @@ class SimpleTextDimensionalWriter extends DimensionalWriter {
|
|||
}
|
||||
});
|
||||
|
||||
// We could have 0 points on merge since all docs with dimensional fields may be deleted:
|
||||
// We could have 0 points on merge since all docs with points may be deleted:
|
||||
if (writer.getPointCount() > 0) {
|
||||
indexFPs.put(fieldInfo.name, writer.finish(dataOut));
|
||||
}
|
||||
|
@ -204,7 +204,7 @@ class SimpleTextDimensionalWriter extends DimensionalWriter {
|
|||
dataOut = null;
|
||||
|
||||
// Write index file
|
||||
String fileName = IndexFileNames.segmentFileName(writeState.segmentInfo.name, writeState.segmentSuffix, SimpleTextDimensionalFormat.DIMENSIONAL_INDEX_EXTENSION);
|
||||
String fileName = IndexFileNames.segmentFileName(writeState.segmentInfo.name, writeState.segmentSuffix, SimpleTextPointFormat.POINT_INDEX_EXTENSION);
|
||||
try (IndexOutput indexOut = writeState.directory.createOutput(fileName, writeState.context)) {
|
||||
int count = indexFPs.size();
|
||||
write(indexOut, FIELD_COUNT);
|
|
@ -23,7 +23,6 @@ import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
|||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
|
||||
import org.apache.lucene.index.DimensionalValues;
|
||||
import org.apache.lucene.util.Attribute;
|
||||
import org.apache.lucene.util.AttributeFactory;
|
||||
import org.apache.lucene.util.AttributeImpl;
|
||||
|
@ -86,7 +85,7 @@ import org.apache.lucene.util.LegacyNumericUtils;
|
|||
* href="../search/LegacyNumericRangeQuery.html#precisionStepDesc"><code>precisionStep</code></a>
|
||||
* parameter as well as how numeric fields work under the hood.</p>
|
||||
*
|
||||
* @deprecated Please switch to {@link DimensionalValues} instead
|
||||
* @deprecated Please switch to {@link org.apache.lucene.index.PointValues} instead
|
||||
*
|
||||
* @since 2.9
|
||||
*/
|
||||
|
|
|
@ -108,8 +108,8 @@ public abstract class Codec implements NamedSPILoader.NamedSPI {
|
|||
/** Encodes/decodes compound files */
|
||||
public abstract CompoundFormat compoundFormat();
|
||||
|
||||
/** Encodes/decodes dimensional index */
|
||||
public abstract DimensionalFormat dimensionalFormat();
|
||||
/** Encodes/decodes points index */
|
||||
public abstract PointFormat pointFormat();
|
||||
|
||||
/** looks up a codec by name */
|
||||
public static Codec forName(String name) {
|
||||
|
|
|
@ -105,7 +105,7 @@ public abstract class FilterCodec extends Codec {
|
|||
}
|
||||
|
||||
@Override
|
||||
public DimensionalFormat dimensionalFormat() {
|
||||
return delegate.dimensionalFormat();
|
||||
public PointFormat pointFormat() {
|
||||
return delegate.pointFormat();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,19 +23,19 @@ import org.apache.lucene.index.SegmentReadState;
|
|||
import org.apache.lucene.index.SegmentWriteState;
|
||||
|
||||
/**
|
||||
* Encodes/decodes indexed dimensional data.
|
||||
* Encodes/decodes indexed points.
|
||||
*
|
||||
* @lucene.experimental */
|
||||
public abstract class DimensionalFormat {
|
||||
public abstract class PointFormat {
|
||||
|
||||
/**
|
||||
* Creates a new dimensional format.
|
||||
* Creates a new point format.
|
||||
*/
|
||||
protected DimensionalFormat() {
|
||||
protected PointFormat() {
|
||||
}
|
||||
|
||||
/** Writes a new segment */
|
||||
public abstract DimensionalWriter fieldsWriter(SegmentWriteState state) throws IOException;
|
||||
public abstract PointWriter fieldsWriter(SegmentWriteState state) throws IOException;
|
||||
|
||||
/** Reads a segment. NOTE: by the time this call
|
||||
* returns, it must hold open any files it will need to
|
||||
|
@ -46,18 +46,18 @@ public abstract class DimensionalFormat {
|
|||
* IOExceptions are expected and will automatically cause a retry of the
|
||||
* segment opening logic with the newly revised segments.
|
||||
* */
|
||||
public abstract DimensionalReader fieldsReader(SegmentReadState state) throws IOException;
|
||||
public abstract PointReader fieldsReader(SegmentReadState state) throws IOException;
|
||||
|
||||
/** A {@code DimensionalFormat} that has nothing indexed */
|
||||
public static final DimensionalFormat EMPTY = new DimensionalFormat() {
|
||||
/** A {@code PointFormat} that has nothing indexed */
|
||||
public static final PointFormat EMPTY = new PointFormat() {
|
||||
@Override
|
||||
public DimensionalWriter fieldsWriter(SegmentWriteState state) {
|
||||
public PointWriter fieldsWriter(SegmentWriteState state) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DimensionalReader fieldsReader(SegmentReadState state) {
|
||||
return new DimensionalReader() {
|
||||
public PointReader fieldsReader(SegmentReadState state) {
|
||||
return new PointReader() {
|
||||
@Override
|
||||
public void close() {
|
||||
}
|
||||
|
@ -73,27 +73,27 @@ public abstract class DimensionalFormat {
|
|||
|
||||
@Override
|
||||
public void intersect(String fieldName, IntersectVisitor visitor) {
|
||||
throw new IllegalArgumentException("field=\"" + fieldName + "\" was not indexed with dimensional values");
|
||||
throw new IllegalArgumentException("field=\"" + fieldName + "\" was not indexed with points");
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getMinPackedValue(String fieldName) {
|
||||
throw new IllegalArgumentException("field=\"" + fieldName + "\" was not indexed with dimensional values");
|
||||
throw new IllegalArgumentException("field=\"" + fieldName + "\" was not indexed with points");
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getMaxPackedValue(String fieldName) {
|
||||
throw new IllegalArgumentException("field=\"" + fieldName + "\" was not indexed with dimensional values");
|
||||
throw new IllegalArgumentException("field=\"" + fieldName + "\" was not indexed with points");
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getNumDimensions(String fieldName) {
|
||||
throw new IllegalArgumentException("field=\"" + fieldName + "\" was not indexed with dimensional values");
|
||||
throw new IllegalArgumentException("field=\"" + fieldName + "\" was not indexed with points");
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getBytesPerDimension(String fieldName) {
|
||||
throw new IllegalArgumentException("field=\"" + fieldName + "\" was not indexed with dimensional values");
|
||||
throw new IllegalArgumentException("field=\"" + fieldName + "\" was not indexed with points");
|
||||
}
|
||||
};
|
||||
}
|
|
@ -20,17 +20,17 @@ package org.apache.lucene.codecs;
|
|||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.DimensionalValues;
|
||||
import org.apache.lucene.index.PointValues;
|
||||
import org.apache.lucene.util.Accountable;
|
||||
|
||||
/** Abstract API to visit dimensional values.
|
||||
/** Abstract API to visit point values.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public abstract class DimensionalReader extends DimensionalValues implements Closeable, Accountable {
|
||||
public abstract class PointReader extends PointValues implements Closeable, Accountable {
|
||||
|
||||
/** Sole constructor. (For invocation by subclass constructors, typically implicit.) */
|
||||
protected DimensionalReader() {}
|
||||
protected PointReader() {}
|
||||
|
||||
/**
|
||||
* Checks consistency of this reader.
|
||||
|
@ -45,7 +45,7 @@ public abstract class DimensionalReader extends DimensionalValues implements Clo
|
|||
* Returns an instance optimized for merging.
|
||||
* <p>
|
||||
* The default implementation returns {@code this} */
|
||||
public DimensionalReader getMergeInstance() throws IOException {
|
||||
public PointReader getMergeInstance() throws IOException {
|
||||
return this;
|
||||
}
|
||||
}
|
|
@ -23,40 +23,40 @@ import java.io.IOException;
|
|||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.MergeState;
|
||||
|
||||
/** Abstract API to write dimensional values
|
||||
/** Abstract API to write points
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
|
||||
public abstract class DimensionalWriter implements Closeable {
|
||||
public abstract class PointWriter implements Closeable {
|
||||
/** Sole constructor. (For invocation by subclass
|
||||
* constructors, typically implicit.) */
|
||||
protected DimensionalWriter() {
|
||||
protected PointWriter() {
|
||||
}
|
||||
|
||||
/** Write all values contained in the provided reader */
|
||||
public abstract void writeField(FieldInfo fieldInfo, DimensionalReader values) throws IOException;
|
||||
public abstract void writeField(FieldInfo fieldInfo, PointReader values) throws IOException;
|
||||
|
||||
/** Default naive merge implemenation for one field: it just re-indexes all the values
|
||||
* from the incoming segment. The default codec overrides this for 1D fields and uses
|
||||
* a faster but more complex implementation. */
|
||||
protected void mergeOneField(MergeState mergeState, FieldInfo fieldInfo) throws IOException {
|
||||
writeField(fieldInfo,
|
||||
new DimensionalReader() {
|
||||
new PointReader() {
|
||||
@Override
|
||||
public void intersect(String fieldName, IntersectVisitor mergedVisitor) throws IOException {
|
||||
if (fieldName.equals(fieldInfo.name) == false) {
|
||||
throw new IllegalArgumentException("field name must match the field being merged");
|
||||
}
|
||||
for (int i=0;i<mergeState.dimensionalReaders.length;i++) {
|
||||
DimensionalReader dimensionalReader = mergeState.dimensionalReaders[i];
|
||||
if (dimensionalReader == null) {
|
||||
// This segment has no dimensional values
|
||||
for (int i=0;i<mergeState.pointReaders.length;i++) {
|
||||
PointReader pointReader = mergeState.pointReaders[i];
|
||||
if (pointReader == null) {
|
||||
// This segment has no points
|
||||
continue;
|
||||
}
|
||||
MergeState.DocMap docMap = mergeState.docMaps[i];
|
||||
int docBase = mergeState.docBase[i];
|
||||
dimensionalReader.intersect(fieldInfo.name,
|
||||
pointReader.intersect(fieldInfo.name,
|
||||
new IntersectVisitor() {
|
||||
@Override
|
||||
public void visit(int docID) {
|
||||
|
@ -75,7 +75,7 @@ public abstract class DimensionalWriter implements Closeable {
|
|||
|
||||
@Override
|
||||
public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
|
||||
// Forces this segment's DimensionalReader to always visit all docs + values:
|
||||
// Forces this segment's PointReader to always visit all docs + values:
|
||||
return Relation.CELL_CROSSES_QUERY;
|
||||
}
|
||||
});
|
||||
|
@ -118,11 +118,11 @@ public abstract class DimensionalWriter implements Closeable {
|
|||
});
|
||||
}
|
||||
|
||||
/** Default merge implementation to merge incoming dimensional readers by visiting all their points and
|
||||
/** Default merge implementation to merge incoming points readers by visiting all their points and
|
||||
* adding to this writer */
|
||||
public void merge(MergeState mergeState) throws IOException {
|
||||
for (FieldInfo fieldInfo : mergeState.mergeFieldInfos) {
|
||||
if (fieldInfo.getDimensionCount() != 0) {
|
||||
if (fieldInfo.getPointDimensionCount() != 0) {
|
||||
mergeOneField(mergeState, fieldInfo);
|
||||
}
|
||||
}
|
|
@ -21,7 +21,7 @@ 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.PointFormat;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.FieldInfosFormat;
|
||||
import org.apache.lucene.codecs.FilterCodec;
|
||||
|
@ -129,8 +129,8 @@ public class Lucene60Codec extends Codec {
|
|||
}
|
||||
|
||||
@Override
|
||||
public final DimensionalFormat dimensionalFormat() {
|
||||
return new Lucene60DimensionalFormat();
|
||||
public final PointFormat pointFormat() {
|
||||
return new Lucene60PointFormat();
|
||||
}
|
||||
|
||||
/** Returns the postings format that should be used for writing
|
||||
|
|
|
@ -24,7 +24,6 @@ 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.DimensionalLongField;
|
||||
import org.apache.lucene.index.CorruptIndexException;
|
||||
import org.apache.lucene.index.DocValuesType;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
|
@ -96,8 +95,8 @@ import org.apache.lucene.store.IndexOutput;
|
|||
* 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, e.g. using {@link DimensionalLongField}</li>
|
||||
* <li>PointDimensionCount, PointNumBytes: these are non-zero only if the field is
|
||||
* indexed as points, e.g. using {@link org.apache.lucene.document.LongPoint}</li>
|
||||
* </ul>
|
||||
*
|
||||
* @lucene.experimental
|
||||
|
@ -149,18 +148,18 @@ public final class Lucene60FieldInfosFormat extends FieldInfosFormat {
|
|||
attributes = lastAttributes;
|
||||
}
|
||||
lastAttributes = attributes;
|
||||
int dimensionCount = input.readVInt();
|
||||
int dimensionNumBytes;
|
||||
if (dimensionCount != 0) {
|
||||
dimensionNumBytes = input.readVInt();
|
||||
int pointDimensionCount = input.readVInt();
|
||||
int pointNumBytes;
|
||||
if (pointDimensionCount != 0) {
|
||||
pointNumBytes = input.readVInt();
|
||||
} else {
|
||||
dimensionNumBytes = 0;
|
||||
pointNumBytes = 0;
|
||||
}
|
||||
|
||||
try {
|
||||
infos[i] = new FieldInfo(name, fieldNumber, storeTermVector, omitNorms, storePayloads,
|
||||
indexOptions, docValuesType, dvGen, attributes,
|
||||
dimensionCount, dimensionNumBytes);
|
||||
pointDimensionCount, pointNumBytes);
|
||||
infos[i].checkConsistency();
|
||||
} catch (IllegalStateException e) {
|
||||
throw new CorruptIndexException("invalid fieldinfo for field: " + name + ", fieldNumber=" + fieldNumber, input, e);
|
||||
|
@ -286,10 +285,10 @@ public final class Lucene60FieldInfosFormat extends FieldInfosFormat {
|
|||
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());
|
||||
int pointDimensionCount = fi.getPointDimensionCount();
|
||||
output.writeVInt(pointDimensionCount);
|
||||
if (pointDimensionCount != 0) {
|
||||
output.writeVInt(fi.getPointNumBytes());
|
||||
}
|
||||
}
|
||||
CodecUtil.writeFooter(output);
|
||||
|
|
|
@ -20,14 +20,14 @@ package org.apache.lucene.codecs.lucene60;
|
|||
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.codecs.PointFormat;
|
||||
import org.apache.lucene.codecs.PointReader;
|
||||
import org.apache.lucene.codecs.PointWriter;
|
||||
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
|
||||
* Lucene 6.0 point 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
|
||||
|
@ -71,9 +71,9 @@ import org.apache.lucene.index.SegmentWriteState;
|
|||
* @lucene.experimental
|
||||
*/
|
||||
|
||||
public final class Lucene60DimensionalFormat extends DimensionalFormat {
|
||||
public final class Lucene60PointFormat extends PointFormat {
|
||||
|
||||
static final String CODEC_NAME = "Lucene60DimensionalFormat";
|
||||
static final String CODEC_NAME = "Lucene60PointFormat";
|
||||
|
||||
/**
|
||||
* Filename extension for the leaf blocks
|
||||
|
@ -92,16 +92,16 @@ public final class Lucene60DimensionalFormat extends DimensionalFormat {
|
|||
static final int INDEX_VERSION_CURRENT = INDEX_VERSION_START;
|
||||
|
||||
/** Sole constructor */
|
||||
public Lucene60DimensionalFormat() {
|
||||
public Lucene60PointFormat() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public DimensionalWriter fieldsWriter(SegmentWriteState state) throws IOException {
|
||||
return new Lucene60DimensionalWriter(state);
|
||||
public PointWriter fieldsWriter(SegmentWriteState state) throws IOException {
|
||||
return new Lucene60PointWriter(state);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DimensionalReader fieldsReader(SegmentReadState state) throws IOException {
|
||||
return new Lucene60DimensionalReader(state);
|
||||
public PointReader fieldsReader(SegmentReadState state) throws IOException {
|
||||
return new Lucene60PointReader(state);
|
||||
}
|
||||
}
|
|
@ -28,7 +28,7 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.codecs.DimensionalReader;
|
||||
import org.apache.lucene.codecs.PointReader;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
|
@ -39,31 +39,31 @@ 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 {
|
||||
/** Reads point values previously written with {@link Lucene60PointWriter} */
|
||||
public class Lucene60PointReader extends PointReader implements Closeable {
|
||||
final IndexInput dataIn;
|
||||
final SegmentReadState readState;
|
||||
final Map<Integer,BKDReader> readers = new HashMap<>();
|
||||
|
||||
/** Sole constructor */
|
||||
public Lucene60DimensionalReader(SegmentReadState readState) throws IOException {
|
||||
public Lucene60PointReader(SegmentReadState readState) throws IOException {
|
||||
this.readState = readState;
|
||||
String dataFileName = IndexFileNames.segmentFileName(readState.segmentInfo.name,
|
||||
readState.segmentSuffix,
|
||||
Lucene60DimensionalFormat.DATA_EXTENSION);
|
||||
Lucene60PointFormat.DATA_EXTENSION);
|
||||
dataIn = readState.directory.openInput(dataFileName, readState.context);
|
||||
String indexFileName = IndexFileNames.segmentFileName(readState.segmentInfo.name,
|
||||
readState.segmentSuffix,
|
||||
Lucene60DimensionalFormat.INDEX_EXTENSION);
|
||||
Lucene60PointFormat.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,
|
||||
Lucene60PointFormat.CODEC_NAME,
|
||||
Lucene60PointFormat.INDEX_VERSION_START,
|
||||
Lucene60PointFormat.INDEX_VERSION_START,
|
||||
readState.segmentInfo.getId(),
|
||||
readState.segmentSuffix);
|
||||
int count = indexIn.readVInt();
|
||||
|
@ -89,8 +89,8 @@ public class Lucene60DimensionalReader extends DimensionalReader implements Clos
|
|||
if (fieldInfo == null) {
|
||||
throw new IllegalArgumentException("field=\"" + fieldName + "\" is unrecognized");
|
||||
}
|
||||
if (fieldInfo.getDimensionCount() == 0) {
|
||||
throw new IllegalArgumentException("field=\"" + fieldName + "\" did not index dimensional values");
|
||||
if (fieldInfo.getPointDimensionCount() == 0) {
|
||||
throw new IllegalArgumentException("field=\"" + fieldName + "\" did not index point values");
|
||||
}
|
||||
|
||||
return readers.get(fieldInfo.number);
|
||||
|
@ -101,8 +101,8 @@ public class Lucene60DimensionalReader extends DimensionalReader implements Clos
|
|||
BKDReader bkdReader = getBKDReader(fieldName);
|
||||
|
||||
if (bkdReader == null) {
|
||||
// Schema ghost corner case! This field did index dimensional values in the past, but
|
||||
// now all docs having this dimensional field were deleted in this segment:
|
||||
// Schema ghost corner case! This field did index points in the past, but
|
||||
// now all docs having this point field were deleted in this segment:
|
||||
return;
|
||||
}
|
||||
|
||||
|
@ -144,8 +144,8 @@ public class Lucene60DimensionalReader extends DimensionalReader implements Clos
|
|||
public byte[] getMinPackedValue(String fieldName) {
|
||||
BKDReader bkdReader = getBKDReader(fieldName);
|
||||
if (bkdReader == null) {
|
||||
// Schema ghost corner case! This field did index dimensional values in the past, but
|
||||
// now all docs having this dimensional field were deleted in this segment:
|
||||
// Schema ghost corner case! This field did index points in the past, but
|
||||
// now all docs having this point field were deleted in this segment:
|
||||
return null;
|
||||
}
|
||||
|
||||
|
@ -156,8 +156,8 @@ public class Lucene60DimensionalReader extends DimensionalReader implements Clos
|
|||
public byte[] getMaxPackedValue(String fieldName) {
|
||||
BKDReader bkdReader = getBKDReader(fieldName);
|
||||
if (bkdReader == null) {
|
||||
// Schema ghost corner case! This field did index dimensional values in the past, but
|
||||
// now all docs having this dimensional field were deleted in this segment:
|
||||
// Schema ghost corner case! This field did index points in the past, but
|
||||
// now all docs having this point field were deleted in this segment:
|
||||
return null;
|
||||
}
|
||||
|
||||
|
@ -168,8 +168,8 @@ public class Lucene60DimensionalReader extends DimensionalReader implements Clos
|
|||
public int getNumDimensions(String fieldName) {
|
||||
BKDReader bkdReader = getBKDReader(fieldName);
|
||||
if (bkdReader == null) {
|
||||
// Schema ghost corner case! This field did index dimensional values in the past, but
|
||||
// now all docs having this dimensional field were deleted in this segment:
|
||||
// Schema ghost corner case! This field did index points in the past, but
|
||||
// now all docs having this point field were deleted in this segment:
|
||||
return 0;
|
||||
}
|
||||
return bkdReader.getNumDimensions();
|
||||
|
@ -179,8 +179,8 @@ public class Lucene60DimensionalReader extends DimensionalReader implements Clos
|
|||
public int getBytesPerDimension(String fieldName) {
|
||||
BKDReader bkdReader = getBKDReader(fieldName);
|
||||
if (bkdReader == null) {
|
||||
// Schema ghost corner case! This field did index dimensional values in the past, but
|
||||
// now all docs having this dimensional field were deleted in this segment:
|
||||
// Schema ghost corner case! This field did index points in the past, but
|
||||
// now all docs having this point field were deleted in this segment:
|
||||
return 0;
|
||||
}
|
||||
return bkdReader.getBytesPerDimension();
|
|
@ -25,10 +25,10 @@ import java.util.List;
|
|||
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.codecs.PointReader;
|
||||
import org.apache.lucene.codecs.PointWriter;
|
||||
import org.apache.lucene.index.PointValues.IntersectVisitor;
|
||||
import org.apache.lucene.index.PointValues.Relation;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
|
@ -40,7 +40,7 @@ import org.apache.lucene.util.bkd.BKDReader;
|
|||
import org.apache.lucene.util.bkd.BKDWriter;
|
||||
|
||||
/** Writes dimensional values */
|
||||
public class Lucene60DimensionalWriter extends DimensionalWriter implements Closeable {
|
||||
public class Lucene60PointWriter extends PointWriter implements Closeable {
|
||||
|
||||
final IndexOutput dataOut;
|
||||
final Map<String,Long> indexFPs = new HashMap<>();
|
||||
|
@ -50,20 +50,20 @@ public class Lucene60DimensionalWriter extends DimensionalWriter implements Clos
|
|||
private boolean closed;
|
||||
|
||||
/** Full constructor */
|
||||
public Lucene60DimensionalWriter(SegmentWriteState writeState, int maxPointsInLeafNode, double maxMBSortInHeap) throws IOException {
|
||||
assert writeState.fieldInfos.hasDimensionalValues();
|
||||
public Lucene60PointWriter(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,
|
||||
Lucene60DimensionalFormat.DATA_EXTENSION);
|
||||
Lucene60PointFormat.DATA_EXTENSION);
|
||||
dataOut = writeState.directory.createOutput(dataFileName, writeState.context);
|
||||
boolean success = false;
|
||||
try {
|
||||
CodecUtil.writeIndexHeader(dataOut,
|
||||
Lucene60DimensionalFormat.CODEC_NAME,
|
||||
Lucene60DimensionalFormat.DATA_VERSION_CURRENT,
|
||||
Lucene60PointFormat.CODEC_NAME,
|
||||
Lucene60PointFormat.DATA_VERSION_CURRENT,
|
||||
writeState.segmentInfo.getId(),
|
||||
writeState.segmentSuffix);
|
||||
success = true;
|
||||
|
@ -75,17 +75,17 @@ public class Lucene60DimensionalWriter extends DimensionalWriter implements Clos
|
|||
}
|
||||
|
||||
/** Uses the defaults values for {@code maxPointsInLeafNode} (1024) and {@code maxMBSortInHeap} (16.0) */
|
||||
public Lucene60DimensionalWriter(SegmentWriteState writeState) throws IOException {
|
||||
public Lucene60PointWriter(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 {
|
||||
public void writeField(FieldInfo fieldInfo, PointReader values) throws IOException {
|
||||
|
||||
try (BKDWriter writer = new BKDWriter(writeState.directory,
|
||||
writeState.segmentInfo.name,
|
||||
fieldInfo.getDimensionCount(),
|
||||
fieldInfo.getDimensionNumBytes(),
|
||||
fieldInfo.getPointDimensionCount(),
|
||||
fieldInfo.getPointNumBytes(),
|
||||
maxPointsInLeafNode,
|
||||
maxMBSortInHeap)) {
|
||||
|
||||
|
@ -114,8 +114,8 @@ public class Lucene60DimensionalWriter extends DimensionalWriter implements Clos
|
|||
|
||||
@Override
|
||||
public void merge(MergeState mergeState) throws IOException {
|
||||
for(DimensionalReader reader : mergeState.dimensionalReaders) {
|
||||
if (reader instanceof Lucene60DimensionalReader == false) {
|
||||
for(PointReader reader : mergeState.pointReaders) {
|
||||
if (reader instanceof Lucene60PointReader == false) {
|
||||
// We can only bulk merge when all to-be-merged segments use our format:
|
||||
super.merge(mergeState);
|
||||
return;
|
||||
|
@ -123,25 +123,25 @@ public class Lucene60DimensionalWriter extends DimensionalWriter implements Clos
|
|||
}
|
||||
|
||||
for (FieldInfo fieldInfo : mergeState.mergeFieldInfos) {
|
||||
if (fieldInfo.getDimensionCount() != 0) {
|
||||
if (fieldInfo.getDimensionCount() == 1) {
|
||||
if (fieldInfo.getPointDimensionCount() != 0) {
|
||||
if (fieldInfo.getPointDimensionCount() == 1) {
|
||||
//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.directory,
|
||||
writeState.segmentInfo.name,
|
||||
fieldInfo.getDimensionCount(),
|
||||
fieldInfo.getDimensionNumBytes(),
|
||||
fieldInfo.getPointDimensionCount(),
|
||||
fieldInfo.getPointNumBytes(),
|
||||
maxPointsInLeafNode,
|
||||
maxMBSortInHeap)) {
|
||||
List<BKDReader> bkdReaders = new ArrayList<>();
|
||||
List<MergeState.DocMap> docMaps = new ArrayList<>();
|
||||
List<Integer> docIDBases = new ArrayList<>();
|
||||
for(int i=0;i<mergeState.dimensionalReaders.length;i++) {
|
||||
DimensionalReader reader = mergeState.dimensionalReaders[i];
|
||||
for(int i=0;i<mergeState.pointReaders.length;i++) {
|
||||
PointReader reader = mergeState.pointReaders[i];
|
||||
|
||||
Lucene60DimensionalReader reader60 = (Lucene60DimensionalReader) reader;
|
||||
Lucene60PointReader reader60 = (Lucene60PointReader) reader;
|
||||
if (reader60 != null) {
|
||||
// TODO: I could just use the merged fieldInfo.number instead of resolving to this
|
||||
// reader's FieldInfo, right? Field numbers are always consistent across segments,
|
||||
|
@ -180,12 +180,12 @@ public class Lucene60DimensionalWriter extends DimensionalWriter implements Clos
|
|||
|
||||
String indexFileName = IndexFileNames.segmentFileName(writeState.segmentInfo.name,
|
||||
writeState.segmentSuffix,
|
||||
Lucene60DimensionalFormat.INDEX_EXTENSION);
|
||||
Lucene60PointFormat.INDEX_EXTENSION);
|
||||
// Write index file
|
||||
try (IndexOutput indexOut = writeState.directory.createOutput(indexFileName, writeState.context)) {
|
||||
CodecUtil.writeIndexHeader(indexOut,
|
||||
Lucene60DimensionalFormat.CODEC_NAME,
|
||||
Lucene60DimensionalFormat.INDEX_VERSION_CURRENT,
|
||||
Lucene60PointFormat.CODEC_NAME,
|
||||
Lucene60PointFormat.INDEX_VERSION_CURRENT,
|
||||
writeState.segmentInfo.getId(),
|
||||
writeState.segmentSuffix);
|
||||
int count = indexFPs.size();
|
|
@ -193,7 +193,7 @@
|
|||
* An optional file indicating which documents are live.
|
||||
* </li>
|
||||
* <li>
|
||||
* {@link org.apache.lucene.codecs.lucene60.Lucene60DimensionalFormat Dimensional values}.
|
||||
* {@link org.apache.lucene.codecs.lucene60.Lucene60PointFormat Point 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).
|
||||
|
@ -322,9 +322,9 @@
|
|||
* <td>Info about what files are live</td>
|
||||
* </tr>
|
||||
* <tr>
|
||||
* <td>{@link org.apache.lucene.codecs.lucene60.Lucene60DimensionalFormat Dimensional values}</td>
|
||||
* <td>{@link org.apache.lucene.codecs.lucene60.Lucene60PointFormat Point values}</td>
|
||||
* <td>.dii, .dim</td>
|
||||
* <td>Holds dimensionally indexed fields, if any</td>
|
||||
* <td>Holds indexed points, if any</td>
|
||||
* </tr>
|
||||
* </table>
|
||||
* </div>
|
||||
|
|
|
@ -24,7 +24,7 @@ import org.apache.lucene.util.BytesRef;
|
|||
* efficient. Muliple values for the same field in one documents
|
||||
* is allowed. */
|
||||
|
||||
public final class DimensionalBinaryField extends Field {
|
||||
public final class BinaryPoint extends Field {
|
||||
|
||||
private static FieldType getType(byte[][] point) {
|
||||
if (point == null) {
|
||||
|
@ -89,22 +89,22 @@ public final class DimensionalBinaryField extends Field {
|
|||
return new BytesRef(packed);
|
||||
}
|
||||
|
||||
/** General purpose API: creates a new DimensionalField, indexing the
|
||||
/** General purpose API: creates a new BinaryPoint, indexing the
|
||||
* provided N-dimensional binary point.
|
||||
*
|
||||
* @param name field name
|
||||
* @param point byte[][] value
|
||||
* @throws IllegalArgumentException if the field name or value is null.
|
||||
*/
|
||||
public DimensionalBinaryField(String name, byte[]... point) {
|
||||
public BinaryPoint(String name, byte[]... point) {
|
||||
super(name, pack(point), getType(point));
|
||||
}
|
||||
|
||||
/** Expert API */
|
||||
public DimensionalBinaryField(String name, byte[] packedPoint, FieldType type) {
|
||||
public BinaryPoint(String name, byte[] packedPoint, FieldType type) {
|
||||
super(name, packedPoint, type);
|
||||
if (packedPoint.length != type.dimensionCount() * type.dimensionNumBytes()) {
|
||||
throw new IllegalArgumentException("packedPoint is length=" + packedPoint.length + " but type.dimensionCount()=" + type.dimensionCount() + " and type.dimensionNumBytes()=" + type.dimensionNumBytes());
|
||||
if (packedPoint.length != type.pointDimensionCount() * type.pointNumBytes()) {
|
||||
throw new IllegalArgumentException("packedPoint is length=" + packedPoint.length + " but type.pointDimensionCount()=" + type.pointDimensionCount() + " and type.pointNumBytes()=" + type.pointNumBytes());
|
||||
}
|
||||
}
|
||||
}
|
|
@ -24,7 +24,6 @@ import java.util.Date;
|
|||
import java.util.Locale;
|
||||
import java.util.TimeZone;
|
||||
|
||||
import org.apache.lucene.search.DimensionalRangeQuery;
|
||||
import org.apache.lucene.search.PrefixQuery;
|
||||
import org.apache.lucene.search.TermRangeQuery;
|
||||
|
||||
|
@ -39,12 +38,12 @@ import org.apache.lucene.search.TermRangeQuery;
|
|||
* {@link TermRangeQuery} and {@link PrefixQuery} will require more memory and become slower.
|
||||
*
|
||||
* <P>
|
||||
* Another approach is {@link DimensionalLongField}, which indexes the
|
||||
* Another approach is {@link LongPoint}, which indexes the
|
||||
* values in sorted order.
|
||||
* For indexing a {@link Date} or {@link Calendar}, just get the unix timestamp as
|
||||
* <code>long</code> using {@link Date#getTime} or {@link Calendar#getTimeInMillis} and
|
||||
* index this as a numeric value with {@link DimensionalLongField}
|
||||
* and use {@link DimensionalRangeQuery} to query it.
|
||||
* index this as a numeric value with {@link LongPoint}
|
||||
* and use {@link org.apache.lucene.search.PointRangeQuery} to query it.
|
||||
*/
|
||||
public class DateTools {
|
||||
|
||||
|
|
|
@ -26,7 +26,7 @@ import org.apache.lucene.util.RamUsageEstimator;
|
|||
* efficient. Muliple values for the same field in one documents
|
||||
* is allowed. */
|
||||
|
||||
public final class DimensionalDoubleField extends Field {
|
||||
public final class DoublePoint extends Field {
|
||||
|
||||
private static FieldType getType(int numDims) {
|
||||
FieldType type = new FieldType();
|
||||
|
@ -73,14 +73,14 @@ public final class DimensionalDoubleField extends Field {
|
|||
return new BytesRef(packed);
|
||||
}
|
||||
|
||||
/** Creates a new DimensionalDoubleField, indexing the
|
||||
/** Creates a new DoublePoint, indexing the
|
||||
* provided N-dimensional int point.
|
||||
*
|
||||
* @param name field name
|
||||
* @param point double[] value
|
||||
* @throws IllegalArgumentException if the field name or value is null.
|
||||
*/
|
||||
public DimensionalDoubleField(String name, double... point) {
|
||||
public DoublePoint(String name, double... point) {
|
||||
super(name, pack(point), getType(point.length));
|
||||
}
|
||||
}
|
|
@ -18,7 +18,6 @@ package org.apache.lucene.document;
|
|||
*/
|
||||
|
||||
import org.apache.lucene.analysis.Analyzer; // javadocs
|
||||
import org.apache.lucene.index.DimensionalValues; // javadocs
|
||||
import org.apache.lucene.index.DocValuesType;
|
||||
import org.apache.lucene.index.IndexOptions;
|
||||
import org.apache.lucene.index.IndexableFieldType;
|
||||
|
@ -32,7 +31,7 @@ public class FieldType implements IndexableFieldType {
|
|||
/** Data type of the numeric value
|
||||
* @since 3.2
|
||||
*
|
||||
* @deprecated Please switch to {@link DimensionalValues} instead
|
||||
* @deprecated Please switch to {@link org.apache.lucene.index.PointValues} instead
|
||||
*/
|
||||
@Deprecated
|
||||
public enum LegacyNumericType {
|
||||
|
@ -304,7 +303,7 @@ public class FieldType implements IndexableFieldType {
|
|||
* future modifications.
|
||||
* @see #numericType()
|
||||
*
|
||||
* @deprecated Please switch to {@link DimensionalValues} instead
|
||||
* @deprecated Please switch to {@link org.apache.lucene.index.PointValues} instead
|
||||
*/
|
||||
@Deprecated
|
||||
public void setNumericType(LegacyNumericType type) {
|
||||
|
@ -320,7 +319,7 @@ public class FieldType implements IndexableFieldType {
|
|||
* The default is <code>null</code> (no numeric type)
|
||||
* @see #setNumericType(org.apache.lucene.document.FieldType.LegacyNumericType)
|
||||
*
|
||||
* @deprecated Please switch to {@link DimensionalValues} instead
|
||||
* @deprecated Please switch to {@link org.apache.lucene.index.PointValues} instead
|
||||
*/
|
||||
@Deprecated
|
||||
public LegacyNumericType numericType() {
|
||||
|
@ -335,7 +334,7 @@ public class FieldType implements IndexableFieldType {
|
|||
* future modifications.
|
||||
* @see #numericPrecisionStep()
|
||||
*
|
||||
* @deprecated Please switch to {@link DimensionalValues} instead
|
||||
* @deprecated Please switch to {@link org.apache.lucene.index.PointValues} instead
|
||||
*/
|
||||
@Deprecated
|
||||
public void setNumericPrecisionStep(int precisionStep) {
|
||||
|
@ -354,7 +353,7 @@ public class FieldType implements IndexableFieldType {
|
|||
* The default is {@link org.apache.lucene.util.LegacyNumericUtils#PRECISION_STEP_DEFAULT}
|
||||
* @see #setNumericPrecisionStep(int)
|
||||
*
|
||||
* @deprecated Please switch to {@link DimensionalValues} instead
|
||||
* @deprecated Please switch to {@link org.apache.lucene.index.PointValues} instead
|
||||
*/
|
||||
@Deprecated
|
||||
public int numericPrecisionStep() {
|
||||
|
@ -362,22 +361,22 @@ public class FieldType implements IndexableFieldType {
|
|||
}
|
||||
|
||||
/**
|
||||
* Enables dimensional indexing.
|
||||
* Enables points indexing.
|
||||
*/
|
||||
public void setDimensions(int dimensionCount, int dimensionNumBytes) {
|
||||
if (dimensionCount < 0) {
|
||||
throw new IllegalArgumentException("dimensionCount must be >= 0; got " + dimensionCount);
|
||||
throw new IllegalArgumentException("pointDimensionCount must be >= 0; got " + dimensionCount);
|
||||
}
|
||||
if (dimensionNumBytes < 0) {
|
||||
throw new IllegalArgumentException("dimensionNumBytes must be >= 0; got " + dimensionNumBytes);
|
||||
throw new IllegalArgumentException("pointNumBytes must be >= 0; got " + dimensionNumBytes);
|
||||
}
|
||||
if (dimensionCount == 0) {
|
||||
if (dimensionNumBytes != 0) {
|
||||
throw new IllegalArgumentException("when dimensionCount is 0 dimensionNumBytes must 0; got " + dimensionNumBytes);
|
||||
throw new IllegalArgumentException("when pointDimensionCount is 0 pointNumBytes must 0; got " + dimensionNumBytes);
|
||||
}
|
||||
} else if (dimensionNumBytes == 0) {
|
||||
if (dimensionCount != 0) {
|
||||
throw new IllegalArgumentException("when dimensionNumBytes is 0 dimensionCount must 0; got " + dimensionCount);
|
||||
throw new IllegalArgumentException("when pointNumBytes is 0 pointDimensionCount must 0; got " + dimensionCount);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -386,12 +385,12 @@ public class FieldType implements IndexableFieldType {
|
|||
}
|
||||
|
||||
@Override
|
||||
public int dimensionCount() {
|
||||
public int pointDimensionCount() {
|
||||
return dimensionCount;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int dimensionNumBytes() {
|
||||
public int pointNumBytes() {
|
||||
return dimensionNumBytes;
|
||||
}
|
||||
|
||||
|
@ -435,9 +434,9 @@ public class FieldType implements IndexableFieldType {
|
|||
result.append(numericPrecisionStep);
|
||||
}
|
||||
if (dimensionCount != 0) {
|
||||
result.append(",dimensionCount=");
|
||||
result.append(",pointDimensionCount=");
|
||||
result.append(dimensionCount);
|
||||
result.append(",dimensionNumBytes=");
|
||||
result.append(",pointNumBytes=");
|
||||
result.append(dimensionNumBytes);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -26,7 +26,7 @@ import org.apache.lucene.util.RamUsageEstimator;
|
|||
* efficient. Muliple values for the same field in one documents
|
||||
* is allowed. */
|
||||
|
||||
public final class DimensionalFloatField extends Field {
|
||||
public final class FloatPoint extends Field {
|
||||
|
||||
private static FieldType getType(int numDims) {
|
||||
FieldType type = new FieldType();
|
||||
|
@ -73,14 +73,14 @@ public final class DimensionalFloatField extends Field {
|
|||
return new BytesRef(packed);
|
||||
}
|
||||
|
||||
/** Creates a new DimensionalFloatField, indexing the
|
||||
/** Creates a new FloatPoint, indexing the
|
||||
* provided N-dimensional float point.
|
||||
*
|
||||
* @param name field name
|
||||
* @param point int[] value
|
||||
* @throws IllegalArgumentException if the field name or value is null.
|
||||
*/
|
||||
public DimensionalFloatField(String name, float... point) {
|
||||
public FloatPoint(String name, float... point) {
|
||||
super(name, pack(point), getType(point.length));
|
||||
}
|
||||
}
|
|
@ -26,7 +26,7 @@ import org.apache.lucene.util.RamUsageEstimator;
|
|||
* efficient. Muliple values for the same field in one documents
|
||||
* is allowed. */
|
||||
|
||||
public final class DimensionalIntField extends Field {
|
||||
public final class IntPoint extends Field {
|
||||
|
||||
private static FieldType getType(int numDims) {
|
||||
FieldType type = new FieldType();
|
||||
|
@ -73,14 +73,14 @@ public final class DimensionalIntField extends Field {
|
|||
return new BytesRef(packed);
|
||||
}
|
||||
|
||||
/** Creates a new DimensionalIntField, indexing the
|
||||
/** Creates a new IntPoint, indexing the
|
||||
* provided N-dimensional int point.
|
||||
*
|
||||
* @param name field name
|
||||
* @param point int[] value
|
||||
* @throws IllegalArgumentException if the field name or value is null.
|
||||
*/
|
||||
public DimensionalIntField(String name, int... point) {
|
||||
public IntPoint(String name, int... point) {
|
||||
super(name, pack(point), getType(point.length));
|
||||
}
|
||||
}
|
|
@ -105,7 +105,7 @@ import org.apache.lucene.index.IndexOptions;
|
|||
* class is a wrapper around this token stream type for
|
||||
* easier, more intuitive usage.</p>
|
||||
*
|
||||
* @deprecated Please use {@link DimensionalDoubleField} instead
|
||||
* @deprecated Please use {@link DoublePoint} instead
|
||||
*
|
||||
* @since 2.9
|
||||
*/
|
||||
|
|
|
@ -105,7 +105,7 @@ import org.apache.lucene.util.LegacyNumericUtils;
|
|||
* class is a wrapper around this token stream type for
|
||||
* easier, more intuitive usage.</p>
|
||||
*
|
||||
* @deprecated Please use {@link DimensionalFloatField} instead
|
||||
* @deprecated Please use {@link FloatPoint} instead
|
||||
*
|
||||
* @since 2.9
|
||||
*/
|
||||
|
|
|
@ -105,7 +105,7 @@ import org.apache.lucene.util.LegacyNumericUtils;
|
|||
* class is a wrapper around this token stream type for
|
||||
* easier, more intuitive usage.</p>
|
||||
*
|
||||
* @deprecated Please use {@link DimensionalIntField} instead
|
||||
* @deprecated Please use {@link IntPoint} instead
|
||||
*
|
||||
* @since 2.9
|
||||
*/
|
||||
|
|
|
@ -115,7 +115,7 @@ import org.apache.lucene.index.IndexOptions;
|
|||
* class is a wrapper around this token stream type for
|
||||
* easier, more intuitive usage.</p>
|
||||
*
|
||||
* @deprecated Please use {@link DimensionalLongField} instead
|
||||
* @deprecated Please use {@link LongPoint} instead
|
||||
*
|
||||
* @since 2.9
|
||||
*/
|
||||
|
|
|
@ -26,7 +26,7 @@ import org.apache.lucene.util.RamUsageEstimator;
|
|||
* efficient. Muliple values for the same field in one documents
|
||||
* is allowed. */
|
||||
|
||||
public final class DimensionalLongField extends Field {
|
||||
public final class LongPoint extends Field {
|
||||
|
||||
private static FieldType getType(int numDims) {
|
||||
FieldType type = new FieldType();
|
||||
|
@ -73,14 +73,14 @@ public final class DimensionalLongField extends Field {
|
|||
return new BytesRef(packed);
|
||||
}
|
||||
|
||||
/** Creates a new DimensionalLongField, indexing the
|
||||
/** Creates a new LongPoint, indexing the
|
||||
* provided N-dimensional int point.
|
||||
*
|
||||
* @param name field name
|
||||
* @param point int[] value
|
||||
* @throws IllegalArgumentException if the field name or value is null.
|
||||
*/
|
||||
public DimensionalLongField(String name, long... point) {
|
||||
public LongPoint(String name, long... point) {
|
||||
super(name, pack(point), getType(point.length));
|
||||
}
|
||||
}
|
|
@ -35,8 +35,8 @@
|
|||
* the process of taking a file and converting it into a Lucene {@link org.apache.lucene.document.Document}.
|
||||
* </p>
|
||||
* <p>The {@link org.apache.lucene.document.DateTools} is a utility class to make dates and times searchable. {@link
|
||||
* org.apache.lucene.document.DimensionalIntField}, {@link org.apache.lucene.document.DimensionalLongField},
|
||||
* {@link org.apache.lucene.document.DimensionalFloatField} and {@link org.apache.lucene.document.DimensionalDoubleField} enable indexing
|
||||
* of numeric values (and also dates) for fast range queries using {@link org.apache.lucene.search.DimensionalRangeQuery}</p>
|
||||
* org.apache.lucene.document.IntPoint}, {@link org.apache.lucene.document.LongPoint},
|
||||
* {@link org.apache.lucene.document.FloatPoint} and {@link org.apache.lucene.document.DoublePoint} enable indexing
|
||||
* of numeric values (and also dates) for fast range queries using {@link org.apache.lucene.search.PointRangeQuery}</p>
|
||||
*/
|
||||
package org.apache.lucene.document;
|
||||
|
|
|
@ -33,7 +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.PointReader;
|
||||
import org.apache.lucene.codecs.DocValuesProducer;
|
||||
import org.apache.lucene.codecs.NormsProducer;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
|
@ -215,8 +215,8 @@ public final 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;
|
||||
/** Status for testing of PointValues (null if PointValues could not be tested). */
|
||||
public PointsStatus pointsStatus;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -358,17 +358,17 @@ public final class CheckIndex implements Closeable {
|
|||
}
|
||||
|
||||
/**
|
||||
* Status from testing DimensionalValues
|
||||
* Status from testing PointValues
|
||||
*/
|
||||
public static final class DimensionalValuesStatus {
|
||||
public static final class PointsStatus {
|
||||
|
||||
DimensionalValuesStatus() {
|
||||
PointsStatus() {
|
||||
}
|
||||
|
||||
/** Total number of dimensional values points tested. */
|
||||
/** Total number of values points tested. */
|
||||
public long totalValuePoints;
|
||||
|
||||
/** Total number of fields with dimensional values. */
|
||||
/** Total number of fields with points. */
|
||||
public int totalValueFields;
|
||||
|
||||
/** Exception thrown during doc values test (null on success) */
|
||||
|
@ -721,8 +721,8 @@ public final class CheckIndex implements Closeable {
|
|||
// Test Docvalues
|
||||
segInfoStat.docValuesStatus = testDocValues(reader, infoStream, failFast);
|
||||
|
||||
// Test DimensionalValues
|
||||
segInfoStat.dimensionalValuesStatus = testDimensionalValues(reader, infoStream, failFast);
|
||||
// Test PointValues
|
||||
segInfoStat.pointsStatus = testPoints(reader, infoStream, failFast);
|
||||
|
||||
// Rethrow the first exception we encountered
|
||||
// This will cause stats for failed segments to be incremented properly
|
||||
|
@ -1681,23 +1681,23 @@ public final class CheckIndex implements Closeable {
|
|||
}
|
||||
|
||||
/**
|
||||
* Test the dimensional values index.
|
||||
* Test the points index
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public static Status.DimensionalValuesStatus testDimensionalValues(CodecReader reader, PrintStream infoStream, boolean failFast) throws IOException {
|
||||
public static Status.PointsStatus testPoints(CodecReader reader, PrintStream infoStream, boolean failFast) throws IOException {
|
||||
FieldInfos fieldInfos = reader.getFieldInfos();
|
||||
Status.DimensionalValuesStatus status = new Status.DimensionalValuesStatus();
|
||||
Status.PointsStatus status = new Status.PointsStatus();
|
||||
try {
|
||||
if (fieldInfos.hasDimensionalValues()) {
|
||||
DimensionalReader values = reader.getDimensionalReader();
|
||||
if (fieldInfos.hasPointValues()) {
|
||||
PointReader values = reader.getPointReader();
|
||||
if (values == null) {
|
||||
throw new RuntimeException("there are fields with dimensional values, but reader.getDimensionalRader() is null");
|
||||
throw new RuntimeException("there are fields with points, but reader.getPointReader() is null");
|
||||
}
|
||||
for (FieldInfo fieldInfo : fieldInfos) {
|
||||
if (fieldInfo.getDimensionCount() > 0) {
|
||||
if (fieldInfo.getPointDimensionCount() > 0) {
|
||||
status.totalValueFields++;
|
||||
int dimCount = fieldInfo.getDimensionCount();
|
||||
int bytesPerDim = fieldInfo.getDimensionNumBytes();
|
||||
int dimCount = fieldInfo.getPointDimensionCount();
|
||||
int bytesPerDim = fieldInfo.getPointNumBytes();
|
||||
byte[] lastMinPackedValue = new byte[dimCount*bytesPerDim];
|
||||
BytesRef lastMinPacked = new BytesRef(lastMinPackedValue);
|
||||
byte[] lastMaxPackedValue = new byte[dimCount*bytesPerDim];
|
||||
|
@ -1707,7 +1707,7 @@ public final class CheckIndex implements Closeable {
|
|||
lastMinPacked.length = bytesPerDim;
|
||||
scratch.length = bytesPerDim;
|
||||
values.intersect(fieldInfo.name,
|
||||
new DimensionalValues.IntersectVisitor() {
|
||||
new PointValues.IntersectVisitor() {
|
||||
@Override
|
||||
public void visit(int docID) {
|
||||
throw new RuntimeException("codec called IntersectVisitor.visit without a packed value for docID=" + docID);
|
||||
|
@ -1737,7 +1737,7 @@ public final class CheckIndex implements Closeable {
|
|||
}
|
||||
|
||||
@Override
|
||||
public DimensionalValues.Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
|
||||
public PointValues.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);
|
||||
|
@ -1745,7 +1745,7 @@ public final class CheckIndex implements Closeable {
|
|||
|
||||
// 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.CELL_CROSSES_QUERY;
|
||||
return PointValues.Relation.CELL_CROSSES_QUERY;
|
||||
}
|
||||
|
||||
private void checkPackedValue(String desc, byte[] packedValue, int docID) {
|
||||
|
|
|
@ -25,7 +25,7 @@ import java.util.HashMap;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.codecs.DimensionalReader;
|
||||
import org.apache.lucene.codecs.PointReader;
|
||||
import org.apache.lucene.codecs.DocValuesProducer;
|
||||
import org.apache.lucene.codecs.FieldsProducer;
|
||||
import org.apache.lucene.codecs.NormsProducer;
|
||||
|
@ -77,10 +77,10 @@ public abstract class CodecReader extends LeafReader implements Accountable {
|
|||
public abstract FieldsProducer getPostingsReader();
|
||||
|
||||
/**
|
||||
* Expert: retrieve underlying DimensionalReader
|
||||
* Expert: retrieve underlying PointReader
|
||||
* @lucene.internal
|
||||
*/
|
||||
public abstract DimensionalReader getDimensionalReader();
|
||||
public abstract PointReader getPointReader();
|
||||
|
||||
@Override
|
||||
public final void document(int docID, StoredFieldVisitor visitor) throws IOException {
|
||||
|
@ -322,9 +322,9 @@ public abstract class CodecReader extends LeafReader implements Accountable {
|
|||
ramBytesUsed += getTermVectorsReader().ramBytesUsed();
|
||||
}
|
||||
|
||||
// dimensional values
|
||||
if (getDimensionalReader() != null) {
|
||||
ramBytesUsed += getDimensionalReader().ramBytesUsed();
|
||||
// points
|
||||
if (getPointReader() != null) {
|
||||
ramBytesUsed += getPointReader().ramBytesUsed();
|
||||
}
|
||||
|
||||
return ramBytesUsed;
|
||||
|
@ -358,9 +358,9 @@ public abstract class CodecReader extends LeafReader implements Accountable {
|
|||
resources.add(Accountables.namedAccountable("term vectors", getTermVectorsReader()));
|
||||
}
|
||||
|
||||
// dimensional values
|
||||
if (getDimensionalReader() != null) {
|
||||
resources.add(Accountables.namedAccountable("dimensional values", getDimensionalReader()));
|
||||
// points
|
||||
if (getPointReader() != null) {
|
||||
resources.add(Accountables.namedAccountable("points", getPointReader()));
|
||||
}
|
||||
|
||||
return Collections.unmodifiableList(resources);
|
||||
|
|
|
@ -23,8 +23,8 @@ import java.util.HashMap;
|
|||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.codecs.DimensionalFormat;
|
||||
import org.apache.lucene.codecs.DimensionalWriter;
|
||||
import org.apache.lucene.codecs.PointFormat;
|
||||
import org.apache.lucene.codecs.PointWriter;
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.NormsConsumer;
|
||||
|
@ -93,7 +93,7 @@ final class DefaultIndexingChain extends DocConsumer {
|
|||
int maxDoc = state.segmentInfo.maxDoc();
|
||||
writeNorms(state);
|
||||
writeDocValues(state);
|
||||
writeDimensionalValues(state);
|
||||
writePoints(state);
|
||||
|
||||
// it's possible all docs hit non-aborting exceptions...
|
||||
initStoredFieldsWriter();
|
||||
|
@ -121,33 +121,33 @@ final class DefaultIndexingChain extends DocConsumer {
|
|||
docWriter.codec.fieldInfosFormat().write(state.directory, state.segmentInfo, "", state.fieldInfos, IOContext.DEFAULT);
|
||||
}
|
||||
|
||||
/** Writes all buffered dimensional values. */
|
||||
private void writeDimensionalValues(SegmentWriteState state) throws IOException {
|
||||
DimensionalWriter dimensionalWriter = null;
|
||||
/** Writes all buffered points. */
|
||||
private void writePoints(SegmentWriteState state) throws IOException {
|
||||
PointWriter pointWriter = null;
|
||||
boolean success = false;
|
||||
try {
|
||||
for (int i=0;i<fieldHash.length;i++) {
|
||||
PerField perField = fieldHash[i];
|
||||
while (perField != null) {
|
||||
if (perField.dimensionalValuesWriter != null) {
|
||||
if (perField.fieldInfo.getDimensionCount() == 0) {
|
||||
if (perField.pointValuesWriter != null) {
|
||||
if (perField.fieldInfo.getPointDimensionCount() == 0) {
|
||||
// BUG
|
||||
throw new AssertionError("segment=" + state.segmentInfo + ": field=\"" + perField.fieldInfo.name + "\" has no dimensional values but wrote them");
|
||||
throw new AssertionError("segment=" + state.segmentInfo + ": field=\"" + perField.fieldInfo.name + "\" has no points but wrote them");
|
||||
}
|
||||
if (dimensionalWriter == null) {
|
||||
if (pointWriter == null) {
|
||||
// lazy init
|
||||
DimensionalFormat fmt = state.segmentInfo.getCodec().dimensionalFormat();
|
||||
PointFormat fmt = state.segmentInfo.getCodec().pointFormat();
|
||||
if (fmt == null) {
|
||||
throw new IllegalStateException("field=\"" + perField.fieldInfo.name + "\" was indexed dimensionally but codec does not support dimensional formats");
|
||||
throw new IllegalStateException("field=\"" + perField.fieldInfo.name + "\" was indexed as points but codec does not support points");
|
||||
}
|
||||
dimensionalWriter = fmt.fieldsWriter(state);
|
||||
pointWriter = fmt.fieldsWriter(state);
|
||||
}
|
||||
|
||||
perField.dimensionalValuesWriter.flush(state, dimensionalWriter);
|
||||
perField.dimensionalValuesWriter = null;
|
||||
} else if (perField.fieldInfo.getDimensionCount() != 0) {
|
||||
perField.pointValuesWriter.flush(state, pointWriter);
|
||||
perField.pointValuesWriter = null;
|
||||
} else if (perField.fieldInfo.getPointDimensionCount() != 0) {
|
||||
// BUG
|
||||
throw new AssertionError("segment=" + state.segmentInfo + ": field=\"" + perField.fieldInfo.name + "\" has dimensional values but did not write them");
|
||||
throw new AssertionError("segment=" + state.segmentInfo + ": field=\"" + perField.fieldInfo.name + "\" has points but did not write them");
|
||||
}
|
||||
perField = perField.next;
|
||||
}
|
||||
|
@ -155,9 +155,9 @@ final class DefaultIndexingChain extends DocConsumer {
|
|||
success = true;
|
||||
} finally {
|
||||
if (success) {
|
||||
IOUtils.close(dimensionalWriter);
|
||||
IOUtils.close(pointWriter);
|
||||
} else {
|
||||
IOUtils.closeWhileHandlingException(dimensionalWriter);
|
||||
IOUtils.closeWhileHandlingException(pointWriter);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -419,11 +419,11 @@ final class DefaultIndexingChain extends DocConsumer {
|
|||
}
|
||||
indexDocValue(fp, dvType, field);
|
||||
}
|
||||
if (fieldType.dimensionCount() != 0) {
|
||||
if (fieldType.pointDimensionCount() != 0) {
|
||||
if (fp == null) {
|
||||
fp = getOrAddField(fieldName, fieldType, false);
|
||||
}
|
||||
indexDimensionalValue(fp, field);
|
||||
indexPoint(fp, field);
|
||||
}
|
||||
|
||||
return fieldCount;
|
||||
|
@ -448,24 +448,24 @@ final class DefaultIndexingChain extends DocConsumer {
|
|||
}
|
||||
}
|
||||
|
||||
/** Called from processDocument to index one field's dimensional value */
|
||||
private void indexDimensionalValue(PerField fp, IndexableField field) throws IOException {
|
||||
int dimensionCount = field.fieldType().dimensionCount();
|
||||
/** Called from processDocument to index one field's point */
|
||||
private void indexPoint(PerField fp, IndexableField field) throws IOException {
|
||||
int pointDimensionCount = field.fieldType().pointDimensionCount();
|
||||
|
||||
int dimensionNumBytes = field.fieldType().dimensionNumBytes();
|
||||
int dimensionNumBytes = field.fieldType().pointNumBytes();
|
||||
|
||||
// Record dimensions for this field; this setter will throw IllegalArgExc if
|
||||
// the dimensions were already set to something different:
|
||||
if (fp.fieldInfo.getDimensionCount() == 0) {
|
||||
fieldInfos.globalFieldNumbers.setDimensions(fp.fieldInfo.number, fp.fieldInfo.name, dimensionCount, dimensionNumBytes);
|
||||
if (fp.fieldInfo.getPointDimensionCount() == 0) {
|
||||
fieldInfos.globalFieldNumbers.setDimensions(fp.fieldInfo.number, fp.fieldInfo.name, pointDimensionCount, dimensionNumBytes);
|
||||
}
|
||||
|
||||
fp.fieldInfo.setDimensions(dimensionCount, dimensionNumBytes);
|
||||
fp.fieldInfo.setPointDimensions(pointDimensionCount, dimensionNumBytes);
|
||||
|
||||
if (fp.dimensionalValuesWriter == null) {
|
||||
fp.dimensionalValuesWriter = new DimensionalValuesWriter(docWriter, fp.fieldInfo);
|
||||
if (fp.pointValuesWriter == null) {
|
||||
fp.pointValuesWriter = new PointValuesWriter(docWriter, fp.fieldInfo);
|
||||
}
|
||||
fp.dimensionalValuesWriter.addPackedValue(docState.docID, field.binaryValue());
|
||||
fp.pointValuesWriter.addPackedValue(docState.docID, field.binaryValue());
|
||||
}
|
||||
|
||||
/** Called from processDocument to index one field's doc value */
|
||||
|
@ -596,8 +596,8 @@ final class DefaultIndexingChain extends DocConsumer {
|
|||
// segment:
|
||||
DocValuesWriter docValuesWriter;
|
||||
|
||||
// Non-null if this field ever had dimensional values in this segment:
|
||||
DimensionalValuesWriter dimensionalValuesWriter;
|
||||
// Non-null if this field ever had points in this segment:
|
||||
PointValuesWriter pointValuesWriter;
|
||||
|
||||
/** We use this to know when a PerField is seen for the
|
||||
* first time in the current document. */
|
||||
|
|
|
@ -20,8 +20,6 @@ package org.apache.lucene.index;
|
|||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
import org.apache.lucene.codecs.DimensionalFormat;
|
||||
|
||||
/**
|
||||
* Access to the Field Info file that describes document fields and whether or
|
||||
* not they are indexed. Each segment has a separate Field Info file. Objects
|
||||
|
@ -50,10 +48,10 @@ public final class FieldInfo {
|
|||
|
||||
private long dvGen;
|
||||
|
||||
/** If both of these are positive it means this is a dimensionally indexed
|
||||
* field (see {@link DimensionalFormat}). */
|
||||
private int dimensionCount;
|
||||
private int dimensionNumBytes;
|
||||
/** If both of these are positive it means this field indexed points
|
||||
* (see {@link org.apache.lucene.codecs.PointFormat}). */
|
||||
private int pointDimensionCount;
|
||||
private int pointNumBytes;
|
||||
|
||||
/**
|
||||
* Sole constructor.
|
||||
|
@ -62,7 +60,7 @@ public final class FieldInfo {
|
|||
*/
|
||||
public FieldInfo(String name, int number, boolean storeTermVector, boolean omitNorms,
|
||||
boolean storePayloads, IndexOptions indexOptions, DocValuesType docValues,
|
||||
long dvGen, Map<String,String> attributes, int dimensionCount, int dimensionNumBytes) {
|
||||
long dvGen, Map<String,String> attributes, int pointDimensionCount, int pointNumBytes) {
|
||||
this.name = Objects.requireNonNull(name);
|
||||
this.number = number;
|
||||
this.docValuesType = Objects.requireNonNull(docValues, "DocValuesType cannot be null (field: \"" + name + "\")");
|
||||
|
@ -78,8 +76,8 @@ public final class FieldInfo {
|
|||
}
|
||||
this.dvGen = dvGen;
|
||||
this.attributes = Objects.requireNonNull(attributes);
|
||||
this.dimensionCount = dimensionCount;
|
||||
this.dimensionNumBytes = dimensionNumBytes;
|
||||
this.pointDimensionCount = pointDimensionCount;
|
||||
this.pointNumBytes = pointNumBytes;
|
||||
assert checkConsistency();
|
||||
}
|
||||
|
||||
|
@ -105,20 +103,20 @@ public final class FieldInfo {
|
|||
}
|
||||
}
|
||||
|
||||
if (dimensionCount < 0) {
|
||||
throw new IllegalStateException("dimensionCount must be >= 0; got " + dimensionCount);
|
||||
if (pointDimensionCount < 0) {
|
||||
throw new IllegalStateException("pointDimensionCount must be >= 0; got " + pointDimensionCount);
|
||||
}
|
||||
|
||||
if (dimensionNumBytes < 0) {
|
||||
throw new IllegalStateException("dimensionNumBytes must be >= 0; got " + dimensionNumBytes);
|
||||
if (pointNumBytes < 0) {
|
||||
throw new IllegalStateException("pointNumBytes must be >= 0; got " + pointNumBytes);
|
||||
}
|
||||
|
||||
if (dimensionCount != 0 && dimensionNumBytes == 0) {
|
||||
throw new IllegalStateException("dimensionNumBytes must be > 0 when dimensionCount=" + dimensionCount);
|
||||
if (pointDimensionCount != 0 && pointNumBytes == 0) {
|
||||
throw new IllegalStateException("pointNumBytes must be > 0 when pointDimensionCount=" + pointDimensionCount);
|
||||
}
|
||||
|
||||
if (dimensionNumBytes != 0 && dimensionCount == 0) {
|
||||
throw new IllegalStateException("dimensionCount must be > 0 when dimensionNumBytes=" + dimensionNumBytes);
|
||||
if (pointNumBytes != 0 && pointDimensionCount == 0) {
|
||||
throw new IllegalStateException("pointDimensionCount must be > 0 when pointNumBytes=" + pointNumBytes);
|
||||
}
|
||||
|
||||
if (dvGen != -1 && docValuesType == DocValuesType.NONE) {
|
||||
|
@ -144,9 +142,9 @@ public final class FieldInfo {
|
|||
}
|
||||
}
|
||||
|
||||
if (this.dimensionCount == 0 && dimensionCount != 0) {
|
||||
this.dimensionCount = dimensionCount;
|
||||
this.dimensionNumBytes = dimensionNumBytes;
|
||||
if (this.pointDimensionCount == 0 && dimensionCount != 0) {
|
||||
this.pointDimensionCount = dimensionCount;
|
||||
this.pointNumBytes = dimensionNumBytes;
|
||||
}
|
||||
|
||||
if (this.indexOptions != IndexOptions.NONE) { // if updated field data is not for indexing, leave the updates out
|
||||
|
@ -165,40 +163,40 @@ public final class FieldInfo {
|
|||
assert checkConsistency();
|
||||
}
|
||||
|
||||
/** Record that this field is indexed dimensionally, with the
|
||||
/** Record that this field is indexed with points, with the
|
||||
* specified number of dimensions and bytes per dimension. */
|
||||
public void setDimensions(int count, int numBytes) {
|
||||
public void setPointDimensions(int count, int numBytes) {
|
||||
if (count <= 0) {
|
||||
throw new IllegalArgumentException("dimension count must be >= 0; got " + count + " for field=\"" + name + "\"");
|
||||
throw new IllegalArgumentException("point dimension count must be >= 0; got " + count + " for field=\"" + name + "\"");
|
||||
}
|
||||
if (count > DimensionalValues.MAX_DIMENSIONS) {
|
||||
throw new IllegalArgumentException("dimension count must be < DimensionalValues.MAX_DIMENSIONS (= " + DimensionalValues.MAX_DIMENSIONS + "); got " + count + " for field=\"" + name + "\"");
|
||||
if (count > PointValues.MAX_DIMENSIONS) {
|
||||
throw new IllegalArgumentException("point dimension count must be < PointValues.MAX_DIMENSIONS (= " + PointValues.MAX_DIMENSIONS + "); got " + count + " for field=\"" + name + "\"");
|
||||
}
|
||||
if (numBytes <= 0) {
|
||||
throw new IllegalArgumentException("dimension numBytes must be >= 0; got " + numBytes + " for field=\"" + name + "\"");
|
||||
throw new IllegalArgumentException("point numBytes must be >= 0; got " + numBytes + " for field=\"" + name + "\"");
|
||||
}
|
||||
if (numBytes > DimensionalValues.MAX_NUM_BYTES) {
|
||||
throw new IllegalArgumentException("dimension numBytes must be <= DimensionalValues.MAX_NUM_BYTES (= " + DimensionalValues.MAX_NUM_BYTES + "); got " + numBytes + " for field=\"" + name + "\"");
|
||||
if (numBytes > PointValues.MAX_NUM_BYTES) {
|
||||
throw new IllegalArgumentException("point numBytes must be <= PointValues.MAX_NUM_BYTES (= " + PointValues.MAX_NUM_BYTES + "); got " + numBytes + " for field=\"" + name + "\"");
|
||||
}
|
||||
if (dimensionCount != 0 && dimensionCount != count) {
|
||||
throw new IllegalArgumentException("cannot change dimension count from " + dimensionCount + " to " + count + " for field=\"" + name + "\"");
|
||||
if (pointDimensionCount != 0 && pointDimensionCount != count) {
|
||||
throw new IllegalArgumentException("cannot change point dimension count from " + pointDimensionCount + " to " + count + " for field=\"" + name + "\"");
|
||||
}
|
||||
if (dimensionNumBytes != 0 && dimensionNumBytes != numBytes) {
|
||||
throw new IllegalArgumentException("cannot change dimension numBytes from " + dimensionNumBytes + " to " + numBytes + " for field=\"" + name + "\"");
|
||||
if (pointNumBytes != 0 && pointNumBytes != numBytes) {
|
||||
throw new IllegalArgumentException("cannot change point numBytes from " + pointNumBytes + " to " + numBytes + " for field=\"" + name + "\"");
|
||||
}
|
||||
|
||||
dimensionCount = count;
|
||||
dimensionNumBytes = numBytes;
|
||||
pointDimensionCount = count;
|
||||
pointNumBytes = numBytes;
|
||||
}
|
||||
|
||||
/** Return dimension count */
|
||||
public int getDimensionCount() {
|
||||
return dimensionCount;
|
||||
/** Return point dimension count */
|
||||
public int getPointDimensionCount() {
|
||||
return pointDimensionCount;
|
||||
}
|
||||
|
||||
/** Return number of bytes per dimension */
|
||||
public int getDimensionNumBytes() {
|
||||
return dimensionNumBytes;
|
||||
public int getPointNumBytes() {
|
||||
return pointNumBytes;
|
||||
}
|
||||
|
||||
void setDocValuesType(DocValuesType type) {
|
||||
|
|
|
@ -39,7 +39,7 @@ public class FieldInfos implements Iterable<FieldInfo> {
|
|||
private final boolean hasVectors;
|
||||
private final boolean hasNorms;
|
||||
private final boolean hasDocValues;
|
||||
private final boolean hasDimensionalValues;
|
||||
private final boolean hasPointValues;
|
||||
|
||||
// used only by fieldInfo(int)
|
||||
private final FieldInfo[] byNumberTable; // contiguous
|
||||
|
@ -59,7 +59,7 @@ public class FieldInfos implements Iterable<FieldInfo> {
|
|||
boolean hasFreq = false;
|
||||
boolean hasNorms = false;
|
||||
boolean hasDocValues = false;
|
||||
boolean hasDimensionalValues = false;
|
||||
boolean hasPointValues = false;
|
||||
|
||||
TreeMap<Integer, FieldInfo> byNumber = new TreeMap<>();
|
||||
for (FieldInfo info : infos) {
|
||||
|
@ -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);
|
||||
hasPointValues |= (info.getPointDimensionCount() != 0);
|
||||
}
|
||||
|
||||
this.hasVectors = hasVectors;
|
||||
|
@ -92,7 +92,7 @@ public class FieldInfos implements Iterable<FieldInfo> {
|
|||
this.hasFreq = hasFreq;
|
||||
this.hasNorms = hasNorms;
|
||||
this.hasDocValues = hasDocValues;
|
||||
this.hasDimensionalValues = hasDimensionalValues;
|
||||
this.hasPointValues = hasPointValues;
|
||||
this.values = Collections.unmodifiableCollection(byNumber.values());
|
||||
Integer max = byNumber.isEmpty() ? null : Collections.max(byNumber.keySet());
|
||||
|
||||
|
@ -147,9 +147,9 @@ public class FieldInfos implements Iterable<FieldInfo> {
|
|||
return hasDocValues;
|
||||
}
|
||||
|
||||
/** Returns true if any fields have DimensionalValues */
|
||||
public boolean hasDimensionalValues() {
|
||||
return hasDimensionalValues;
|
||||
/** Returns true if any fields have PointValues */
|
||||
public boolean hasPointValues() {
|
||||
return hasPointValues;
|
||||
}
|
||||
|
||||
/** Returns the number of fields */
|
||||
|
@ -249,10 +249,10 @@ public class FieldInfos implements Iterable<FieldInfo> {
|
|||
FieldDimensions dims = dimensions.get(fieldName);
|
||||
if (dims != null) {
|
||||
if (dims.dimensionCount != dimensionCount) {
|
||||
throw new IllegalArgumentException("cannot change dimension count from " + dims.dimensionCount + " to " + dimensionCount + " for field=\"" + fieldName + "\"");
|
||||
throw new IllegalArgumentException("cannot change point dimension count from " + dims.dimensionCount + " to " + dimensionCount + " for field=\"" + fieldName + "\"");
|
||||
}
|
||||
if (dims.dimensionNumBytes != dimensionNumBytes) {
|
||||
throw new IllegalArgumentException("cannot change dimension numBytes from " + dims.dimensionNumBytes + " to " + dimensionNumBytes + " for field=\"" + fieldName + "\"");
|
||||
throw new IllegalArgumentException("cannot change point numBytes from " + dims.dimensionNumBytes + " to " + dimensionNumBytes + " for field=\"" + fieldName + "\"");
|
||||
}
|
||||
} else {
|
||||
dimensions.put(fieldName, new FieldDimensions(dimensionCount, dimensionNumBytes));
|
||||
|
@ -302,10 +302,10 @@ public class FieldInfos implements Iterable<FieldInfo> {
|
|||
FieldDimensions dim = dimensions.get(name);
|
||||
if (dim != null) {
|
||||
if (dim.dimensionCount != dimensionCount) {
|
||||
throw new IllegalArgumentException("cannot change dimension count from " + dim.dimensionCount + " to " + dimensionCount + " for field=\"" + name + "\"");
|
||||
throw new IllegalArgumentException("cannot change point dimension count from " + dim.dimensionCount + " to " + dimensionCount + " for field=\"" + name + "\"");
|
||||
}
|
||||
if (dim.dimensionNumBytes != dimensionNumBytes) {
|
||||
throw new IllegalArgumentException("cannot change dimension numBytes from " + dim.dimensionNumBytes + " to " + dimensionNumBytes + " for field=\"" + name + "\"");
|
||||
throw new IllegalArgumentException("cannot change point numBytes from " + dim.dimensionNumBytes + " to " + dimensionNumBytes + " for field=\"" + name + "\"");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -337,11 +337,11 @@ public class FieldInfos implements Iterable<FieldInfo> {
|
|||
}
|
||||
|
||||
synchronized void setDimensions(int number, String name, int dimensionCount, int dimensionNumBytes) {
|
||||
if (dimensionNumBytes > DimensionalValues.MAX_NUM_BYTES) {
|
||||
throw new IllegalArgumentException("dimension numBytes must be <= DimensionalValues.MAX_NUM_BYTES (= " + DimensionalValues.MAX_NUM_BYTES + "); got " + dimensionNumBytes + " for field=\"" + name + "\"");
|
||||
if (dimensionNumBytes > PointValues.MAX_NUM_BYTES) {
|
||||
throw new IllegalArgumentException("dimension numBytes must be <= PointValues.MAX_NUM_BYTES (= " + PointValues.MAX_NUM_BYTES + "); got " + dimensionNumBytes + " for field=\"" + name + "\"");
|
||||
}
|
||||
if (dimensionCount > DimensionalValues.MAX_DIMENSIONS) {
|
||||
throw new IllegalArgumentException("dimensionCount must be <= DimensionalValues.MAX_DIMENSIONS (= " + DimensionalValues.MAX_DIMENSIONS + "); got " + dimensionCount + " for field=\"" + name + "\"");
|
||||
if (dimensionCount > PointValues.MAX_DIMENSIONS) {
|
||||
throw new IllegalArgumentException("pointDimensionCount must be <= PointValues.MAX_DIMENSIONS (= " + PointValues.MAX_DIMENSIONS + "); got " + dimensionCount + " for field=\"" + name + "\"");
|
||||
}
|
||||
verifyConsistentDimensions(number, name, dimensionCount, dimensionNumBytes);
|
||||
dimensions.put(name, new FieldDimensions(dimensionCount, dimensionNumBytes));
|
||||
|
@ -432,7 +432,7 @@ public class FieldInfos implements Iterable<FieldInfo> {
|
|||
return addOrUpdateInternal(fi.name, fi.number, fi.hasVectors(),
|
||||
fi.omitsNorms(), fi.hasPayloads(),
|
||||
fi.getIndexOptions(), fi.getDocValuesType(),
|
||||
fi.getDimensionCount(), fi.getDimensionNumBytes());
|
||||
fi.getPointDimensionCount(), fi.getPointNumBytes());
|
||||
}
|
||||
|
||||
public FieldInfo fieldInfo(String fieldName) {
|
||||
|
|
|
@ -19,7 +19,7 @@ package org.apache.lucene.index;
|
|||
|
||||
import java.util.Objects;
|
||||
|
||||
import org.apache.lucene.codecs.DimensionalReader;
|
||||
import org.apache.lucene.codecs.PointReader;
|
||||
import org.apache.lucene.codecs.DocValuesProducer;
|
||||
import org.apache.lucene.codecs.FieldsProducer;
|
||||
import org.apache.lucene.codecs.NormsProducer;
|
||||
|
@ -82,13 +82,13 @@ public class FilterCodecReader extends CodecReader {
|
|||
}
|
||||
|
||||
@Override
|
||||
public DimensionalReader getDimensionalReader() {
|
||||
return in.getDimensionalReader();
|
||||
public PointReader getPointReader() {
|
||||
return in.getPointReader();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DimensionalValues getDimensionalValues() {
|
||||
return in.getDimensionalValues();
|
||||
public PointValues getPointValues() {
|
||||
return in.getPointValues();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -381,8 +381,8 @@ public class FilterLeafReader extends LeafReader {
|
|||
}
|
||||
|
||||
@Override
|
||||
public DimensionalValues getDimensionalValues() {
|
||||
return in.getDimensionalValues();
|
||||
public PointValues getPointValues() {
|
||||
return in.getPointValues();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -1018,7 +1018,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
for(SegmentCommitInfo info : segmentInfos) {
|
||||
FieldInfos fis = readFieldInfos(info);
|
||||
for(FieldInfo fi : fis) {
|
||||
map.addOrGet(fi.name, fi.number, fi.getDocValuesType(), fi.getDimensionCount(), fi.getDimensionNumBytes());
|
||||
map.addOrGet(fi.name, fi.number, fi.getDocValuesType(), fi.getPointDimensionCount(), fi.getPointNumBytes());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -2495,7 +2495,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
FieldInfos fis = readFieldInfos(info);
|
||||
for(FieldInfo fi : fis) {
|
||||
// This will throw exceptions if any of the incoming fields have an illegal schema change:
|
||||
globalFieldNumberMap.addOrGet(fi.name, fi.number, fi.getDocValuesType(), fi.getDimensionCount(), fi.getDimensionNumBytes());
|
||||
globalFieldNumberMap.addOrGet(fi.name, fi.number, fi.getDocValuesType(), fi.getPointDimensionCount(), fi.getPointNumBytes());
|
||||
}
|
||||
infos.add(copySegmentAsIs(info, newSegName, context));
|
||||
}
|
||||
|
|
|
@ -98,12 +98,12 @@ public interface IndexableFieldType {
|
|||
public DocValuesType docValuesType();
|
||||
|
||||
/**
|
||||
* If this is positive, the field is indexed dimensionally.
|
||||
* If this is positive, the field is indexed as a point.
|
||||
*/
|
||||
public int dimensionCount();
|
||||
public int pointDimensionCount();
|
||||
|
||||
/**
|
||||
* The number of bytes in each dimension's values.
|
||||
*/
|
||||
public int dimensionNumBytes();
|
||||
public int pointNumBytes();
|
||||
}
|
||||
|
|
|
@ -19,7 +19,6 @@ package org.apache.lucene.index;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.codecs.DimensionalReader;
|
||||
import org.apache.lucene.index.IndexReader.ReaderClosedListener;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
||||
|
@ -301,9 +300,9 @@ public abstract class LeafReader extends IndexReader {
|
|||
*/
|
||||
public abstract Bits getLiveDocs();
|
||||
|
||||
/** Returns the {@link DimensionalReader} used for numeric or
|
||||
* spatial searches, or null if there are no dimensional fields. */
|
||||
public abstract DimensionalValues getDimensionalValues();
|
||||
/** Returns the {@link org.apache.lucene.codecs.PointReader} used for numeric or
|
||||
* spatial searches, or null if there are no point fields. */
|
||||
public abstract PointValues getPointValues();
|
||||
|
||||
/**
|
||||
* Checks consistency of this reader.
|
||||
|
|
|
@ -20,7 +20,7 @@ package org.apache.lucene.index;
|
|||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.codecs.DimensionalReader;
|
||||
import org.apache.lucene.codecs.PointReader;
|
||||
import org.apache.lucene.codecs.DocValuesProducer;
|
||||
import org.apache.lucene.codecs.FieldsProducer;
|
||||
import org.apache.lucene.codecs.NormsProducer;
|
||||
|
@ -66,8 +66,8 @@ public class MergeState {
|
|||
/** Postings to merge */
|
||||
public final FieldsProducer[] fieldsProducers;
|
||||
|
||||
/** Dimensional readers to merge */
|
||||
public final DimensionalReader[] dimensionalReaders;
|
||||
/** Point readers to merge */
|
||||
public final PointReader[] pointReaders;
|
||||
|
||||
/** New docID base per reader. */
|
||||
public final int[] docBase;
|
||||
|
@ -90,7 +90,7 @@ public class MergeState {
|
|||
storedFieldsReaders = new StoredFieldsReader[numReaders];
|
||||
termVectorsReaders = new TermVectorsReader[numReaders];
|
||||
docValuesProducers = new DocValuesProducer[numReaders];
|
||||
dimensionalReaders = new DimensionalReader[numReaders];
|
||||
pointReaders = new PointReader[numReaders];
|
||||
fieldInfos = new FieldInfos[numReaders];
|
||||
liveDocs = new Bits[numReaders];
|
||||
|
||||
|
@ -122,9 +122,9 @@ public class MergeState {
|
|||
}
|
||||
|
||||
fieldsProducers[i] = reader.getPostingsReader().getMergeInstance();
|
||||
dimensionalReaders[i] = reader.getDimensionalReader();
|
||||
if (dimensionalReaders[i] != null) {
|
||||
dimensionalReaders[i] = dimensionalReaders[i].getMergeInstance();
|
||||
pointReaders[i] = reader.getPointReader();
|
||||
if (pointReaders[i] != null) {
|
||||
pointReaders[i] = pointReaders[i].getMergeInstance();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -23,30 +23,30 @@ import java.util.List;
|
|||
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
|
||||
class MultiDimensionalValues extends DimensionalValues {
|
||||
class MultiPointValues extends PointValues {
|
||||
|
||||
private final List<DimensionalValues> subs;
|
||||
private final List<PointValues> subs;
|
||||
private final List<Integer> docBases;
|
||||
|
||||
private MultiDimensionalValues(List<DimensionalValues> subs, List<Integer> docBases) {
|
||||
private MultiPointValues(List<PointValues> subs, List<Integer> docBases) {
|
||||
this.subs = subs;
|
||||
this.docBases = docBases;
|
||||
}
|
||||
|
||||
public static DimensionalValues get(IndexReader r) {
|
||||
public static PointValues get(IndexReader r) {
|
||||
final List<LeafReaderContext> leaves = r.leaves();
|
||||
final int size = leaves.size();
|
||||
if (size == 0) {
|
||||
return null;
|
||||
} else if (size == 1) {
|
||||
return leaves.get(0).reader().getDimensionalValues();
|
||||
return leaves.get(0).reader().getPointValues();
|
||||
}
|
||||
|
||||
List<DimensionalValues> values = new ArrayList<>();
|
||||
List<PointValues> values = new ArrayList<>();
|
||||
List<Integer> docBases = new ArrayList<>();
|
||||
for (int i = 0; i < size; i++) {
|
||||
LeafReaderContext context = leaves.get(i);
|
||||
DimensionalValues v = context.reader().getDimensionalValues();
|
||||
PointValues v = context.reader().getPointValues();
|
||||
if (v != null) {
|
||||
values.add(v);
|
||||
docBases.add(context.docBase);
|
||||
|
@ -57,7 +57,7 @@ class MultiDimensionalValues extends DimensionalValues {
|
|||
return null;
|
||||
}
|
||||
|
||||
return new MultiDimensionalValues(values, docBases);
|
||||
return new MultiPointValues(values, docBases);
|
||||
}
|
||||
|
||||
/** Finds all documents and points matching the provided visitor */
|
||||
|
@ -85,7 +85,7 @@ class MultiDimensionalValues extends DimensionalValues {
|
|||
@Override
|
||||
public String toString() {
|
||||
StringBuilder b = new StringBuilder();
|
||||
b.append("MultiDimensionalValues(");
|
||||
b.append("MultiPointValues(");
|
||||
for(int i=0;i<subs.size();i++) {
|
||||
if (i > 0) {
|
||||
b.append(", ");
|
|
@ -315,15 +315,15 @@ public class ParallelLeafReader extends LeafReader {
|
|||
}
|
||||
|
||||
@Override
|
||||
public DimensionalValues getDimensionalValues() {
|
||||
return new DimensionalValues() {
|
||||
public PointValues getPointValues() {
|
||||
return new PointValues() {
|
||||
@Override
|
||||
public void intersect(String fieldName, IntersectVisitor visitor) throws IOException {
|
||||
LeafReader reader = fieldToReader.get(fieldName);
|
||||
if (reader == null) {
|
||||
return;
|
||||
}
|
||||
DimensionalValues dimValues = reader.getDimensionalValues();
|
||||
PointValues dimValues = reader.getPointValues();
|
||||
if (dimValues == null) {
|
||||
return;
|
||||
}
|
||||
|
@ -336,7 +336,7 @@ public class ParallelLeafReader extends LeafReader {
|
|||
if (reader == null) {
|
||||
return null;
|
||||
}
|
||||
DimensionalValues dimValues = reader.getDimensionalValues();
|
||||
PointValues dimValues = reader.getPointValues();
|
||||
if (dimValues == null) {
|
||||
return null;
|
||||
}
|
||||
|
@ -349,7 +349,7 @@ public class ParallelLeafReader extends LeafReader {
|
|||
if (reader == null) {
|
||||
return null;
|
||||
}
|
||||
DimensionalValues dimValues = reader.getDimensionalValues();
|
||||
PointValues dimValues = reader.getPointValues();
|
||||
if (dimValues == null) {
|
||||
return null;
|
||||
}
|
||||
|
@ -362,7 +362,7 @@ public class ParallelLeafReader extends LeafReader {
|
|||
if (reader == null) {
|
||||
return 0;
|
||||
}
|
||||
DimensionalValues dimValues = reader.getDimensionalValues();
|
||||
PointValues dimValues = reader.getPointValues();
|
||||
if (dimValues == null) {
|
||||
return 0;
|
||||
}
|
||||
|
@ -375,7 +375,7 @@ public class ParallelLeafReader extends LeafReader {
|
|||
if (reader == null) {
|
||||
return 0;
|
||||
}
|
||||
DimensionalValues dimValues = reader.getDimensionalValues();
|
||||
PointValues dimValues = reader.getPointValues();
|
||||
if (dimValues == null) {
|
||||
return 0;
|
||||
}
|
||||
|
|
|
@ -2,11 +2,6 @@ package org.apache.lucene.index;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.document.DimensionalBinaryField;
|
||||
import org.apache.lucene.document.DimensionalDoubleField;
|
||||
import org.apache.lucene.document.DimensionalFloatField;
|
||||
import org.apache.lucene.document.DimensionalIntField;
|
||||
import org.apache.lucene.document.DimensionalLongField;
|
||||
import org.apache.lucene.util.bkd.BKDWriter;
|
||||
|
||||
/*
|
||||
|
@ -26,12 +21,12 @@ import org.apache.lucene.util.bkd.BKDWriter;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
/** Allows recursively visiting dimensional values indexed with {@link DimensionalIntField},
|
||||
* {@link DimensionalFloatField}, {@link DimensionalLongField}, {@link DimensionalDoubleField}
|
||||
* or {@link DimensionalBinaryField}.
|
||||
/** Allows recursively visiting point values indexed with {@link org.apache.lucene.document.IntPoint},
|
||||
* {@link org.apache.lucene.document.FloatPoint}, {@link org.apache.lucene.document.LongPoint}, {@link org.apache.lucene.document.DoublePoint}
|
||||
* or {@link org.apache.lucene.document.BinaryPoint}.
|
||||
*
|
||||
* @lucene.experimental */
|
||||
public abstract class DimensionalValues {
|
||||
public abstract class PointValues {
|
||||
|
||||
/** Maximum number of bytes for each dimension */
|
||||
public static final int MAX_NUM_BYTES = 16;
|
||||
|
@ -40,7 +35,7 @@ public abstract class DimensionalValues {
|
|||
public static final int MAX_DIMENSIONS = BKDWriter.MAX_DIMS;
|
||||
|
||||
/** Default constructor */
|
||||
protected DimensionalValues() {
|
||||
protected PointValues() {
|
||||
}
|
||||
|
||||
/** Used by {@link #intersect} to check how each recursive cell corresponds to the query. */
|
|
@ -19,8 +19,8 @@ package org.apache.lucene.index;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.codecs.DimensionalReader;
|
||||
import org.apache.lucene.codecs.DimensionalWriter;
|
||||
import org.apache.lucene.codecs.PointReader;
|
||||
import org.apache.lucene.codecs.PointWriter;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.ByteBlockPool;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
@ -28,7 +28,7 @@ import org.apache.lucene.util.Counter;
|
|||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
/** Buffers up pending byte[][] value(s) per doc, then flushes when segment flushes. */
|
||||
class DimensionalValuesWriter {
|
||||
class PointValuesWriter {
|
||||
private final FieldInfo fieldInfo;
|
||||
private final ByteBlockPool bytes;
|
||||
private final Counter iwBytesUsed;
|
||||
|
@ -36,21 +36,21 @@ class DimensionalValuesWriter {
|
|||
private int numDocs;
|
||||
private final byte[] packedValue;
|
||||
|
||||
public DimensionalValuesWriter(DocumentsWriterPerThread docWriter, FieldInfo fieldInfo) {
|
||||
public PointValuesWriter(DocumentsWriterPerThread docWriter, FieldInfo fieldInfo) {
|
||||
this.fieldInfo = fieldInfo;
|
||||
this.iwBytesUsed = docWriter.bytesUsed;
|
||||
this.bytes = new ByteBlockPool(docWriter.byteBlockAllocator);
|
||||
docIDs = new int[16];
|
||||
iwBytesUsed.addAndGet(16 * RamUsageEstimator.NUM_BYTES_INT);
|
||||
packedValue = new byte[fieldInfo.getDimensionCount() * fieldInfo.getDimensionNumBytes()];
|
||||
packedValue = new byte[fieldInfo.getPointDimensionCount() * fieldInfo.getPointNumBytes()];
|
||||
}
|
||||
|
||||
public void addPackedValue(int docID, BytesRef value) {
|
||||
if (value == null) {
|
||||
throw new IllegalArgumentException("field=" + fieldInfo.name + ": dimensional value cannot be null");
|
||||
throw new IllegalArgumentException("field=" + fieldInfo.name + ": point value cannot be null");
|
||||
}
|
||||
if (value.length != fieldInfo.getDimensionCount() * fieldInfo.getDimensionNumBytes()) {
|
||||
throw new IllegalArgumentException("field=" + fieldInfo.name + ": this field's value has length=" + value.length + " but should be " + (fieldInfo.getDimensionCount() * fieldInfo.getDimensionNumBytes()));
|
||||
if (value.length != fieldInfo.getPointDimensionCount() * fieldInfo.getPointNumBytes()) {
|
||||
throw new IllegalArgumentException("field=" + fieldInfo.name + ": this field's value has length=" + value.length + " but should be " + (fieldInfo.getPointDimensionCount() * fieldInfo.getPointNumBytes()));
|
||||
}
|
||||
if (docIDs.length == numDocs) {
|
||||
docIDs = ArrayUtil.grow(docIDs, numDocs+1);
|
||||
|
@ -61,10 +61,10 @@ class DimensionalValuesWriter {
|
|||
numDocs++;
|
||||
}
|
||||
|
||||
public void flush(SegmentWriteState state, DimensionalWriter writer) throws IOException {
|
||||
public void flush(SegmentWriteState state, PointWriter writer) throws IOException {
|
||||
|
||||
writer.writeField(fieldInfo,
|
||||
new DimensionalReader() {
|
||||
new PointReader() {
|
||||
@Override
|
||||
public void intersect(String fieldName, IntersectVisitor visitor) throws IOException {
|
||||
if (fieldName.equals(fieldInfo.name) == false) {
|
|
@ -24,7 +24,7 @@ import java.util.Set;
|
|||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.DimensionalReader;
|
||||
import org.apache.lucene.codecs.PointReader;
|
||||
import org.apache.lucene.codecs.FieldsProducer;
|
||||
import org.apache.lucene.codecs.NormsProducer;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
|
@ -54,7 +54,7 @@ final class SegmentCoreReaders {
|
|||
|
||||
final StoredFieldsReader fieldsReaderOrig;
|
||||
final TermVectorsReader termVectorsReaderOrig;
|
||||
final DimensionalReader dimensionalReader;
|
||||
final PointReader pointReader;
|
||||
final Directory cfsReader;
|
||||
/**
|
||||
* fieldinfos for this core: means gen=-1.
|
||||
|
@ -124,10 +124,10 @@ final class SegmentCoreReaders {
|
|||
termVectorsReaderOrig = null;
|
||||
}
|
||||
|
||||
if (coreFieldInfos.hasDimensionalValues()) {
|
||||
dimensionalReader = codec.dimensionalFormat().fieldsReader(segmentReadState);
|
||||
if (coreFieldInfos.hasPointValues()) {
|
||||
pointReader = codec.pointFormat().fieldsReader(segmentReadState);
|
||||
} else {
|
||||
dimensionalReader = null;
|
||||
pointReader = null;
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
|
@ -157,7 +157,7 @@ final class SegmentCoreReaders {
|
|||
Throwable th = null;
|
||||
try {
|
||||
IOUtils.close(termVectorsLocal, fieldsReaderLocal, fields, termVectorsReaderOrig, fieldsReaderOrig,
|
||||
cfsReader, normsProducer, dimensionalReader);
|
||||
cfsReader, normsProducer, pointReader);
|
||||
} catch (Throwable throwable) {
|
||||
th = throwable;
|
||||
} finally {
|
||||
|
|
|
@ -21,7 +21,7 @@ import java.io.IOException;
|
|||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.DimensionalWriter;
|
||||
import org.apache.lucene.codecs.PointWriter;
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.codecs.FieldsConsumer;
|
||||
import org.apache.lucene.codecs.NormsConsumer;
|
||||
|
@ -113,12 +113,12 @@ final class SegmentMerger {
|
|||
if (mergeState.infoStream.isEnabled("SM")) {
|
||||
t0 = System.nanoTime();
|
||||
}
|
||||
if (mergeState.mergeFieldInfos.hasDimensionalValues()) {
|
||||
mergeDimensionalValues(segmentWriteState);
|
||||
if (mergeState.mergeFieldInfos.hasPointValues()) {
|
||||
mergePoints(segmentWriteState);
|
||||
}
|
||||
if (mergeState.infoStream.isEnabled("SM")) {
|
||||
long t1 = System.nanoTime();
|
||||
mergeState.infoStream.message("SM", ((t1-t0)/1000000) + " msec to merge dimensional values [" + numMerged + " docs]");
|
||||
mergeState.infoStream.message("SM", ((t1-t0)/1000000) + " msec to merge points [" + numMerged + " docs]");
|
||||
}
|
||||
|
||||
if (mergeState.mergeFieldInfos.hasNorms()) {
|
||||
|
@ -163,8 +163,8 @@ final class SegmentMerger {
|
|||
}
|
||||
}
|
||||
|
||||
private void mergeDimensionalValues(SegmentWriteState segmentWriteState) throws IOException {
|
||||
try (DimensionalWriter writer = codec.dimensionalFormat().fieldsWriter(segmentWriteState)) {
|
||||
private void mergePoints(SegmentWriteState segmentWriteState) throws IOException {
|
||||
try (PointWriter writer = codec.pointFormat().fieldsWriter(segmentWriteState)) {
|
||||
writer.merge(mergeState);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,7 +21,7 @@ import java.io.IOException;
|
|||
import java.util.Collections;
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.DimensionalReader;
|
||||
import org.apache.lucene.codecs.PointReader;
|
||||
import org.apache.lucene.codecs.DocValuesProducer;
|
||||
import org.apache.lucene.codecs.FieldInfosFormat;
|
||||
import org.apache.lucene.codecs.FieldsProducer;
|
||||
|
@ -218,9 +218,9 @@ public final class SegmentReader extends CodecReader {
|
|||
}
|
||||
|
||||
@Override
|
||||
public DimensionalValues getDimensionalValues() {
|
||||
public PointValues getPointValues() {
|
||||
ensureOpen();
|
||||
return core.dimensionalReader;
|
||||
return core.pointReader;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -242,9 +242,9 @@ public final class SegmentReader extends CodecReader {
|
|||
}
|
||||
|
||||
@Override
|
||||
public DimensionalReader getDimensionalReader() {
|
||||
public PointReader getPointReader() {
|
||||
ensureOpen();
|
||||
return core.dimensionalReader;
|
||||
return core.pointReader;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -20,7 +20,7 @@ package org.apache.lucene.index;
|
|||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
|
||||
import org.apache.lucene.codecs.DimensionalReader;
|
||||
import org.apache.lucene.codecs.PointReader;
|
||||
import org.apache.lucene.codecs.DocValuesProducer;
|
||||
import org.apache.lucene.codecs.FieldsProducer;
|
||||
import org.apache.lucene.codecs.NormsProducer;
|
||||
|
@ -92,13 +92,13 @@ public final class SlowCodecReaderWrapper {
|
|||
}
|
||||
|
||||
@Override
|
||||
public DimensionalValues getDimensionalValues() {
|
||||
return reader.getDimensionalValues();
|
||||
public PointValues getPointValues() {
|
||||
return reader.getPointValues();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DimensionalReader getDimensionalReader() {
|
||||
return dimensionalValuesToReader(reader.getDimensionalValues());
|
||||
public PointReader getPointReader() {
|
||||
return pointValuesToReader(reader.getPointValues());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -129,11 +129,11 @@ public final class SlowCodecReaderWrapper {
|
|||
}
|
||||
}
|
||||
|
||||
private static DimensionalReader dimensionalValuesToReader(DimensionalValues values) {
|
||||
private static PointReader pointValuesToReader(PointValues values) {
|
||||
if (values == null) {
|
||||
return null;
|
||||
}
|
||||
return new DimensionalReader() {
|
||||
return new PointReader() {
|
||||
@Override
|
||||
public void intersect(String fieldName, IntersectVisitor visitor) throws IOException {
|
||||
values.intersect(fieldName, visitor);
|
||||
|
|
|
@ -234,9 +234,9 @@ public final class SlowCompositeReaderWrapper extends LeafReader {
|
|||
}
|
||||
|
||||
@Override
|
||||
public DimensionalValues getDimensionalValues() {
|
||||
public PointValues getPointValues() {
|
||||
ensureOpen();
|
||||
return MultiDimensionalValues.get(in);
|
||||
return MultiPointValues.get(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -0,0 +1,153 @@
|
|||
package org.apache.lucene.search;
|
||||
|
||||
/*
|
||||
* 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.Arrays;
|
||||
import java.util.Objects;
|
||||
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.PointValues.IntersectVisitor;
|
||||
import org.apache.lucene.index.PointValues.Relation;
|
||||
import org.apache.lucene.index.PointValues;
|
||||
import org.apache.lucene.util.DocIdSetBuilder;
|
||||
import org.apache.lucene.util.NumericUtils;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
|
||||
/** Searches for single points in fields previously indexed using points
|
||||
* e.g. {@link org.apache.lucene.document.LongPoint}. */
|
||||
|
||||
public class ExactPointQuery extends Query {
|
||||
final String field;
|
||||
final int numDims;
|
||||
final byte[][] point;
|
||||
final int bytesPerDim;
|
||||
|
||||
public ExactPointQuery(String field, byte[][] point) {
|
||||
this.field = field;
|
||||
if (point == null) {
|
||||
throw new IllegalArgumentException("point must not be null");
|
||||
}
|
||||
this.point = point;
|
||||
this.numDims = point.length;
|
||||
|
||||
int bytesPerDim = -1;
|
||||
for(byte[] value : point) {
|
||||
if (value == null) {
|
||||
throw new IllegalArgumentException("point's dimensional values must not be null");
|
||||
}
|
||||
if (bytesPerDim == -1) {
|
||||
bytesPerDim = value.length;
|
||||
} else if (value.length != bytesPerDim) {
|
||||
throw new IllegalArgumentException("all dimensions must have same bytes length, but saw " + bytesPerDim + " and " + value.length);
|
||||
}
|
||||
}
|
||||
this.bytesPerDim = bytesPerDim;
|
||||
}
|
||||
|
||||
/** Use in the 1D case when you indexed 1D int values using {@link org.apache.lucene.document.IntPoint} */
|
||||
public static ExactPointQuery new1DIntExact(String field, int value) {
|
||||
return new ExactPointQuery(field, pack(value));
|
||||
}
|
||||
|
||||
/** Use in the 1D case when you indexed 1D long values using {@link org.apache.lucene.document.LongPoint} */
|
||||
public static ExactPointQuery new1DLongExact(String field, long value) {
|
||||
return new ExactPointQuery(field, pack(value));
|
||||
}
|
||||
|
||||
/** Use in the 1D case when you indexed 1D float values using {@link org.apache.lucene.document.FloatPoint} */
|
||||
public static ExactPointQuery new1DFloatExact(String field, float value) {
|
||||
return new ExactPointQuery(field, pack(value));
|
||||
}
|
||||
|
||||
/** Use in the 1D case when you indexed 1D double values using {@link org.apache.lucene.document.DoublePoint} */
|
||||
public static ExactPointQuery new1DDoubleExact(String field, double value) {
|
||||
return new ExactPointQuery(field, pack(value));
|
||||
}
|
||||
|
||||
private static byte[][] pack(long value) {
|
||||
byte[][] result = new byte[][] {new byte[RamUsageEstimator.NUM_BYTES_LONG]};
|
||||
NumericUtils.longToBytes(value, result[0], 0);
|
||||
return result;
|
||||
}
|
||||
|
||||
private static byte[][] pack(double value) {
|
||||
byte[][] result = new byte[][] {new byte[RamUsageEstimator.NUM_BYTES_LONG]};
|
||||
NumericUtils.longToBytesDirect(NumericUtils.doubleToSortableLong(value), result[0], 0);
|
||||
return result;
|
||||
}
|
||||
|
||||
private static byte[][] pack(int value) {
|
||||
byte[][] result = new byte[][] {new byte[RamUsageEstimator.NUM_BYTES_INT]};
|
||||
NumericUtils.intToBytes(value, result[0], 0);
|
||||
return result;
|
||||
}
|
||||
|
||||
private static byte[][] pack(float value) {
|
||||
byte[][] result = new byte[][] {new byte[RamUsageEstimator.NUM_BYTES_INT]};
|
||||
NumericUtils.intToBytesDirect(NumericUtils.floatToSortableInt(value), result[0], 0);
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Query rewrite(IndexReader reader) throws IOException {
|
||||
boolean[] inclusive = new boolean[] {true};
|
||||
return new PointRangeQuery(field, point, inclusive, point, inclusive);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
int hash = super.hashCode();
|
||||
hash += Arrays.hashCode(point)^0x14fa55fb;
|
||||
hash += numDims^0x14fa55fb;
|
||||
hash += Objects.hashCode(bytesPerDim);
|
||||
return hash;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object other) {
|
||||
if (super.equals(other)) {
|
||||
final ExactPointQuery q = (ExactPointQuery) other;
|
||||
return q.numDims == numDims &&
|
||||
q.bytesPerDim == bytesPerDim &&
|
||||
Arrays.equals(point, q.point);
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString(String field) {
|
||||
final StringBuilder sb = new StringBuilder();
|
||||
sb.append(getClass().getSimpleName());
|
||||
sb.append(':');
|
||||
if (this.field.equals(field) == false) {
|
||||
sb.append("field=");
|
||||
sb.append(this.field);
|
||||
sb.append(':');
|
||||
}
|
||||
|
||||
return sb.append(" point=")
|
||||
.append(Arrays.toString(point))
|
||||
.toString();
|
||||
}
|
||||
}
|
|
@ -157,7 +157,7 @@ import org.apache.lucene.index.Term; // for javadocs
|
|||
* precision step). This query type was developed for a geographic portal, where the performance for
|
||||
* e.g. bounding boxes or exact date/time stamps is important.</p>
|
||||
*
|
||||
* @deprecated Please use {@link DimensionalRangeQuery} instead
|
||||
* @deprecated Please use {@link PointRangeQuery} instead
|
||||
*
|
||||
* @since 2.9
|
||||
**/
|
||||
|
|
|
@ -21,14 +21,9 @@ import java.io.IOException;
|
|||
import java.util.Arrays;
|
||||
import java.util.Objects;
|
||||
|
||||
import org.apache.lucene.document.DimensionalBinaryField;
|
||||
import org.apache.lucene.document.DimensionalDoubleField;
|
||||
import org.apache.lucene.document.DimensionalFloatField;
|
||||
import org.apache.lucene.document.DimensionalIntField;
|
||||
import org.apache.lucene.document.DimensionalLongField;
|
||||
import org.apache.lucene.index.DimensionalValues;
|
||||
import org.apache.lucene.index.DimensionalValues.IntersectVisitor;
|
||||
import org.apache.lucene.index.DimensionalValues.Relation;
|
||||
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.FieldInfo;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
|
@ -37,11 +32,11 @@ import org.apache.lucene.util.NumericUtils;
|
|||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
|
||||
/** Searches for ranges in fields previously indexed using dimensional
|
||||
* fields, e.g. {@link DimensionalLongField}. In a 1D field this is
|
||||
/** Searches for ranges in fields previously indexed using points e.g.
|
||||
* {@link org.apache.lucene.document.LongPoint}. In a 1D field this is
|
||||
* a simple range query; in a multi-dimensional field it's a box shape. */
|
||||
|
||||
public class DimensionalRangeQuery extends Query {
|
||||
public class PointRangeQuery extends Query {
|
||||
final String field;
|
||||
final int numDims;
|
||||
final byte[][] lowerPoint;
|
||||
|
@ -51,9 +46,9 @@ public class DimensionalRangeQuery extends Query {
|
|||
// This is null only in the "fully open range" case
|
||||
final Integer bytesPerDim;
|
||||
|
||||
public DimensionalRangeQuery(String field,
|
||||
byte[][] lowerPoint, boolean[] lowerInclusive,
|
||||
byte[][] upperPoint, boolean[] upperInclusive) {
|
||||
public PointRangeQuery(String field,
|
||||
byte[][] lowerPoint, boolean[] lowerInclusive,
|
||||
byte[][] upperPoint, boolean[] upperInclusive) {
|
||||
this.field = field;
|
||||
if (lowerPoint == null) {
|
||||
throw new IllegalArgumentException("lowerPoint must not be null");
|
||||
|
@ -96,29 +91,29 @@ public class DimensionalRangeQuery extends Query {
|
|||
}
|
||||
}
|
||||
|
||||
/** Use in the 1D case when you indexed 1D int values using {@link DimensionalIntField} */
|
||||
public static DimensionalRangeQuery new1DIntRange(String field, Integer lowerValue, boolean lowerInclusive, Integer upperValue, boolean upperInclusive) {
|
||||
return new DimensionalRangeQuery(field, pack(lowerValue), new boolean[] {lowerInclusive}, pack(upperValue), new boolean[] {upperInclusive});
|
||||
/** Use in the 1D case when you indexed 1D int values using {@link org.apache.lucene.document.IntPoint} */
|
||||
public static PointRangeQuery new1DIntRange(String field, Integer lowerValue, boolean lowerInclusive, Integer upperValue, boolean upperInclusive) {
|
||||
return new PointRangeQuery(field, pack(lowerValue), new boolean[] {lowerInclusive}, pack(upperValue), new boolean[] {upperInclusive});
|
||||
}
|
||||
|
||||
/** Use in the 1D case when you indexed 1D long values using {@link DimensionalLongField} */
|
||||
public static DimensionalRangeQuery new1DLongRange(String field, Long lowerValue, boolean lowerInclusive, Long upperValue, boolean upperInclusive) {
|
||||
return new DimensionalRangeQuery(field, pack(lowerValue), new boolean[] {lowerInclusive}, pack(upperValue), new boolean[] {upperInclusive});
|
||||
/** Use in the 1D case when you indexed 1D long values using {@link org.apache.lucene.document.LongPoint} */
|
||||
public static PointRangeQuery new1DLongRange(String field, Long lowerValue, boolean lowerInclusive, Long upperValue, boolean upperInclusive) {
|
||||
return new PointRangeQuery(field, pack(lowerValue), new boolean[] {lowerInclusive}, pack(upperValue), new boolean[] {upperInclusive});
|
||||
}
|
||||
|
||||
/** Use in the 1D case when you indexed 1D float values using {@link DimensionalFloatField} */
|
||||
public static DimensionalRangeQuery new1DFloatRange(String field, Float lowerValue, boolean lowerInclusive, Float upperValue, boolean upperInclusive) {
|
||||
return new DimensionalRangeQuery(field, pack(lowerValue), new boolean[] {lowerInclusive}, pack(upperValue), new boolean[] {upperInclusive});
|
||||
/** Use in the 1D case when you indexed 1D float values using {@link org.apache.lucene.document.FloatPoint} */
|
||||
public static PointRangeQuery new1DFloatRange(String field, Float lowerValue, boolean lowerInclusive, Float upperValue, boolean upperInclusive) {
|
||||
return new PointRangeQuery(field, pack(lowerValue), new boolean[] {lowerInclusive}, pack(upperValue), new boolean[] {upperInclusive});
|
||||
}
|
||||
|
||||
/** Use in the 1D case when you indexed 1D double values using {@link DimensionalDoubleField} */
|
||||
public static DimensionalRangeQuery new1DDoubleRange(String field, Double lowerValue, boolean lowerInclusive, Double upperValue, boolean upperInclusive) {
|
||||
return new DimensionalRangeQuery(field, pack(lowerValue), new boolean[] {lowerInclusive}, pack(upperValue), new boolean[] {upperInclusive});
|
||||
/** Use in the 1D case when you indexed 1D double values using {@link org.apache.lucene.document.DoublePoint} */
|
||||
public static PointRangeQuery new1DDoubleRange(String field, Double lowerValue, boolean lowerInclusive, Double upperValue, boolean upperInclusive) {
|
||||
return new PointRangeQuery(field, pack(lowerValue), new boolean[] {lowerInclusive}, pack(upperValue), new boolean[] {upperInclusive});
|
||||
}
|
||||
|
||||
/** Use in the 1D case when you indexed binary values using {@link DimensionalBinaryField} */
|
||||
public static DimensionalRangeQuery new1DBinaryRange(String field, byte[] lowerValue, boolean lowerInclusive, byte[] upperValue, boolean upperInclusive) {
|
||||
return new DimensionalRangeQuery(field, new byte[][] {lowerValue}, new boolean[] {lowerInclusive}, new byte[][] {upperValue}, new boolean[] {upperInclusive});
|
||||
/** Use in the 1D case when you indexed binary values using {@link org.apache.lucene.document.BinaryPoint} */
|
||||
public static PointRangeQuery new1DBinaryRange(String field, byte[] lowerValue, boolean lowerInclusive, byte[] upperValue, boolean upperInclusive) {
|
||||
return new PointRangeQuery(field, new byte[][] {lowerValue}, new boolean[] {lowerInclusive}, new byte[][] {upperValue}, new boolean[] {upperInclusive});
|
||||
}
|
||||
|
||||
private static byte[][] pack(Long value) {
|
||||
|
@ -172,9 +167,9 @@ public class DimensionalRangeQuery extends Query {
|
|||
@Override
|
||||
public Scorer scorer(LeafReaderContext context) throws IOException {
|
||||
LeafReader reader = context.reader();
|
||||
DimensionalValues values = reader.getDimensionalValues();
|
||||
PointValues values = reader.getPointValues();
|
||||
if (values == null) {
|
||||
// No docs in this segment indexed any field dimensionally
|
||||
// No docs in this segment indexed any points
|
||||
return null;
|
||||
}
|
||||
FieldInfo fieldInfo = reader.getFieldInfos().fieldInfo(field);
|
||||
|
@ -182,13 +177,13 @@ public class DimensionalRangeQuery extends Query {
|
|||
// No docs in this segment indexed this field at all
|
||||
return null;
|
||||
}
|
||||
if (fieldInfo.getDimensionCount() != numDims) {
|
||||
throw new IllegalArgumentException("field=\"" + field + "\" was indexed with numDims=" + fieldInfo.getDimensionCount() + " but this query has numDims=" + numDims);
|
||||
if (fieldInfo.getPointDimensionCount() != numDims) {
|
||||
throw new IllegalArgumentException("field=\"" + field + "\" was indexed with numDims=" + fieldInfo.getPointDimensionCount() + " but this query has numDims=" + numDims);
|
||||
}
|
||||
if (bytesPerDim != null && bytesPerDim.intValue() != fieldInfo.getDimensionNumBytes()) {
|
||||
throw new IllegalArgumentException("field=\"" + field + "\" was indexed with bytesPerDim=" + fieldInfo.getDimensionNumBytes() + " but this query has bytesPerDim=" + bytesPerDim);
|
||||
if (bytesPerDim != null && bytesPerDim.intValue() != fieldInfo.getPointNumBytes()) {
|
||||
throw new IllegalArgumentException("field=\"" + field + "\" was indexed with bytesPerDim=" + fieldInfo.getPointNumBytes() + " but this query has bytesPerDim=" + bytesPerDim);
|
||||
}
|
||||
int bytesPerDim = fieldInfo.getDimensionNumBytes();
|
||||
int bytesPerDim = fieldInfo.getPointNumBytes();
|
||||
|
||||
byte[] packedLowerIncl = new byte[numDims * bytesPerDim];
|
||||
byte[] packedUpperIncl = new byte[numDims * bytesPerDim];
|
||||
|
@ -320,7 +315,7 @@ public class DimensionalRangeQuery extends Query {
|
|||
@Override
|
||||
public boolean equals(Object other) {
|
||||
if (super.equals(other)) {
|
||||
final DimensionalRangeQuery q = (DimensionalRangeQuery) other;
|
||||
final PointRangeQuery q = (PointRangeQuery) other;
|
||||
return q.numDims == numDims &&
|
||||
q.bytesPerDim == bytesPerDim &&
|
||||
Arrays.equals(lowerPoint, q.lowerPoint) &&
|
|
@ -33,7 +33,7 @@ import org.apache.lucene.index.IndexReader;
|
|||
<li> {@link FuzzyQuery}
|
||||
<li> {@link RegexpQuery}
|
||||
<li> {@link TermRangeQuery}
|
||||
<li> {@link DimensionalRangeQuery}
|
||||
<li> {@link PointRangeQuery}
|
||||
<li> {@link ConstantScoreQuery}
|
||||
<li> {@link DisjunctionMaxQuery}
|
||||
<li> {@link MatchAllDocsQuery}
|
||||
|
|
|
@ -43,7 +43,7 @@ public final class UsageTrackingQueryCachingPolicy implements QueryCachingPolicy
|
|||
// DocIdSet in the first place
|
||||
return query instanceof MultiTermQuery ||
|
||||
query instanceof MultiTermQueryConstantScoreWrapper ||
|
||||
query instanceof DimensionalRangeQuery;
|
||||
query instanceof PointRangeQuery;
|
||||
}
|
||||
|
||||
static boolean isCheap(Query query) {
|
||||
|
|
|
@ -160,22 +160,22 @@
|
|||
* and an upper
|
||||
* {@link org.apache.lucene.index.Term Term}
|
||||
* according to {@link org.apache.lucene.util.BytesRef#compareTo BytesRef.compareTo()}. It is not intended
|
||||
* for numerical ranges; use {@link org.apache.lucene.search.DimensionalRangeQuery DimensionalRangeQuery} instead.
|
||||
* for numerical ranges; use {@link org.apache.lucene.search.PointRangeQuery PointRangeQuery} instead.
|
||||
*
|
||||
* For example, one could find all documents
|
||||
* that have terms beginning with the letters <tt>a</tt> through <tt>c</tt>.
|
||||
*
|
||||
* <h3>
|
||||
* {@link org.apache.lucene.search.DimensionalRangeQuery DimensionalRangeQuery}
|
||||
* {@link org.apache.lucene.search.PointRangeQuery PointRangeQuery}
|
||||
* </h3>
|
||||
*
|
||||
* <p>The
|
||||
* {@link org.apache.lucene.search.DimensionalRangeQuery DimensionalRangeQuery}
|
||||
* {@link org.apache.lucene.search.PointRangeQuery PointRangeQuery}
|
||||
* matches all documents that occur in a numeric range.
|
||||
* For DimensionalRangeQuery to work, you must index the values
|
||||
* using a one of the numeric fields ({@link org.apache.lucene.document.DimensionalIntField DimensionalIntField},
|
||||
* {@link org.apache.lucene.document.DimensionalLongField DimensionalLongField}, {@link org.apache.lucene.document.DimensionalFloatField DimensionalFloatField},
|
||||
* or {@link org.apache.lucene.document.DimensionalDoubleField DimensionalDoubleField}).
|
||||
* For PointRangeQuery to work, you must index the values
|
||||
* using a one of the numeric fields ({@link org.apache.lucene.document.IntPoint IntPoint},
|
||||
* {@link org.apache.lucene.document.LongPoint LongPoint}, {@link org.apache.lucene.document.FloatPoint FloatPoint},
|
||||
* or {@link org.apache.lucene.document.DoublePoint DoublePoint}).
|
||||
*
|
||||
* <h3>
|
||||
* {@link org.apache.lucene.search.PrefixQuery PrefixQuery},
|
||||
|
|
|
@ -19,7 +19,6 @@ package org.apache.lucene.util;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.DimensionalValues;
|
||||
import org.apache.lucene.index.FilterLeafReader;
|
||||
import org.apache.lucene.index.FilteredTermsEnum;
|
||||
import org.apache.lucene.index.Terms;
|
||||
|
@ -60,7 +59,7 @@ import org.apache.lucene.index.TermsEnum;
|
|||
*
|
||||
* @lucene.internal
|
||||
*
|
||||
* @deprecated Please use {@link DimensionalValues} instead.
|
||||
* @deprecated Please use {@link org.apache.lucene.index.PointValues} instead.
|
||||
*
|
||||
* @since 2.9, API changed non backwards-compliant in 4.0
|
||||
*/
|
||||
|
|
|
@ -21,8 +21,8 @@ import java.io.IOException;
|
|||
import java.util.Arrays;
|
||||
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.index.DimensionalValues.IntersectVisitor;
|
||||
import org.apache.lucene.index.DimensionalValues.Relation;
|
||||
import org.apache.lucene.index.PointValues.IntersectVisitor;
|
||||
import org.apache.lucene.index.PointValues.Relation;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.Accountable;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
|
|
@ -89,7 +89,7 @@ public class TestFieldType extends LuceneTestCase {
|
|||
if ((method.getModifiers() & Modifier.PUBLIC) != 0 && method.getName().startsWith("set")) {
|
||||
final Class<?>[] parameterTypes = method.getParameterTypes();
|
||||
final Object[] args = new Object[parameterTypes.length];
|
||||
if (method.getName().equals("setDimensions")) {
|
||||
if (method.getName().equals("setPointDimensions")) {
|
||||
args[0] = 1 + random().nextInt(15);
|
||||
args[1] = 1 + random().nextInt(100);
|
||||
} else {
|
||||
|
|
|
@ -32,11 +32,11 @@ import java.util.concurrent.ConcurrentHashMap;
|
|||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.lucene.document.DimensionalLongField;
|
||||
import org.apache.lucene.document.LongPoint;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.NumericDocValuesField;
|
||||
import org.apache.lucene.search.DimensionalRangeQuery;
|
||||
import org.apache.lucene.search.PointRangeQuery;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.MatchAllDocsQuery;
|
||||
import org.apache.lucene.search.ScoreDoc;
|
||||
|
@ -681,7 +681,7 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
|
|||
Document newDoc = new Document();
|
||||
long value = Long.parseLong(oldDoc.get("text").split(" ")[1]);
|
||||
newDoc.add(new NumericDocValuesField("number", value));
|
||||
newDoc.add(new DimensionalLongField("number", value));
|
||||
newDoc.add(new LongPoint("number", value));
|
||||
w.addDocument(newDoc);
|
||||
}
|
||||
|
||||
|
@ -737,7 +737,7 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
|
|||
Document newDoc = new Document();
|
||||
long value = Long.parseLong(oldDoc.get("text").split(" ")[1]);
|
||||
newDoc.add(new NumericDocValuesField("number_" + newSchemaGen, value));
|
||||
newDoc.add(new DimensionalLongField("number", value));
|
||||
newDoc.add(new LongPoint("number", value));
|
||||
w.addDocument(newDoc);
|
||||
}
|
||||
} else {
|
||||
|
@ -832,7 +832,7 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
|
|||
Document newDoc = new Document();
|
||||
long value = Long.parseLong(oldDoc.get("text").split(" ")[1]);
|
||||
newDoc.add(new NumericDocValuesField("number", newSchemaGen*value));
|
||||
newDoc.add(new DimensionalLongField("number", value));
|
||||
newDoc.add(new LongPoint("number", value));
|
||||
w.addDocument(newDoc);
|
||||
}
|
||||
} else {
|
||||
|
@ -1168,7 +1168,7 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
|
|||
checkAllNumberDVs(r);
|
||||
IndexSearcher s = newSearcher(r);
|
||||
testNumericDVSort(s);
|
||||
testDimensionalRangeQuery(s);
|
||||
testPointRangeQuery(s);
|
||||
} finally {
|
||||
reindexer.mgr.release(r);
|
||||
}
|
||||
|
@ -1190,7 +1190,7 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
|
|||
checkAllNumberDVs(r);
|
||||
IndexSearcher s = newSearcher(r);
|
||||
testNumericDVSort(s);
|
||||
testDimensionalRangeQuery(s);
|
||||
testPointRangeQuery(s);
|
||||
} finally {
|
||||
reindexer.mgr.release(r);
|
||||
}
|
||||
|
@ -1209,7 +1209,7 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
|
|||
checkAllNumberDVs(r);
|
||||
IndexSearcher s = newSearcher(r);
|
||||
testNumericDVSort(s);
|
||||
testDimensionalRangeQuery(s);
|
||||
testPointRangeQuery(s);
|
||||
} finally {
|
||||
reindexer.mgr.release(r);
|
||||
}
|
||||
|
@ -1261,7 +1261,7 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
|
|||
checkAllNumberDVs(r);
|
||||
IndexSearcher s = newSearcher(r);
|
||||
testNumericDVSort(s);
|
||||
testDimensionalRangeQuery(s);
|
||||
testPointRangeQuery(s);
|
||||
} finally {
|
||||
reindexer.mgr.release(r);
|
||||
}
|
||||
|
@ -1340,7 +1340,7 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
private static void testDimensionalRangeQuery(IndexSearcher s) throws IOException {
|
||||
private static void testPointRangeQuery(IndexSearcher s) throws IOException {
|
||||
NumericDocValues numbers = MultiDocValues.getNumericValues(s.getIndexReader(), "number");
|
||||
for(int i=0;i<100;i++) {
|
||||
// Confirm we can range search by the new indexed (numeric) field:
|
||||
|
@ -1352,7 +1352,7 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
|
|||
max = x;
|
||||
}
|
||||
|
||||
TopDocs hits = s.search(DimensionalRangeQuery.new1DLongRange("number", min, true, max, true), 100);
|
||||
TopDocs hits = s.search(PointRangeQuery.new1DLongRange("number", min, true, max, true), 100);
|
||||
for(ScoreDoc scoreDoc : hits.scoreDocs) {
|
||||
long value = Long.parseLong(s.doc(scoreDoc.doc).get("text").split(" ")[1]);
|
||||
assertTrue(value >= min);
|
||||
|
|
|
@ -92,12 +92,12 @@ public class TestIndexableField extends LuceneTestCase {
|
|||
}
|
||||
|
||||
@Override
|
||||
public int dimensionCount() {
|
||||
public int pointDimensionCount() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int dimensionNumBytes() {
|
||||
public int pointNumBytes() {
|
||||
return 0;
|
||||
}
|
||||
};
|
||||
|
|
|
@ -26,20 +26,20 @@ import java.util.List;
|
|||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
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.PointFormat;
|
||||
import org.apache.lucene.codecs.PointReader;
|
||||
import org.apache.lucene.codecs.PointWriter;
|
||||
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.DimensionalBinaryField;
|
||||
import org.apache.lucene.document.DimensionalIntField;
|
||||
import org.apache.lucene.codecs.lucene60.Lucene60PointReader;
|
||||
import org.apache.lucene.codecs.lucene60.Lucene60PointWriter;
|
||||
import org.apache.lucene.document.BinaryPoint;
|
||||
import org.apache.lucene.document.IntPoint;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.NumericDocValuesField;
|
||||
import org.apache.lucene.document.StringField;
|
||||
import org.apache.lucene.index.DimensionalValues.IntersectVisitor;
|
||||
import org.apache.lucene.index.DimensionalValues.Relation;
|
||||
import org.apache.lucene.index.PointValues.IntersectVisitor;
|
||||
import org.apache.lucene.index.PointValues.Relation;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.MockDirectoryWrapper;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
@ -52,10 +52,10 @@ import org.apache.lucene.util.TestUtil;
|
|||
|
||||
// TODO: factor out a BaseTestDimensionFormat
|
||||
|
||||
public class TestDimensionalValues extends LuceneTestCase {
|
||||
public class TestPointValues extends LuceneTestCase {
|
||||
public void testBasic() throws Exception {
|
||||
Directory dir = getDirectory(20);
|
||||
// TODO: randomize codec once others support dimensional format
|
||||
// TODO: randomize codec once others support points format
|
||||
IndexWriterConfig iwc = newIndexWriterConfig();
|
||||
iwc.setMergePolicy(newLogMergePolicy());
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
|
@ -63,7 +63,7 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
for(int i=0;i<20;i++) {
|
||||
Document doc = new Document();
|
||||
NumericUtils.intToBytes(i, point, 0);
|
||||
doc.add(new DimensionalBinaryField("dim", point));
|
||||
doc.add(new BinaryPoint("dim", point));
|
||||
w.addDocument(doc);
|
||||
}
|
||||
w.forceMerge(1);
|
||||
|
@ -71,7 +71,7 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
|
||||
DirectoryReader r = DirectoryReader.open(dir);
|
||||
LeafReader sub = getOnlySegmentReader(r);
|
||||
DimensionalValues values = sub.getDimensionalValues();
|
||||
PointValues values = sub.getPointValues();
|
||||
|
||||
// Simple test: make sure intersect can visit every doc:
|
||||
BitSet seen = new BitSet();
|
||||
|
@ -102,7 +102,7 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
for(int i=0;i<20;i++) {
|
||||
Document doc = new Document();
|
||||
NumericUtils.intToBytes(i, point, 0);
|
||||
doc.add(new DimensionalBinaryField("dim", point));
|
||||
doc.add(new BinaryPoint("dim", point));
|
||||
w.addDocument(doc);
|
||||
if (i == 10) {
|
||||
w.commit();
|
||||
|
@ -113,7 +113,7 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
|
||||
DirectoryReader r = DirectoryReader.open(dir);
|
||||
LeafReader sub = getOnlySegmentReader(r);
|
||||
DimensionalValues values = sub.getDimensionalValues();
|
||||
PointValues values = sub.getPointValues();
|
||||
|
||||
// Simple test: make sure intersect can visit every doc:
|
||||
BitSet seen = new BitSet();
|
||||
|
@ -135,7 +135,7 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
IOUtils.close(r, dir);
|
||||
}
|
||||
|
||||
public void testAllDimensionalDocsDeletedInSegment() throws Exception {
|
||||
public void testAllPointDocsDeletedInSegment() throws Exception {
|
||||
Directory dir = getDirectory(20);
|
||||
IndexWriterConfig iwc = newIndexWriterConfig();
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
|
@ -143,7 +143,7 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
for(int i=0;i<10;i++) {
|
||||
Document doc = new Document();
|
||||
NumericUtils.intToBytes(i, point, 0);
|
||||
doc.add(new DimensionalBinaryField("dim", point));
|
||||
doc.add(new BinaryPoint("dim", point));
|
||||
doc.add(new NumericDocValuesField("id", i));
|
||||
doc.add(newStringField("x", "x", Field.Store.NO));
|
||||
w.addDocument(doc);
|
||||
|
@ -156,7 +156,7 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
w.close();
|
||||
DirectoryReader r = DirectoryReader.open(dir);
|
||||
assertEquals(1, r.numDocs());
|
||||
DimensionalValues values = MultiDimensionalValues.get(r);
|
||||
PointValues values = MultiPointValues.get(r);
|
||||
Bits liveDocs = MultiFields.getLiveDocs(r);
|
||||
NumericDocValues idValues = MultiDocValues.getNumericValues(r, "id");
|
||||
|
||||
|
@ -186,8 +186,8 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
/** Make sure we close open files, delete temp files, etc., on exception */
|
||||
public void testWithExceptions() throws Exception {
|
||||
int numDocs = atLeast(10000);
|
||||
int numBytesPerDim = TestUtil.nextInt(random(), 2, DimensionalValues.MAX_NUM_BYTES);
|
||||
int numDims = TestUtil.nextInt(random(), 1, DimensionalValues.MAX_DIMENSIONS);
|
||||
int numBytesPerDim = TestUtil.nextInt(random(), 2, PointValues.MAX_NUM_BYTES);
|
||||
int numDims = TestUtil.nextInt(random(), 1, PointValues.MAX_DIMENSIONS);
|
||||
|
||||
byte[][][] docValues = new byte[numDocs][][];
|
||||
|
||||
|
@ -246,8 +246,8 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
}
|
||||
|
||||
public void testMultiValued() throws Exception {
|
||||
int numBytesPerDim = TestUtil.nextInt(random(), 2, DimensionalValues.MAX_NUM_BYTES);
|
||||
int numDims = TestUtil.nextInt(random(), 1, DimensionalValues.MAX_DIMENSIONS);
|
||||
int numBytesPerDim = TestUtil.nextInt(random(), 2, PointValues.MAX_NUM_BYTES);
|
||||
int numDims = TestUtil.nextInt(random(), 1, PointValues.MAX_DIMENSIONS);
|
||||
|
||||
int numDocs = atLeast(1000);
|
||||
List<byte[][]> docValues = new ArrayList<>();
|
||||
|
@ -276,8 +276,8 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
}
|
||||
|
||||
public void testAllEqual() throws Exception {
|
||||
int numBytesPerDim = TestUtil.nextInt(random(), 2, DimensionalValues.MAX_NUM_BYTES);
|
||||
int numDims = TestUtil.nextInt(random(), 1, DimensionalValues.MAX_DIMENSIONS);
|
||||
int numBytesPerDim = TestUtil.nextInt(random(), 2, PointValues.MAX_NUM_BYTES);
|
||||
int numDims = TestUtil.nextInt(random(), 1, PointValues.MAX_DIMENSIONS);
|
||||
|
||||
int numDocs = atLeast(1000);
|
||||
byte[][][] docValues = new byte[numDocs][][];
|
||||
|
@ -299,8 +299,8 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
}
|
||||
|
||||
public void testOneDimEqual() throws Exception {
|
||||
int numBytesPerDim = TestUtil.nextInt(random(), 2, DimensionalValues.MAX_NUM_BYTES);
|
||||
int numDims = TestUtil.nextInt(random(), 1, DimensionalValues.MAX_DIMENSIONS);
|
||||
int numBytesPerDim = TestUtil.nextInt(random(), 2, PointValues.MAX_NUM_BYTES);
|
||||
int numDims = TestUtil.nextInt(random(), 1, PointValues.MAX_DIMENSIONS);
|
||||
|
||||
int numDocs = atLeast(1000);
|
||||
int theEqualDim = random().nextInt(numDims);
|
||||
|
@ -326,8 +326,8 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
|
||||
int numDocs = atLeast(1000);
|
||||
try (Directory dir = getDirectory(numDocs)) {
|
||||
int numBytesPerDim = TestUtil.nextInt(random(), 2, DimensionalValues.MAX_NUM_BYTES);
|
||||
int numDims = TestUtil.nextInt(random(), 1, DimensionalValues.MAX_DIMENSIONS);
|
||||
int numBytesPerDim = TestUtil.nextInt(random(), 2, PointValues.MAX_NUM_BYTES);
|
||||
int numDims = TestUtil.nextInt(random(), 1, PointValues.MAX_DIMENSIONS);
|
||||
IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random()));
|
||||
// We rely on docIDs not changing:
|
||||
iwc.setMergePolicy(newLogMergePolicy());
|
||||
|
@ -350,14 +350,14 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
}
|
||||
docs[docID] = values;
|
||||
Document doc = new Document();
|
||||
doc.add(new DimensionalBinaryField("field", bytes));
|
||||
doc.add(new BinaryPoint("field", bytes));
|
||||
w.addDocument(doc);
|
||||
}
|
||||
|
||||
DirectoryReader r = w.getReader();
|
||||
w.close();
|
||||
|
||||
DimensionalValues dimValues = MultiDimensionalValues.get(r);
|
||||
PointValues dimValues = MultiPointValues.get(r);
|
||||
|
||||
int iters = atLeast(100);
|
||||
for(int iter=0;iter<iters;iter++) {
|
||||
|
@ -459,8 +459,8 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
doTestRandomBinary(200000);
|
||||
}
|
||||
|
||||
// Suddenly add dimensional values to an existing field:
|
||||
public void testUpgradeFieldToDimensional() throws Exception {
|
||||
// Suddenly add points to an existing field:
|
||||
public void testUpgradeFieldToPoints() throws Exception {
|
||||
Directory dir = getDirectory(1);
|
||||
IndexWriterConfig iwc = newIndexWriterConfig();
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
|
@ -471,7 +471,7 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
|
||||
iwc = newIndexWriterConfig();
|
||||
w = new IndexWriter(dir, iwc);
|
||||
doc.add(new DimensionalBinaryField("dim", new byte[4]));
|
||||
doc.add(new BinaryPoint("dim", new byte[4]));
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
|
@ -483,13 +483,13 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new DimensionalBinaryField("dim", new byte[4]));
|
||||
doc.add(new DimensionalBinaryField("dim", new byte[4], new byte[4]));
|
||||
doc.add(new BinaryPoint("dim", new byte[4]));
|
||||
doc.add(new BinaryPoint("dim", new byte[4], new byte[4]));
|
||||
try {
|
||||
w.addDocument(doc);
|
||||
} catch (IllegalArgumentException iae) {
|
||||
// expected
|
||||
assertEquals("cannot change dimension count from 1 to 2 for field=\"dim\"", iae.getMessage());
|
||||
assertEquals("cannot change point dimension count from 1 to 2 for field=\"dim\"", iae.getMessage());
|
||||
}
|
||||
w.close();
|
||||
dir.close();
|
||||
|
@ -500,15 +500,15 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new DimensionalBinaryField("dim", new byte[4]));
|
||||
doc.add(new BinaryPoint("dim", new byte[4]));
|
||||
w.addDocument(doc);
|
||||
doc = new Document();
|
||||
doc.add(new DimensionalBinaryField("dim", new byte[4], new byte[4]));
|
||||
doc.add(new BinaryPoint("dim", new byte[4], new byte[4]));
|
||||
try {
|
||||
w.addDocument(doc);
|
||||
} catch (IllegalArgumentException iae) {
|
||||
// expected
|
||||
assertEquals("cannot change dimension count from 1 to 2 for field=\"dim\"", iae.getMessage());
|
||||
assertEquals("cannot change point dimension count from 1 to 2 for field=\"dim\"", iae.getMessage());
|
||||
}
|
||||
w.close();
|
||||
dir.close();
|
||||
|
@ -519,16 +519,16 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new DimensionalBinaryField("dim", new byte[4]));
|
||||
doc.add(new BinaryPoint("dim", new byte[4]));
|
||||
w.addDocument(doc);
|
||||
w.commit();
|
||||
doc = new Document();
|
||||
doc.add(new DimensionalBinaryField("dim", new byte[4], new byte[4]));
|
||||
doc.add(new BinaryPoint("dim", new byte[4], new byte[4]));
|
||||
try {
|
||||
w.addDocument(doc);
|
||||
} catch (IllegalArgumentException iae) {
|
||||
// expected
|
||||
assertEquals("cannot change dimension count from 1 to 2 for field=\"dim\"", iae.getMessage());
|
||||
assertEquals("cannot change point dimension count from 1 to 2 for field=\"dim\"", iae.getMessage());
|
||||
}
|
||||
w.close();
|
||||
dir.close();
|
||||
|
@ -539,18 +539,18 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new DimensionalBinaryField("dim", new byte[4]));
|
||||
doc.add(new BinaryPoint("dim", new byte[4]));
|
||||
w.addDocument(doc);
|
||||
w.close();
|
||||
iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
w = new IndexWriter(dir, iwc);
|
||||
doc = new Document();
|
||||
doc.add(new DimensionalBinaryField("dim", new byte[4], new byte[4]));
|
||||
doc.add(new BinaryPoint("dim", new byte[4], new byte[4]));
|
||||
try {
|
||||
w.addDocument(doc);
|
||||
} catch (IllegalArgumentException iae) {
|
||||
// expected
|
||||
assertEquals("cannot change dimension count from 1 to 2 for field=\"dim\"", iae.getMessage());
|
||||
assertEquals("cannot change point dimension count from 1 to 2 for field=\"dim\"", iae.getMessage());
|
||||
}
|
||||
w.close();
|
||||
dir.close();
|
||||
|
@ -561,7 +561,7 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new DimensionalBinaryField("dim", new byte[4]));
|
||||
doc.add(new BinaryPoint("dim", new byte[4]));
|
||||
w.addDocument(doc);
|
||||
w.close();
|
||||
|
||||
|
@ -569,12 +569,12 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
w = new IndexWriter(dir2, iwc);
|
||||
doc = new Document();
|
||||
doc.add(new DimensionalBinaryField("dim", new byte[4], new byte[4]));
|
||||
doc.add(new BinaryPoint("dim", new byte[4], new byte[4]));
|
||||
w.addDocument(doc);
|
||||
try {
|
||||
w.addIndexes(new Directory[] {dir});
|
||||
} catch (IllegalArgumentException iae) {
|
||||
assertEquals("cannot change dimension count from 2 to 1 for field=\"dim\"", iae.getMessage());
|
||||
assertEquals("cannot change point dimension count from 2 to 1 for field=\"dim\"", iae.getMessage());
|
||||
}
|
||||
IOUtils.close(w, dir, dir2);
|
||||
}
|
||||
|
@ -584,7 +584,7 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new DimensionalBinaryField("dim", new byte[4]));
|
||||
doc.add(new BinaryPoint("dim", new byte[4]));
|
||||
w.addDocument(doc);
|
||||
w.close();
|
||||
|
||||
|
@ -592,13 +592,13 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
w = new IndexWriter(dir2, iwc);
|
||||
doc = new Document();
|
||||
doc.add(new DimensionalBinaryField("dim", new byte[4], new byte[4]));
|
||||
doc.add(new BinaryPoint("dim", new byte[4], new byte[4]));
|
||||
w.addDocument(doc);
|
||||
DirectoryReader r = DirectoryReader.open(dir);
|
||||
try {
|
||||
w.addIndexes(new CodecReader[] {getOnlySegmentReader(r)});
|
||||
} catch (IllegalArgumentException iae) {
|
||||
assertEquals("cannot change dimension count from 2 to 1 for field=\"dim\"", iae.getMessage());
|
||||
assertEquals("cannot change point dimension count from 2 to 1 for field=\"dim\"", iae.getMessage());
|
||||
}
|
||||
IOUtils.close(r, w, dir, dir2);
|
||||
}
|
||||
|
@ -608,7 +608,7 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new DimensionalBinaryField("dim", new byte[4]));
|
||||
doc.add(new BinaryPoint("dim", new byte[4]));
|
||||
w.addDocument(doc);
|
||||
w.close();
|
||||
|
||||
|
@ -616,13 +616,13 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
w = new IndexWriter(dir2, iwc);
|
||||
doc = new Document();
|
||||
doc.add(new DimensionalBinaryField("dim", new byte[4], new byte[4]));
|
||||
doc.add(new BinaryPoint("dim", new byte[4], new byte[4]));
|
||||
w.addDocument(doc);
|
||||
DirectoryReader r = DirectoryReader.open(dir);
|
||||
try {
|
||||
TestUtil.addIndexesSlowly(w, r);
|
||||
} catch (IllegalArgumentException iae) {
|
||||
assertEquals("cannot change dimension count from 2 to 1 for field=\"dim\"", iae.getMessage());
|
||||
assertEquals("cannot change point dimension count from 2 to 1 for field=\"dim\"", iae.getMessage());
|
||||
}
|
||||
IOUtils.close(r, w, dir, dir2);
|
||||
}
|
||||
|
@ -632,13 +632,13 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new DimensionalBinaryField("dim", new byte[4]));
|
||||
doc.add(new DimensionalBinaryField("dim", new byte[6]));
|
||||
doc.add(new BinaryPoint("dim", new byte[4]));
|
||||
doc.add(new BinaryPoint("dim", new byte[6]));
|
||||
try {
|
||||
w.addDocument(doc);
|
||||
} catch (IllegalArgumentException iae) {
|
||||
// expected
|
||||
assertEquals("cannot change dimension numBytes from 4 to 6 for field=\"dim\"", iae.getMessage());
|
||||
assertEquals("cannot change point numBytes from 4 to 6 for field=\"dim\"", iae.getMessage());
|
||||
}
|
||||
w.close();
|
||||
dir.close();
|
||||
|
@ -649,15 +649,15 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new DimensionalBinaryField("dim", new byte[4]));
|
||||
doc.add(new BinaryPoint("dim", new byte[4]));
|
||||
w.addDocument(doc);
|
||||
doc = new Document();
|
||||
doc.add(new DimensionalBinaryField("dim", new byte[6]));
|
||||
doc.add(new BinaryPoint("dim", new byte[6]));
|
||||
try {
|
||||
w.addDocument(doc);
|
||||
} catch (IllegalArgumentException iae) {
|
||||
// expected
|
||||
assertEquals("cannot change dimension numBytes from 4 to 6 for field=\"dim\"", iae.getMessage());
|
||||
assertEquals("cannot change point numBytes from 4 to 6 for field=\"dim\"", iae.getMessage());
|
||||
}
|
||||
w.close();
|
||||
dir.close();
|
||||
|
@ -668,16 +668,16 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new DimensionalBinaryField("dim", new byte[4]));
|
||||
doc.add(new BinaryPoint("dim", new byte[4]));
|
||||
w.addDocument(doc);
|
||||
w.commit();
|
||||
doc = new Document();
|
||||
doc.add(new DimensionalBinaryField("dim", new byte[6]));
|
||||
doc.add(new BinaryPoint("dim", new byte[6]));
|
||||
try {
|
||||
w.addDocument(doc);
|
||||
} catch (IllegalArgumentException iae) {
|
||||
// expected
|
||||
assertEquals("cannot change dimension numBytes from 4 to 6 for field=\"dim\"", iae.getMessage());
|
||||
assertEquals("cannot change point numBytes from 4 to 6 for field=\"dim\"", iae.getMessage());
|
||||
}
|
||||
w.close();
|
||||
dir.close();
|
||||
|
@ -688,18 +688,18 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new DimensionalBinaryField("dim", new byte[4]));
|
||||
doc.add(new BinaryPoint("dim", new byte[4]));
|
||||
w.addDocument(doc);
|
||||
w.close();
|
||||
iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
w = new IndexWriter(dir, iwc);
|
||||
doc = new Document();
|
||||
doc.add(new DimensionalBinaryField("dim", new byte[6]));
|
||||
doc.add(new BinaryPoint("dim", new byte[6]));
|
||||
try {
|
||||
w.addDocument(doc);
|
||||
} catch (IllegalArgumentException iae) {
|
||||
// expected
|
||||
assertEquals("cannot change dimension numBytes from 4 to 6 for field=\"dim\"", iae.getMessage());
|
||||
assertEquals("cannot change point numBytes from 4 to 6 for field=\"dim\"", iae.getMessage());
|
||||
}
|
||||
w.close();
|
||||
dir.close();
|
||||
|
@ -710,7 +710,7 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new DimensionalBinaryField("dim", new byte[4]));
|
||||
doc.add(new BinaryPoint("dim", new byte[4]));
|
||||
w.addDocument(doc);
|
||||
w.close();
|
||||
|
||||
|
@ -718,12 +718,12 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
w = new IndexWriter(dir2, iwc);
|
||||
doc = new Document();
|
||||
doc.add(new DimensionalBinaryField("dim", new byte[6]));
|
||||
doc.add(new BinaryPoint("dim", new byte[6]));
|
||||
w.addDocument(doc);
|
||||
try {
|
||||
w.addIndexes(new Directory[] {dir});
|
||||
} catch (IllegalArgumentException iae) {
|
||||
assertEquals("cannot change dimension numBytes from 6 to 4 for field=\"dim\"", iae.getMessage());
|
||||
assertEquals("cannot change point numBytes from 6 to 4 for field=\"dim\"", iae.getMessage());
|
||||
}
|
||||
IOUtils.close(w, dir, dir2);
|
||||
}
|
||||
|
@ -733,7 +733,7 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new DimensionalBinaryField("dim", new byte[4]));
|
||||
doc.add(new BinaryPoint("dim", new byte[4]));
|
||||
w.addDocument(doc);
|
||||
w.close();
|
||||
|
||||
|
@ -741,13 +741,13 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
w = new IndexWriter(dir2, iwc);
|
||||
doc = new Document();
|
||||
doc.add(new DimensionalBinaryField("dim", new byte[6]));
|
||||
doc.add(new BinaryPoint("dim", new byte[6]));
|
||||
w.addDocument(doc);
|
||||
DirectoryReader r = DirectoryReader.open(dir);
|
||||
try {
|
||||
w.addIndexes(new CodecReader[] {getOnlySegmentReader(r)});
|
||||
} catch (IllegalArgumentException iae) {
|
||||
assertEquals("cannot change dimension numBytes from 6 to 4 for field=\"dim\"", iae.getMessage());
|
||||
assertEquals("cannot change point numBytes from 6 to 4 for field=\"dim\"", iae.getMessage());
|
||||
}
|
||||
IOUtils.close(r, w, dir, dir2);
|
||||
}
|
||||
|
@ -757,7 +757,7 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new DimensionalBinaryField("dim", new byte[4]));
|
||||
doc.add(new BinaryPoint("dim", new byte[4]));
|
||||
w.addDocument(doc);
|
||||
w.close();
|
||||
|
||||
|
@ -765,13 +765,13 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
w = new IndexWriter(dir2, iwc);
|
||||
doc = new Document();
|
||||
doc.add(new DimensionalBinaryField("dim", new byte[6]));
|
||||
doc.add(new BinaryPoint("dim", new byte[6]));
|
||||
w.addDocument(doc);
|
||||
DirectoryReader r = DirectoryReader.open(dir);
|
||||
try {
|
||||
TestUtil.addIndexesSlowly(w, r);
|
||||
} catch (IllegalArgumentException iae) {
|
||||
assertEquals("cannot change dimension numBytes from 6 to 4 for field=\"dim\"", iae.getMessage());
|
||||
assertEquals("cannot change point numBytes from 6 to 4 for field=\"dim\"", iae.getMessage());
|
||||
}
|
||||
IOUtils.close(r, w, dir, dir2);
|
||||
}
|
||||
|
@ -781,7 +781,7 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new DimensionalBinaryField("dim", new byte[DimensionalValues.MAX_NUM_BYTES+1]));
|
||||
doc.add(new BinaryPoint("dim", new byte[PointValues.MAX_NUM_BYTES+1]));
|
||||
try {
|
||||
w.addDocument(doc);
|
||||
fail("did not hit exception");
|
||||
|
@ -789,7 +789,7 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
// expected
|
||||
}
|
||||
doc = new Document();
|
||||
doc.add(new DimensionalIntField("dim", 17));
|
||||
doc.add(new IntPoint("dim", 17));
|
||||
w.addDocument(doc);
|
||||
w.close();
|
||||
dir.close();
|
||||
|
@ -800,11 +800,11 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
byte[][] values = new byte[DimensionalValues.MAX_DIMENSIONS+1][];
|
||||
byte[][] values = new byte[PointValues.MAX_DIMENSIONS+1][];
|
||||
for(int i=0;i<values.length;i++) {
|
||||
values[i] = new byte[4];
|
||||
}
|
||||
doc.add(new DimensionalBinaryField("dim", values));
|
||||
doc.add(new BinaryPoint("dim", values));
|
||||
try {
|
||||
w.addDocument(doc);
|
||||
fail("did not hit exception");
|
||||
|
@ -812,7 +812,7 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
// expected
|
||||
}
|
||||
doc = new Document();
|
||||
doc.add(new DimensionalIntField("dim", 17));
|
||||
doc.add(new IntPoint("dim", 17));
|
||||
w.addDocument(doc);
|
||||
w.close();
|
||||
dir.close();
|
||||
|
@ -820,8 +820,8 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
|
||||
private void doTestRandomBinary(int count) throws Exception {
|
||||
int numDocs = TestUtil.nextInt(random(), count, count*2);
|
||||
int numBytesPerDim = TestUtil.nextInt(random(), 2, DimensionalValues.MAX_NUM_BYTES);
|
||||
int numDims = TestUtil.nextInt(random(), 1, DimensionalValues.MAX_DIMENSIONS);
|
||||
int numBytesPerDim = TestUtil.nextInt(random(), 2, PointValues.MAX_NUM_BYTES);
|
||||
int numDims = TestUtil.nextInt(random(), 1, PointValues.MAX_DIMENSIONS);
|
||||
|
||||
byte[][][] docValues = new byte[numDocs][][];
|
||||
|
||||
|
@ -843,21 +843,21 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
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);
|
||||
System.out.println("TEST: using Lucene60PointFormat with maxPointsInLeafNode=" + maxPointsInLeafNode + " and maxMBSortInHeap=" + maxMBSortInHeap);
|
||||
}
|
||||
|
||||
return new FilterCodec("Lucene60", Codec.getDefault()) {
|
||||
@Override
|
||||
public DimensionalFormat dimensionalFormat() {
|
||||
return new DimensionalFormat() {
|
||||
public PointFormat pointFormat() {
|
||||
return new PointFormat() {
|
||||
@Override
|
||||
public DimensionalWriter fieldsWriter(SegmentWriteState writeState) throws IOException {
|
||||
return new Lucene60DimensionalWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap);
|
||||
public PointWriter fieldsWriter(SegmentWriteState writeState) throws IOException {
|
||||
return new Lucene60PointWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DimensionalReader fieldsReader(SegmentReadState readState) throws IOException {
|
||||
return new Lucene60DimensionalReader(readState);
|
||||
public PointReader fieldsReader(SegmentReadState readState) throws IOException {
|
||||
return new Lucene60PointReader(readState);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -931,7 +931,7 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
}
|
||||
|
||||
// 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:
|
||||
// addIndexes to bring the indexed point values to the main directory:
|
||||
Directory saveDir;
|
||||
RandomIndexWriter saveW;
|
||||
int addIndexesAt;
|
||||
|
@ -981,7 +981,7 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", id));
|
||||
}
|
||||
doc.add(new DimensionalBinaryField("field", docValues[ord]));
|
||||
doc.add(new BinaryPoint("field", docValues[ord]));
|
||||
lastID = id;
|
||||
|
||||
if (random().nextInt(30) == 17) {
|
||||
|
@ -999,7 +999,7 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
if (random().nextInt(30) == 17) {
|
||||
// randomly index some documents with this field, but we will delete them:
|
||||
Document xdoc = new Document();
|
||||
xdoc.add(new DimensionalBinaryField("field", docValues[ord]));
|
||||
xdoc.add(new BinaryPoint("field", docValues[ord]));
|
||||
xdoc.add(new StringField("nukeme", "yes", Field.Store.NO));
|
||||
if (useRealWriter) {
|
||||
w.w.addDocument(xdoc);
|
||||
|
@ -1051,7 +1051,7 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
System.out.println("TEST: reader=" + r);
|
||||
}
|
||||
|
||||
DimensionalValues dimValues = MultiDimensionalValues.get(r);
|
||||
PointValues dimValues = MultiPointValues.get(r);
|
||||
if (VERBOSE) {
|
||||
System.out.println(" dimValues=" + dimValues);
|
||||
}
|
||||
|
@ -1102,7 +1102,7 @@ public class TestDimensionalValues extends LuceneTestCase {
|
|||
|
||||
final BitSet hits = new BitSet();
|
||||
|
||||
dimValues.intersect("field", new DimensionalValues.IntersectVisitor() {
|
||||
dimValues.intersect("field", new PointValues.IntersectVisitor() {
|
||||
@Override
|
||||
public void visit(int docID) {
|
||||
if (liveDocs == null || liveDocs.get(docID)) {
|
|
@ -21,16 +21,12 @@ import java.io.IOException;
|
|||
import java.util.Random;
|
||||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.document.DimensionalDoubleField;
|
||||
import org.apache.lucene.document.DimensionalFloatField;
|
||||
import org.apache.lucene.document.DimensionalIntField;
|
||||
import org.apache.lucene.document.DimensionalLongField;
|
||||
import org.apache.lucene.document.DoublePoint;
|
||||
import org.apache.lucene.document.FloatPoint;
|
||||
import org.apache.lucene.document.IntPoint;
|
||||
import org.apache.lucene.document.LongPoint;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.LegacyDoubleField;
|
||||
import org.apache.lucene.document.LegacyFloatField;
|
||||
import org.apache.lucene.document.LegacyIntField;
|
||||
import org.apache.lucene.document.LegacyLongField;
|
||||
import org.apache.lucene.document.NumericDocValuesField;
|
||||
import org.apache.lucene.document.SortedDocValuesField;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
|
@ -127,13 +123,13 @@ public class BaseTestRangeFilter extends LuceneTestCase {
|
|||
Document doc = new Document();
|
||||
Field idField = newStringField(random, "id", "", Field.Store.YES);
|
||||
Field idDVField = new SortedDocValuesField("id", new BytesRef());
|
||||
Field intIdField = new DimensionalIntField("id_int", 0);
|
||||
Field intIdField = new IntPoint("id_int", 0);
|
||||
Field intDVField = new NumericDocValuesField("id_int", 0);
|
||||
Field floatIdField = new DimensionalFloatField("id_float", 0);
|
||||
Field floatIdField = new FloatPoint("id_float", 0);
|
||||
Field floatDVField = new NumericDocValuesField("id_float", 0);
|
||||
Field longIdField = new DimensionalLongField("id_long", 0);
|
||||
Field longIdField = new LongPoint("id_long", 0);
|
||||
Field longDVField = new NumericDocValuesField("id_long", 0);
|
||||
Field doubleIdField = new DimensionalDoubleField("id_double", 0);
|
||||
Field doubleIdField = new DoublePoint("id_double", 0);
|
||||
Field doubleDVField = new NumericDocValuesField("id_double", 0);
|
||||
Field randField = newStringField(random, "rand", "", Field.Store.YES);
|
||||
Field randDVField = new SortedDocValuesField("rand", new BytesRef());
|
||||
|
|
|
@ -27,26 +27,30 @@ import java.util.concurrent.CountDownLatch;
|
|||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
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.DimensionalBinaryField;
|
||||
import org.apache.lucene.document.DimensionalLongField;
|
||||
import org.apache.lucene.codecs.PointFormat;
|
||||
import org.apache.lucene.codecs.PointReader;
|
||||
import org.apache.lucene.codecs.PointWriter;
|
||||
import org.apache.lucene.codecs.lucene60.Lucene60PointReader;
|
||||
import org.apache.lucene.codecs.lucene60.Lucene60PointWriter;
|
||||
import org.apache.lucene.document.BinaryPoint;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.DoublePoint;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.FloatPoint;
|
||||
import org.apache.lucene.document.IntPoint;
|
||||
import org.apache.lucene.document.LongPoint;
|
||||
import org.apache.lucene.document.NumericDocValuesField;
|
||||
import org.apache.lucene.document.SortedNumericDocValuesField;
|
||||
import org.apache.lucene.document.StringField;
|
||||
import org.apache.lucene.index.DimensionalValues;
|
||||
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.LeafReaderContext;
|
||||
import org.apache.lucene.index.MultiDocValues;
|
||||
import org.apache.lucene.index.NumericDocValues;
|
||||
import org.apache.lucene.index.PointValues;
|
||||
import org.apache.lucene.index.RandomIndexWriter;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
|
@ -60,7 +64,7 @@ import org.apache.lucene.util.NumericUtils;
|
|||
import org.apache.lucene.util.TestUtil;
|
||||
import org.junit.BeforeClass;
|
||||
|
||||
public class TestDimensionalRangeQuery extends LuceneTestCase {
|
||||
public class TestPointQueries extends LuceneTestCase {
|
||||
|
||||
// Controls what range of values we randomly generate, so we sometimes test narrow ranges:
|
||||
static long valueMid;
|
||||
|
@ -225,10 +229,10 @@ public class TestDimensionalRangeQuery extends LuceneTestCase {
|
|||
}
|
||||
|
||||
if (missing.get(id) == false) {
|
||||
doc.add(new DimensionalLongField("sn_value", values[id]));
|
||||
doc.add(new LongPoint("sn_value", values[id]));
|
||||
byte[] bytes = new byte[8];
|
||||
NumericUtils.longToBytes(values[id], bytes, 0);
|
||||
doc.add(new DimensionalBinaryField("ss_value", bytes));
|
||||
doc.add(new BinaryPoint("ss_value", bytes));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -301,7 +305,7 @@ public class TestDimensionalRangeQuery extends LuceneTestCase {
|
|||
}
|
||||
|
||||
if (random().nextBoolean()) {
|
||||
query = DimensionalRangeQuery.new1DLongRange("sn_value", lower, includeLower, upper, includeUpper);
|
||||
query = PointRangeQuery.new1DLongRange("sn_value", lower, includeLower, upper, includeUpper);
|
||||
} else {
|
||||
byte[] lowerBytes;
|
||||
if (lower == null) {
|
||||
|
@ -317,7 +321,7 @@ public class TestDimensionalRangeQuery extends LuceneTestCase {
|
|||
upperBytes = new byte[8];
|
||||
NumericUtils.longToBytes(upper, upperBytes, 0);
|
||||
}
|
||||
query = DimensionalRangeQuery.new1DBinaryRange("ss_value", lowerBytes, includeLower, upperBytes, includeUpper);
|
||||
query = PointRangeQuery.new1DBinaryRange("ss_value", lowerBytes, includeLower, upperBytes, includeUpper);
|
||||
}
|
||||
|
||||
if (VERBOSE) {
|
||||
|
@ -386,8 +390,8 @@ public class TestDimensionalRangeQuery extends LuceneTestCase {
|
|||
|
||||
private void doTestRandomBinary(int count) throws Exception {
|
||||
int numValues = TestUtil.nextInt(random(), count, count*2);
|
||||
int numBytesPerDim = TestUtil.nextInt(random(), 2, DimensionalValues.MAX_NUM_BYTES);
|
||||
int numDims = TestUtil.nextInt(random(), 1, DimensionalValues.MAX_DIMENSIONS);
|
||||
int numBytesPerDim = TestUtil.nextInt(random(), 2, PointValues.MAX_NUM_BYTES);
|
||||
int numDims = TestUtil.nextInt(random(), 1, PointValues.MAX_DIMENSIONS);
|
||||
|
||||
int sameValuePct = random().nextInt(100);
|
||||
|
||||
|
@ -435,7 +439,7 @@ public class TestDimensionalRangeQuery extends LuceneTestCase {
|
|||
|
||||
Directory dir;
|
||||
if (docValues.length > 100000) {
|
||||
dir = noVirusChecker(newFSDirectory(createTempDir("TestDimensionalRangeQuery")));
|
||||
dir = noVirusChecker(newFSDirectory(createTempDir("TestPointRangeQuery")));
|
||||
} else {
|
||||
dir = getDirectory();
|
||||
}
|
||||
|
@ -489,7 +493,7 @@ public class TestDimensionalRangeQuery extends LuceneTestCase {
|
|||
}
|
||||
|
||||
if (missing.get(id) == false) {
|
||||
doc.add(new DimensionalBinaryField("value", docValues[ord]));
|
||||
doc.add(new BinaryPoint("value", docValues[ord]));
|
||||
if (VERBOSE) {
|
||||
System.out.println("id=" + id);
|
||||
for(int dim=0;dim<numDims;dim++) {
|
||||
|
@ -583,7 +587,7 @@ public class TestDimensionalRangeQuery extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
Query query = new DimensionalRangeQuery("value", lower, includeLower, upper, includeUpper);
|
||||
Query query = new PointRangeQuery("value", lower, includeLower, upper, includeUpper);
|
||||
|
||||
if (VERBOSE) {
|
||||
System.out.println(Thread.currentThread().getName() + ": using query: " + query);
|
||||
|
@ -722,20 +726,20 @@ public class TestDimensionalRangeQuery extends LuceneTestCase {
|
|||
iwc.setCodec(getCodec());
|
||||
RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new DimensionalLongField("value", Long.MIN_VALUE));
|
||||
doc.add(new LongPoint("value", Long.MIN_VALUE));
|
||||
w.addDocument(doc);
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new DimensionalLongField("value", Long.MAX_VALUE));
|
||||
doc.add(new LongPoint("value", Long.MAX_VALUE));
|
||||
w.addDocument(doc);
|
||||
|
||||
IndexReader r = w.getReader();
|
||||
|
||||
IndexSearcher s = newSearcher(r);
|
||||
|
||||
assertEquals(1, s.count(DimensionalRangeQuery.new1DLongRange("value", Long.MIN_VALUE, true, 0L, true)));
|
||||
assertEquals(1, s.count(DimensionalRangeQuery.new1DLongRange("value", 0L, true, Long.MAX_VALUE, true)));
|
||||
assertEquals(2, s.count(DimensionalRangeQuery.new1DLongRange("value", Long.MIN_VALUE, true, Long.MAX_VALUE, true)));
|
||||
assertEquals(1, s.count(PointRangeQuery.new1DLongRange("value", Long.MIN_VALUE, true, 0L, true)));
|
||||
assertEquals(1, s.count(PointRangeQuery.new1DLongRange("value", 0L, true, Long.MAX_VALUE, true)));
|
||||
assertEquals(2, s.count(PointRangeQuery.new1DLongRange("value", Long.MIN_VALUE, true, Long.MAX_VALUE, true)));
|
||||
|
||||
IOUtils.close(r, w, dir);
|
||||
}
|
||||
|
@ -761,61 +765,61 @@ public class TestDimensionalRangeQuery extends LuceneTestCase {
|
|||
iwc.setCodec(getCodec());
|
||||
RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new DimensionalBinaryField("value", toUTF8("abc")));
|
||||
doc.add(new BinaryPoint("value", toUTF8("abc")));
|
||||
w.addDocument(doc);
|
||||
doc = new Document();
|
||||
doc.add(new DimensionalBinaryField("value", toUTF8("def")));
|
||||
doc.add(new BinaryPoint("value", toUTF8("def")));
|
||||
w.addDocument(doc);
|
||||
|
||||
IndexReader r = w.getReader();
|
||||
|
||||
IndexSearcher s = newSearcher(r);
|
||||
|
||||
assertEquals(1, s.count(DimensionalRangeQuery.new1DBinaryRange("value",
|
||||
toUTF8("aaa"),
|
||||
true,
|
||||
toUTF8("bbb"),
|
||||
true)));
|
||||
assertEquals(1, s.count(DimensionalRangeQuery.new1DBinaryRange("value",
|
||||
toUTF8("c", 3),
|
||||
true,
|
||||
toUTF8("e", 3),
|
||||
true)));
|
||||
assertEquals(2, s.count(DimensionalRangeQuery.new1DBinaryRange("value",
|
||||
toUTF8("a", 3),
|
||||
true,
|
||||
toUTF8("z", 3),
|
||||
true)));
|
||||
assertEquals(1, s.count(DimensionalRangeQuery.new1DBinaryRange("value",
|
||||
null,
|
||||
true,
|
||||
toUTF8("abc"),
|
||||
true)));
|
||||
assertEquals(1, s.count(DimensionalRangeQuery.new1DBinaryRange("value",
|
||||
toUTF8("a", 3),
|
||||
true,
|
||||
toUTF8("abc"),
|
||||
true)));
|
||||
assertEquals(0, s.count(DimensionalRangeQuery.new1DBinaryRange("value",
|
||||
toUTF8("a", 3),
|
||||
true,
|
||||
toUTF8("abc"),
|
||||
false)));
|
||||
assertEquals(1, s.count(DimensionalRangeQuery.new1DBinaryRange("value",
|
||||
toUTF8("def"),
|
||||
true,
|
||||
null,
|
||||
false)));
|
||||
assertEquals(1, s.count(DimensionalRangeQuery.new1DBinaryRange("value",
|
||||
toUTF8(("def")),
|
||||
true,
|
||||
toUTF8("z", 3),
|
||||
true)));
|
||||
assertEquals(0, s.count(DimensionalRangeQuery.new1DBinaryRange("value",
|
||||
toUTF8("def"),
|
||||
false,
|
||||
toUTF8("z", 3),
|
||||
true)));
|
||||
assertEquals(1, s.count(PointRangeQuery.new1DBinaryRange("value",
|
||||
toUTF8("aaa"),
|
||||
true,
|
||||
toUTF8("bbb"),
|
||||
true)));
|
||||
assertEquals(1, s.count(PointRangeQuery.new1DBinaryRange("value",
|
||||
toUTF8("c", 3),
|
||||
true,
|
||||
toUTF8("e", 3),
|
||||
true)));
|
||||
assertEquals(2, s.count(PointRangeQuery.new1DBinaryRange("value",
|
||||
toUTF8("a", 3),
|
||||
true,
|
||||
toUTF8("z", 3),
|
||||
true)));
|
||||
assertEquals(1, s.count(PointRangeQuery.new1DBinaryRange("value",
|
||||
null,
|
||||
true,
|
||||
toUTF8("abc"),
|
||||
true)));
|
||||
assertEquals(1, s.count(PointRangeQuery.new1DBinaryRange("value",
|
||||
toUTF8("a", 3),
|
||||
true,
|
||||
toUTF8("abc"),
|
||||
true)));
|
||||
assertEquals(0, s.count(PointRangeQuery.new1DBinaryRange("value",
|
||||
toUTF8("a", 3),
|
||||
true,
|
||||
toUTF8("abc"),
|
||||
false)));
|
||||
assertEquals(1, s.count(PointRangeQuery.new1DBinaryRange("value",
|
||||
toUTF8("def"),
|
||||
true,
|
||||
null,
|
||||
false)));
|
||||
assertEquals(1, s.count(PointRangeQuery.new1DBinaryRange("value",
|
||||
toUTF8(("def")),
|
||||
true,
|
||||
toUTF8("z", 3),
|
||||
true)));
|
||||
assertEquals(0, s.count(PointRangeQuery.new1DBinaryRange("value",
|
||||
toUTF8("def"),
|
||||
false,
|
||||
toUTF8("z", 3),
|
||||
true)));
|
||||
|
||||
IOUtils.close(r, w, dir);
|
||||
}
|
||||
|
@ -826,22 +830,22 @@ public class TestDimensionalRangeQuery extends LuceneTestCase {
|
|||
iwc.setCodec(getCodec());
|
||||
RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new DimensionalLongField("value", Long.MIN_VALUE));
|
||||
doc.add(new LongPoint("value", Long.MIN_VALUE));
|
||||
w.addDocument(doc);
|
||||
doc = new Document();
|
||||
doc.add(new DimensionalLongField("value", Long.MAX_VALUE));
|
||||
doc.add(new LongPoint("value", Long.MAX_VALUE));
|
||||
w.addDocument(doc);
|
||||
|
||||
IndexReader r = w.getReader();
|
||||
|
||||
IndexSearcher s = newSearcher(r);
|
||||
|
||||
assertEquals(2, s.count(DimensionalRangeQuery.new1DLongRange("value", Long.MIN_VALUE, true, Long.MAX_VALUE, true)));
|
||||
assertEquals(1, s.count(DimensionalRangeQuery.new1DLongRange("value", Long.MIN_VALUE, true, Long.MAX_VALUE, false)));
|
||||
assertEquals(1, s.count(DimensionalRangeQuery.new1DLongRange("value", Long.MIN_VALUE, false, Long.MAX_VALUE, true)));
|
||||
assertEquals(0, s.count(DimensionalRangeQuery.new1DLongRange("value", Long.MIN_VALUE, false, Long.MAX_VALUE, false)));
|
||||
assertEquals(2, s.count(PointRangeQuery.new1DLongRange("value", Long.MIN_VALUE, true, Long.MAX_VALUE, true)));
|
||||
assertEquals(1, s.count(PointRangeQuery.new1DLongRange("value", Long.MIN_VALUE, true, Long.MAX_VALUE, false)));
|
||||
assertEquals(1, s.count(PointRangeQuery.new1DLongRange("value", Long.MIN_VALUE, false, Long.MAX_VALUE, true)));
|
||||
assertEquals(0, s.count(PointRangeQuery.new1DLongRange("value", Long.MIN_VALUE, false, Long.MAX_VALUE, false)));
|
||||
|
||||
assertEquals(2, s.count(DimensionalRangeQuery.new1DBinaryRange("value", (byte[]) null, true, null, true)));
|
||||
assertEquals(2, s.count(PointRangeQuery.new1DBinaryRange("value", (byte[]) null, true, null, true)));
|
||||
|
||||
IOUtils.close(r, w, dir);
|
||||
}
|
||||
|
@ -852,10 +856,10 @@ public class TestDimensionalRangeQuery extends LuceneTestCase {
|
|||
iwc.setCodec(getCodec());
|
||||
RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new DimensionalLongField("value", Long.MIN_VALUE));
|
||||
doc.add(new LongPoint("value", Long.MIN_VALUE));
|
||||
w.addDocument(doc);
|
||||
doc = new Document();
|
||||
doc.add(new DimensionalLongField("value", Long.MAX_VALUE));
|
||||
doc.add(new LongPoint("value", Long.MAX_VALUE));
|
||||
w.addDocument(doc);
|
||||
|
||||
IndexReader r = w.getReader();
|
||||
|
@ -863,12 +867,12 @@ public class TestDimensionalRangeQuery extends LuceneTestCase {
|
|||
// We can't wrap with "exotic" readers because the query must see the RangeTreeDVFormat:
|
||||
IndexSearcher s = newSearcher(r, false);
|
||||
|
||||
assertEquals(2, s.count(DimensionalRangeQuery.new1DLongRange("value", Long.MIN_VALUE, true, Long.MAX_VALUE, true)));
|
||||
assertEquals(1, s.count(DimensionalRangeQuery.new1DLongRange("value", Long.MIN_VALUE, true, Long.MAX_VALUE, false)));
|
||||
assertEquals(1, s.count(DimensionalRangeQuery.new1DLongRange("value", Long.MIN_VALUE, false, Long.MAX_VALUE, true)));
|
||||
assertEquals(0, s.count(DimensionalRangeQuery.new1DLongRange("value", Long.MIN_VALUE, false, Long.MAX_VALUE, false)));
|
||||
assertEquals(2, s.count(PointRangeQuery.new1DLongRange("value", Long.MIN_VALUE, true, Long.MAX_VALUE, true)));
|
||||
assertEquals(1, s.count(PointRangeQuery.new1DLongRange("value", Long.MIN_VALUE, true, Long.MAX_VALUE, false)));
|
||||
assertEquals(1, s.count(PointRangeQuery.new1DLongRange("value", Long.MIN_VALUE, false, Long.MAX_VALUE, true)));
|
||||
assertEquals(0, s.count(PointRangeQuery.new1DLongRange("value", Long.MIN_VALUE, false, Long.MAX_VALUE, false)));
|
||||
|
||||
assertEquals(2, s.count(DimensionalRangeQuery.new1DLongRange("value", (Long) null, true, null, true)));
|
||||
assertEquals(2, s.count(PointRangeQuery.new1DLongRange("value", (Long) null, true, null, true)));
|
||||
|
||||
IOUtils.close(r, w, dir);
|
||||
}
|
||||
|
@ -879,18 +883,18 @@ public class TestDimensionalRangeQuery extends LuceneTestCase {
|
|||
iwc.setCodec(getCodec());
|
||||
RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new DimensionalBinaryField("value", toUTF8("a")));
|
||||
doc.add(new BinaryPoint("value", toUTF8("a")));
|
||||
w.addDocument(doc);
|
||||
doc = new Document();
|
||||
doc.add(new DimensionalBinaryField("value", toUTF8("z")));
|
||||
doc.add(new BinaryPoint("value", toUTF8("z")));
|
||||
w.addDocument(doc);
|
||||
|
||||
IndexReader r = w.getReader();
|
||||
|
||||
IndexSearcher s = newSearcher(r);
|
||||
assertEquals(0, s.count(DimensionalRangeQuery.new1DBinaryRange("value", toUTF8("m"), true, toUTF8("n"), false)));
|
||||
assertEquals(0, s.count(PointRangeQuery.new1DBinaryRange("value", toUTF8("m"), true, toUTF8("n"), false)));
|
||||
|
||||
assertEquals(2, s.count(DimensionalRangeQuery.new1DBinaryRange("value", (byte[]) null, true, null, true)));
|
||||
assertEquals(2, s.count(PointRangeQuery.new1DBinaryRange("value", (byte[]) null, true, null, true)));
|
||||
|
||||
IOUtils.close(r, w, dir);
|
||||
}
|
||||
|
@ -910,7 +914,7 @@ public class TestDimensionalRangeQuery extends LuceneTestCase {
|
|||
IndexReader r = w.getReader();
|
||||
|
||||
IndexSearcher s = new IndexSearcher(r);
|
||||
assertEquals(0, s.count(DimensionalRangeQuery.new1DLongRange("value", 17L, true, 13L, false)));
|
||||
assertEquals(0, s.count(PointRangeQuery.new1DLongRange("value", 17L, true, 13L, false)));
|
||||
|
||||
IOUtils.close(r, w, dir);
|
||||
}
|
||||
|
@ -925,7 +929,7 @@ public class TestDimensionalRangeQuery extends LuceneTestCase {
|
|||
IndexReader r = w.getReader();
|
||||
|
||||
IndexSearcher s = newSearcher(r);
|
||||
assertEquals(0, s.count(DimensionalRangeQuery.new1DLongRange("value", 17L, true, 13L, false)));
|
||||
assertEquals(0, s.count(PointRangeQuery.new1DLongRange("value", 17L, true, 13L, false)));
|
||||
|
||||
IOUtils.close(r, w, dir);
|
||||
}
|
||||
|
@ -936,7 +940,7 @@ public class TestDimensionalRangeQuery extends LuceneTestCase {
|
|||
iwc.setCodec(getCodec());
|
||||
RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new DimensionalLongField("value", Long.MIN_VALUE));
|
||||
doc.add(new LongPoint("value", Long.MIN_VALUE));
|
||||
w.addDocument(doc);
|
||||
|
||||
IndexReader r = w.getReader();
|
||||
|
@ -945,7 +949,7 @@ public class TestDimensionalRangeQuery extends LuceneTestCase {
|
|||
IndexSearcher s = new IndexSearcher(r);
|
||||
byte[][] point = new byte[2][];
|
||||
try {
|
||||
s.count(new DimensionalRangeQuery("value", point, new boolean[] {true, true}, point, new boolean[] {true, true}));
|
||||
s.count(new PointRangeQuery("value", point, new boolean[] {true, true}, point, new boolean[] {true, true}));
|
||||
} catch (IllegalArgumentException iae) {
|
||||
assertEquals("field=\"value\" was indexed with numDims=1 but this query has numDims=2", iae.getMessage());
|
||||
}
|
||||
|
@ -959,7 +963,7 @@ public class TestDimensionalRangeQuery extends LuceneTestCase {
|
|||
iwc.setCodec(getCodec());
|
||||
RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new DimensionalLongField("value", Long.MIN_VALUE));
|
||||
doc.add(new LongPoint("value", Long.MIN_VALUE));
|
||||
w.addDocument(doc);
|
||||
|
||||
IndexReader r = w.getReader();
|
||||
|
@ -969,7 +973,7 @@ public class TestDimensionalRangeQuery extends LuceneTestCase {
|
|||
byte[][] point = new byte[1][];
|
||||
point[0] = new byte[10];
|
||||
try {
|
||||
s.count(new DimensionalRangeQuery("value", point, new boolean[] {true}, point, new boolean[] {true}));
|
||||
s.count(new PointRangeQuery("value", point, new boolean[] {true}, point, new boolean[] {true}));
|
||||
} catch (IllegalArgumentException iae) {
|
||||
assertEquals("field=\"value\" was indexed with bytesPerDim=8 but this query has bytesPerDim=10", iae.getMessage());
|
||||
}
|
||||
|
@ -977,14 +981,14 @@ public class TestDimensionalRangeQuery extends LuceneTestCase {
|
|||
IOUtils.close(r, w, dir);
|
||||
}
|
||||
|
||||
public void testAllDimensionalDocsWereDeletedAndThenMergedAgain() throws Exception {
|
||||
public void testAllPointDocsWereDeletedAndThenMergedAgain() throws Exception {
|
||||
Directory dir = getDirectory();
|
||||
IndexWriterConfig iwc = newIndexWriterConfig();
|
||||
iwc.setCodec(getCodec());
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new StringField("id", "0", Field.Store.NO));
|
||||
doc.add(new DimensionalLongField("value", 0L));
|
||||
doc.add(new LongPoint("value", 0L));
|
||||
w.addDocument(doc);
|
||||
|
||||
// Add document that won't be deleted to avoid IW dropping
|
||||
|
@ -995,7 +999,7 @@ public class TestDimensionalRangeQuery extends LuceneTestCase {
|
|||
// Need another segment so we invoke BKDWriter.merge
|
||||
doc = new Document();
|
||||
doc.add(new StringField("id", "0", Field.Store.NO));
|
||||
doc.add(new DimensionalLongField("value", 0L));
|
||||
doc.add(new LongPoint("value", 0L));
|
||||
w.addDocument(doc);
|
||||
w.addDocument(new Document());
|
||||
|
||||
|
@ -1004,7 +1008,7 @@ public class TestDimensionalRangeQuery extends LuceneTestCase {
|
|||
|
||||
doc = new Document();
|
||||
doc.add(new StringField("id", "0", Field.Store.NO));
|
||||
doc.add(new DimensionalLongField("value", 0L));
|
||||
doc.add(new LongPoint("value", 0L));
|
||||
w.addDocument(doc);
|
||||
w.addDocument(new Document());
|
||||
|
||||
|
@ -1030,21 +1034,21 @@ public class TestDimensionalRangeQuery extends LuceneTestCase {
|
|||
int maxPointsInLeafNode = TestUtil.nextInt(random(), 16, 2048);
|
||||
double maxMBSortInHeap = 3.0 + (3*random().nextDouble());
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: using Lucene60DimensionalFormat with maxPointsInLeafNode=" + maxPointsInLeafNode + " and maxMBSortInHeap=" + maxMBSortInHeap);
|
||||
System.out.println("TEST: using Lucene60PointFormat with maxPointsInLeafNode=" + maxPointsInLeafNode + " and maxMBSortInHeap=" + maxMBSortInHeap);
|
||||
}
|
||||
|
||||
return new FilterCodec("Lucene60", Codec.getDefault()) {
|
||||
@Override
|
||||
public DimensionalFormat dimensionalFormat() {
|
||||
return new DimensionalFormat() {
|
||||
public PointFormat pointFormat() {
|
||||
return new PointFormat() {
|
||||
@Override
|
||||
public DimensionalWriter fieldsWriter(SegmentWriteState writeState) throws IOException {
|
||||
return new Lucene60DimensionalWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap);
|
||||
public PointWriter fieldsWriter(SegmentWriteState writeState) throws IOException {
|
||||
return new Lucene60PointWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DimensionalReader fieldsReader(SegmentReadState readState) throws IOException {
|
||||
return new Lucene60DimensionalReader(readState);
|
||||
public PointReader fieldsReader(SegmentReadState readState) throws IOException {
|
||||
return new Lucene60PointReader(readState);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -1053,4 +1057,44 @@ public class TestDimensionalRangeQuery extends LuceneTestCase {
|
|||
return Codec.getDefault();
|
||||
}
|
||||
}
|
||||
|
||||
public void testExactPointQuery() throws Exception {
|
||||
Directory dir = getDirectory();
|
||||
IndexWriterConfig iwc = newIndexWriterConfig();
|
||||
iwc.setCodec(getCodec());
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
|
||||
Document doc = new Document();
|
||||
doc.add(new LongPoint("long", 5L));
|
||||
w.addDocument(doc);
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new IntPoint("int", 42));
|
||||
w.addDocument(doc);
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new FloatPoint("float", 2.0f));
|
||||
w.addDocument(doc);
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new DoublePoint("double", 1.0));
|
||||
w.addDocument(doc);
|
||||
|
||||
IndexReader r = DirectoryReader.open(w);
|
||||
IndexSearcher s = newSearcher(r);
|
||||
assertEquals(1, s.count(ExactPointQuery.new1DIntExact("int", 42)));
|
||||
assertEquals(0, s.count(ExactPointQuery.new1DIntExact("int", 41)));
|
||||
|
||||
assertEquals(1, s.count(ExactPointQuery.new1DLongExact("long", 5L)));
|
||||
assertEquals(0, s.count(ExactPointQuery.new1DLongExact("long", -1L)));
|
||||
|
||||
assertEquals(1, s.count(ExactPointQuery.new1DFloatExact("float", 2.0f)));
|
||||
assertEquals(0, s.count(ExactPointQuery.new1DFloatExact("float", 1.0f)));
|
||||
|
||||
assertEquals(1, s.count(ExactPointQuery.new1DDoubleExact("double", 1.0)));
|
||||
assertEquals(0, s.count(ExactPointQuery.new1DDoubleExact("double", 2.0)));
|
||||
w.close();
|
||||
r.close();
|
||||
dir.close();
|
||||
}
|
||||
}
|
|
@ -26,7 +26,7 @@ public class TestUsageTrackingFilterCachingPolicy extends LuceneTestCase {
|
|||
|
||||
public void testCostlyFilter() {
|
||||
assertTrue(UsageTrackingQueryCachingPolicy.isCostly(new PrefixQuery(new Term("field", "prefix"))));
|
||||
assertTrue(UsageTrackingQueryCachingPolicy.isCostly(DimensionalRangeQuery.new1DIntRange("intField", 1, true, 1000, true)));
|
||||
assertTrue(UsageTrackingQueryCachingPolicy.isCostly(PointRangeQuery.new1DIntRange("intField", 1, true, 1000, true)));
|
||||
assertFalse(UsageTrackingQueryCachingPolicy.isCostly(new TermQuery(new Term("field", "value"))));
|
||||
}
|
||||
|
||||
|
|
|
@ -24,8 +24,8 @@ import java.util.Arrays;
|
|||
import java.util.BitSet;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.index.DimensionalValues.IntersectVisitor;
|
||||
import org.apache.lucene.index.DimensionalValues.Relation;
|
||||
import org.apache.lucene.index.PointValues.IntersectVisitor;
|
||||
import org.apache.lucene.index.PointValues.Relation;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
|
|
|
@ -32,7 +32,7 @@ import java.util.Date;
|
|||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.standard.StandardAnalyzer;
|
||||
import org.apache.lucene.document.DimensionalLongField;
|
||||
import org.apache.lucene.document.LongPoint;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.StringField;
|
||||
|
@ -178,13 +178,13 @@ public class IndexFiles {
|
|||
doc.add(pathField);
|
||||
|
||||
// Add the last modified date of the file a field named "modified".
|
||||
// Use a DimensionalLongField that is indexed (i.e. efficiently filterable with
|
||||
// DimensionalRangeQuery). This indexes to milli-second resolution, which
|
||||
// Use a LongPoint that is indexed (i.e. efficiently filterable with
|
||||
// PointRangeQuery). This indexes to milli-second resolution, which
|
||||
// is often too fine. You could instead create a number based on
|
||||
// year/month/day/hour/minutes/seconds, down the resolution you require.
|
||||
// For example the long value 2011021714 would mean
|
||||
// February 17, 2011, 2-3 PM.
|
||||
doc.add(new DimensionalLongField("modified", lastModified));
|
||||
doc.add(new LongPoint("modified", lastModified));
|
||||
|
||||
// Add the contents of the file to a field named "contents". Specify a Reader,
|
||||
// so that the text of the file is tokenized and indexed, but not stored.
|
||||
|
|
|
@ -18,7 +18,7 @@ package org.apache.lucene.demo.facet;
|
|||
*/
|
||||
|
||||
import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
|
||||
import org.apache.lucene.document.DimensionalDoubleField;
|
||||
import org.apache.lucene.document.DoublePoint;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.NumericDocValuesField;
|
||||
import org.apache.lucene.expressions.Expression;
|
||||
|
@ -40,7 +40,7 @@ import org.apache.lucene.index.IndexWriterConfig.OpenMode;
|
|||
import org.apache.lucene.queries.function.ValueSource;
|
||||
import org.apache.lucene.search.BooleanClause;
|
||||
import org.apache.lucene.search.BooleanQuery;
|
||||
import org.apache.lucene.search.DimensionalRangeQuery;
|
||||
import org.apache.lucene.search.PointRangeQuery;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.MatchAllDocsQuery;
|
||||
import org.apache.lucene.search.Query;
|
||||
|
@ -91,25 +91,25 @@ public class DistanceFacetsExample implements Closeable {
|
|||
// TODO: we could index in radians instead ... saves all the conversions in getBoundingBoxFilter
|
||||
|
||||
// Add documents with latitude/longitude location:
|
||||
// we index these both as DimensionalDoubleFields (for bounding box/ranges) and as NumericDocValuesFields (for scoring)
|
||||
// we index these both as DoublePoints (for bounding box/ranges) and as NumericDocValuesFields (for scoring)
|
||||
Document doc = new Document();
|
||||
doc.add(new DimensionalDoubleField("latitude", 40.759011));
|
||||
doc.add(new DoublePoint("latitude", 40.759011));
|
||||
doc.add(new NumericDocValuesField("latitude", Double.doubleToRawLongBits(40.759011)));
|
||||
doc.add(new DimensionalDoubleField("longitude", -73.9844722));
|
||||
doc.add(new DoublePoint("longitude", -73.9844722));
|
||||
doc.add(new NumericDocValuesField("longitude", Double.doubleToRawLongBits(-73.9844722)));
|
||||
writer.addDocument(doc);
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new DimensionalDoubleField("latitude", 40.718266));
|
||||
doc.add(new DoublePoint("latitude", 40.718266));
|
||||
doc.add(new NumericDocValuesField("latitude", Double.doubleToRawLongBits(40.718266)));
|
||||
doc.add(new DimensionalDoubleField("longitude", -74.007819));
|
||||
doc.add(new DoublePoint("longitude", -74.007819));
|
||||
doc.add(new NumericDocValuesField("longitude", Double.doubleToRawLongBits(-74.007819)));
|
||||
writer.addDocument(doc);
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new DimensionalDoubleField("latitude", 40.7051157));
|
||||
doc.add(new DoublePoint("latitude", 40.7051157));
|
||||
doc.add(new NumericDocValuesField("latitude", Double.doubleToRawLongBits(40.7051157)));
|
||||
doc.add(new DimensionalDoubleField("longitude", -74.0088305));
|
||||
doc.add(new DoublePoint("longitude", -74.0088305));
|
||||
doc.add(new NumericDocValuesField("longitude", Double.doubleToRawLongBits(-74.0088305)));
|
||||
writer.addDocument(doc);
|
||||
|
||||
|
@ -181,7 +181,7 @@ public class DistanceFacetsExample implements Closeable {
|
|||
BooleanQuery.Builder f = new BooleanQuery.Builder();
|
||||
|
||||
// Add latitude range filter:
|
||||
f.add(DimensionalRangeQuery.new1DDoubleRange("latitude", Math.toDegrees(minLat), true, Math.toDegrees(maxLat), true),
|
||||
f.add(PointRangeQuery.new1DDoubleRange("latitude", Math.toDegrees(minLat), true, Math.toDegrees(maxLat), true),
|
||||
BooleanClause.Occur.FILTER);
|
||||
|
||||
// Add longitude range filter:
|
||||
|
@ -189,13 +189,13 @@ public class DistanceFacetsExample implements Closeable {
|
|||
// The bounding box crosses the international date
|
||||
// line:
|
||||
BooleanQuery.Builder lonF = new BooleanQuery.Builder();
|
||||
lonF.add(DimensionalRangeQuery.new1DDoubleRange("longitude", Math.toDegrees(minLng), true, null, true),
|
||||
lonF.add(PointRangeQuery.new1DDoubleRange("longitude", Math.toDegrees(minLng), true, null, true),
|
||||
BooleanClause.Occur.SHOULD);
|
||||
lonF.add(DimensionalRangeQuery.new1DDoubleRange("longitude", null, true, Math.toDegrees(maxLng), true),
|
||||
lonF.add(PointRangeQuery.new1DDoubleRange("longitude", null, true, Math.toDegrees(maxLng), true),
|
||||
BooleanClause.Occur.SHOULD);
|
||||
f.add(lonF.build(), BooleanClause.Occur.MUST);
|
||||
} else {
|
||||
f.add(DimensionalRangeQuery.new1DDoubleRange("longitude", Math.toDegrees(minLng), true, Math.toDegrees(maxLng), true),
|
||||
f.add(PointRangeQuery.new1DDoubleRange("longitude", Math.toDegrees(minLng), true, Math.toDegrees(maxLng), true),
|
||||
BooleanClause.Occur.FILTER);
|
||||
}
|
||||
|
||||
|
|
|
@ -18,7 +18,7 @@ package org.apache.lucene.demo.facet;
|
|||
*/
|
||||
|
||||
import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
|
||||
import org.apache.lucene.document.DimensionalLongField;
|
||||
import org.apache.lucene.document.LongPoint;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.NumericDocValuesField;
|
||||
import org.apache.lucene.facet.DrillDownQuery;
|
||||
|
@ -32,7 +32,7 @@ import org.apache.lucene.index.DirectoryReader;
|
|||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.index.IndexWriterConfig;
|
||||
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
|
||||
import org.apache.lucene.search.DimensionalRangeQuery;
|
||||
import org.apache.lucene.search.PointRangeQuery;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.MatchAllDocsQuery;
|
||||
import org.apache.lucene.search.TopDocs;
|
||||
|
@ -69,7 +69,7 @@ public class RangeFacetsExample implements Closeable {
|
|||
// Add as doc values field, so we can compute range facets:
|
||||
doc.add(new NumericDocValuesField("timestamp", then));
|
||||
// Add as numeric field so we can drill-down:
|
||||
doc.add(new DimensionalLongField("timestamp", then));
|
||||
doc.add(new LongPoint("timestamp", then));
|
||||
indexWriter.addDocument(doc);
|
||||
}
|
||||
|
||||
|
@ -107,7 +107,7 @@ public class RangeFacetsExample implements Closeable {
|
|||
// documents ("browse only"):
|
||||
DrillDownQuery q = new DrillDownQuery(getConfig());
|
||||
|
||||
q.add("timestamp", DimensionalRangeQuery.new1DLongRange("timestamp", range.min, range.minInclusive, range.max, range.maxInclusive));
|
||||
q.add("timestamp", PointRangeQuery.new1DLongRange("timestamp", range.min, range.minInclusive, range.max, range.maxInclusive));
|
||||
|
||||
return searcher.search(q, 10);
|
||||
}
|
||||
|
|
|
@ -19,7 +19,6 @@ package org.apache.lucene.facet.range;
|
|||
|
||||
import org.apache.lucene.facet.DrillDownQuery; // javadocs
|
||||
import org.apache.lucene.queries.function.ValueSource;
|
||||
import org.apache.lucene.search.DimensionalRangeQuery; // javadocs
|
||||
import org.apache.lucene.search.Query;
|
||||
|
||||
/** Base class for a single labeled range.
|
||||
|
@ -46,7 +45,7 @@ public abstract class Range {
|
|||
* or when intersected with another query that can lead the
|
||||
* iteration. If the {@link ValueSource} is static, e.g. an
|
||||
* indexed numeric field, then it may be more efficient to use
|
||||
* {@link DimensionalRangeQuery}. The provided fastMatchQuery,
|
||||
* {@link org.apache.lucene.search.PointRangeQuery}. The provided fastMatchQuery,
|
||||
* if non-null, will first be consulted, and only if
|
||||
* that is set for each document will the range then be
|
||||
* checked. */
|
||||
|
@ -60,7 +59,7 @@ public abstract class Range {
|
|||
* or when intersected with another query that can lead the
|
||||
* iteration. If the {@link ValueSource} is static, e.g. an
|
||||
* indexed numeric field, then it may be more efficient to
|
||||
* use {@link DimensionalRangeQuery}. */
|
||||
* use {@link org.apache.lucene.search.PointRangeQuery}. */
|
||||
public Query getQuery(ValueSource valueSource) {
|
||||
return getQuery(null, valueSource);
|
||||
}
|
||||
|
|
|
@ -23,9 +23,9 @@ import java.util.Map;
|
|||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import org.apache.lucene.document.DimensionalDoubleField;
|
||||
import org.apache.lucene.document.DimensionalFloatField;
|
||||
import org.apache.lucene.document.DimensionalLongField;
|
||||
import org.apache.lucene.document.DoublePoint;
|
||||
import org.apache.lucene.document.FloatPoint;
|
||||
import org.apache.lucene.document.LongPoint;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.DoubleDocValuesField;
|
||||
import org.apache.lucene.document.FloatDocValuesField;
|
||||
|
@ -55,7 +55,7 @@ import org.apache.lucene.queries.function.docvalues.DoubleDocValues;
|
|||
import org.apache.lucene.queries.function.valuesource.DoubleFieldSource;
|
||||
import org.apache.lucene.queries.function.valuesource.FloatFieldSource;
|
||||
import org.apache.lucene.queries.function.valuesource.LongFieldSource;
|
||||
import org.apache.lucene.search.DimensionalRangeQuery;
|
||||
import org.apache.lucene.search.PointRangeQuery;
|
||||
import org.apache.lucene.search.Explanation;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.MatchAllDocsQuery;
|
||||
|
@ -219,7 +219,7 @@ public class TestRangeFacetCounts extends FacetTestCase {
|
|||
// For computing range facet counts:
|
||||
doc.add(new NumericDocValuesField("field", l));
|
||||
// For drill down by numeric range:
|
||||
doc.add(new DimensionalLongField("field", l));
|
||||
doc.add(new LongPoint("field", l));
|
||||
|
||||
if ((l&3) == 0) {
|
||||
doc.add(new FacetField("dim", "a"));
|
||||
|
@ -295,7 +295,7 @@ public class TestRangeFacetCounts extends FacetTestCase {
|
|||
|
||||
// Third search, drill down on "less than or equal to 10":
|
||||
ddq = new DrillDownQuery(config);
|
||||
ddq.add("field", DimensionalRangeQuery.new1DLongRange("field", 0L, true, 10L, true));
|
||||
ddq.add("field", PointRangeQuery.new1DLongRange("field", 0L, true, 10L, true));
|
||||
dsr = ds.search(null, ddq, 10);
|
||||
|
||||
assertEquals(11, dsr.hits.totalHits);
|
||||
|
@ -383,7 +383,7 @@ public class TestRangeFacetCounts extends FacetTestCase {
|
|||
long v = random().nextLong();
|
||||
values[i] = v;
|
||||
doc.add(new NumericDocValuesField("field", v));
|
||||
doc.add(new DimensionalLongField("field", v));
|
||||
doc.add(new LongPoint("field", v));
|
||||
w.addDocument(doc);
|
||||
minValue = Math.min(minValue, v);
|
||||
maxValue = Math.max(maxValue, v);
|
||||
|
@ -475,9 +475,9 @@ public class TestRangeFacetCounts extends FacetTestCase {
|
|||
Query fastMatchQuery;
|
||||
if (random().nextBoolean()) {
|
||||
if (random().nextBoolean()) {
|
||||
fastMatchQuery = DimensionalRangeQuery.new1DLongRange("field", minValue, true, maxValue, true);
|
||||
fastMatchQuery = PointRangeQuery.new1DLongRange("field", minValue, true, maxValue, true);
|
||||
} else {
|
||||
fastMatchQuery = DimensionalRangeQuery.new1DLongRange("field", minAcceptedValue, true, maxAcceptedValue, true);
|
||||
fastMatchQuery = PointRangeQuery.new1DLongRange("field", minAcceptedValue, true, maxAcceptedValue, true);
|
||||
}
|
||||
} else {
|
||||
fastMatchQuery = null;
|
||||
|
@ -499,7 +499,7 @@ public class TestRangeFacetCounts extends FacetTestCase {
|
|||
// Test drill-down:
|
||||
DrillDownQuery ddq = new DrillDownQuery(config);
|
||||
if (random().nextBoolean()) {
|
||||
ddq.add("field", DimensionalRangeQuery.new1DLongRange("field", range.min, range.minInclusive, range.max, range.maxInclusive));
|
||||
ddq.add("field", PointRangeQuery.new1DLongRange("field", range.min, range.minInclusive, range.max, range.maxInclusive));
|
||||
} else {
|
||||
ddq.add("field", range.getQuery(fastMatchQuery, vs));
|
||||
}
|
||||
|
@ -524,7 +524,7 @@ public class TestRangeFacetCounts extends FacetTestCase {
|
|||
float v = random().nextFloat();
|
||||
values[i] = v;
|
||||
doc.add(new FloatDocValuesField("field", v));
|
||||
doc.add(new DimensionalFloatField("field", v));
|
||||
doc.add(new FloatPoint("field", v));
|
||||
w.addDocument(doc);
|
||||
minValue = Math.min(minValue, v);
|
||||
maxValue = Math.max(maxValue, v);
|
||||
|
@ -630,9 +630,9 @@ public class TestRangeFacetCounts extends FacetTestCase {
|
|||
Query fastMatchQuery;
|
||||
if (random().nextBoolean()) {
|
||||
if (random().nextBoolean()) {
|
||||
fastMatchQuery = DimensionalRangeQuery.new1DFloatRange("field", minValue, true, maxValue, true);
|
||||
fastMatchQuery = PointRangeQuery.new1DFloatRange("field", minValue, true, maxValue, true);
|
||||
} else {
|
||||
fastMatchQuery = DimensionalRangeQuery.new1DFloatRange("field", minAcceptedValue, true, maxAcceptedValue, true);
|
||||
fastMatchQuery = PointRangeQuery.new1DFloatRange("field", minAcceptedValue, true, maxAcceptedValue, true);
|
||||
}
|
||||
} else {
|
||||
fastMatchQuery = null;
|
||||
|
@ -654,7 +654,7 @@ public class TestRangeFacetCounts extends FacetTestCase {
|
|||
// Test drill-down:
|
||||
DrillDownQuery ddq = new DrillDownQuery(config);
|
||||
if (random().nextBoolean()) {
|
||||
ddq.add("field", DimensionalRangeQuery.new1DFloatRange("field", (float) range.min, range.minInclusive, (float) range.max, range.maxInclusive));
|
||||
ddq.add("field", PointRangeQuery.new1DFloatRange("field", (float) range.min, range.minInclusive, (float) range.max, range.maxInclusive));
|
||||
} else {
|
||||
ddq.add("field", range.getQuery(fastMatchQuery, vs));
|
||||
}
|
||||
|
@ -679,7 +679,7 @@ public class TestRangeFacetCounts extends FacetTestCase {
|
|||
double v = random().nextDouble();
|
||||
values[i] = v;
|
||||
doc.add(new DoubleDocValuesField("field", v));
|
||||
doc.add(new DimensionalDoubleField("field", v));
|
||||
doc.add(new DoublePoint("field", v));
|
||||
w.addDocument(doc);
|
||||
minValue = Math.min(minValue, v);
|
||||
maxValue = Math.max(maxValue, v);
|
||||
|
@ -769,9 +769,9 @@ public class TestRangeFacetCounts extends FacetTestCase {
|
|||
Query fastMatchFilter;
|
||||
if (random().nextBoolean()) {
|
||||
if (random().nextBoolean()) {
|
||||
fastMatchFilter = DimensionalRangeQuery.new1DDoubleRange("field", minValue, true, maxValue, true);
|
||||
fastMatchFilter = PointRangeQuery.new1DDoubleRange("field", minValue, true, maxValue, true);
|
||||
} else {
|
||||
fastMatchFilter = DimensionalRangeQuery.new1DDoubleRange("field", minAcceptedValue, true, maxAcceptedValue, true);
|
||||
fastMatchFilter = PointRangeQuery.new1DDoubleRange("field", minAcceptedValue, true, maxAcceptedValue, true);
|
||||
}
|
||||
} else {
|
||||
fastMatchFilter = null;
|
||||
|
@ -793,7 +793,7 @@ public class TestRangeFacetCounts extends FacetTestCase {
|
|||
// Test drill-down:
|
||||
DrillDownQuery ddq = new DrillDownQuery(config);
|
||||
if (random().nextBoolean()) {
|
||||
ddq.add("field", DimensionalRangeQuery.new1DDoubleRange("field", range.min, range.minInclusive, range.max, range.maxInclusive));
|
||||
ddq.add("field", PointRangeQuery.new1DDoubleRange("field", range.min, range.minInclusive, range.max, range.maxInclusive));
|
||||
} else {
|
||||
ddq.add("field", range.getQuery(fastMatchFilter, vs));
|
||||
}
|
||||
|
|
|
@ -22,7 +22,7 @@ import java.util.Collections;
|
|||
import java.util.Iterator;
|
||||
|
||||
import org.apache.lucene.index.BinaryDocValues;
|
||||
import org.apache.lucene.index.DimensionalValues;
|
||||
import org.apache.lucene.index.PointValues;
|
||||
import org.apache.lucene.index.DocValuesType;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
|
@ -149,7 +149,7 @@ public class TermVectorLeafReader extends LeafReader {
|
|||
}
|
||||
|
||||
@Override
|
||||
public DimensionalValues getDimensionalValues() {
|
||||
public PointValues getPointValues() {
|
||||
return null;
|
||||
}
|
||||
|
||||
|
|
|
@ -45,7 +45,7 @@ import org.apache.lucene.analysis.Tokenizer;
|
|||
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||
import org.apache.lucene.document.DimensionalIntField;
|
||||
import org.apache.lucene.document.IntPoint;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.FieldType;
|
||||
|
@ -63,7 +63,7 @@ import org.apache.lucene.queries.payloads.SpanPayloadCheckQuery;
|
|||
import org.apache.lucene.search.BooleanClause.Occur;
|
||||
import org.apache.lucene.search.BooleanQuery;
|
||||
import org.apache.lucene.search.ConstantScoreQuery;
|
||||
import org.apache.lucene.search.DimensionalRangeQuery;
|
||||
import org.apache.lucene.search.PointRangeQuery;
|
||||
import org.apache.lucene.search.FuzzyQuery;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.MultiPhraseQuery;
|
||||
|
@ -585,7 +585,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
|
|||
|
||||
public void testDimensionalRangeQuery() throws Exception {
|
||||
// doesn't currently highlight, but make sure it doesn't cause exception either
|
||||
query = DimensionalRangeQuery.new1DIntRange(NUMERIC_FIELD_NAME, 2, true, 6, true);
|
||||
query = PointRangeQuery.new1DIntRange(NUMERIC_FIELD_NAME, 2, true, 6, true);
|
||||
searcher = newSearcher(reader);
|
||||
hits = searcher.search(query, 100);
|
||||
int maxNumFragmentsRequired = 2;
|
||||
|
@ -2076,22 +2076,22 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
|
|||
|
||||
// a few tests need other docs...:
|
||||
Document doc = new Document();
|
||||
doc.add(new DimensionalIntField(NUMERIC_FIELD_NAME, 1));
|
||||
doc.add(new IntPoint(NUMERIC_FIELD_NAME, 1));
|
||||
doc.add(new StoredField(NUMERIC_FIELD_NAME, 1));
|
||||
writer.addDocument(doc);
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new DimensionalIntField(NUMERIC_FIELD_NAME, 3));
|
||||
doc.add(new IntPoint(NUMERIC_FIELD_NAME, 3));
|
||||
doc.add(new StoredField(NUMERIC_FIELD_NAME, 3));
|
||||
writer.addDocument(doc);
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new DimensionalIntField(NUMERIC_FIELD_NAME, 5));
|
||||
doc.add(new IntPoint(NUMERIC_FIELD_NAME, 5));
|
||||
doc.add(new StoredField(NUMERIC_FIELD_NAME, 5));
|
||||
writer.addDocument(doc);
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new DimensionalIntField(NUMERIC_FIELD_NAME, 7));
|
||||
doc.add(new IntPoint(NUMERIC_FIELD_NAME, 7));
|
||||
doc.add(new StoredField(NUMERIC_FIELD_NAME, 7));
|
||||
writer.addDocument(doc);
|
||||
|
||||
|
|
|
@ -34,7 +34,7 @@ import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
|
|||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
|
||||
import org.apache.lucene.index.BinaryDocValues;
|
||||
import org.apache.lucene.index.DimensionalValues;
|
||||
import org.apache.lucene.index.PointValues;
|
||||
import org.apache.lucene.index.DocValuesType;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
|
@ -814,7 +814,7 @@ public class MemoryIndex {
|
|||
}
|
||||
|
||||
@Override
|
||||
public DimensionalValues getDimensionalValues() {
|
||||
public PointValues getPointValues() {
|
||||
return null;
|
||||
}
|
||||
|
||||
|
|
|
@ -211,8 +211,8 @@ class MergeReaderWrapper extends LeafReader {
|
|||
}
|
||||
|
||||
@Override
|
||||
public DimensionalValues getDimensionalValues() {
|
||||
return in.getDimensionalValues();
|
||||
public PointValues getPointValues() {
|
||||
return in.getPointValues();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -254,12 +254,12 @@ public class SortingLeafReader extends FilterLeafReader {
|
|||
}
|
||||
}
|
||||
|
||||
private static class SortingDimensionalValues extends DimensionalValues {
|
||||
private static class SortingPointValues extends PointValues {
|
||||
|
||||
private final DimensionalValues in;
|
||||
private final PointValues in;
|
||||
private final Sorter.DocMap docMap;
|
||||
|
||||
public SortingDimensionalValues(final DimensionalValues in, Sorter.DocMap docMap) {
|
||||
public SortingPointValues(final PointValues in, Sorter.DocMap docMap) {
|
||||
this.in = in;
|
||||
this.docMap = docMap;
|
||||
}
|
||||
|
@ -851,13 +851,13 @@ public class SortingLeafReader extends FilterLeafReader {
|
|||
}
|
||||
|
||||
@Override
|
||||
public DimensionalValues getDimensionalValues() {
|
||||
final DimensionalValues inDimensionalValues = in.getDimensionalValues();
|
||||
if (inDimensionalValues == null) {
|
||||
public PointValues getPointValues() {
|
||||
final PointValues inPointValues = in.getPointValues();
|
||||
if (inPointValues == null) {
|
||||
return null;
|
||||
} else {
|
||||
// TODO: this is untested!
|
||||
return new SortingDimensionalValues(inDimensionalValues, docMap);
|
||||
return new SortingPointValues(inPointValues, docMap);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -19,7 +19,6 @@ package org.apache.lucene.uninverting;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.document.BinaryDocValuesField; // javadocs
|
||||
|
@ -213,7 +212,7 @@ public class UninvertingReader extends FilterLeafReader {
|
|||
}
|
||||
filteredInfos.add(new FieldInfo(fi.name, fi.number, fi.hasVectors(), fi.omitsNorms(),
|
||||
fi.hasPayloads(), fi.getIndexOptions(), type, fi.getDocValuesGen(), fi.attributes(),
|
||||
fi.getDimensionCount(), fi.getDimensionNumBytes()));
|
||||
fi.getPointDimensionCount(), fi.getPointNumBytes()));
|
||||
}
|
||||
fieldInfos = new FieldInfos(filteredInfos.toArray(new FieldInfo[filteredInfos.size()]));
|
||||
}
|
||||
|
|
|
@ -30,7 +30,7 @@ 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.DimensionalBinaryField;
|
||||
import org.apache.lucene.document.BinaryPoint;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field.Store;
|
||||
import org.apache.lucene.document.Field;
|
||||
|
@ -41,8 +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.PointValues.IntersectVisitor;
|
||||
import org.apache.lucene.index.PointValues.Relation;
|
||||
import org.apache.lucene.index.SortingLeafReader.SortingDocsEnum;
|
||||
import org.apache.lucene.index.TermsEnum.SeekStatus;
|
||||
import org.apache.lucene.search.CollectionStatistics;
|
||||
|
@ -175,7 +175,7 @@ public abstract class SorterTestBase extends LuceneTestCase {
|
|||
doc.add(new Field(TERM_VECTORS_FIELD, Integer.toString(id), TERM_VECTORS_TYPE));
|
||||
byte[] bytes = new byte[4];
|
||||
NumericUtils.intToBytes(id, bytes, 0);
|
||||
doc.add(new DimensionalBinaryField(DIMENSIONAL_FIELD, bytes));
|
||||
doc.add(new BinaryPoint(DIMENSIONAL_FIELD, bytes));
|
||||
return doc;
|
||||
}
|
||||
|
||||
|
@ -379,8 +379,8 @@ public abstract class SorterTestBase extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
public void testDimensionalValues() throws Exception {
|
||||
DimensionalValues values = sortedReader.getDimensionalValues();
|
||||
public void testPoints() throws Exception {
|
||||
PointValues values = sortedReader.getPointValues();
|
||||
values.intersect(DIMENSIONAL_FIELD,
|
||||
new IntersectVisitor() {
|
||||
@Override
|
||||
|
|
|
@ -26,7 +26,7 @@ import java.util.Map;
|
|||
import java.util.Set;
|
||||
import java.util.Collections;
|
||||
|
||||
import org.apache.lucene.document.DimensionalIntField;
|
||||
import org.apache.lucene.document.IntPoint;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.FieldType;
|
||||
|
@ -356,7 +356,7 @@ public class TestUninvertingReader extends LuceneTestCase {
|
|||
doc.add(new StringField("id", idBytes, Store.YES));
|
||||
doc.add(new LegacyIntField("int", 5, Store.YES));
|
||||
doc.add(new NumericDocValuesField("dv", 5));
|
||||
doc.add(new DimensionalIntField("dint", 5));
|
||||
doc.add(new IntPoint("dint", 5));
|
||||
doc.add(new StoredField("stored", 5)); // not indexed
|
||||
iw.addDocument(doc);
|
||||
|
||||
|
@ -373,12 +373,12 @@ public class TestUninvertingReader extends LuceneTestCase {
|
|||
|
||||
FieldInfo intFInfo = leafReader.getFieldInfos().fieldInfo("int");
|
||||
assertEquals(DocValuesType.NUMERIC, intFInfo.getDocValuesType());
|
||||
assertEquals(0, intFInfo.getDimensionCount());
|
||||
assertEquals(0, intFInfo.getDimensionNumBytes());
|
||||
assertEquals(0, intFInfo.getPointDimensionCount());
|
||||
assertEquals(0, intFInfo.getPointNumBytes());
|
||||
|
||||
FieldInfo dintFInfo = leafReader.getFieldInfos().fieldInfo("dint");
|
||||
assertEquals(1, dintFInfo.getDimensionCount());
|
||||
assertEquals(4, dintFInfo.getDimensionNumBytes());
|
||||
assertEquals(1, dintFInfo.getPointDimensionCount());
|
||||
assertEquals(4, dintFInfo.getPointNumBytes());
|
||||
|
||||
FieldInfo dvFInfo = leafReader.getFieldInfos().fieldInfo("dv");
|
||||
assertEquals(DocValuesType.NUMERIC, dvFInfo.getDocValuesType());
|
||||
|
|
|
@ -22,8 +22,7 @@ import org.apache.lucene.util.GeoUtils;
|
|||
import org.apache.lucene.util.NumericUtils;
|
||||
|
||||
/** Add this to a document to index lat/lon point dimensionally */
|
||||
public final class DimensionalLatLonField extends Field {
|
||||
|
||||
public class LatLonPoint extends Field {
|
||||
public static final FieldType TYPE = new FieldType();
|
||||
static {
|
||||
TYPE.setDimensions(2, 4);
|
||||
|
@ -31,13 +30,13 @@ public final class DimensionalLatLonField extends Field {
|
|||
}
|
||||
|
||||
/**
|
||||
* Creates a new DimensionalLatLonField with the specified lat and lon
|
||||
* Creates a new LatLonPoint with the specified lat and lon
|
||||
* @param name field name
|
||||
* @param lat double latitude
|
||||
* @param lon double longitude
|
||||
* @throws IllegalArgumentException if the field name is null or lat or lon are out of bounds
|
||||
*/
|
||||
public DimensionalLatLonField(String name, double lat, double lon) {
|
||||
public LatLonPoint(String name, double lat, double lon) {
|
||||
super(name, TYPE);
|
||||
if (GeoUtils.isValidLat(lat) == false) {
|
||||
throw new IllegalArgumentException("invalid lat (" + lat + "): must be -90 to 90");
|
|
@ -33,7 +33,7 @@ import org.apache.lucene.util.BytesRef;
|
|||
* A range query that works on top of the doc values APIs. Such queries are
|
||||
* usually slow since they do not use an inverted index. However, in the
|
||||
* dense case where most documents match this query, it <b>might</b> be as
|
||||
* fast or faster than a regular {@link DimensionalRangeQuery}.
|
||||
* fast or faster than a regular {@link PointRangeQuery}.
|
||||
*
|
||||
* <p>
|
||||
* <b>NOTE</b>: be very careful using this query: it is
|
||||
|
|
|
@ -20,10 +20,10 @@ package org.apache.lucene.search;
|
|||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
|
||||
import org.apache.lucene.document.DimensionalLatLonField;
|
||||
import org.apache.lucene.index.DimensionalValues.IntersectVisitor;
|
||||
import org.apache.lucene.index.DimensionalValues.Relation;
|
||||
import org.apache.lucene.index.DimensionalValues;
|
||||
import org.apache.lucene.document.LatLonPoint;
|
||||
import org.apache.lucene.index.PointValues.IntersectVisitor;
|
||||
import org.apache.lucene.index.PointValues.Relation;
|
||||
import org.apache.lucene.index.PointValues;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.util.DocIdSetBuilder;
|
||||
|
@ -33,11 +33,11 @@ import org.apache.lucene.util.NumericUtils;
|
|||
|
||||
/** Finds all previously indexed points that fall within the specified polygon.
|
||||
*
|
||||
* <p>The field must be indexed with using {@link DimensionalLatLonField} added per document.
|
||||
* <p>The field must be indexed with using {@link org.apache.lucene.document.LatLonPoint} added per document.
|
||||
*
|
||||
* @lucene.experimental */
|
||||
|
||||
public class DimensionalPointInPolygonQuery extends Query {
|
||||
public class PointInPolygonQuery extends Query {
|
||||
final String field;
|
||||
final double minLat;
|
||||
final double maxLat;
|
||||
|
@ -47,7 +47,7 @@ public class DimensionalPointInPolygonQuery extends Query {
|
|||
final double[] polyLons;
|
||||
|
||||
/** The lats/lons must be clockwise or counter-clockwise. */
|
||||
public DimensionalPointInPolygonQuery(String field, double[] polyLats, double[] polyLons) {
|
||||
public PointInPolygonQuery(String field, double[] polyLats, double[] polyLons) {
|
||||
this.field = field;
|
||||
if (polyLats.length != polyLons.length) {
|
||||
throw new IllegalArgumentException("polyLats and polyLons must be equal length");
|
||||
|
@ -105,9 +105,9 @@ public class DimensionalPointInPolygonQuery extends Query {
|
|||
@Override
|
||||
public Scorer scorer(LeafReaderContext context) throws IOException {
|
||||
LeafReader reader = context.reader();
|
||||
DimensionalValues values = reader.getDimensionalValues();
|
||||
PointValues values = reader.getPointValues();
|
||||
if (values == null) {
|
||||
// No docs in this segment had any dimensional fields
|
||||
// No docs in this segment had any points fields
|
||||
return null;
|
||||
}
|
||||
|
||||
|
@ -124,8 +124,8 @@ public class DimensionalPointInPolygonQuery extends Query {
|
|||
@Override
|
||||
public void visit(int docID, byte[] packedValue) {
|
||||
assert packedValue.length == 8;
|
||||
double lat = DimensionalLatLonField.decodeLat(NumericUtils.bytesToInt(packedValue, 0));
|
||||
double lon = DimensionalLatLonField.decodeLon(NumericUtils.bytesToInt(packedValue, 1));
|
||||
double lat = LatLonPoint.decodeLat(NumericUtils.bytesToInt(packedValue, 0));
|
||||
double lon = LatLonPoint.decodeLon(NumericUtils.bytesToInt(packedValue, 1));
|
||||
if (GeoRelationUtils.pointInPolygon(polyLons, polyLats, lat, lon)) {
|
||||
hitCount[0]++;
|
||||
result.add(docID);
|
||||
|
@ -134,10 +134,10 @@ public class DimensionalPointInPolygonQuery extends Query {
|
|||
|
||||
@Override
|
||||
public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
|
||||
double cellMinLat = DimensionalLatLonField.decodeLat(NumericUtils.bytesToInt(minPackedValue, 0));
|
||||
double cellMinLon = DimensionalLatLonField.decodeLon(NumericUtils.bytesToInt(minPackedValue, 1));
|
||||
double cellMaxLat = DimensionalLatLonField.decodeLat(NumericUtils.bytesToInt(maxPackedValue, 0));
|
||||
double cellMaxLon = DimensionalLatLonField.decodeLon(NumericUtils.bytesToInt(maxPackedValue, 1));
|
||||
double cellMinLat = LatLonPoint.decodeLat(NumericUtils.bytesToInt(minPackedValue, 0));
|
||||
double cellMinLon = LatLonPoint.decodeLon(NumericUtils.bytesToInt(minPackedValue, 1));
|
||||
double cellMaxLat = LatLonPoint.decodeLat(NumericUtils.bytesToInt(maxPackedValue, 0));
|
||||
double cellMaxLon = LatLonPoint.decodeLon(NumericUtils.bytesToInt(maxPackedValue, 1));
|
||||
|
||||
if (cellMinLat <= minLat && cellMaxLat >= maxLat && cellMinLon <= minLon && cellMaxLon >= maxLon) {
|
||||
// Cell fully encloses the query
|
||||
|
@ -169,7 +169,7 @@ public class DimensionalPointInPolygonQuery extends Query {
|
|||
if (o == null || getClass() != o.getClass()) return false;
|
||||
if (!super.equals(o)) return false;
|
||||
|
||||
DimensionalPointInPolygonQuery that = (DimensionalPointInPolygonQuery) o;
|
||||
PointInPolygonQuery that = (PointInPolygonQuery) o;
|
||||
|
||||
if (Arrays.equals(polyLons, that.polyLons) == false) {
|
||||
return false;
|
|
@ -19,10 +19,10 @@ package org.apache.lucene.search;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.document.DimensionalLatLonField;
|
||||
import org.apache.lucene.index.DimensionalValues.IntersectVisitor;
|
||||
import org.apache.lucene.index.DimensionalValues.Relation;
|
||||
import org.apache.lucene.index.DimensionalValues;
|
||||
import org.apache.lucene.document.LatLonPoint;
|
||||
import org.apache.lucene.index.PointValues.IntersectVisitor;
|
||||
import org.apache.lucene.index.PointValues.Relation;
|
||||
import org.apache.lucene.index.PointValues;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
|
@ -32,11 +32,11 @@ import org.apache.lucene.util.NumericUtils;
|
|||
|
||||
/** Finds all previously indexed points that fall within the specified boundings box.
|
||||
*
|
||||
* <p>The field must be indexed with using {@link DimensionalLatLonField} added per document.
|
||||
* <p>The field must be indexed with using {@link org.apache.lucene.document.LatLonPoint} added per document.
|
||||
*
|
||||
* @lucene.experimental */
|
||||
|
||||
public class DimensionalPointInRectQuery extends Query {
|
||||
public class PointInRectQuery extends Query {
|
||||
final String field;
|
||||
final double minLat;
|
||||
final double maxLat;
|
||||
|
@ -44,7 +44,7 @@ public class DimensionalPointInRectQuery extends Query {
|
|||
final double maxLon;
|
||||
|
||||
/** Matches all points >= minLon, minLat (inclusive) and < maxLon, maxLat (exclusive). */
|
||||
public DimensionalPointInRectQuery(String field, double minLat, double maxLat, double minLon, double maxLon) {
|
||||
public PointInRectQuery(String field, double minLat, double maxLat, double minLon, double maxLon) {
|
||||
this.field = field;
|
||||
if (GeoUtils.isValidLat(minLat) == false) {
|
||||
throw new IllegalArgumentException("minLat=" + minLat + " is not a valid latitude");
|
||||
|
@ -74,9 +74,9 @@ public class DimensionalPointInRectQuery extends Query {
|
|||
@Override
|
||||
public Scorer scorer(LeafReaderContext context) throws IOException {
|
||||
LeafReader reader = context.reader();
|
||||
DimensionalValues values = reader.getDimensionalValues();
|
||||
PointValues values = reader.getPointValues();
|
||||
if (values == null) {
|
||||
// No docs in this segment had any dimensional fields
|
||||
// No docs in this segment had any points fields
|
||||
return null;
|
||||
}
|
||||
|
||||
|
@ -98,8 +98,8 @@ public class DimensionalPointInRectQuery extends Query {
|
|||
@Override
|
||||
public void visit(int docID, byte[] packedValue) {
|
||||
assert packedValue.length == 8;
|
||||
double lat = DimensionalLatLonField.decodeLat(NumericUtils.bytesToInt(packedValue, 0));
|
||||
double lon = DimensionalLatLonField.decodeLon(NumericUtils.bytesToInt(packedValue, 1));
|
||||
double lat = LatLonPoint.decodeLat(NumericUtils.bytesToInt(packedValue, 0));
|
||||
double lon = LatLonPoint.decodeLon(NumericUtils.bytesToInt(packedValue, 1));
|
||||
if (lat >= minLat && lat <= maxLat && lon >= minLon && lon <= maxLon) {
|
||||
hitCount[0]++;
|
||||
result.add(docID);
|
||||
|
@ -108,10 +108,10 @@ public class DimensionalPointInRectQuery extends Query {
|
|||
|
||||
@Override
|
||||
public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
|
||||
double cellMinLat = DimensionalLatLonField.decodeLat(NumericUtils.bytesToInt(minPackedValue, 0));
|
||||
double cellMinLon = DimensionalLatLonField.decodeLon(NumericUtils.bytesToInt(minPackedValue, 1));
|
||||
double cellMaxLat = DimensionalLatLonField.decodeLat(NumericUtils.bytesToInt(maxPackedValue, 0));
|
||||
double cellMaxLon = DimensionalLatLonField.decodeLon(NumericUtils.bytesToInt(maxPackedValue, 1));
|
||||
double cellMinLat = LatLonPoint.decodeLat(NumericUtils.bytesToInt(minPackedValue, 0));
|
||||
double cellMinLon = LatLonPoint.decodeLon(NumericUtils.bytesToInt(minPackedValue, 1));
|
||||
double cellMaxLat = LatLonPoint.decodeLat(NumericUtils.bytesToInt(maxPackedValue, 0));
|
||||
double cellMaxLon = LatLonPoint.decodeLon(NumericUtils.bytesToInt(maxPackedValue, 1));
|
||||
|
||||
if (minLat <= cellMinLat && maxLat >= cellMaxLat && minLon <= cellMinLon && maxLon >= cellMaxLon) {
|
||||
return Relation.CELL_INSIDE_QUERY;
|
||||
|
@ -141,9 +141,9 @@ public class DimensionalPointInRectQuery extends Query {
|
|||
q.setDisableCoord(true);
|
||||
|
||||
// E.g.: maxLon = -179, minLon = 179
|
||||
DimensionalPointInRectQuery left = new DimensionalPointInRectQuery(field, minLat, maxLat, GeoUtils.MIN_LON_INCL, maxLon);
|
||||
PointInRectQuery left = new PointInRectQuery(field, minLat, maxLat, GeoUtils.MIN_LON_INCL, maxLon);
|
||||
q.add(new BooleanClause(left, BooleanClause.Occur.SHOULD));
|
||||
DimensionalPointInRectQuery right = new DimensionalPointInRectQuery(field, minLat, maxLat, minLon, GeoUtils.MAX_LON_INCL);
|
||||
PointInRectQuery right = new PointInRectQuery(field, minLat, maxLat, minLon, GeoUtils.MAX_LON_INCL);
|
||||
q.add(new BooleanClause(right, BooleanClause.Occur.SHOULD));
|
||||
return new ConstantScoreQuery(q.build());
|
||||
} else {
|
||||
|
@ -163,8 +163,8 @@ public class DimensionalPointInRectQuery extends Query {
|
|||
|
||||
@Override
|
||||
public boolean equals(Object other) {
|
||||
if (super.equals(other) && other instanceof DimensionalPointInRectQuery) {
|
||||
final DimensionalPointInRectQuery q = (DimensionalPointInRectQuery) other;
|
||||
if (super.equals(other) && other instanceof PointInRectQuery) {
|
||||
final PointInRectQuery q = (PointInRectQuery) other;
|
||||
return field.equals(q.field) &&
|
||||
minLat == q.minLat &&
|
||||
maxLat == q.maxLat &&
|
|
@ -19,7 +19,7 @@ package org.apache.lucene.search;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.document.DimensionalLongField;
|
||||
import org.apache.lucene.document.LongPoint;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field.Store;
|
||||
import org.apache.lucene.document.NumericDocValuesField;
|
||||
|
@ -53,12 +53,12 @@ public class TestDocValuesRangeQuery extends LuceneTestCase {
|
|||
for (int j = 0; j < numValues; ++j) {
|
||||
final long value = TestUtil.nextLong(random(), -100, 10000);
|
||||
doc.add(new SortedNumericDocValuesField("dv", value));
|
||||
doc.add(new DimensionalLongField("idx", value));
|
||||
doc.add(new LongPoint("idx", value));
|
||||
}
|
||||
iw.addDocument(doc);
|
||||
}
|
||||
if (random().nextBoolean()) {
|
||||
iw.deleteDocuments(DimensionalRangeQuery.new1DLongRange("idx", 0L, true, 10L, true));
|
||||
iw.deleteDocuments(PointRangeQuery.new1DLongRange("idx", 0L, true, 10L, true));
|
||||
}
|
||||
iw.commit();
|
||||
final IndexReader reader = iw.getReader();
|
||||
|
@ -70,7 +70,7 @@ public class TestDocValuesRangeQuery extends LuceneTestCase {
|
|||
final Long max = random().nextBoolean() ? null : TestUtil.nextLong(random(), -100, 1000);
|
||||
final boolean minInclusive = random().nextBoolean();
|
||||
final boolean maxInclusive = random().nextBoolean();
|
||||
final Query q1 = DimensionalRangeQuery.new1DLongRange("idx", min, minInclusive, max, maxInclusive);
|
||||
final Query q1 = PointRangeQuery.new1DLongRange("idx", min, minInclusive, max, maxInclusive);
|
||||
final Query q2 = DocValuesRangeQuery.newLongRange("dv", min, max, minInclusive, maxInclusive);
|
||||
assertSameMatches(searcher, q1, q2, false);
|
||||
}
|
||||
|
@ -180,13 +180,13 @@ public class TestDocValuesRangeQuery extends LuceneTestCase {
|
|||
final long value = TestUtil.nextLong(random(), -100, 10000);
|
||||
doc.add(new SortedNumericDocValuesField("dv1", value));
|
||||
doc.add(new SortedSetDocValuesField("dv2", toSortableBytes(value)));
|
||||
doc.add(new DimensionalLongField("idx", value));
|
||||
doc.add(new LongPoint("idx", value));
|
||||
doc.add(new StringField("f", random().nextBoolean() ? "a" : "b", Store.NO));
|
||||
}
|
||||
iw.addDocument(doc);
|
||||
}
|
||||
if (random().nextBoolean()) {
|
||||
iw.deleteDocuments(DimensionalRangeQuery.new1DLongRange("idx", 0L, true, 10L, true));
|
||||
iw.deleteDocuments(PointRangeQuery.new1DLongRange("idx", 0L, true, 10L, true));
|
||||
}
|
||||
iw.commit();
|
||||
final IndexReader reader = iw.getReader();
|
||||
|
@ -200,7 +200,7 @@ public class TestDocValuesRangeQuery extends LuceneTestCase {
|
|||
final boolean maxInclusive = random().nextBoolean();
|
||||
|
||||
BooleanQuery.Builder ref = new BooleanQuery.Builder();
|
||||
ref.add(DimensionalRangeQuery.new1DLongRange("idx", min, minInclusive, max, maxInclusive), Occur.FILTER);
|
||||
ref.add(PointRangeQuery.new1DLongRange("idx", min, minInclusive, max, maxInclusive), Occur.FILTER);
|
||||
ref.add(new TermQuery(new Term("f", "a")), Occur.MUST);
|
||||
|
||||
BooleanQuery.Builder bq1 = new BooleanQuery.Builder();
|
||||
|
|
|
@ -17,24 +17,22 @@ package org.apache.lucene.search;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.document.DimensionalLatLonField;
|
||||
import org.apache.lucene.document.LatLonPoint;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BaseGeoPointTestCase;
|
||||
import org.apache.lucene.util.GeoDistanceUtils;
|
||||
import org.apache.lucene.util.GeoRect;
|
||||
import org.apache.lucene.util.SloppyMath;
|
||||
|
||||
public class TestDimensionalQueries extends BaseGeoPointTestCase {
|
||||
public class TestLatLonPointQueries extends BaseGeoPointTestCase {
|
||||
|
||||
@Override
|
||||
protected void addPointToDoc(String field, Document doc, double lat, double lon) {
|
||||
doc.add(new DimensionalLatLonField(field, lat, lon));
|
||||
doc.add(new LatLonPoint(field, lat, lon));
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Query newRectQuery(String field, GeoRect rect) {
|
||||
return new DimensionalPointInRectQuery(field, rect.minLat, rect.maxLat, rect.minLon, rect.maxLon);
|
||||
return new PointInRectQuery(field, rect.minLat, rect.maxLat, rect.minLon, rect.maxLon);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -50,7 +48,7 @@ public class TestDimensionalQueries extends BaseGeoPointTestCase {
|
|||
|
||||
@Override
|
||||
protected Query newPolygonQuery(String field, double[] lats, double[] lons) {
|
||||
return new DimensionalPointInPolygonQuery(FIELD_NAME, lats, lons);
|
||||
return new PointInPolygonQuery(FIELD_NAME, lats, lons);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -58,13 +56,13 @@ public class TestDimensionalQueries extends BaseGeoPointTestCase {
|
|||
|
||||
assert Double.isNaN(pointLat) == false;
|
||||
|
||||
int rectLatMinEnc = DimensionalLatLonField.encodeLat(rect.minLat);
|
||||
int rectLatMaxEnc = DimensionalLatLonField.encodeLat(rect.maxLat);
|
||||
int rectLonMinEnc = DimensionalLatLonField.encodeLon(rect.minLon);
|
||||
int rectLonMaxEnc = DimensionalLatLonField.encodeLon(rect.maxLon);
|
||||
int rectLatMinEnc = LatLonPoint.encodeLat(rect.minLat);
|
||||
int rectLatMaxEnc = LatLonPoint.encodeLat(rect.maxLat);
|
||||
int rectLonMinEnc = LatLonPoint.encodeLon(rect.minLon);
|
||||
int rectLonMaxEnc = LatLonPoint.encodeLon(rect.maxLon);
|
||||
|
||||
int pointLatEnc = DimensionalLatLonField.encodeLat(pointLat);
|
||||
int pointLonEnc = DimensionalLatLonField.encodeLon(pointLon);
|
||||
int pointLatEnc = LatLonPoint.encodeLat(pointLat);
|
||||
int pointLonEnc = LatLonPoint.encodeLon(pointLon);
|
||||
|
||||
if (rect.minLon < rect.maxLon) {
|
||||
return pointLatEnc >= rectLatMinEnc &&
|
||||
|
@ -114,12 +112,12 @@ public class TestDimensionalQueries extends BaseGeoPointTestCase {
|
|||
boolean small = random().nextBoolean();
|
||||
for(int iter=0;iter<iters;iter++) {
|
||||
double lat = randomLat(small);
|
||||
double latQuantized = DimensionalLatLonField.decodeLat(DimensionalLatLonField.encodeLat(lat));
|
||||
assertEquals(lat, latQuantized, DimensionalLatLonField.TOLERANCE);
|
||||
double latQuantized = LatLonPoint.decodeLat(LatLonPoint.encodeLat(lat));
|
||||
assertEquals(lat, latQuantized, LatLonPoint.TOLERANCE);
|
||||
|
||||
double lon = randomLon(small);
|
||||
double lonQuantized = DimensionalLatLonField.decodeLon(DimensionalLatLonField.encodeLon(lon));
|
||||
assertEquals(lon, lonQuantized, DimensionalLatLonField.TOLERANCE);
|
||||
double lonQuantized = LatLonPoint.decodeLon(LatLonPoint.encodeLon(lon));
|
||||
assertEquals(lon, lonQuantized, LatLonPoint.TOLERANCE);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -24,11 +24,11 @@ import org.apache.lucene.util.NumericUtils;
|
|||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
/** Add this to a document to index lat/lon or x/y/z point, indexed as a dimensional value.
|
||||
* Multiple values are allowed: just add multiple Geo3DPointField to the document with the
|
||||
* Multiple values are allowed: just add multiple Geo3DPoint to the document with the
|
||||
* same field name.
|
||||
*
|
||||
* @lucene.experimental */
|
||||
public final class Geo3DPointField extends Field {
|
||||
public final class Geo3DPoint extends Field {
|
||||
|
||||
/** Indexing {@link FieldType}. */
|
||||
public static final FieldType TYPE = new FieldType();
|
||||
|
@ -38,11 +38,11 @@ public final class Geo3DPointField extends Field {
|
|||
}
|
||||
|
||||
/**
|
||||
* Creates a new Geo3DPointField field with the specified lat, lon (in radians), given a planet model.
|
||||
* Creates a new Geo3DPoint field with the specified lat, lon (in radians), given a planet model.
|
||||
*
|
||||
* @throws IllegalArgumentException if the field name is null or lat or lon are out of bounds
|
||||
*/
|
||||
public Geo3DPointField(String name, PlanetModel planetModel, double lat, double lon) {
|
||||
public Geo3DPoint(String name, PlanetModel planetModel, double lat, double lon) {
|
||||
super(name, TYPE);
|
||||
// Translate lat/lon to x,y,z:
|
||||
final GeoPoint point = new GeoPoint(planetModel, lat, lon);
|
||||
|
@ -50,11 +50,11 @@ public final class Geo3DPointField extends Field {
|
|||
}
|
||||
|
||||
/**
|
||||
* Creates a new Geo3DPointField field with the specified x,y,z.
|
||||
* Creates a new Geo3DPoint field with the specified x,y,z.
|
||||
*
|
||||
* @throws IllegalArgumentException if the field name is null or lat or lon are out of bounds
|
||||
*/
|
||||
public Geo3DPointField(String name, PlanetModel planetModel, double x, double y, double z) {
|
||||
public Geo3DPoint(String name, PlanetModel planetModel, double x, double y, double z) {
|
||||
super(name, TYPE);
|
||||
fillFieldsData(planetModel.getMaximumMagnitude(), x, y, z);
|
||||
}
|
|
@ -19,9 +19,9 @@ package org.apache.lucene.geo3d;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.DimensionalValues.IntersectVisitor;
|
||||
import org.apache.lucene.index.DimensionalValues.Relation;
|
||||
import org.apache.lucene.index.DimensionalValues;
|
||||
import org.apache.lucene.index.PointValues.IntersectVisitor;
|
||||
import org.apache.lucene.index.PointValues;
|
||||
import org.apache.lucene.index.PointValues.Relation;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.search.ConstantScoreScorer;
|
||||
|
@ -35,7 +35,7 @@ import org.apache.lucene.util.NumericUtils;
|
|||
|
||||
/** Finds all previously indexed points that fall within the specified polygon.
|
||||
*
|
||||
* <p>The field must be indexed using {@link Geo3DPointField}.
|
||||
* <p>The field must be indexed using {@link Geo3DPoint}.
|
||||
*
|
||||
* @lucene.experimental */
|
||||
|
||||
|
@ -62,7 +62,7 @@ public class PointInGeo3DShapeQuery extends Query {
|
|||
@Override
|
||||
public Scorer scorer(LeafReaderContext context) throws IOException {
|
||||
LeafReader reader = context.reader();
|
||||
DimensionalValues values = reader.getDimensionalValues();
|
||||
PointValues values = reader.getPointValues();
|
||||
if (values == null) {
|
||||
return null;
|
||||
}
|
||||
|
|
|
@ -28,12 +28,12 @@ import java.util.concurrent.CountDownLatch;
|
|||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
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.PointFormat;
|
||||
import org.apache.lucene.codecs.PointReader;
|
||||
import org.apache.lucene.codecs.PointWriter;
|
||||
import org.apache.lucene.codecs.FilterCodec;
|
||||
import org.apache.lucene.codecs.lucene60.Lucene60DimensionalReader;
|
||||
import org.apache.lucene.codecs.lucene60.Lucene60DimensionalWriter;
|
||||
import org.apache.lucene.codecs.lucene60.Lucene60PointReader;
|
||||
import org.apache.lucene.codecs.lucene60.Lucene60PointWriter;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.NumericDocValuesField;
|
||||
|
@ -60,7 +60,7 @@ import org.junit.BeforeClass;
|
|||
|
||||
import com.carrotsearch.randomizedtesting.generators.RandomInts;
|
||||
|
||||
public class TestGeo3DPointField extends LuceneTestCase {
|
||||
public class TestGeo3DPoint extends LuceneTestCase {
|
||||
|
||||
private static boolean smallBBox;
|
||||
|
||||
|
@ -77,21 +77,21 @@ public class TestGeo3DPointField extends LuceneTestCase {
|
|||
int maxPointsInLeafNode = TestUtil.nextInt(random(), 16, 2048);
|
||||
double maxMBSortInHeap = 3.0 + (3*random().nextDouble());
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: using Lucene60DimensionalFormat with maxPointsInLeafNode=" + maxPointsInLeafNode + " and maxMBSortInHeap=" + maxMBSortInHeap);
|
||||
System.out.println("TEST: using Lucene60PointFormat with maxPointsInLeafNode=" + maxPointsInLeafNode + " and maxMBSortInHeap=" + maxMBSortInHeap);
|
||||
}
|
||||
|
||||
return new FilterCodec("Lucene60", Codec.getDefault()) {
|
||||
@Override
|
||||
public DimensionalFormat dimensionalFormat() {
|
||||
return new DimensionalFormat() {
|
||||
public PointFormat pointFormat() {
|
||||
return new PointFormat() {
|
||||
@Override
|
||||
public DimensionalWriter fieldsWriter(SegmentWriteState writeState) throws IOException {
|
||||
return new Lucene60DimensionalWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap);
|
||||
public PointWriter fieldsWriter(SegmentWriteState writeState) throws IOException {
|
||||
return new Lucene60PointWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DimensionalReader fieldsReader(SegmentReadState readState) throws IOException {
|
||||
return new Lucene60DimensionalReader(readState);
|
||||
public PointReader fieldsReader(SegmentReadState readState) throws IOException {
|
||||
return new Lucene60PointReader(readState);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -107,7 +107,7 @@ public class TestGeo3DPointField extends LuceneTestCase {
|
|||
iwc.setCodec(getCodec());
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new Geo3DPointField("field", PlanetModel.WGS84, toRadians(50.7345267), toRadians(-97.5303555)));
|
||||
doc.add(new Geo3DPoint("field", PlanetModel.WGS84, toRadians(50.7345267), toRadians(-97.5303555)));
|
||||
w.addDocument(doc);
|
||||
IndexReader r = DirectoryReader.open(w, true);
|
||||
// We can't wrap with "exotic" readers because the query must see the BKD3DDVFormat:
|
||||
|
@ -663,7 +663,7 @@ public class TestGeo3DPointField extends LuceneTestCase {
|
|||
doc.add(newStringField("id", ""+id, Field.Store.NO));
|
||||
doc.add(new NumericDocValuesField("id", id));
|
||||
if (Double.isNaN(lats[id]) == false) {
|
||||
doc.add(new Geo3DPointField("point", planetModel, lats[id], lons[id]));
|
||||
doc.add(new Geo3DPoint("point", planetModel, lats[id], lons[id]));
|
||||
}
|
||||
w.addDocument(doc);
|
||||
if (id > 0 && random().nextInt(100) == 42) {
|
|
@ -34,7 +34,7 @@ import org.apache.lucene.analysis.TokenStream;
|
|||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene60.Lucene60Codec;
|
||||
import org.apache.lucene.document.DimensionalIntField;
|
||||
import org.apache.lucene.document.IntPoint;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.StoredField;
|
||||
|
@ -44,7 +44,7 @@ import org.apache.lucene.index.IndexWriterConfig;
|
|||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.RandomIndexWriter;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.DimensionalRangeQuery;
|
||||
import org.apache.lucene.search.PointRangeQuery;
|
||||
import org.apache.lucene.search.ScoreDoc;
|
||||
import org.apache.lucene.search.TopDocs;
|
||||
import org.apache.lucene.search.suggest.BitsProducer;
|
||||
|
@ -305,7 +305,7 @@ public class TestSuggestField extends LuceneTestCase {
|
|||
Document document = new Document();
|
||||
document.add(new SuggestField("suggest_field", "abc_" + i, i));
|
||||
document.add(new StoredField("weight_fld", i));
|
||||
document.add(new DimensionalIntField("weight_fld", i));
|
||||
document.add(new IntPoint("weight_fld", i));
|
||||
iw.addDocument(document);
|
||||
|
||||
if (usually()) {
|
||||
|
@ -313,7 +313,7 @@ public class TestSuggestField extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
iw.deleteDocuments(DimensionalRangeQuery.new1DIntRange("weight_fld", 2, true, null, false));
|
||||
iw.deleteDocuments(PointRangeQuery.new1DIntRange("weight_fld", 2, true, null, false));
|
||||
|
||||
DirectoryReader reader = DirectoryReader.open(iw);
|
||||
SuggestIndexSearcher indexSearcher = new SuggestIndexSearcher(reader);
|
||||
|
|
|
@ -17,7 +17,7 @@ package org.apache.lucene.codecs.asserting;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.codecs.DimensionalFormat;
|
||||
import org.apache.lucene.codecs.PointFormat;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.FilterCodec;
|
||||
import org.apache.lucene.codecs.LiveDocsFormat;
|
||||
|
@ -54,7 +54,7 @@ public class AssertingCodec extends FilterCodec {
|
|||
private final LiveDocsFormat liveDocs = new AssertingLiveDocsFormat();
|
||||
private final PostingsFormat defaultFormat = new AssertingPostingsFormat();
|
||||
private final DocValuesFormat defaultDVFormat = new AssertingDocValuesFormat();
|
||||
private final DimensionalFormat dimensionalFormat = new AssertingDimensionalFormat();
|
||||
private final PointFormat pointFormat = new AssertingPointFormat();
|
||||
|
||||
public AssertingCodec() {
|
||||
super("Asserting", TestUtil.getDefaultCodec());
|
||||
|
@ -91,8 +91,8 @@ public class AssertingCodec extends FilterCodec {
|
|||
}
|
||||
|
||||
@Override
|
||||
public DimensionalFormat dimensionalFormat() {
|
||||
return dimensionalFormat;
|
||||
public PointFormat pointFormat() {
|
||||
return pointFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -20,9 +20,9 @@ package org.apache.lucene.codecs.asserting;
|
|||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
|
||||
import org.apache.lucene.codecs.DimensionalFormat;
|
||||
import org.apache.lucene.codecs.DimensionalReader;
|
||||
import org.apache.lucene.codecs.DimensionalWriter;
|
||||
import org.apache.lucene.codecs.PointFormat;
|
||||
import org.apache.lucene.codecs.PointReader;
|
||||
import org.apache.lucene.codecs.PointWriter;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.MergeState;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
|
@ -31,26 +31,26 @@ import org.apache.lucene.util.Accountable;
|
|||
import org.apache.lucene.util.TestUtil;
|
||||
|
||||
/**
|
||||
* Just like the default dimensional format but with additional asserts.
|
||||
* Just like the default point format but with additional asserts.
|
||||
*/
|
||||
|
||||
public final class AssertingDimensionalFormat extends DimensionalFormat {
|
||||
private final DimensionalFormat in = TestUtil.getDefaultCodec().dimensionalFormat();
|
||||
public final class AssertingPointFormat extends PointFormat {
|
||||
private final PointFormat in = TestUtil.getDefaultCodec().pointFormat();
|
||||
|
||||
@Override
|
||||
public DimensionalWriter fieldsWriter(SegmentWriteState state) throws IOException {
|
||||
return new AssertingDimensionalWriter(state, in.fieldsWriter(state));
|
||||
public PointWriter fieldsWriter(SegmentWriteState state) throws IOException {
|
||||
return new AssertingPointWriter(state, in.fieldsWriter(state));
|
||||
}
|
||||
|
||||
@Override
|
||||
public DimensionalReader fieldsReader(SegmentReadState state) throws IOException {
|
||||
return new AssertingDimensionalReader(in.fieldsReader(state));
|
||||
public PointReader fieldsReader(SegmentReadState state) throws IOException {
|
||||
return new AssertingPointReader(in.fieldsReader(state));
|
||||
}
|
||||
|
||||
static class AssertingDimensionalReader extends DimensionalReader {
|
||||
private final DimensionalReader in;
|
||||
static class AssertingPointReader extends PointReader {
|
||||
private final PointReader in;
|
||||
|
||||
AssertingDimensionalReader(DimensionalReader in) {
|
||||
AssertingPointReader(PointReader in) {
|
||||
this.in = in;
|
||||
// do a few simple checks on init
|
||||
assert toString() != null;
|
||||
|
@ -90,8 +90,8 @@ public final class AssertingDimensionalFormat extends DimensionalFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public DimensionalReader getMergeInstance() throws IOException {
|
||||
return new AssertingDimensionalReader(in.getMergeInstance());
|
||||
public PointReader getMergeInstance() throws IOException {
|
||||
return new AssertingPointReader(in.getMergeInstance());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -120,17 +120,17 @@ public final class AssertingDimensionalFormat extends DimensionalFormat {
|
|||
}
|
||||
}
|
||||
|
||||
static class AssertingDimensionalWriter extends DimensionalWriter {
|
||||
private final DimensionalWriter in;
|
||||
static class AssertingPointWriter extends PointWriter {
|
||||
private final PointWriter in;
|
||||
|
||||
AssertingDimensionalWriter(SegmentWriteState writeState, DimensionalWriter in) {
|
||||
AssertingPointWriter(SegmentWriteState writeState, PointWriter in) {
|
||||
this.in = in;
|
||||
}
|
||||
|
||||
@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");
|
||||
public void writeField(FieldInfo fieldInfo, PointReader values) throws IOException {
|
||||
if (fieldInfo.getPointDimensionCount() == 0) {
|
||||
throw new IllegalArgumentException("writing field=\"" + fieldInfo.name + "\" but pointDimensionalCount is 0");
|
||||
}
|
||||
in.writeField(fieldInfo, values);
|
||||
}
|
|
@ -312,7 +312,7 @@ abstract class BaseIndexFileFormatTestCase extends LuceneTestCase {
|
|||
FieldInfo proto = oneDocReader.getFieldInfos().fieldInfo("field");
|
||||
FieldInfo field = new FieldInfo(proto.name, proto.number, proto.hasVectors(), proto.omitsNorms(), proto.hasPayloads(),
|
||||
proto.getIndexOptions(), proto.getDocValuesType(), proto.getDocValuesGen(), new HashMap<>(),
|
||||
proto.getDimensionCount(), proto.getDimensionNumBytes());
|
||||
proto.getPointDimensionCount(), proto.getPointNumBytes());
|
||||
|
||||
FieldInfos fieldInfos = new FieldInfos(new FieldInfo[] { field } );
|
||||
|
||||
|
|
|
@ -67,8 +67,8 @@ public class MismatchedLeafReader extends FilterLeafReader {
|
|||
oldInfo.getDocValuesType(), // docValuesType
|
||||
oldInfo.getDocValuesGen(), // dvGen
|
||||
oldInfo.attributes(), // attributes
|
||||
oldInfo.getDimensionCount(), // dimension count
|
||||
oldInfo.getDimensionNumBytes()); // dimension numBytes
|
||||
oldInfo.getPointDimensionCount(), // dimension count
|
||||
oldInfo.getPointNumBytes()); // dimension numBytes
|
||||
shuffled.set(i, newInfo);
|
||||
}
|
||||
|
||||
|
|
|
@ -24,7 +24,7 @@ import java.util.List;
|
|||
import java.util.Random;
|
||||
|
||||
import org.apache.lucene.index.BinaryDocValues;
|
||||
import org.apache.lucene.index.DimensionalValues;
|
||||
import org.apache.lucene.index.PointValues;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.Fields;
|
||||
|
@ -257,7 +257,7 @@ public class QueryUtils {
|
|||
}
|
||||
|
||||
@Override
|
||||
public DimensionalValues getDimensionalValues() {
|
||||
public PointValues getPointValues() {
|
||||
return null;
|
||||
}
|
||||
|
||||
|
|
|
@ -34,10 +34,11 @@ import java.util.Random;
|
|||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.zip.GZIPInputStream;
|
||||
|
||||
import org.apache.lucene.document.DimensionalIntField;
|
||||
import org.apache.lucene.document.IntPoint;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.FieldType;
|
||||
import org.apache.lucene.document.IntPoint;
|
||||
import org.apache.lucene.document.NumericDocValuesField;
|
||||
import org.apache.lucene.document.SortedDocValuesField;
|
||||
import org.apache.lucene.document.StringField;
|
||||
|
@ -186,7 +187,7 @@ public class LineFileDocs implements Closeable {
|
|||
id = new StringField("docid", "", Field.Store.YES);
|
||||
doc.add(id);
|
||||
|
||||
idNum = new DimensionalIntField("docid_int", 0);
|
||||
idNum = new IntPoint("docid_int", 0);
|
||||
doc.add(idNum);
|
||||
|
||||
date = new StringField("date", "", Field.Store.YES);
|
||||
|
|
|
@ -57,7 +57,7 @@ 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;
|
||||
import org.apache.lucene.document.DimensionalBinaryField;
|
||||
import org.apache.lucene.document.BinaryPoint;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.FieldType.LegacyNumericType;
|
||||
|
@ -1037,7 +1037,7 @@ public final class TestUtil {
|
|||
final Field field1 = (Field) f;
|
||||
final Field field2;
|
||||
final DocValuesType dvType = field1.fieldType().docValuesType();
|
||||
final int dimCount = field1.fieldType().dimensionCount();
|
||||
final int dimCount = field1.fieldType().pointDimensionCount();
|
||||
final LegacyNumericType numType = field1.fieldType().numericType();
|
||||
if (dvType != DocValuesType.NONE) {
|
||||
switch(dvType) {
|
||||
|
@ -1057,7 +1057,7 @@ public final class TestUtil {
|
|||
BytesRef br = field1.binaryValue();
|
||||
byte[] bytes = new byte[br.length];
|
||||
System.arraycopy(br.bytes, br.offset, bytes, 0, br.length);
|
||||
field2 = new DimensionalBinaryField(field1.name(), bytes, field1.fieldType());
|
||||
field2 = new BinaryPoint(field1.name(), bytes, field1.fieldType());
|
||||
} else if (numType != null) {
|
||||
switch (numType) {
|
||||
case INT:
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue