You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by nk...@apache.org on 2018/10/10 17:01:42 UTC
[6/6] lucene-solr:branch_7x: LUCENE-8496: revert Selective indexing -
modify BKDReader/BKDWriter to allow users to select a fewer number of
dimensions to be used for creating the index than the total number of
dimensions used for field encoding. i.e., di
LUCENE-8496: revert Selective indexing - modify BKDReader/BKDWriter to allow users to select a fewer number of dimensions to be used for creating the index than the total number of dimensions used for field encoding. i.e., dimensions 0 to N may be used to determine how to split the inner nodes, and dimensions N+1 to D are ignored and stored as data dimensions at the leaves.
Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/ac11c9e5
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/ac11c9e5
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/ac11c9e5
Branch: refs/heads/branch_7x
Commit: ac11c9e5b17dc7f9abd151dfe0ee880374a38542
Parents: 7ff5298
Author: Nicholas Knize <nk...@gmail.com>
Authored: Wed Oct 10 12:00:50 2018 -0500
Committer: Nicholas Knize <nk...@gmail.com>
Committed: Wed Oct 10 12:00:50 2018 -0500
----------------------------------------------------------------------
lucene/CHANGES.txt | 8 -
.../codecs/simpletext/SimpleTextBKDReader.java | 59 +++----
.../codecs/simpletext/SimpleTextBKDWriter.java | 144 +++++++--------
.../simpletext/SimpleTextFieldInfosFormat.java | 21 +--
.../simpletext/SimpleTextPointsReader.java | 20 +--
.../simpletext/SimpleTextPointsWriter.java | 6 +-
.../org/apache/lucene/codecs/PointsWriter.java | 13 +-
.../lucene50/Lucene50FieldInfosFormat.java | 2 +-
.../lucene60/Lucene60FieldInfosFormat.java | 21 +--
.../codecs/lucene60/Lucene60PointsReader.java | 2 +-
.../codecs/lucene60/Lucene60PointsWriter.java | 14 +-
.../codecs/perfield/PerFieldMergeState.java | 2 +-
.../org/apache/lucene/document/BinaryPoint.java | 4 +-
.../org/apache/lucene/document/DoublePoint.java | 10 +-
.../org/apache/lucene/document/DoubleRange.java | 10 +-
.../org/apache/lucene/document/FieldType.java | 66 ++-----
.../org/apache/lucene/document/FloatPoint.java | 10 +-
.../org/apache/lucene/document/FloatRange.java | 10 +-
.../org/apache/lucene/document/IntPoint.java | 10 +-
.../org/apache/lucene/document/IntRange.java | 10 +-
.../org/apache/lucene/document/LatLonPoint.java | 6 +-
.../org/apache/lucene/document/LongPoint.java | 10 +-
.../org/apache/lucene/document/LongRange.java | 10 +-
.../apache/lucene/document/RangeFieldQuery.java | 4 +-
.../org/apache/lucene/index/CheckIndex.java | 37 ++--
.../org/apache/lucene/index/CodecReader.java | 2 +-
.../lucene/index/DefaultIndexingChain.java | 15 +-
.../java/org/apache/lucene/index/FieldInfo.java | 73 +++-----
.../org/apache/lucene/index/FieldInfos.java | 61 +++----
.../org/apache/lucene/index/IndexWriter.java | 6 +-
.../apache/lucene/index/IndexableFieldType.java | 9 +-
.../org/apache/lucene/index/PointValues.java | 11 +-
.../apache/lucene/index/PointValuesWriter.java | 20 +--
.../apache/lucene/index/SortingLeafReader.java | 9 +-
.../apache/lucene/search/PointInSetQuery.java | 4 +-
.../apache/lucene/search/PointRangeQuery.java | 4 +-
.../org/apache/lucene/util/bkd/BKDReader.java | 147 +++++++---------
.../org/apache/lucene/util/bkd/BKDWriter.java | 174 +++++++++----------
.../apache/lucene/document/TestFieldType.java | 11 +-
.../apache/lucene/index/TestIndexableField.java | 7 +-
.../lucene/index/TestPendingSoftDeletes.java | 10 +-
.../apache/lucene/index/TestPointValues.java | 14 +-
.../apache/lucene/search/TestPointQueries.java | 2 +-
.../apache/lucene/util/TestDocIdSetBuilder.java | 7 +-
.../apache/lucene/util/bkd/Test2BBKDPoints.java | 4 +-
.../org/apache/lucene/util/bkd/TestBKD.java | 142 +++++++--------
.../util/bkd/TestMutablePointsReaderUtils.java | 7 +-
.../search/highlight/TermVectorLeafReader.java | 2 +-
.../join/PointInSetIncludingScoreQuery.java | 4 +-
.../apache/lucene/index/memory/MemoryIndex.java | 24 +--
.../apache/lucene/document/BigIntegerPoint.java | 10 +-
.../apache/lucene/document/HalfFloatPoint.java | 10 +-
.../lucene/spatial/bbox/BBoxStrategy.java | 2 +-
.../spatial/vector/PointVectorStrategy.java | 2 +-
.../codecs/asserting/AssertingPointsFormat.java | 4 +-
.../codecs/cranky/CrankyPointsFormat.java | 12 +-
.../lucene/index/AssertingLeafReader.java | 41 ++---
.../index/BaseIndexFileFormatTestCase.java | 2 +-
.../lucene/index/BasePointsFormatTestCase.java | 76 +++-----
.../lucene/index/MismatchedLeafReader.java | 3 +-
.../org/apache/lucene/index/RandomCodec.java | 9 +-
.../lucene/index/RandomPostingsTester.java | 4 +-
.../org/apache/lucene/util/LuceneTestCase.java | 16 +-
.../java/org/apache/lucene/util/TestUtil.java | 2 +-
.../solr/handler/component/ExpandComponent.java | 3 +-
.../org/apache/solr/legacy/BBoxStrategy.java | 2 +-
.../apache/solr/legacy/PointVectorStrategy.java | 2 +-
.../org/apache/solr/schema/SchemaField.java | 7 +-
.../solr/search/CollapsingQParserPlugin.java | 2 +-
.../java/org/apache/solr/search/Insanity.java | 2 +-
.../apache/solr/uninverting/FieldCacheImpl.java | 6 +-
.../solr/uninverting/UninvertingReader.java | 6 +-
.../solr/uninverting/TestUninvertingReader.java | 6 +-
73 files changed, 611 insertions(+), 896 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index d66d980..729809f 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -19,14 +19,6 @@ Bug fixes:
* LUCENE-8479: QueryBuilder#analyzeGraphPhrase now throws TooManyClause exception
if the number of expanded path reaches the BooleanQuery#maxClause limit. (Jim Ferenczi)
-New Features
-
-* LUCENE-8496: Selective indexing - modify BKDReader/BKDWriter to allow users
- to select a fewer number of dimensions to be used for creating the index than
- the total number of dimensions used for field encoding. i.e., dimensions 0 to N
- may be used to determine how to split the inner nodes, and dimensions N+1 to D
- are ignored and stored as data dimensions at the leaves. (Nick Knize)
-
======================= Lucene 7.5.1 =======================
Bug Fixes:
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextBKDReader.java
----------------------------------------------------------------------
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextBKDReader.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextBKDReader.java
index 5227d6d..b7af45a 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextBKDReader.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextBKDReader.java
@@ -41,8 +41,7 @@ final class SimpleTextBKDReader extends PointValues implements Accountable {
final private byte[] splitPackedValues;
final long[] leafBlockFPs;
final private int leafNodeOffset;
- final int numDataDims;
- final int numIndexDims;
+ final int numDims;
final int bytesPerDim;
final int bytesPerIndexEntry;
final IndexInput in;
@@ -53,19 +52,16 @@ final class SimpleTextBKDReader extends PointValues implements Accountable {
final int docCount;
final int version;
protected final int packedBytesLength;
- protected final int packedIndexBytesLength;
- public SimpleTextBKDReader(IndexInput in, int numDataDims, int numIndexDims, int maxPointsInLeafNode, int bytesPerDim, long[] leafBlockFPs, byte[] splitPackedValues,
+ public SimpleTextBKDReader(IndexInput in, int numDims, int maxPointsInLeafNode, int bytesPerDim, long[] leafBlockFPs, byte[] splitPackedValues,
byte[] minPackedValue, byte[] maxPackedValue, long pointCount, int docCount) throws IOException {
this.in = in;
- this.numDataDims = numDataDims;
- this.numIndexDims = numIndexDims;
+ this.numDims = numDims;
this.maxPointsInLeafNode = maxPointsInLeafNode;
this.bytesPerDim = bytesPerDim;
// no version check here because callers of this API (SimpleText) have no back compat:
- bytesPerIndexEntry = numIndexDims == 1 ? bytesPerDim : bytesPerDim + 1;
- packedBytesLength = numDataDims * bytesPerDim;
- packedIndexBytesLength = numIndexDims * bytesPerDim;
+ bytesPerIndexEntry = numDims == 1 ? bytesPerDim : bytesPerDim + 1;
+ packedBytesLength = numDims * bytesPerDim;
this.leafNodeOffset = leafBlockFPs.length;
this.leafBlockFPs = leafBlockFPs;
this.splitPackedValues = splitPackedValues;
@@ -74,8 +70,8 @@ final class SimpleTextBKDReader extends PointValues implements Accountable {
this.pointCount = pointCount;
this.docCount = docCount;
this.version = SimpleTextBKDWriter.VERSION_CURRENT;
- assert minPackedValue.length == packedIndexBytesLength;
- assert maxPackedValue.length == packedIndexBytesLength;
+ assert minPackedValue.length == packedBytesLength;
+ assert maxPackedValue.length == packedBytesLength;
}
/** Used to track all state for a single call to {@link #intersect}. */
@@ -119,7 +115,7 @@ final class SimpleTextBKDReader extends PointValues implements Accountable {
/** Create a new {@link IntersectState} */
public IntersectState getIntersectState(IntersectVisitor visitor) {
- return new IntersectState(in.clone(), numDataDims,
+ return new IntersectState(in.clone(), numDims,
packedBytesLength,
maxPointsInLeafNode,
visitor);
@@ -185,7 +181,7 @@ final class SimpleTextBKDReader extends PointValues implements Accountable {
scratchPackedValue[compressedByteOffset] = in.readByte();
final int runLen = Byte.toUnsignedInt(in.readByte());
for (int j = 0; j < runLen; ++j) {
- for(int dim=0;dim<numDataDims;dim++) {
+ for(int dim=0;dim<numDims;dim++) {
int prefix = commonPrefixLengths[dim];
in.readBytes(scratchPackedValue, dim*bytesPerDim + prefix, bytesPerDim - prefix);
}
@@ -200,14 +196,14 @@ final class SimpleTextBKDReader extends PointValues implements Accountable {
private int readCompressedDim(IndexInput in) throws IOException {
int compressedDim = in.readByte();
- if (compressedDim < -1 || compressedDim >= numIndexDims) {
+ if (compressedDim < -1 || compressedDim >= numDims) {
throw new CorruptIndexException("Got compressedDim="+compressedDim, in);
}
return compressedDim;
}
private void readCommonPrefixes(int[] commonPrefixLengths, byte[] scratchPackedValue, IndexInput in) throws IOException {
- for(int dim=0;dim<numDataDims;dim++) {
+ for(int dim=0;dim<numDims;dim++) {
int prefix = in.readVInt();
commonPrefixLengths[dim] = prefix;
if (prefix > 0) {
@@ -262,27 +258,27 @@ final class SimpleTextBKDReader extends PointValues implements Accountable {
int address = nodeID * bytesPerIndexEntry;
int splitDim;
- if (numIndexDims == 1) {
+ if (numDims == 1) {
splitDim = 0;
} else {
splitDim = splitPackedValues[address++] & 0xff;
}
- assert splitDim < numIndexDims;
+ assert splitDim < numDims;
// TODO: can we alloc & reuse this up front?
- byte[] splitPackedValue = new byte[packedIndexBytesLength];
+ byte[] splitPackedValue = new byte[packedBytesLength];
// Recurse on left sub-tree:
- System.arraycopy(cellMaxPacked, 0, splitPackedValue, 0, packedIndexBytesLength);
+ System.arraycopy(cellMaxPacked, 0, splitPackedValue, 0, packedBytesLength);
System.arraycopy(splitPackedValues, address, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
intersect(state,
2*nodeID,
cellMinPacked, splitPackedValue);
// Recurse on right sub-tree:
- System.arraycopy(cellMinPacked, 0, splitPackedValue, 0, packedIndexBytesLength);
+ System.arraycopy(cellMinPacked, 0, splitPackedValue, 0, packedBytesLength);
System.arraycopy(splitPackedValues, address, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
intersect(state,
2*nodeID+1,
@@ -311,27 +307,27 @@ final class SimpleTextBKDReader extends PointValues implements Accountable {
int address = nodeID * bytesPerIndexEntry;
int splitDim;
- if (numIndexDims == 1) {
+ if (numDims == 1) {
splitDim = 0;
} else {
splitDim = splitPackedValues[address++] & 0xff;
}
- assert splitDim < numIndexDims;
+ assert splitDim < numDims;
// TODO: can we alloc & reuse this up front?
- byte[] splitPackedValue = new byte[packedIndexBytesLength];
+ byte[] splitPackedValue = new byte[packedBytesLength];
// Recurse on left sub-tree:
- System.arraycopy(cellMaxPacked, 0, splitPackedValue, 0, packedIndexBytesLength);
+ System.arraycopy(cellMaxPacked, 0, splitPackedValue, 0, packedBytesLength);
System.arraycopy(splitPackedValues, address, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
final long leftCost = estimatePointCount(state,
2*nodeID,
cellMinPacked, splitPackedValue);
// Recurse on right sub-tree:
- System.arraycopy(cellMinPacked, 0, splitPackedValue, 0, packedIndexBytesLength);
+ System.arraycopy(cellMinPacked, 0, splitPackedValue, 0, packedBytesLength);
System.arraycopy(splitPackedValues, address, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
final long rightCost = estimatePointCount(state,
2*nodeID+1,
@@ -344,13 +340,13 @@ final class SimpleTextBKDReader extends PointValues implements Accountable {
public void copySplitValue(int nodeID, byte[] splitPackedValue) {
int address = nodeID * bytesPerIndexEntry;
int splitDim;
- if (numIndexDims == 1) {
+ if (numDims == 1) {
splitDim = 0;
} else {
splitDim = splitPackedValues[address++] & 0xff;
}
- assert splitDim < numIndexDims;
+ assert splitDim < numDims;
System.arraycopy(splitPackedValues, address, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
}
@@ -371,13 +367,8 @@ final class SimpleTextBKDReader extends PointValues implements Accountable {
}
@Override
- public int getNumDataDimensions() {
- return numDataDims;
- }
-
- @Override
- public int getNumIndexDimensions() {
- return numIndexDims;
+ public int getNumDimensions() {
+ return numDims;
}
@Override
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextBKDWriter.java
----------------------------------------------------------------------
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextBKDWriter.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextBKDWriter.java
index 0e3d5d6..3a161e8 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextBKDWriter.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextBKDWriter.java
@@ -64,8 +64,7 @@ import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.INDEX_C
import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.MAX_LEAF_POINTS;
import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.MAX_VALUE;
import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.MIN_VALUE;
-import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.NUM_DATA_DIMS;
-import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.NUM_INDEX_DIMS;
+import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.NUM_DIMS;
import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.POINT_COUNT;
import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.SPLIT_COUNT;
import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.SPLIT_DIM;
@@ -105,21 +104,15 @@ final class SimpleTextBKDWriter implements Closeable {
/** Maximum number of dimensions */
public static final int MAX_DIMS = 8;
- /** How many dimensions we are storing at the leaf (data) nodes */
- protected final int numDataDims;
-
- /** How many dimensions we are indexing in the internal nodes */
- protected final int numIndexDims;
+ /** How many dimensions we are indexing */
+ protected final int numDims;
/** How many bytes each value in each dimension takes. */
protected final int bytesPerDim;
- /** numDataDims * bytesPerDim */
+ /** numDims * bytesPerDim */
protected final int packedBytesLength;
- /** numIndexDims * bytesPerDim */
- protected final int packedIndexBytesLength;
-
final BytesRefBuilder scratch = new BytesRefBuilder();
final TrackingDirectoryWrapper tempDir;
@@ -167,39 +160,37 @@ final class SimpleTextBKDWriter implements Closeable {
private final int maxDoc;
- public SimpleTextBKDWriter(int maxDoc, Directory tempDir, String tempFileNamePrefix, int numDataDims, int numIndexDims, int bytesPerDim,
+ public SimpleTextBKDWriter(int maxDoc, Directory tempDir, String tempFileNamePrefix, int numDims, int bytesPerDim,
int maxPointsInLeafNode, double maxMBSortInHeap, long totalPointCount, boolean singleValuePerDoc) throws IOException {
- this(maxDoc, tempDir, tempFileNamePrefix, numDataDims, numIndexDims, bytesPerDim, maxPointsInLeafNode, maxMBSortInHeap, totalPointCount, singleValuePerDoc,
+ this(maxDoc, tempDir, tempFileNamePrefix, numDims, bytesPerDim, maxPointsInLeafNode, maxMBSortInHeap, totalPointCount, singleValuePerDoc,
totalPointCount > Integer.MAX_VALUE, Math.max(1, (long) maxMBSortInHeap), OfflineSorter.MAX_TEMPFILES);
}
- private SimpleTextBKDWriter(int maxDoc, Directory tempDir, String tempFileNamePrefix, int numDataDims, int numIndexDims, int bytesPerDim,
+ private SimpleTextBKDWriter(int maxDoc, Directory tempDir, String tempFileNamePrefix, int numDims, int bytesPerDim,
int maxPointsInLeafNode, double maxMBSortInHeap, long totalPointCount,
boolean singleValuePerDoc, boolean longOrds, long offlineSorterBufferMB, int offlineSorterMaxTempFiles) throws IOException {
- verifyParams(numDataDims, numIndexDims, maxPointsInLeafNode, maxMBSortInHeap, totalPointCount);
+ verifyParams(numDims, maxPointsInLeafNode, maxMBSortInHeap, totalPointCount);
// We use tracking dir to deal with removing files on exception, so each place that
// creates temp files doesn't need crazy try/finally/sucess logic:
this.tempDir = new TrackingDirectoryWrapper(tempDir);
this.tempFileNamePrefix = tempFileNamePrefix;
this.maxPointsInLeafNode = maxPointsInLeafNode;
- this.numDataDims = numDataDims;
- this.numIndexDims = numIndexDims;
+ this.numDims = numDims;
this.bytesPerDim = bytesPerDim;
this.totalPointCount = totalPointCount;
this.maxDoc = maxDoc;
this.offlineSorterBufferMB = OfflineSorter.BufferSize.megabytes(offlineSorterBufferMB);
this.offlineSorterMaxTempFiles = offlineSorterMaxTempFiles;
docsSeen = new FixedBitSet(maxDoc);
- packedBytesLength = numDataDims * bytesPerDim;
- packedIndexBytesLength = numIndexDims * bytesPerDim;
+ packedBytesLength = numDims * bytesPerDim;
scratchDiff = new byte[bytesPerDim];
scratch1 = new byte[packedBytesLength];
scratch2 = new byte[packedBytesLength];
- commonPrefixLengths = new int[numDataDims];
+ commonPrefixLengths = new int[numDims];
- minPackedValue = new byte[packedIndexBytesLength];
- maxPackedValue = new byte[packedIndexBytesLength];
+ minPackedValue = new byte[packedBytesLength];
+ maxPackedValue = new byte[packedBytesLength];
// If we may have more than 1+Integer.MAX_VALUE values, then we must encode ords with long (8 bytes), else we can use int (4 bytes).
this.longOrds = longOrds;
@@ -226,7 +217,7 @@ final class SimpleTextBKDWriter implements Closeable {
// bytes to points here. Each dimension has its own sorted partition, so
// we must divide by numDims as wel.
- maxPointsSortInHeap = (int) (0.5 * (maxMBSortInHeap * 1024 * 1024) / (bytesPerDoc * numDataDims));
+ maxPointsSortInHeap = (int) (0.5 * (maxMBSortInHeap * 1024 * 1024) / (bytesPerDoc * numDims));
// Finally, we must be able to hold at least the leaf node in heap during build:
if (maxPointsSortInHeap < maxPointsInLeafNode) {
@@ -239,14 +230,11 @@ final class SimpleTextBKDWriter implements Closeable {
this.maxMBSortInHeap = maxMBSortInHeap;
}
- public static void verifyParams(int numDataDims, int numIndexDims, int maxPointsInLeafNode, double maxMBSortInHeap, long totalPointCount) {
+ public static void verifyParams(int numDims, int maxPointsInLeafNode, double maxMBSortInHeap, long totalPointCount) {
// We encode dim in a single byte in the splitPackedValues, but we only expose 4 bits for it now, in case we want to use
// remaining 4 bits for another purpose later
- if (numDataDims < 1 || numDataDims > MAX_DIMS) {
- throw new IllegalArgumentException("numDataDims must be 1 .. " + MAX_DIMS + " (got: " + numDataDims + ")");
- }
- if (numIndexDims < 1 || numIndexDims > numDataDims) {
- throw new IllegalArgumentException("numIndexDims must be 1 .. " + numDataDims + " (got: " + numIndexDims + ")");
+ if (numDims < 1 || numDims > MAX_DIMS) {
+ throw new IllegalArgumentException("numDims must be 1 .. " + MAX_DIMS + " (got: " + numDims + ")");
}
if (maxPointsInLeafNode <= 0) {
throw new IllegalArgumentException("maxPointsInLeafNode must be > 0; got " + maxPointsInLeafNode);
@@ -295,10 +283,10 @@ final class SimpleTextBKDWriter implements Closeable {
// TODO: we could specialize for the 1D case:
if (pointCount == 0) {
- System.arraycopy(packedValue, 0, minPackedValue, 0, packedIndexBytesLength);
- System.arraycopy(packedValue, 0, maxPackedValue, 0, packedIndexBytesLength);
+ System.arraycopy(packedValue, 0, minPackedValue, 0, packedBytesLength);
+ System.arraycopy(packedValue, 0, maxPackedValue, 0, packedBytesLength);
} else {
- for(int dim=0;dim<numIndexDims;dim++) {
+ for(int dim=0;dim<numDims;dim++) {
int offset = dim*bytesPerDim;
if (FutureArrays.compareUnsigned(packedValue, offset, offset + bytesPerDim, minPackedValue, offset, offset + bytesPerDim) < 0) {
System.arraycopy(packedValue, offset, minPackedValue, offset, bytesPerDim);
@@ -445,7 +433,7 @@ final class SimpleTextBKDWriter implements Closeable {
* disk. This method does not use transient disk in order to reorder points.
*/
public long writeField(IndexOutput out, String fieldName, MutablePointValues reader) throws IOException {
- if (numIndexDims == 1) {
+ if (numDims == 1) {
return writeField1Dim(out, fieldName, reader);
} else {
return writeFieldNDims(out, fieldName, reader);
@@ -488,7 +476,7 @@ final class SimpleTextBKDWriter implements Closeable {
Arrays.fill(maxPackedValue, (byte) 0);
for (int i = 0; i < Math.toIntExact(pointCount); ++i) {
values.getValue(i, scratchBytesRef1);
- for(int dim=0;dim<numIndexDims;dim++) {
+ for(int dim=0;dim<numDims;dim++) {
int offset = dim*bytesPerDim;
if (FutureArrays.compareUnsigned(scratchBytesRef1.bytes, scratchBytesRef1.offset + offset, scratchBytesRef1.offset + offset + bytesPerDim, minPackedValue, offset, offset + bytesPerDim) < 0) {
System.arraycopy(scratchBytesRef1.bytes, scratchBytesRef1.offset + offset, minPackedValue, offset, bytesPerDim);
@@ -514,7 +502,7 @@ final class SimpleTextBKDWriter implements Closeable {
/* In the 1D case, we can simply sort points in ascending order and use the
* same writing logic as we use at merge time. */
private long writeField1Dim(IndexOutput out, String fieldName, MutablePointValues reader) throws IOException {
- MutablePointsReaderUtils.sort(maxDoc, packedIndexBytesLength, reader, 0, Math.toIntExact(reader.size()));
+ MutablePointsReaderUtils.sort(maxDoc, packedBytesLength, reader, 0, Math.toIntExact(reader.size()));
final OneDimensionBKDWriter oneDimWriter = new OneDimensionBKDWriter(out);
@@ -594,8 +582,8 @@ final class SimpleTextBKDWriter implements Closeable {
int leafCount;
OneDimensionBKDWriter(IndexOutput out) {
- if (numIndexDims != 1) {
- throw new UnsupportedOperationException("numIndexDims must be 1 but got " + numIndexDims);
+ if (numDims != 1) {
+ throw new UnsupportedOperationException("numDims must be 1 but got " + numDims);
}
if (pointCount != 0) {
throw new IllegalStateException("cannot mix add and merge");
@@ -673,9 +661,9 @@ final class SimpleTextBKDWriter implements Closeable {
private void writeLeafBlock() throws IOException {
assert leafCount != 0;
if (valueCount == 0) {
- System.arraycopy(leafValues, 0, minPackedValue, 0, packedIndexBytesLength);
+ System.arraycopy(leafValues, 0, minPackedValue, 0, packedBytesLength);
}
- System.arraycopy(leafValues, (leafCount - 1) * packedBytesLength, maxPackedValue, 0, packedIndexBytesLength);
+ System.arraycopy(leafValues, (leafCount - 1) * packedBytesLength, maxPackedValue, 0, packedBytesLength);
valueCount += leafCount;
@@ -688,7 +676,7 @@ final class SimpleTextBKDWriter implements Closeable {
Arrays.fill(commonPrefixLengths, bytesPerDim);
// Find per-dim common prefix:
- for(int dim=0;dim<numDataDims;dim++) {
+ for(int dim=0;dim<numDims;dim++) {
int offset1 = dim * bytesPerDim;
int offset2 = (leafCount - 1) * packedBytesLength + offset1;
for(int j=0;j<commonPrefixLengths[dim];j++) {
@@ -834,7 +822,7 @@ final class SimpleTextBKDWriter implements Closeable {
}
private PointWriter sort(int dim) throws IOException {
- assert dim >= 0 && dim < numDataDims;
+ assert dim >= 0 && dim < numDims;
if (heapPointWriter != null) {
@@ -867,7 +855,7 @@ final class SimpleTextBKDWriter implements Closeable {
final int offset = bytesPerDim * dim;
Comparator<BytesRef> cmp;
- if (dim == numDataDims - 1) {
+ if (dim == numDims - 1) {
// in that case the bytes for the dimension and for the doc id are contiguous,
// so we don't need a branch
cmp = new BytesRefComparator(bytesPerDim + Integer.BYTES) {
@@ -952,7 +940,7 @@ final class SimpleTextBKDWriter implements Closeable {
}
LongBitSet ordBitSet;
- if (numDataDims > 1) {
+ if (numDims > 1) {
if (singleValuePerDoc) {
ordBitSet = new LongBitSet(maxDoc);
} else {
@@ -987,7 +975,7 @@ final class SimpleTextBKDWriter implements Closeable {
assert pointCount / numLeaves <= maxPointsInLeafNode: "pointCount=" + pointCount + " numLeaves=" + numLeaves + " maxPointsInLeafNode=" + maxPointsInLeafNode;
// Sort all docs once by each dimension:
- PathSlice[] sortedPointWriters = new PathSlice[numDataDims];
+ PathSlice[] sortedPointWriters = new PathSlice[numDims];
// This is only used on exception; on normal code paths we close all files we opened:
List<Closeable> toCloseHeroically = new ArrayList<>();
@@ -995,7 +983,7 @@ final class SimpleTextBKDWriter implements Closeable {
boolean success = false;
try {
//long t0 = System.nanoTime();
- for(int dim=0;dim<numDataDims;dim++) {
+ for(int dim=0;dim<numDims;dim++) {
sortedPointWriters[dim] = new PathSlice(sort(dim), 0, pointCount);
}
//long t1 = System.nanoTime();
@@ -1043,12 +1031,8 @@ final class SimpleTextBKDWriter implements Closeable {
/** Subclass can change how it writes the index. */
private void writeIndex(IndexOutput out, long[] leafBlockFPs, byte[] splitPackedValues) throws IOException {
- write(out, NUM_DATA_DIMS);
- writeInt(out, numDataDims);
- newline(out);
-
- write(out, NUM_INDEX_DIMS);
- writeInt(out, numIndexDims);
+ write(out, NUM_DIMS);
+ writeInt(out, numDims);
newline(out);
write(out, BYTES_PER_DIM);
@@ -1132,7 +1116,7 @@ final class SimpleTextBKDWriter implements Closeable {
BytesRef ref = packedValues.apply(i);
assert ref.length == packedBytesLength;
- for(int dim=0;dim<numDataDims;dim++) {
+ for(int dim=0;dim<numDims;dim++) {
int prefix = commonPrefixLengths[dim];
out.writeBytes(ref.bytes, ref.offset + dim*bytesPerDim + prefix, bytesPerDim-prefix);
}
@@ -1216,7 +1200,7 @@ final class SimpleTextBKDWriter implements Closeable {
boolean result = reader.next();
assert result;
System.arraycopy(reader.packedValue(), splitDim*bytesPerDim, scratch1, 0, bytesPerDim);
- if (numDataDims > 1) {
+ if (numDims > 1) {
assert ordBitSet.get(reader.ord()) == false;
ordBitSet.set(reader.ord());
// Subtract 1 from rightCount because we already did the first value above (so we could record the split value):
@@ -1231,7 +1215,7 @@ final class SimpleTextBKDWriter implements Closeable {
/** Called only in assert */
private boolean valueInBounds(BytesRef packedValue, byte[] minPackedValue, byte[] maxPackedValue) {
- for(int dim=0;dim<numIndexDims;dim++) {
+ for(int dim=0;dim<numDims;dim++) {
int offset = bytesPerDim*dim;
if (FutureArrays.compareUnsigned(packedValue.bytes, packedValue.offset + offset, packedValue.offset + offset + bytesPerDim, minPackedValue, offset, offset + bytesPerDim) < 0) {
return false;
@@ -1247,7 +1231,7 @@ final class SimpleTextBKDWriter implements Closeable {
protected int split(byte[] minPackedValue, byte[] maxPackedValue) {
// Find which dim has the largest span so we can split on it:
int splitDim = -1;
- for(int dim=0;dim<numIndexDims;dim++) {
+ for(int dim=0;dim<numDims;dim++) {
NumericUtils.subtract(bytesPerDim, dim, maxPackedValue, minPackedValue, scratchDiff);
if (splitDim == -1 || FutureArrays.compareUnsigned(scratchDiff, 0, bytesPerDim, scratch1, 0, bytesPerDim) > 0) {
System.arraycopy(scratchDiff, 0, scratch1, 0, bytesPerDim);
@@ -1295,7 +1279,7 @@ final class SimpleTextBKDWriter implements Closeable {
reader.getValue(from, scratchBytesRef1);
for (int i = from + 1; i < to; ++i) {
reader.getValue(i, scratchBytesRef2);
- for (int dim=0;dim<numDataDims;dim++) {
+ for (int dim=0;dim<numDims;dim++) {
final int offset = dim * bytesPerDim;
for(int j=0;j<commonPrefixLengths[dim];j++) {
if (scratchBytesRef1.bytes[scratchBytesRef1.offset+offset+j] != scratchBytesRef2.bytes[scratchBytesRef2.offset+offset+j]) {
@@ -1307,14 +1291,14 @@ final class SimpleTextBKDWriter implements Closeable {
}
// Find the dimension that has the least number of unique bytes at commonPrefixLengths[dim]
- FixedBitSet[] usedBytes = new FixedBitSet[numDataDims];
- for (int dim = 0; dim < numDataDims; ++dim) {
+ FixedBitSet[] usedBytes = new FixedBitSet[numDims];
+ for (int dim = 0; dim < numDims; ++dim) {
if (commonPrefixLengths[dim] < bytesPerDim) {
usedBytes[dim] = new FixedBitSet(256);
}
}
for (int i = from + 1; i < to; ++i) {
- for (int dim=0;dim<numDataDims;dim++) {
+ for (int dim=0;dim<numDims;dim++) {
if (usedBytes[dim] != null) {
byte b = reader.getByteAt(i, dim * bytesPerDim + commonPrefixLengths[dim]);
usedBytes[dim].set(Byte.toUnsignedInt(b));
@@ -1323,7 +1307,7 @@ final class SimpleTextBKDWriter implements Closeable {
}
int sortedDim = 0;
int sortedDimCardinality = Integer.MAX_VALUE;
- for (int dim = 0; dim < numDataDims; ++dim) {
+ for (int dim = 0; dim < numDims; ++dim) {
if (usedBytes[dim] != null) {
final int cardinality = usedBytes[dim].cardinality();
if (cardinality < sortedDimCardinality) {
@@ -1386,8 +1370,8 @@ final class SimpleTextBKDWriter implements Closeable {
reader.getValue(mid, scratchBytesRef1);
System.arraycopy(scratchBytesRef1.bytes, scratchBytesRef1.offset + splitDim * bytesPerDim, splitPackedValues, address + 1, bytesPerDim);
- byte[] minSplitPackedValue = ArrayUtil.copyOfSubArray(minPackedValue, 0, packedIndexBytesLength);
- byte[] maxSplitPackedValue = ArrayUtil.copyOfSubArray(maxPackedValue, 0, packedIndexBytesLength);
+ byte[] minSplitPackedValue = ArrayUtil.copyOfSubArray(minPackedValue, 0, packedBytesLength);
+ byte[] maxSplitPackedValue = ArrayUtil.copyOfSubArray(maxPackedValue, 0, packedBytesLength);
System.arraycopy(scratchBytesRef1.bytes, scratchBytesRef1.offset + splitDim * bytesPerDim,
minSplitPackedValue, splitDim * bytesPerDim, bytesPerDim);
System.arraycopy(scratchBytesRef1.bytes, scratchBytesRef1.offset + splitDim * bytesPerDim,
@@ -1415,7 +1399,7 @@ final class SimpleTextBKDWriter implements Closeable {
assert slice.count == slices[0].count;
}
- if (numDataDims == 1 && slices[0].writer instanceof OfflinePointWriter && slices[0].count <= maxPointsSortInHeap) {
+ if (numDims == 1 && slices[0].writer instanceof OfflinePointWriter && slices[0].count <= maxPointsSortInHeap) {
// Special case for 1D, to cutover to heap once we recurse deeply enough:
slices[0] = switchToHeap(slices[0], toCloseHeroically);
}
@@ -1428,7 +1412,7 @@ final class SimpleTextBKDWriter implements Closeable {
int sortedDim = 0;
int sortedDimCardinality = Integer.MAX_VALUE;
- for (int dim=0;dim<numDataDims;dim++) {
+ for (int dim=0;dim<numDims;dim++) {
if (slices[dim].writer instanceof HeapPointWriter == false) {
// Adversarial cases can cause this, e.g. very lopsided data, all equal points, such that we started
// offline, but then kept splitting only in one dimension, and so never had to rewrite into heap writer
@@ -1513,7 +1497,7 @@ final class SimpleTextBKDWriter implements Closeable {
// Inner node: partition/recurse
int splitDim;
- if (numIndexDims > 1) {
+ if (numDims > 1) {
splitDim = split(minPackedValue, maxPackedValue);
} else {
splitDim = 0;
@@ -1534,24 +1518,24 @@ final class SimpleTextBKDWriter implements Closeable {
// Partition all PathSlice that are not the split dim into sorted left and right sets, so we can recurse:
- PathSlice[] leftSlices = new PathSlice[numDataDims];
- PathSlice[] rightSlices = new PathSlice[numDataDims];
+ PathSlice[] leftSlices = new PathSlice[numDims];
+ PathSlice[] rightSlices = new PathSlice[numDims];
- byte[] minSplitPackedValue = new byte[packedIndexBytesLength];
- System.arraycopy(minPackedValue, 0, minSplitPackedValue, 0, packedIndexBytesLength);
+ byte[] minSplitPackedValue = new byte[packedBytesLength];
+ System.arraycopy(minPackedValue, 0, minSplitPackedValue, 0, packedBytesLength);
- byte[] maxSplitPackedValue = new byte[packedIndexBytesLength];
- System.arraycopy(maxPackedValue, 0, maxSplitPackedValue, 0, packedIndexBytesLength);
+ byte[] maxSplitPackedValue = new byte[packedBytesLength];
+ System.arraycopy(maxPackedValue, 0, maxSplitPackedValue, 0, packedBytesLength);
// When we are on this dim, below, we clear the ordBitSet:
int dimToClear;
- if (numDataDims - 1 == splitDim) {
- dimToClear = numDataDims - 2;
+ if (numDims - 1 == splitDim) {
+ dimToClear = numDims - 2;
} else {
- dimToClear = numDataDims - 1;
+ dimToClear = numDims - 1;
}
- for(int dim=0;dim<numDataDims;dim++) {
+ for(int dim=0;dim<numDims;dim++) {
if (dim == splitDim) {
// No need to partition on this dim since it's a simple slice of the incoming already sorted slice, and we
@@ -1572,7 +1556,7 @@ final class SimpleTextBKDWriter implements Closeable {
long nextRightCount = reader.split(source.count, ordBitSet, leftPointWriter, rightPointWriter, dim == dimToClear);
if (rightCount != nextRightCount) {
- throw new IllegalStateException("wrong number of points in split: expected=" + rightCount + " but actual=" + nextRightCount + " in dim " + dim);
+ throw new IllegalStateException("wrong number of points in split: expected=" + rightCount + " but actual=" + nextRightCount);
}
leftSlices[dim] = new PathSlice(leftPointWriter, 0, leftCount);
@@ -1587,7 +1571,7 @@ final class SimpleTextBKDWriter implements Closeable {
ordBitSet, out,
minPackedValue, maxSplitPackedValue,
splitPackedValues, leafBlockFPs, toCloseHeroically);
- for(int dim=0;dim<numDataDims;dim++) {
+ for(int dim=0;dim<numDims;dim++) {
// Don't destroy the dim we split on because we just re-used what our caller above gave us for that dim:
if (dim != splitDim) {
leftSlices[dim].writer.destroy();
@@ -1600,7 +1584,7 @@ final class SimpleTextBKDWriter implements Closeable {
ordBitSet, out,
minSplitPackedValue, maxPackedValue,
splitPackedValues, leafBlockFPs, toCloseHeroically);
- for(int dim=0;dim<numDataDims;dim++) {
+ for(int dim=0;dim<numDims;dim++) {
// Don't destroy the dim we split on because we just re-used what our caller above gave us for that dim:
if (dim != splitDim) {
rightSlices[dim].writer.destroy();
@@ -1634,10 +1618,10 @@ final class SimpleTextBKDWriter implements Closeable {
if (ord > 0) {
int cmp = FutureArrays.compareUnsigned(lastPackedValue, dimOffset, dimOffset + bytesPerDim, packedValue, packedValueOffset + dimOffset, packedValueOffset + dimOffset + bytesPerDim);
if (cmp > 0) {
- throw new AssertionError("values out of order: last value=" + new BytesRef(lastPackedValue) + " current value=" + new BytesRef(packedValue, packedValueOffset, packedBytesLength) + " ord=" + ord + " sortedDim=" + sortedDim);
+ throw new AssertionError("values out of order: last value=" + new BytesRef(lastPackedValue) + " current value=" + new BytesRef(packedValue, packedValueOffset, packedBytesLength) + " ord=" + ord);
}
if (cmp == 0 && doc < lastDoc) {
- throw new AssertionError("docs out of order: last doc=" + lastDoc + " current doc=" + doc + " ord=" + ord + " sortedDim=" + sortedDim);
+ throw new AssertionError("docs out of order: last doc=" + lastDoc + " current doc=" + doc + " ord=" + ord);
}
}
System.arraycopy(packedValue, packedValueOffset, lastPackedValue, 0, packedBytesLength);
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosFormat.java
----------------------------------------------------------------------
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosFormat.java
index f52a84a..1c40cbd 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosFormat.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosFormat.java
@@ -64,8 +64,7 @@ public class SimpleTextFieldInfosFormat extends FieldInfosFormat {
static final BytesRef NUM_ATTS = new BytesRef(" attributes ");
static final BytesRef ATT_KEY = new BytesRef(" key ");
static final BytesRef ATT_VALUE = new BytesRef(" value ");
- static final BytesRef DATA_DIM_COUNT = new BytesRef(" data dimensional count ");
- static final BytesRef INDEX_DIM_COUNT = new BytesRef(" index dimensional count ");
+ static final BytesRef DIM_COUNT = new BytesRef(" dimensional count ");
static final BytesRef DIM_NUM_BYTES = new BytesRef(" dimensional num bytes ");
static final BytesRef SOFT_DELETES = new BytesRef(" soft-deletes ");
@@ -135,12 +134,8 @@ public class SimpleTextFieldInfosFormat extends FieldInfosFormat {
}
SimpleTextUtil.readLine(input, scratch);
- assert StringHelper.startsWith(scratch.get(), DATA_DIM_COUNT);
- int dataDimensionalCount = Integer.parseInt(readString(DATA_DIM_COUNT.length, scratch));
-
- SimpleTextUtil.readLine(input, scratch);
- assert StringHelper.startsWith(scratch.get(), INDEX_DIM_COUNT);
- int indexDimensionalCount = Integer.parseInt(readString(INDEX_DIM_COUNT.length, scratch));
+ assert StringHelper.startsWith(scratch.get(), DIM_COUNT);
+ int dimensionalCount = Integer.parseInt(readString(DIM_COUNT.length, scratch));
SimpleTextUtil.readLine(input, scratch);
assert StringHelper.startsWith(scratch.get(), DIM_NUM_BYTES);
@@ -152,7 +147,7 @@ public class SimpleTextFieldInfosFormat extends FieldInfosFormat {
infos[i] = new FieldInfo(name, fieldNumber, storeTermVector,
omitNorms, storePayloads, indexOptions, docValuesType, dvGen, Collections.unmodifiableMap(atts),
- dataDimensionalCount, indexDimensionalCount, dimensionalNumBytes, isSoftDeletesField);
+ dimensionalCount, dimensionalNumBytes, isSoftDeletesField);
}
SimpleTextUtil.checkFooter(input);
@@ -241,12 +236,8 @@ public class SimpleTextFieldInfosFormat extends FieldInfosFormat {
}
}
- SimpleTextUtil.write(out, DATA_DIM_COUNT);
- SimpleTextUtil.write(out, Integer.toString(fi.getPointDataDimensionCount()), scratch);
- SimpleTextUtil.writeNewline(out);
-
- SimpleTextUtil.write(out, INDEX_DIM_COUNT);
- SimpleTextUtil.write(out, Integer.toString(fi.getPointIndexDimensionCount()), scratch);
+ SimpleTextUtil.write(out, DIM_COUNT);
+ SimpleTextUtil.write(out, Integer.toString(fi.getPointDimensionCount()), scratch);
SimpleTextUtil.writeNewline(out);
SimpleTextUtil.write(out, DIM_NUM_BYTES);
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPointsReader.java
----------------------------------------------------------------------
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPointsReader.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPointsReader.java
index 728aad1..453bd23 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPointsReader.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPointsReader.java
@@ -47,8 +47,7 @@ import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.INDEX_C
import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.MAX_LEAF_POINTS;
import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.MAX_VALUE;
import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.MIN_VALUE;
-import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.NUM_DATA_DIMS;
-import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.NUM_INDEX_DIMS;
+import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.NUM_DIMS;
import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.POINT_COUNT;
import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.SPLIT_COUNT;
import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.SPLIT_DIM;
@@ -102,10 +101,7 @@ class SimpleTextPointsReader extends PointsReader {
// NOTE: matches what writeIndex does in SimpleTextPointsWriter
dataIn.seek(fp);
readLine(dataIn);
- int numDataDims = parseInt(NUM_DATA_DIMS);
-
- readLine(dataIn);
- int numIndexDims = parseInt(NUM_INDEX_DIMS);
+ int numDims = parseInt(NUM_DIMS);
readLine(dataIn);
int bytesPerDim = parseInt(BYTES_PER_DIM);
@@ -119,12 +115,12 @@ class SimpleTextPointsReader extends PointsReader {
readLine(dataIn);
assert startsWith(MIN_VALUE);
BytesRef minValue = SimpleTextUtil.fromBytesRefString(stripPrefix(MIN_VALUE));
- assert minValue.length == numIndexDims*bytesPerDim;
+ assert minValue.length == numDims*bytesPerDim;
readLine(dataIn);
assert startsWith(MAX_VALUE);
BytesRef maxValue = SimpleTextUtil.fromBytesRefString(stripPrefix(MAX_VALUE));
- assert maxValue.length == numIndexDims*bytesPerDim;
+ assert maxValue.length == numDims*bytesPerDim;
readLine(dataIn);
assert startsWith(POINT_COUNT);
@@ -144,7 +140,7 @@ class SimpleTextPointsReader extends PointsReader {
byte[] splitPackedValues;
int bytesPerIndexEntry;
- if (numIndexDims == 1) {
+ if (numDims == 1) {
bytesPerIndexEntry = bytesPerDim;
} else {
bytesPerIndexEntry = 1 + bytesPerDim;
@@ -154,7 +150,7 @@ class SimpleTextPointsReader extends PointsReader {
readLine(dataIn);
int address = bytesPerIndexEntry * i;
int splitDim = parseInt(SPLIT_DIM);
- if (numIndexDims != 1) {
+ if (numDims != 1) {
splitPackedValues[address++] = (byte) splitDim;
}
readLine(dataIn);
@@ -164,7 +160,7 @@ class SimpleTextPointsReader extends PointsReader {
System.arraycopy(br.bytes, br.offset, splitPackedValues, address, bytesPerDim);
}
- return new SimpleTextBKDReader(dataIn, numDataDims, numIndexDims, maxPointsInLeafNode, bytesPerDim, leafBlockFPs, splitPackedValues, minValue.bytes, maxValue.bytes, pointCount, docCount);
+ return new SimpleTextBKDReader(dataIn, numDims, maxPointsInLeafNode, bytesPerDim, leafBlockFPs, splitPackedValues, minValue.bytes, maxValue.bytes, pointCount, docCount);
}
private void readLine(IndexInput in) throws IOException {
@@ -195,7 +191,7 @@ class SimpleTextPointsReader extends PointsReader {
if (fieldInfo == null) {
throw new IllegalArgumentException("field=\"" + fieldName + "\" is unrecognized");
}
- if (fieldInfo.getPointDataDimensionCount() == 0) {
+ if (fieldInfo.getPointDimensionCount() == 0) {
throw new IllegalArgumentException("field=\"" + fieldName + "\" did not index points");
}
return readers.get(fieldName);
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPointsWriter.java
----------------------------------------------------------------------
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPointsWriter.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPointsWriter.java
index 2da74d6..c3217f3 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPointsWriter.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPointsWriter.java
@@ -35,8 +35,7 @@ import org.apache.lucene.util.BytesRefBuilder;
class SimpleTextPointsWriter extends PointsWriter {
- public final static BytesRef NUM_DATA_DIMS = new BytesRef("num data dims ");
- public final static BytesRef NUM_INDEX_DIMS = new BytesRef("num index dims ");
+ public final static BytesRef NUM_DIMS = new BytesRef("num dims ");
public final static BytesRef BYTES_PER_DIM = new BytesRef("bytes per dim ");
public final static BytesRef MAX_LEAF_POINTS = new BytesRef("max leaf points ");
public final static BytesRef INDEX_COUNT = new BytesRef("index count ");
@@ -77,8 +76,7 @@ class SimpleTextPointsWriter extends PointsWriter {
try (SimpleTextBKDWriter writer = new SimpleTextBKDWriter(writeState.segmentInfo.maxDoc(),
writeState.directory,
writeState.segmentInfo.name,
- fieldInfo.getPointDataDimensionCount(),
- fieldInfo.getPointIndexDimensionCount(),
+ fieldInfo.getPointDimensionCount(),
fieldInfo.getPointNumBytes(),
SimpleTextBKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE,
SimpleTextBKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP,
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/core/src/java/org/apache/lucene/codecs/PointsWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/PointsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/PointsWriter.java
index dd1eaeb..d9a0b30 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/PointsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/PointsWriter.java
@@ -48,7 +48,7 @@ public abstract class PointsWriter implements Closeable {
PointsReader pointsReader = mergeState.pointsReaders[i];
if (pointsReader != null) {
FieldInfo readerFieldInfo = mergeState.fieldInfos[i].fieldInfo(fieldInfo.name);
- if (readerFieldInfo != null && readerFieldInfo.getPointDataDimensionCount() > 0) {
+ if (readerFieldInfo != null && readerFieldInfo.getPointDimensionCount() > 0) {
PointValues values = pointsReader.getValues(fieldInfo.name);
if (values != null) {
maxPointCount += values.size();
@@ -92,7 +92,7 @@ public abstract class PointsWriter implements Closeable {
continue;
}
- if (readerFieldInfo.getPointDataDimensionCount() == 0) {
+ if (readerFieldInfo.getPointDimensionCount() == 0) {
// This segment saw this field, but the field did not index points in it:
continue;
}
@@ -143,12 +143,7 @@ public abstract class PointsWriter implements Closeable {
}
@Override
- public int getNumDataDimensions() {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public int getNumIndexDimensions() {
+ public int getNumDimensions() {
throw new UnsupportedOperationException();
}
@@ -187,7 +182,7 @@ public abstract class PointsWriter implements Closeable {
}
// merge field at a time
for (FieldInfo fieldInfo : mergeState.mergeFieldInfos) {
- if (fieldInfo.getPointDataDimensionCount() != 0) {
+ if (fieldInfo.getPointDimensionCount() != 0) {
mergeOneField(mergeState, fieldInfo);
}
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java
index 0ad0cad..30dca70 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java
@@ -148,7 +148,7 @@ public final class Lucene50FieldInfosFormat extends FieldInfosFormat {
lastAttributes = attributes;
try {
infos[i] = new FieldInfo(name, fieldNumber, storeTermVector, omitNorms, storePayloads,
- indexOptions, docValuesType, dvGen, attributes, 0, 0, 0, false);
+ indexOptions, docValuesType, dvGen, attributes, 0, 0, false);
infos[i].checkConsistency();
} catch (IllegalStateException e) {
throw new CorruptIndexException("invalid fieldinfo for field: " + name + ", fieldNumber=" + fieldNumber, input, e);
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60FieldInfosFormat.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60FieldInfosFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60FieldInfosFormat.java
index e2ca9ee..522a73f 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60FieldInfosFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60FieldInfosFormat.java
@@ -114,7 +114,7 @@ public final class Lucene60FieldInfosFormat extends FieldInfosFormat {
Throwable priorE = null;
FieldInfo infos[] = null;
try {
- int version = CodecUtil.checkIndexHeader(input,
+ CodecUtil.checkIndexHeader(input,
Lucene60FieldInfosFormat.CODEC_NAME,
Lucene60FieldInfosFormat.FORMAT_START,
Lucene60FieldInfosFormat.FORMAT_CURRENT,
@@ -149,13 +149,9 @@ public final class Lucene60FieldInfosFormat extends FieldInfosFormat {
attributes = lastAttributes;
}
lastAttributes = attributes;
- int pointDataDimensionCount = input.readVInt();
+ int pointDimensionCount = input.readVInt();
int pointNumBytes;
- int pointIndexDimensionCount = pointDataDimensionCount;
- if (pointDataDimensionCount != 0) {
- if (version >= Lucene60FieldInfosFormat.FORMAT_SELECTIVE_INDEXING) {
- pointIndexDimensionCount = input.readVInt();
- }
+ if (pointDimensionCount != 0) {
pointNumBytes = input.readVInt();
} else {
pointNumBytes = 0;
@@ -164,7 +160,7 @@ public final class Lucene60FieldInfosFormat extends FieldInfosFormat {
try {
infos[i] = new FieldInfo(name, fieldNumber, storeTermVector, omitNorms, storePayloads,
indexOptions, docValuesType, dvGen, attributes,
- pointDataDimensionCount, pointIndexDimensionCount, pointNumBytes, isSoftDeletesField);
+ pointDimensionCount, pointNumBytes, isSoftDeletesField);
infos[i].checkConsistency();
} catch (IllegalStateException e) {
throw new CorruptIndexException("invalid fieldinfo for field: " + name + ", fieldNumber=" + fieldNumber, input, e);
@@ -291,9 +287,9 @@ public final class Lucene60FieldInfosFormat extends FieldInfosFormat {
output.writeByte(docValuesByte(fi.getDocValuesType()));
output.writeLong(fi.getDocValuesGen());
output.writeMapOfStrings(fi.attributes());
- output.writeVInt(fi.getPointDataDimensionCount());
- if (fi.getPointDataDimensionCount() != 0) {
- output.writeVInt(fi.getPointIndexDimensionCount());
+ int pointDimensionCount = fi.getPointDimensionCount();
+ output.writeVInt(pointDimensionCount);
+ if (pointDimensionCount != 0) {
output.writeVInt(fi.getPointNumBytes());
}
}
@@ -308,8 +304,7 @@ public final class Lucene60FieldInfosFormat extends FieldInfosFormat {
static final String CODEC_NAME = "Lucene60FieldInfos";
static final int FORMAT_START = 0;
static final int FORMAT_SOFT_DELETES = 1;
- static final int FORMAT_SELECTIVE_INDEXING = 2;
- static final int FORMAT_CURRENT = FORMAT_SELECTIVE_INDEXING;
+ static final int FORMAT_CURRENT = FORMAT_SOFT_DELETES;
// Field flags
static final byte STORE_TERMVECTOR = 0x1;
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsReader.java
index b5ff680..0eecdbb 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsReader.java
@@ -124,7 +124,7 @@ public class Lucene60PointsReader extends PointsReader implements Closeable {
if (fieldInfo == null) {
throw new IllegalArgumentException("field=\"" + fieldName + "\" is unrecognized");
}
- if (fieldInfo.getPointDataDimensionCount() == 0) {
+ if (fieldInfo.getPointDimensionCount() == 0) {
throw new IllegalArgumentException("field=\"" + fieldName + "\" did not index point values");
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java
index fddf08c..4f51e26 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java
@@ -94,8 +94,7 @@ public class Lucene60PointsWriter extends PointsWriter implements Closeable {
try (BKDWriter writer = new BKDWriter(writeState.segmentInfo.maxDoc(),
writeState.directory,
writeState.segmentInfo.name,
- fieldInfo.getPointDataDimensionCount(),
- fieldInfo.getPointIndexDimensionCount(),
+ fieldInfo.getPointDimensionCount(),
fieldInfo.getPointNumBytes(),
maxPointsInLeafNode,
maxMBSortInHeap,
@@ -153,8 +152,8 @@ public class Lucene60PointsWriter extends PointsWriter implements Closeable {
}
for (FieldInfo fieldInfo : mergeState.mergeFieldInfos) {
- if (fieldInfo.getPointDataDimensionCount() != 0) {
- if (fieldInfo.getPointDataDimensionCount() == 1) {
+ if (fieldInfo.getPointDimensionCount() != 0) {
+ if (fieldInfo.getPointDimensionCount() == 1) {
boolean singleValuePerDoc = true;
@@ -165,7 +164,7 @@ public class Lucene60PointsWriter extends PointsWriter implements Closeable {
if (reader != null) {
FieldInfos readerFieldInfos = mergeState.fieldInfos[i];
FieldInfo readerFieldInfo = readerFieldInfos.fieldInfo(fieldInfo.name);
- if (readerFieldInfo != null && readerFieldInfo.getPointDataDimensionCount() > 0) {
+ if (readerFieldInfo != null && readerFieldInfo.getPointDimensionCount() > 0) {
PointValues values = reader.getValues(fieldInfo.name);
if (values != null) {
totMaxSize += values.size();
@@ -182,8 +181,7 @@ public class Lucene60PointsWriter extends PointsWriter implements Closeable {
try (BKDWriter writer = new BKDWriter(writeState.segmentInfo.maxDoc(),
writeState.directory,
writeState.segmentInfo.name,
- fieldInfo.getPointDataDimensionCount(),
- fieldInfo.getPointIndexDimensionCount(),
+ fieldInfo.getPointDimensionCount(),
fieldInfo.getPointNumBytes(),
maxPointsInLeafNode,
maxMBSortInHeap,
@@ -206,7 +204,7 @@ public class Lucene60PointsWriter extends PointsWriter implements Closeable {
FieldInfos readerFieldInfos = mergeState.fieldInfos[i];
FieldInfo readerFieldInfo = readerFieldInfos.fieldInfo(fieldInfo.name);
- if (readerFieldInfo != null && readerFieldInfo.getPointDataDimensionCount() > 0) {
+ if (readerFieldInfo != null && readerFieldInfo.getPointDimensionCount() > 0) {
BKDReader bkdReader = reader60.readers.get(readerFieldInfo.number);
if (bkdReader != null) {
bkdReaders.add(bkdReader);
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldMergeState.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldMergeState.java b/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldMergeState.java
index 291a384..991eedf 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldMergeState.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldMergeState.java
@@ -134,7 +134,7 @@ final class PerFieldMergeState {
hasNorms |= fi.hasNorms();
hasDocValues |= fi.getDocValuesType() != DocValuesType.NONE;
hasPayloads |= fi.hasPayloads();
- hasPointValues |= (fi.getPointDataDimensionCount() != 0);
+ hasPointValues |= (fi.getPointDimensionCount() != 0);
}
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/core/src/java/org/apache/lucene/document/BinaryPoint.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/BinaryPoint.java b/lucene/core/src/java/org/apache/lucene/document/BinaryPoint.java
index f4b625e..693a00f 100644
--- a/lucene/core/src/java/org/apache/lucene/document/BinaryPoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/BinaryPoint.java
@@ -124,8 +124,8 @@ public final class BinaryPoint extends Field {
/** Expert API */
public BinaryPoint(String name, byte[] packedPoint, IndexableFieldType type) {
super(name, packedPoint, type);
- if (packedPoint.length != type.pointDataDimensionCount() * type.pointNumBytes()) {
- throw new IllegalArgumentException("packedPoint is length=" + packedPoint.length + " but type.pointDimensionCount()=" + type.pointDataDimensionCount() + " and type.pointNumBytes()=" + type.pointNumBytes());
+ 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());
}
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java b/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java
index e442bec..6547402 100644
--- a/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java
@@ -85,8 +85,8 @@ public final class DoublePoint extends Field {
/** Change the values of this field */
public void setDoubleValues(double... point) {
- if (type.pointDataDimensionCount() != point.length) {
- throw new IllegalArgumentException("this field (name=" + name + ") uses " + type.pointDataDimensionCount() + " dimensions; cannot change to (incoming) " + point.length + " dimensions");
+ if (type.pointDimensionCount() != point.length) {
+ throw new IllegalArgumentException("this field (name=" + name + ") uses " + type.pointDimensionCount() + " dimensions; cannot change to (incoming) " + point.length + " dimensions");
}
fieldsData = pack(point);
}
@@ -98,8 +98,8 @@ public final class DoublePoint extends Field {
@Override
public Number numericValue() {
- if (type.pointDataDimensionCount() != 1) {
- throw new IllegalStateException("this field (name=" + name + ") uses " + type.pointDataDimensionCount() + " dimensions; cannot convert to a single numeric value");
+ if (type.pointDimensionCount() != 1) {
+ throw new IllegalStateException("this field (name=" + name + ") uses " + type.pointDimensionCount() + " dimensions; cannot convert to a single numeric value");
}
BytesRef bytes = (BytesRef) fieldsData;
assert bytes.length == Double.BYTES;
@@ -142,7 +142,7 @@ public final class DoublePoint extends Field {
result.append(':');
BytesRef bytes = (BytesRef) fieldsData;
- for (int dim = 0; dim < type.pointDataDimensionCount(); dim++) {
+ for (int dim = 0; dim < type.pointDimensionCount(); dim++) {
if (dim > 0) {
result.append(',');
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/core/src/java/org/apache/lucene/document/DoubleRange.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/DoubleRange.java b/lucene/core/src/java/org/apache/lucene/document/DoubleRange.java
index 726d0a1..cf308c3 100644
--- a/lucene/core/src/java/org/apache/lucene/document/DoubleRange.java
+++ b/lucene/core/src/java/org/apache/lucene/document/DoubleRange.java
@@ -77,8 +77,8 @@ public class DoubleRange extends Field {
*/
public void setRangeValues(double[] min, double[] max) {
checkArgs(min, max);
- if (min.length*2 != type.pointDataDimensionCount() || max.length*2 != type.pointDataDimensionCount()) {
- throw new IllegalArgumentException("field (name=" + name + ") uses " + type.pointDataDimensionCount()/2
+ if (min.length*2 != type.pointDimensionCount() || max.length*2 != type.pointDimensionCount()) {
+ throw new IllegalArgumentException("field (name=" + name + ") uses " + type.pointDimensionCount()/2
+ " dimensions; cannot change to (incoming) " + min.length + " dimensions");
}
@@ -148,7 +148,7 @@ public class DoubleRange extends Field {
* @return the decoded min value
*/
public double getMin(int dimension) {
- FutureObjects.checkIndex(dimension, type.pointDataDimensionCount()/2);
+ FutureObjects.checkIndex(dimension, type.pointDimensionCount()/2);
return decodeMin(((BytesRef)fieldsData).bytes, dimension);
}
@@ -158,7 +158,7 @@ public class DoubleRange extends Field {
* @return the decoded max value
*/
public double getMax(int dimension) {
- FutureObjects.checkIndex(dimension, type.pointDataDimensionCount()/2);
+ FutureObjects.checkIndex(dimension, type.pointDimensionCount()/2);
return decodeMax(((BytesRef)fieldsData).bytes, dimension);
}
@@ -244,7 +244,7 @@ public class DoubleRange extends Field {
sb.append(':');
byte[] b = ((BytesRef)fieldsData).bytes;
toString(b, 0);
- for (int d = 0; d < type.pointDataDimensionCount() / 2; ++d) {
+ for (int d = 0; d < type.pointDimensionCount() / 2; ++d) {
sb.append(' ');
sb.append(toString(b, d));
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/core/src/java/org/apache/lucene/document/FieldType.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/FieldType.java b/lucene/core/src/java/org/apache/lucene/document/FieldType.java
index a21572e..3c7d276 100644
--- a/lucene/core/src/java/org/apache/lucene/document/FieldType.java
+++ b/lucene/core/src/java/org/apache/lucene/document/FieldType.java
@@ -38,8 +38,7 @@ public class FieldType implements IndexableFieldType {
private IndexOptions indexOptions = IndexOptions.NONE;
private boolean frozen;
private DocValuesType docValuesType = DocValuesType.NONE;
- private int dataDimensionCount;
- private int indexDimensionCount;
+ private int dimensionCount;
private int dimensionNumBytes;
/**
@@ -55,8 +54,7 @@ public class FieldType implements IndexableFieldType {
this.omitNorms = ref.omitNorms();
this.indexOptions = ref.indexOptions();
this.docValuesType = ref.docValuesType();
- this.dataDimensionCount = ref.pointDataDimensionCount();
- this.indexDimensionCount = ref.pointIndexDimensionCount();
+ this.dimensionCount = ref.pointDimensionCount();
this.dimensionNumBytes = ref.pointNumBytes();
// Do not copy frozen!
}
@@ -281,24 +279,11 @@ public class FieldType implements IndexableFieldType {
* Enables points indexing.
*/
public void setDimensions(int dimensionCount, int dimensionNumBytes) {
- this.setDimensions(dimensionCount, dimensionCount, dimensionNumBytes);
- }
-
- /**
- * Enables points indexing with selectable dimension indexing.
- */
- public void setDimensions(int dataDimensionCount, int indexDimensionCount, int dimensionNumBytes) {
- if (dataDimensionCount < 0) {
- throw new IllegalArgumentException("dataDimensionCount must be >= 0; got " + dataDimensionCount);
- }
- if (dataDimensionCount > PointValues.MAX_DIMENSIONS) {
- throw new IllegalArgumentException("dataDimensionCount must be <= " + PointValues.MAX_DIMENSIONS + "; got " + dataDimensionCount);
+ if (dimensionCount < 0) {
+ throw new IllegalArgumentException("dimensionCount must be >= 0; got " + dimensionCount);
}
- if (indexDimensionCount < 0) {
- throw new IllegalArgumentException("indexDimensionCount must be >= 0; got " + indexDimensionCount);
- }
- if (indexDimensionCount > dataDimensionCount) {
- throw new IllegalArgumentException("indexDimensionCount must be <= dataDimensionCount: " + dataDimensionCount + "; got " + indexDimensionCount);
+ if (dimensionCount > PointValues.MAX_DIMENSIONS) {
+ throw new IllegalArgumentException("dimensionCount must be <= " + PointValues.MAX_DIMENSIONS + "; got " + dimensionCount);
}
if (dimensionNumBytes < 0) {
throw new IllegalArgumentException("dimensionNumBytes must be >= 0; got " + dimensionNumBytes);
@@ -306,34 +291,23 @@ public class FieldType implements IndexableFieldType {
if (dimensionNumBytes > PointValues.MAX_NUM_BYTES) {
throw new IllegalArgumentException("dimensionNumBytes must be <= " + PointValues.MAX_NUM_BYTES + "; got " + dimensionNumBytes);
}
- if (dataDimensionCount == 0) {
- if (indexDimensionCount != 0) {
- throw new IllegalArgumentException("when dataDimensionCount is 0, indexDimensionCount must be 0; got " + indexDimensionCount);
- }
+ if (dimensionCount == 0) {
if (dimensionNumBytes != 0) {
- throw new IllegalArgumentException("when dataDimensionCount is 0, dimensionNumBytes must be 0; got " + dimensionNumBytes);
+ throw new IllegalArgumentException("when dimensionCount is 0, dimensionNumBytes must 0; got " + dimensionNumBytes);
}
- } else if (indexDimensionCount == 0) {
- throw new IllegalArgumentException("when dataDimensionCount is > 0, indexDimensionCount must be > 0; got " + indexDimensionCount);
} else if (dimensionNumBytes == 0) {
- if (dataDimensionCount != 0) {
- throw new IllegalArgumentException("when dimensionNumBytes is 0, dataDimensionCount must be 0; got " + dataDimensionCount);
+ if (dimensionCount != 0) {
+ throw new IllegalArgumentException("when dimensionNumBytes is 0, dimensionCount must 0; got " + dimensionCount);
}
}
- this.dataDimensionCount = dataDimensionCount;
- this.indexDimensionCount = indexDimensionCount;
+ this.dimensionCount = dimensionCount;
this.dimensionNumBytes = dimensionNumBytes;
}
@Override
- public int pointDataDimensionCount() {
- return dataDimensionCount;
- }
-
- @Override
- public int pointIndexDimensionCount() {
- return indexDimensionCount;
+ public int pointDimensionCount() {
+ return dimensionCount;
}
@Override
@@ -375,14 +349,12 @@ public class FieldType implements IndexableFieldType {
result.append(indexOptions);
}
}
- if (dataDimensionCount != 0) {
+ if (dimensionCount != 0) {
if (result.length() > 0) {
result.append(",");
}
- result.append("pointDataDimensionCount=");
- result.append(dataDimensionCount);
- result.append(",pointIndexDimensionCount=");
- result.append(indexDimensionCount);
+ result.append("pointDimensionCount=");
+ result.append(dimensionCount);
result.append(",pointNumBytes=");
result.append(dimensionNumBytes);
}
@@ -427,8 +399,7 @@ public class FieldType implements IndexableFieldType {
public int hashCode() {
final int prime = 31;
int result = 1;
- result = prime * result + dataDimensionCount;
- result = prime * result + indexDimensionCount;
+ result = prime * result + dimensionCount;
result = prime * result + dimensionNumBytes;
result = prime * result + ((docValuesType == null) ? 0 : docValuesType.hashCode());
result = prime * result + indexOptions.hashCode();
@@ -448,8 +419,7 @@ public class FieldType implements IndexableFieldType {
if (obj == null) return false;
if (getClass() != obj.getClass()) return false;
FieldType other = (FieldType) obj;
- if (dataDimensionCount != other.dataDimensionCount) return false;
- if (indexDimensionCount != other.indexDimensionCount) return false;
+ if (dimensionCount != other.dimensionCount) return false;
if (dimensionNumBytes != other.dimensionNumBytes) return false;
if (docValuesType != other.docValuesType) return false;
if (indexOptions != other.indexOptions) return false;
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java b/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java
index b6d508f..0ec67fd 100644
--- a/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java
@@ -85,8 +85,8 @@ public final class FloatPoint extends Field {
/** Change the values of this field */
public void setFloatValues(float... point) {
- if (type.pointDataDimensionCount() != point.length) {
- throw new IllegalArgumentException("this field (name=" + name + ") uses " + type.pointDataDimensionCount() + " dimensions; cannot change to (incoming) " + point.length + " dimensions");
+ if (type.pointDimensionCount() != point.length) {
+ throw new IllegalArgumentException("this field (name=" + name + ") uses " + type.pointDimensionCount() + " dimensions; cannot change to (incoming) " + point.length + " dimensions");
}
fieldsData = pack(point);
}
@@ -98,8 +98,8 @@ public final class FloatPoint extends Field {
@Override
public Number numericValue() {
- if (type.pointDataDimensionCount() != 1) {
- throw new IllegalStateException("this field (name=" + name + ") uses " + type.pointDataDimensionCount() + " dimensions; cannot convert to a single numeric value");
+ if (type.pointDimensionCount() != 1) {
+ throw new IllegalStateException("this field (name=" + name + ") uses " + type.pointDimensionCount() + " dimensions; cannot convert to a single numeric value");
}
BytesRef bytes = (BytesRef) fieldsData;
assert bytes.length == Float.BYTES;
@@ -142,7 +142,7 @@ public final class FloatPoint extends Field {
result.append(':');
BytesRef bytes = (BytesRef) fieldsData;
- for (int dim = 0; dim < type.pointDataDimensionCount(); dim++) {
+ for (int dim = 0; dim < type.pointDimensionCount(); dim++) {
if (dim > 0) {
result.append(',');
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/core/src/java/org/apache/lucene/document/FloatRange.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/FloatRange.java b/lucene/core/src/java/org/apache/lucene/document/FloatRange.java
index 191631b..9b555d6 100644
--- a/lucene/core/src/java/org/apache/lucene/document/FloatRange.java
+++ b/lucene/core/src/java/org/apache/lucene/document/FloatRange.java
@@ -77,8 +77,8 @@ public class FloatRange extends Field {
*/
public void setRangeValues(float[] min, float[] max) {
checkArgs(min, max);
- if (min.length*2 != type.pointDataDimensionCount() || max.length*2 != type.pointDataDimensionCount()) {
- throw new IllegalArgumentException("field (name=" + name + ") uses " + type.pointDataDimensionCount()/2
+ if (min.length*2 != type.pointDimensionCount() || max.length*2 != type.pointDimensionCount()) {
+ throw new IllegalArgumentException("field (name=" + name + ") uses " + type.pointDimensionCount()/2
+ " dimensions; cannot change to (incoming) " + min.length + " dimensions");
}
@@ -148,7 +148,7 @@ public class FloatRange extends Field {
* @return the decoded min value
*/
public float getMin(int dimension) {
- FutureObjects.checkIndex(dimension, type.pointDataDimensionCount()/2);
+ FutureObjects.checkIndex(dimension, type.pointDimensionCount()/2);
return decodeMin(((BytesRef)fieldsData).bytes, dimension);
}
@@ -158,7 +158,7 @@ public class FloatRange extends Field {
* @return the decoded max value
*/
public float getMax(int dimension) {
- FutureObjects.checkIndex(dimension, type.pointDataDimensionCount()/2);
+ FutureObjects.checkIndex(dimension, type.pointDimensionCount()/2);
return decodeMax(((BytesRef)fieldsData).bytes, dimension);
}
@@ -244,7 +244,7 @@ public class FloatRange extends Field {
sb.append(':');
byte[] b = ((BytesRef)fieldsData).bytes;
toString(b, 0);
- for (int d = 0; d < type.pointDataDimensionCount() / 2; ++d) {
+ for (int d = 0; d < type.pointDimensionCount() / 2; ++d) {
sb.append(' ');
sb.append(toString(b, d));
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/IntPoint.java b/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
index da4f391..8b3484b 100644
--- a/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
@@ -59,8 +59,8 @@ public final class IntPoint extends Field {
/** Change the values of this field */
public void setIntValues(int... point) {
- if (type.pointDataDimensionCount() != point.length) {
- throw new IllegalArgumentException("this field (name=" + name + ") uses " + type.pointDataDimensionCount() + " dimensions; cannot change to (incoming) " + point.length + " dimensions");
+ if (type.pointDimensionCount() != point.length) {
+ throw new IllegalArgumentException("this field (name=" + name + ") uses " + type.pointDimensionCount() + " dimensions; cannot change to (incoming) " + point.length + " dimensions");
}
fieldsData = pack(point);
}
@@ -72,8 +72,8 @@ public final class IntPoint extends Field {
@Override
public Number numericValue() {
- if (type.pointDataDimensionCount() != 1) {
- throw new IllegalStateException("this field (name=" + name + ") uses " + type.pointDataDimensionCount() + " dimensions; cannot convert to a single numeric value");
+ if (type.pointDimensionCount() != 1) {
+ throw new IllegalStateException("this field (name=" + name + ") uses " + type.pointDimensionCount() + " dimensions; cannot convert to a single numeric value");
}
BytesRef bytes = (BytesRef) fieldsData;
assert bytes.length == Integer.BYTES;
@@ -116,7 +116,7 @@ public final class IntPoint extends Field {
result.append(':');
BytesRef bytes = (BytesRef) fieldsData;
- for (int dim = 0; dim < type.pointDataDimensionCount(); dim++) {
+ for (int dim = 0; dim < type.pointDimensionCount(); dim++) {
if (dim > 0) {
result.append(',');
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/core/src/java/org/apache/lucene/document/IntRange.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/IntRange.java b/lucene/core/src/java/org/apache/lucene/document/IntRange.java
index b24e80a..e67b94f 100644
--- a/lucene/core/src/java/org/apache/lucene/document/IntRange.java
+++ b/lucene/core/src/java/org/apache/lucene/document/IntRange.java
@@ -77,8 +77,8 @@ public class IntRange extends Field {
*/
public void setRangeValues(int[] min, int[] max) {
checkArgs(min, max);
- if (min.length*2 != type.pointDataDimensionCount() || max.length*2 != type.pointDataDimensionCount()) {
- throw new IllegalArgumentException("field (name=" + name + ") uses " + type.pointDataDimensionCount()/2
+ if (min.length*2 != type.pointDimensionCount() || max.length*2 != type.pointDimensionCount()) {
+ throw new IllegalArgumentException("field (name=" + name + ") uses " + type.pointDimensionCount()/2
+ " dimensions; cannot change to (incoming) " + min.length + " dimensions");
}
@@ -148,7 +148,7 @@ public class IntRange extends Field {
* @return the decoded min value
*/
public int getMin(int dimension) {
- FutureObjects.checkIndex(dimension, type.pointDataDimensionCount()/2);
+ FutureObjects.checkIndex(dimension, type.pointDimensionCount()/2);
return decodeMin(((BytesRef)fieldsData).bytes, dimension);
}
@@ -158,7 +158,7 @@ public class IntRange extends Field {
* @return the decoded max value
*/
public int getMax(int dimension) {
- FutureObjects.checkIndex(dimension, type.pointDataDimensionCount()/2);
+ FutureObjects.checkIndex(dimension, type.pointDimensionCount()/2);
return decodeMax(((BytesRef)fieldsData).bytes, dimension);
}
@@ -244,7 +244,7 @@ public class IntRange extends Field {
sb.append(':');
byte[] b = ((BytesRef)fieldsData).bytes;
toString(b, 0);
- for (int d = 0; d < type.pointDataDimensionCount() / 2; ++d) {
+ for (int d = 0; d < type.pointDimensionCount() / 2; ++d) {
sb.append(' ');
sb.append(toString(b, d));
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/core/src/java/org/apache/lucene/document/LatLonPoint.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/LatLonPoint.java b/lucene/core/src/java/org/apache/lucene/document/LatLonPoint.java
index 9cb94ae..a44795d 100644
--- a/lucene/core/src/java/org/apache/lucene/document/LatLonPoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/LatLonPoint.java
@@ -145,9 +145,9 @@ public class LatLonPoint extends Field {
/** helper: checks a fieldinfo and throws exception if its definitely not a LatLonPoint */
static void checkCompatible(FieldInfo fieldInfo) {
// point/dv properties could be "unset", if you e.g. used only StoredField with this same name in the segment.
- if (fieldInfo.getPointDataDimensionCount() != 0 && fieldInfo.getPointDataDimensionCount() != TYPE.pointDataDimensionCount()) {
- throw new IllegalArgumentException("field=\"" + fieldInfo.name + "\" was indexed with numDims=" + fieldInfo.getPointDataDimensionCount() +
- " but this point type has numDims=" + TYPE.pointDataDimensionCount() +
+ if (fieldInfo.getPointDimensionCount() != 0 && fieldInfo.getPointDimensionCount() != TYPE.pointDimensionCount()) {
+ throw new IllegalArgumentException("field=\"" + fieldInfo.name + "\" was indexed with numDims=" + fieldInfo.getPointDimensionCount() +
+ " but this point type has numDims=" + TYPE.pointDimensionCount() +
", is the field really a LatLonPoint?");
}
if (fieldInfo.getPointNumBytes() != 0 && fieldInfo.getPointNumBytes() != TYPE.pointNumBytes()) {
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/LongPoint.java b/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
index b3189ca..686086c 100644
--- a/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
@@ -59,8 +59,8 @@ public final class LongPoint extends Field {
/** Change the values of this field */
public void setLongValues(long... point) {
- if (type.pointDataDimensionCount() != point.length) {
- throw new IllegalArgumentException("this field (name=" + name + ") uses " + type.pointDataDimensionCount() + " dimensions; cannot change to (incoming) " + point.length + " dimensions");
+ if (type.pointDimensionCount() != point.length) {
+ throw new IllegalArgumentException("this field (name=" + name + ") uses " + type.pointDimensionCount() + " dimensions; cannot change to (incoming) " + point.length + " dimensions");
}
fieldsData = pack(point);
}
@@ -72,8 +72,8 @@ public final class LongPoint extends Field {
@Override
public Number numericValue() {
- if (type.pointDataDimensionCount() != 1) {
- throw new IllegalStateException("this field (name=" + name + ") uses " + type.pointDataDimensionCount() + " dimensions; cannot convert to a single numeric value");
+ if (type.pointDimensionCount() != 1) {
+ throw new IllegalStateException("this field (name=" + name + ") uses " + type.pointDimensionCount() + " dimensions; cannot convert to a single numeric value");
}
BytesRef bytes = (BytesRef) fieldsData;
assert bytes.length == Long.BYTES;
@@ -116,7 +116,7 @@ public final class LongPoint extends Field {
result.append(':');
BytesRef bytes = (BytesRef) fieldsData;
- for (int dim = 0; dim < type.pointDataDimensionCount(); dim++) {
+ for (int dim = 0; dim < type.pointDimensionCount(); dim++) {
if (dim > 0) {
result.append(',');
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/core/src/java/org/apache/lucene/document/LongRange.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/LongRange.java b/lucene/core/src/java/org/apache/lucene/document/LongRange.java
index 9056d2d..1a1b19a 100644
--- a/lucene/core/src/java/org/apache/lucene/document/LongRange.java
+++ b/lucene/core/src/java/org/apache/lucene/document/LongRange.java
@@ -77,8 +77,8 @@ public class LongRange extends Field {
*/
public void setRangeValues(long[] min, long[] max) {
checkArgs(min, max);
- if (min.length*2 != type.pointDataDimensionCount() || max.length*2 != type.pointDataDimensionCount()) {
- throw new IllegalArgumentException("field (name=" + name + ") uses " + type.pointDataDimensionCount()/2
+ if (min.length*2 != type.pointDimensionCount() || max.length*2 != type.pointDimensionCount()) {
+ throw new IllegalArgumentException("field (name=" + name + ") uses " + type.pointDimensionCount()/2
+ " dimensions; cannot change to (incoming) " + min.length + " dimensions");
}
@@ -146,7 +146,7 @@ public class LongRange extends Field {
* @return the decoded min value
*/
public long getMin(int dimension) {
- FutureObjects.checkIndex(dimension, type.pointDataDimensionCount()/2);
+ FutureObjects.checkIndex(dimension, type.pointDimensionCount()/2);
return decodeMin(((BytesRef)fieldsData).bytes, dimension);
}
@@ -156,7 +156,7 @@ public class LongRange extends Field {
* @return the decoded max value
*/
public long getMax(int dimension) {
- FutureObjects.checkIndex(dimension, type.pointDataDimensionCount()/2);
+ FutureObjects.checkIndex(dimension, type.pointDimensionCount()/2);
return decodeMax(((BytesRef)fieldsData).bytes, dimension);
}
@@ -242,7 +242,7 @@ public class LongRange extends Field {
sb.append(':');
byte[] b = ((BytesRef)fieldsData).bytes;
toString(b, 0);
- for (int d = 0; d < type.pointDataDimensionCount() / 2; ++d) {
+ for (int d = 0; d < type.pointDimensionCount() / 2; ++d) {
sb.append(' ');
sb.append(toString(b, d));
}