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/18 16:41:15 UTC

[3/4] lucene-solr:branch_7x: 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., dimension

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/804afbfd/lucene/core/src/java/org/apache/lucene/document/RangeFieldQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/RangeFieldQuery.java b/lucene/core/src/java/org/apache/lucene/document/RangeFieldQuery.java
index 5c9a811..9c85fa8 100644
--- a/lucene/core/src/java/org/apache/lucene/document/RangeFieldQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/document/RangeFieldQuery.java
@@ -254,9 +254,9 @@ abstract class RangeFieldQuery extends Query {
 
   /** Check indexed field info against the provided query data. */
   private void checkFieldInfo(FieldInfo fieldInfo) {
-    if (fieldInfo.getPointDimensionCount()/2 != numDims) {
+    if (fieldInfo.getPointDataDimensionCount()/2 != numDims) {
       throw new IllegalArgumentException("field=\"" + field + "\" was indexed with numDims="
-          + fieldInfo.getPointDimensionCount()/2 + " but this query has numDims=" + numDims);
+          + fieldInfo.getPointDataDimensionCount()/2 + " but this query has numDims=" + numDims);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/804afbfd/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
index 2fe54c8..57336ac 100644
--- a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
+++ b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
@@ -1782,7 +1782,7 @@ public final class CheckIndex implements Closeable {
           throw new RuntimeException("there are fields with points, but reader.getPointsReader() is null");
         }
         for (FieldInfo fieldInfo : fieldInfos) {
-          if (fieldInfo.getPointDimensionCount() > 0) {
+          if (fieldInfo.getPointDataDimensionCount() > 0) {
             PointValues values = pointsReader.getValues(fieldInfo.name);
             if (values == null) {
               continue;
@@ -1852,7 +1852,9 @@ public final class CheckIndex implements Closeable {
     private final byte[] globalMinPackedValue;
     private final byte[] globalMaxPackedValue;
     private final int packedBytesCount;
-    private final int numDims;
+    private final int packedIndexBytesCount;
+    private final int numDataDims;
+    private final int numIndexDims;
     private final int bytesPerDim;
     private final String fieldName;
 
@@ -1860,14 +1862,16 @@ public final class CheckIndex implements Closeable {
     public VerifyPointsVisitor(String fieldName, int maxDoc, PointValues values) throws IOException {
       this.maxDoc = maxDoc;
       this.fieldName = fieldName;
-      numDims = values.getNumDimensions();
+      numDataDims = values.getNumDataDimensions();
+      numIndexDims = values.getNumIndexDimensions();
       bytesPerDim = values.getBytesPerDimension();
-      packedBytesCount = numDims * bytesPerDim;
+      packedBytesCount = numDataDims * bytesPerDim;
+      packedIndexBytesCount = numIndexDims * bytesPerDim;
       globalMinPackedValue = values.getMinPackedValue();
       globalMaxPackedValue = values.getMaxPackedValue();
       docsSeen = new FixedBitSet(maxDoc);
-      lastMinPackedValue = new byte[packedBytesCount];
-      lastMaxPackedValue = new byte[packedBytesCount];
+      lastMinPackedValue = new byte[packedIndexBytesCount];
+      lastMaxPackedValue = new byte[packedIndexBytesCount];
       lastPackedValue = new byte[packedBytesCount];
 
       if (values.getDocCount() > values.size()) {
@@ -1882,14 +1886,14 @@ public final class CheckIndex implements Closeable {
         if (values.size() != 0) {
           throw new RuntimeException("getMinPackedValue is null points for field \"" + fieldName + "\" yet size=" + values.size());
         }
-      } else if (globalMinPackedValue.length != packedBytesCount) {
+      } else if (globalMinPackedValue.length != packedIndexBytesCount) {
         throw new RuntimeException("getMinPackedValue for field \"" + fieldName + "\" return length=" + globalMinPackedValue.length + " array, but should be " + packedBytesCount);
       }
       if (globalMaxPackedValue == null) {
         if (values.size() != 0) {
           throw new RuntimeException("getMaxPackedValue is null points for field \"" + fieldName + "\" yet size=" + values.size());
         }
-      } else if (globalMaxPackedValue.length != packedBytesCount) {
+      } else if (globalMaxPackedValue.length != packedIndexBytesCount) {
         throw new RuntimeException("getMaxPackedValue for field \"" + fieldName + "\" return length=" + globalMaxPackedValue.length + " array, but should be " + packedBytesCount);
       }
     }
@@ -1915,7 +1919,7 @@ public final class CheckIndex implements Closeable {
       pointCountSeen++;
       docsSeen.set(docID);
 
-      for(int dim=0;dim<numDims;dim++) {
+      for(int dim=0;dim<numIndexDims;dim++) {
         int offset = bytesPerDim * dim;
 
         // Compare to last cell:
@@ -1930,9 +1934,10 @@ public final class CheckIndex implements Closeable {
         }
       }
 
-      // In the 1D case, PointValues must make a single in-order sweep through all values, and tie-break by
+      // In the 1D data case, PointValues must make a single in-order sweep through all values, and tie-break by
       // increasing docID:
-      if (numDims == 1) {
+      // for data dimension > 1, leaves are sorted by the dimension with the lowest cardinality to improve block compression
+      if (numDataDims == 1) {
         int cmp = FutureArrays.compareUnsigned(lastPackedValue, 0, bytesPerDim, packedValue, 0, bytesPerDim);
         if (cmp > 0) {
           throw new RuntimeException("packed points value " + Arrays.toString(packedValue) + " for field=\"" + fieldName + "\", for docID=" + docID + " is out-of-order vs the previous document's value " + Arrays.toString(lastPackedValue));
@@ -1949,11 +1954,11 @@ public final class CheckIndex implements Closeable {
     @Override
     public PointValues.Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
       checkPackedValue("min packed value", minPackedValue, -1);
-      System.arraycopy(minPackedValue, 0, lastMinPackedValue, 0, packedBytesCount);
+      System.arraycopy(minPackedValue, 0, lastMinPackedValue, 0, packedIndexBytesCount);
       checkPackedValue("max packed value", maxPackedValue, -1);
-      System.arraycopy(maxPackedValue, 0, lastMaxPackedValue, 0, packedBytesCount);
+      System.arraycopy(maxPackedValue, 0, lastMaxPackedValue, 0, packedIndexBytesCount);
 
-      for(int dim=0;dim<numDims;dim++) {
+      for(int dim=0;dim<numIndexDims;dim++) {
         int offset = bytesPerDim * dim;
 
         if (FutureArrays.compareUnsigned(minPackedValue, offset, offset + bytesPerDim, maxPackedValue, offset, offset + bytesPerDim) > 0) {
@@ -1992,8 +1997,8 @@ public final class CheckIndex implements Closeable {
         throw new RuntimeException(desc + " is null for docID=" + docID + " field=\"" + fieldName + "\"");
       }
 
-      if (packedValue.length != packedBytesCount) {
-        throw new RuntimeException(desc + " has incorrect length=" + packedValue.length + " vs expected=" + packedBytesCount + " for docID=" + docID + " field=\"" + fieldName + "\"");
+      if (packedValue.length != (docID < 0 ? packedIndexBytesCount : packedBytesCount)) {
+        throw new RuntimeException(desc + " has incorrect length=" + packedValue.length + " vs expected=" + packedIndexBytesCount + " for docID=" + docID + " field=\"" + fieldName + "\"");
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/804afbfd/lucene/core/src/java/org/apache/lucene/index/CodecReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/CodecReader.java b/lucene/core/src/java/org/apache/lucene/index/CodecReader.java
index 50aaa42..ca6ac7d 100644
--- a/lucene/core/src/java/org/apache/lucene/index/CodecReader.java
+++ b/lucene/core/src/java/org/apache/lucene/index/CodecReader.java
@@ -194,7 +194,7 @@ public abstract class CodecReader extends LeafReader implements Accountable {
   public final PointValues getPointValues(String field) throws IOException {
     ensureOpen();
     FieldInfo fi = getFieldInfos().fieldInfo(field);
-    if (fi == null || fi.getPointDimensionCount() == 0) {
+    if (fi == null || fi.getPointDataDimensionCount() == 0) {
       // Field does not exist or does not index points
       return null;
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/804afbfd/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java b/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
index 1f7adbd..dd2945e 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
@@ -190,7 +190,7 @@ final class DefaultIndexingChain extends DocConsumer {
         PerField perField = fieldHash[i];
         while (perField != null) {
           if (perField.pointValuesWriter != null) {
-            if (perField.fieldInfo.getPointDimensionCount() == 0) {
+            if (perField.fieldInfo.getPointDataDimensionCount() == 0) {
               // BUG
               throw new AssertionError("segment=" + state.segmentInfo + ": field=\"" + perField.fieldInfo.name + "\" has no points but wrote them");
             }
@@ -205,7 +205,7 @@ final class DefaultIndexingChain extends DocConsumer {
 
             perField.pointValuesWriter.flush(state, sortMap, pointsWriter);
             perField.pointValuesWriter = null;
-          } else if (perField.fieldInfo.getPointDimensionCount() != 0) {
+          } else if (perField.fieldInfo.getPointDataDimensionCount() != 0) {
             // BUG
             throw new AssertionError("segment=" + state.segmentInfo + ": field=\"" + perField.fieldInfo.name + "\" has points but did not write them");
           }
@@ -466,7 +466,7 @@ final class DefaultIndexingChain extends DocConsumer {
       }
       indexDocValue(fp, dvType, field);
     }
-    if (fieldType.pointDimensionCount() != 0) {
+    if (fieldType.pointDataDimensionCount() != 0) {
       if (fp == null) {
         fp = getOrAddField(fieldName, fieldType, false);
       }
@@ -497,17 +497,18 @@ final class DefaultIndexingChain extends DocConsumer {
 
   /** Called from processDocument to index one field's point */
   private void indexPoint(PerField fp, IndexableField field) throws IOException {
-    int pointDimensionCount = field.fieldType().pointDimensionCount();
+    int pointDataDimensionCount = field.fieldType().pointDataDimensionCount();
+    int pointIndexDimensionCount = field.fieldType().pointIndexDimensionCount();
 
     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.getPointDimensionCount() == 0) {
-      fieldInfos.globalFieldNumbers.setDimensions(fp.fieldInfo.number, fp.fieldInfo.name, pointDimensionCount, dimensionNumBytes);
+    if (fp.fieldInfo.getPointDataDimensionCount() == 0) {
+      fieldInfos.globalFieldNumbers.setDimensions(fp.fieldInfo.number, fp.fieldInfo.name, pointDataDimensionCount, pointIndexDimensionCount, dimensionNumBytes);
     }
 
-    fp.fieldInfo.setPointDimensions(pointDimensionCount, dimensionNumBytes);
+    fp.fieldInfo.setPointDimensions(pointDataDimensionCount, pointIndexDimensionCount, dimensionNumBytes);
 
     if (fp.pointValuesWriter == null) {
       fp.pointValuesWriter = new PointValuesWriter(docWriter, fp.fieldInfo);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/804afbfd/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java b/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
index 250c94b..31f4970 100644
--- a/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
+++ b/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
@@ -50,7 +50,8 @@ public final class FieldInfo {
 
   /** If both of these are positive it means this field indexed points
    *  (see {@link org.apache.lucene.codecs.PointsFormat}). */
-  private int pointDimensionCount;
+  private int pointDataDimensionCount;
+  private int pointIndexDimensionCount;
   private int pointNumBytes;
 
   // whether this field is used as the soft-deletes field
@@ -63,7 +64,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 pointDimensionCount, int pointNumBytes, boolean softDeletesField) {
+                   int pointDataDimensionCount, int pointIndexDimensionCount, int pointNumBytes, boolean softDeletesField) {
     this.name = Objects.requireNonNull(name);
     this.number = number;
     this.docValuesType = Objects.requireNonNull(docValues, "DocValuesType must not be null (field: \"" + name + "\")");
@@ -79,7 +80,8 @@ public final class FieldInfo {
     }
     this.dvGen = dvGen;
     this.attributes = Objects.requireNonNull(attributes);
-    this.pointDimensionCount = pointDimensionCount;
+    this.pointDataDimensionCount = pointDataDimensionCount;
+    this.pointIndexDimensionCount = pointIndexDimensionCount;
     this.pointNumBytes = pointNumBytes;
     this.softDeletesField = softDeletesField;
     assert checkConsistency();
@@ -107,20 +109,28 @@ public final class FieldInfo {
       }
     }
 
-    if (pointDimensionCount < 0) {
-      throw new IllegalStateException("pointDimensionCount must be >= 0; got " + pointDimensionCount);
+    if (pointDataDimensionCount < 0) {
+      throw new IllegalStateException("pointDataDimensionCount must be >= 0; got " + pointDataDimensionCount);
+    }
+
+    if (pointIndexDimensionCount < 0) {
+      throw new IllegalStateException("pointIndexDimensionCount must be >= 0; got " + pointIndexDimensionCount);
     }
 
     if (pointNumBytes < 0) {
       throw new IllegalStateException("pointNumBytes must be >= 0; got " + pointNumBytes);
     }
 
-    if (pointDimensionCount != 0 && pointNumBytes == 0) {
-      throw new IllegalStateException("pointNumBytes must be > 0 when pointDimensionCount=" + pointDimensionCount);
+    if (pointDataDimensionCount != 0 && pointNumBytes == 0) {
+      throw new IllegalStateException("pointNumBytes must be > 0 when pointDataDimensionCount=" + pointDataDimensionCount);
+    }
+
+    if (pointIndexDimensionCount != 0 && pointDataDimensionCount == 0) {
+      throw new IllegalStateException("pointIndexDimensionCount must be 0 when pointDataDimensionCount=0");
     }
 
-    if (pointNumBytes != 0 && pointDimensionCount == 0) {
-      throw new IllegalStateException("pointDimensionCount must be > 0 when pointNumBytes=" + pointNumBytes);
+    if (pointNumBytes != 0 && pointDataDimensionCount == 0) {
+      throw new IllegalStateException("pointDataDimensionCount must be > 0 when pointNumBytes=" + pointNumBytes);
     }
     
     if (dvGen != -1 && docValuesType == DocValuesType.NONE) {
@@ -132,7 +142,7 @@ public final class FieldInfo {
 
   // should only be called by FieldInfos#addOrUpdate
   void update(boolean storeTermVector, boolean omitNorms, boolean storePayloads, IndexOptions indexOptions,
-              int dimensionCount, int dimensionNumBytes) {
+              int dataDimensionCount, int indexDimensionCount, int dimensionNumBytes) {
     if (indexOptions == null) {
       throw new NullPointerException("IndexOptions must not be null (field: \"" + name + "\")");
     }
@@ -146,11 +156,12 @@ public final class FieldInfo {
       }
     }
 
-    if (this.pointDimensionCount == 0 && dimensionCount != 0) {
-      this.pointDimensionCount = dimensionCount;
+    if (this.pointDataDimensionCount == 0 && dataDimensionCount != 0) {
+      this.pointDataDimensionCount = dataDimensionCount;
+      this.pointIndexDimensionCount = indexDimensionCount;
       this.pointNumBytes = dimensionNumBytes;
-    } else if (dimensionCount != 0 && (this.pointDimensionCount != dimensionCount || this.pointNumBytes != dimensionNumBytes)) {
-      throw new IllegalArgumentException("cannot change field \"" + name + "\" from points dimensionCount=" + this.pointDimensionCount + ", numBytes=" + this.pointNumBytes + " to inconsistent dimensionCount=" + dimensionCount + ", numBytes=" + dimensionNumBytes);
+    } else if (dataDimensionCount != 0 && (this.pointDataDimensionCount != dataDimensionCount || this.pointIndexDimensionCount != indexDimensionCount || this.pointNumBytes != dimensionNumBytes)) {
+      throw new IllegalArgumentException("cannot change field \"" + name + "\" from points dataDimensionCount=" + this.pointDataDimensionCount + ", indexDimensionCount=" + this.pointIndexDimensionCount + ", numBytes=" + this.pointNumBytes + " to inconsistent dataDimensionCount=" + dataDimensionCount +", indexDimensionCount=" + indexDimensionCount + ", numBytes=" + dimensionNumBytes);
     }
 
     if (this.indexOptions != IndexOptions.NONE) { // if updated field data is not for indexing, leave the updates out
@@ -171,12 +182,15 @@ public final class FieldInfo {
 
   /** Record that this field is indexed with points, with the
    *  specified number of dimensions and bytes per dimension. */
-  public void setPointDimensions(int count, int numBytes) {
-    if (count <= 0) {
-      throw new IllegalArgumentException("point dimension count must be >= 0; got " + count + " for field=\"" + name + "\"");
+  public void setPointDimensions(int dataDimensionCount, int indexDimensionCount, int numBytes) {
+    if (dataDimensionCount <= 0) {
+      throw new IllegalArgumentException("point data dimension count must be >= 0; got " + dataDimensionCount + " 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 (dataDimensionCount > PointValues.MAX_DIMENSIONS) {
+      throw new IllegalArgumentException("point data dimension count must be < PointValues.MAX_DIMENSIONS (= " + PointValues.MAX_DIMENSIONS + "); got " + dataDimensionCount + " for field=\"" + name + "\"");
+    }
+    if (indexDimensionCount > dataDimensionCount) {
+      throw new IllegalArgumentException("point index dimension count must be <= point data dimension count (= " + dataDimensionCount + "); got " + indexDimensionCount + " for field=\"" + name + "\"");
     }
     if (numBytes <= 0) {
       throw new IllegalArgumentException("point numBytes must be >= 0; got " + numBytes + " for field=\"" + name + "\"");
@@ -184,22 +198,31 @@ public final class FieldInfo {
     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 (pointDimensionCount != 0 && pointDimensionCount != count) {
-      throw new IllegalArgumentException("cannot change point dimension count from " + pointDimensionCount + " to " + count + " for field=\"" + name + "\"");
+    if (pointDataDimensionCount != 0 && pointDataDimensionCount != dataDimensionCount) {
+      throw new IllegalArgumentException("cannot change point data dimension count from " + pointDataDimensionCount + " to " + dataDimensionCount + " for field=\"" + name + "\"");
+    }
+    if (pointIndexDimensionCount != 0 && pointIndexDimensionCount != indexDimensionCount) {
+      throw new IllegalArgumentException("cannot change point index dimension count from " + pointIndexDimensionCount + " to " + indexDimensionCount + " for field=\"" + name + "\"");
     }
     if (pointNumBytes != 0 && pointNumBytes != numBytes) {
       throw new IllegalArgumentException("cannot change point numBytes from " + pointNumBytes + " to " + numBytes + " for field=\"" + name + "\"");
     }
 
-    pointDimensionCount = count;
+    pointDataDimensionCount = dataDimensionCount;
+    pointIndexDimensionCount = indexDimensionCount;
     pointNumBytes = numBytes;
 
     assert checkConsistency();
   }
 
-  /** Return point dimension count */
-  public int getPointDimensionCount() {
-    return pointDimensionCount;
+  /** Return point data dimension count */
+  public int getPointDataDimensionCount() {
+    return pointDataDimensionCount;
+  }
+
+  /** Return point data dimension count */
+  public int getPointIndexDimensionCount() {
+    return pointIndexDimensionCount;
   }
 
   /** Return number of bytes per dimension */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/804afbfd/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java b/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
index 22ca9de7..518263c 100644
--- a/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
+++ b/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
@@ -93,7 +93,7 @@ public class FieldInfos implements Iterable<FieldInfo> {
       hasNorms |= info.hasNorms();
       hasDocValues |= info.getDocValuesType() != DocValuesType.NONE;
       hasPayloads |= info.hasPayloads();
-      hasPointValues |= (info.getPointDimensionCount() != 0);
+      hasPointValues |= (info.getPointDataDimensionCount() != 0);
       if (info.isSoftDeletesField()) {
         if (softDeletesField != null && softDeletesField.equals(info.name) == false) {
           throw new IllegalArgumentException("multiple soft-deletes fields [" + info.name + ", " + softDeletesField + "]");
@@ -210,11 +210,13 @@ public class FieldInfos implements Iterable<FieldInfo> {
   }
 
   static final class FieldDimensions {
-    public final int dimensionCount;
+    public final int dataDimensionCount;
+    public final int indexDimensionCount;
     public final int dimensionNumBytes;
 
-    public FieldDimensions(int dimensionCount, int dimensionNumBytes) {
-      this.dimensionCount = dimensionCount;
+    public FieldDimensions(int dataDimensionCount, int indexDimensionCount, int dimensionNumBytes) {
+      this.dataDimensionCount = dataDimensionCount;
+      this.indexDimensionCount = indexDimensionCount;
       this.dimensionNumBytes = dimensionNumBytes;
     }
   }
@@ -252,7 +254,7 @@ public class FieldInfos implements Iterable<FieldInfo> {
      * number assigned if possible otherwise the first unassigned field number
      * is used as the field number.
      */
-    synchronized int addOrGet(String fieldName, int preferredFieldNumber, DocValuesType dvType, int dimensionCount, int dimensionNumBytes, boolean isSoftDeletesField) {
+    synchronized int addOrGet(String fieldName, int preferredFieldNumber, DocValuesType dvType, int dataDimensionCount, int indexDimensionCount, int dimensionNumBytes, boolean isSoftDeletesField) {
       if (dvType != DocValuesType.NONE) {
         DocValuesType currentDVType = docValuesType.get(fieldName);
         if (currentDVType == null) {
@@ -261,17 +263,20 @@ public class FieldInfos implements Iterable<FieldInfo> {
           throw new IllegalArgumentException("cannot change DocValues type from " + currentDVType + " to " + dvType + " for field \"" + fieldName + "\"");
         }
       }
-      if (dimensionCount != 0) {
+      if (dataDimensionCount != 0) {
         FieldDimensions dims = dimensions.get(fieldName);
         if (dims != null) {
-          if (dims.dimensionCount != dimensionCount) {
-            throw new IllegalArgumentException("cannot change point dimension count from " + dims.dimensionCount + " to " + dimensionCount + " for field=\"" + fieldName + "\"");
+          if (dims.dataDimensionCount != dataDimensionCount) {
+            throw new IllegalArgumentException("cannot change point data dimension count from " + dims.dataDimensionCount + " to " + dataDimensionCount + " for field=\"" + fieldName + "\"");
+          }
+          if (dims.indexDimensionCount != indexDimensionCount) {
+            throw new IllegalArgumentException("cannot change point index dimension count from " + dims.indexDimensionCount + " to " + indexDimensionCount + " for field=\"" + fieldName + "\"");
           }
           if (dims.dimensionNumBytes != dimensionNumBytes) {
             throw new IllegalArgumentException("cannot change point numBytes from " + dims.dimensionNumBytes + " to " + dimensionNumBytes + " for field=\"" + fieldName + "\"");
           }
         } else {
-          dimensions.put(fieldName, new FieldDimensions(dimensionCount, dimensionNumBytes));
+          dimensions.put(fieldName, new FieldDimensions(dataDimensionCount, indexDimensionCount, dimensionNumBytes));
         }
       }
       Integer fieldNumber = nameToNumber.get(fieldName);
@@ -318,7 +323,7 @@ public class FieldInfos implements Iterable<FieldInfo> {
       }
     }
 
-    synchronized void verifyConsistentDimensions(Integer number, String name, int dimensionCount, int dimensionNumBytes) {
+    synchronized void verifyConsistentDimensions(Integer number, String name, int dataDimensionCount, int indexDimensionCount, int dimensionNumBytes) {
       if (name.equals(numberToName.get(number)) == false) {
         throw new IllegalArgumentException("field number " + number + " is already mapped to field name \"" + numberToName.get(number) + "\", not \"" + name + "\"");
       }
@@ -327,8 +332,11 @@ public class FieldInfos implements Iterable<FieldInfo> {
       }
       FieldDimensions dim = dimensions.get(name);
       if (dim != null) {
-        if (dim.dimensionCount != dimensionCount) {
-          throw new IllegalArgumentException("cannot change point dimension count from " + dim.dimensionCount + " to " + dimensionCount + " for field=\"" + name + "\"");
+        if (dim.dataDimensionCount != dataDimensionCount) {
+          throw new IllegalArgumentException("cannot change point data dimension count from " + dim.dataDimensionCount + " to " + dataDimensionCount + " for field=\"" + name + "\"");
+        }
+        if (dim.indexDimensionCount != indexDimensionCount) {
+          throw new IllegalArgumentException("cannot change point index dimension count from " + dim.indexDimensionCount + " to " + indexDimensionCount + " for field=\"" + name + "\"");
         }
         if (dim.dimensionNumBytes != dimensionNumBytes) {
           throw new IllegalArgumentException("cannot change point numBytes from " + dim.dimensionNumBytes + " to " + dimensionNumBytes + " for field=\"" + name + "\"");
@@ -366,15 +374,18 @@ public class FieldInfos implements Iterable<FieldInfo> {
       docValuesType.put(name, dvType);
     }
 
-    synchronized void setDimensions(int number, String name, int dimensionCount, int dimensionNumBytes) {
+    synchronized void setDimensions(int number, String name, int dataDimensionCount, int indexDimensionCount, int dimensionNumBytes) {
       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 > PointValues.MAX_DIMENSIONS) {
-        throw new IllegalArgumentException("pointDimensionCount must be <= PointValues.MAX_DIMENSIONS (= " + PointValues.MAX_DIMENSIONS + "); got " + dimensionCount + " for field=\"" + name + "\"");
+      if (dataDimensionCount > PointValues.MAX_DIMENSIONS) {
+        throw new IllegalArgumentException("pointDataDimensionCount must be <= PointValues.MAX_DIMENSIONS (= " + PointValues.MAX_DIMENSIONS + "); got " + dataDimensionCount + " for field=\"" + name + "\"");
+      }
+      if (indexDimensionCount > dataDimensionCount) {
+        throw new IllegalArgumentException("pointIndexDimensionCount must be <= pointDataDimensionCount (= " + dataDimensionCount + "); got " + indexDimensionCount + " for field=\"" + name + "\"");
       }
-      verifyConsistentDimensions(number, name, dimensionCount, dimensionNumBytes);
-      dimensions.put(name, new FieldDimensions(dimensionCount, dimensionNumBytes));
+      verifyConsistentDimensions(number, name, dataDimensionCount, indexDimensionCount, dimensionNumBytes);
+      dimensions.put(name, new FieldDimensions(dataDimensionCount, indexDimensionCount, dimensionNumBytes));
     }
   }
   
@@ -409,8 +420,8 @@ public class FieldInfos implements Iterable<FieldInfo> {
         // before then we'll get the same name and number,
         // else we'll allocate a new one:
         final boolean isSoftDeletesField = name.equals(globalFieldNumbers.softDeletesFieldName);
-        final int fieldNumber = globalFieldNumbers.addOrGet(name, -1, DocValuesType.NONE, 0, 0, isSoftDeletesField);
-        fi = new FieldInfo(name, fieldNumber, false, false, false, IndexOptions.NONE, DocValuesType.NONE, -1, new HashMap<>(), 0, 0, isSoftDeletesField);
+        final int fieldNumber = globalFieldNumbers.addOrGet(name, -1, DocValuesType.NONE, 0, 0, 0, isSoftDeletesField);
+        fi = new FieldInfo(name, fieldNumber, false, false, false, IndexOptions.NONE, DocValuesType.NONE, -1, new HashMap<>(), 0, 0, 0, isSoftDeletesField);
         assert !byName.containsKey(fi.name);
         globalFieldNumbers.verifyConsistent(Integer.valueOf(fi.number), fi.name, DocValuesType.NONE);
         byName.put(fi.name, fi);
@@ -423,7 +434,8 @@ public class FieldInfos implements Iterable<FieldInfo> {
                                           boolean storeTermVector,
                                           boolean omitNorms, boolean storePayloads, IndexOptions indexOptions,
                                           DocValuesType docValues, long dvGen,
-                                          int dimensionCount, int dimensionNumBytes, boolean isSoftDeletesField) {
+                                          int dataDimensionCount, int indexDimensionCount, int dimensionNumBytes,
+                                          boolean isSoftDeletesField) {
       assert assertNotFinished();
       if (docValues == null) {
         throw new NullPointerException("DocValuesType must not be null");
@@ -435,13 +447,13 @@ public class FieldInfos implements Iterable<FieldInfo> {
         // number for this field.  If the field was seen
         // before then we'll get the same name and number,
         // else we'll allocate a new one:
-        final int fieldNumber = globalFieldNumbers.addOrGet(name, preferredFieldNumber, docValues, dimensionCount, dimensionNumBytes, isSoftDeletesField);
-        fi = new FieldInfo(name, fieldNumber, storeTermVector, omitNorms, storePayloads, indexOptions, docValues, dvGen, new HashMap<>(), dimensionCount, dimensionNumBytes, isSoftDeletesField);
+        final int fieldNumber = globalFieldNumbers.addOrGet(name, preferredFieldNumber, docValues, dataDimensionCount, indexDimensionCount, dimensionNumBytes, isSoftDeletesField);
+        fi = new FieldInfo(name, fieldNumber, storeTermVector, omitNorms, storePayloads, indexOptions, docValues, dvGen, new HashMap<>(), dataDimensionCount, indexDimensionCount, dimensionNumBytes, isSoftDeletesField);
         assert !byName.containsKey(fi.name);
         globalFieldNumbers.verifyConsistent(Integer.valueOf(fi.number), fi.name, fi.getDocValuesType());
         byName.put(fi.name, fi);
       } else {
-        fi.update(storeTermVector, omitNorms, storePayloads, indexOptions, dimensionCount, dimensionNumBytes);
+        fi.update(storeTermVector, omitNorms, storePayloads, indexOptions, dataDimensionCount, indexDimensionCount, dimensionNumBytes);
 
         if (docValues != DocValuesType.NONE) {
           // Only pay the synchronization cost if fi does not already have a DVType
@@ -469,7 +481,8 @@ public class FieldInfos implements Iterable<FieldInfo> {
       return addOrUpdateInternal(fi.name, fi.number, fi.hasVectors(),
                                  fi.omitsNorms(), fi.hasPayloads(),
                                  fi.getIndexOptions(), fi.getDocValuesType(), dvGen,
-                                 fi.getPointDimensionCount(), fi.getPointNumBytes(), fi.isSoftDeletesField());
+                                 fi.getPointDataDimensionCount(), fi.getPointIndexDimensionCount(), fi.getPointNumBytes(),
+                                 fi.isSoftDeletesField());
     }
     
     public FieldInfo fieldInfo(String fieldName) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/804afbfd/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
index 0ce5882..4410656 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
@@ -983,7 +983,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.getPointDimensionCount(), fi.getPointNumBytes(), fi.isSoftDeletesField());
+        map.addOrGet(fi.name, fi.number, fi.getDocValuesType(), fi.getPointDataDimensionCount(), fi.getPointIndexDimensionCount(), fi.getPointNumBytes(), fi.isSoftDeletesField());
       }
     }
 
@@ -1805,7 +1805,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
       if (globalFieldNumberMap.contains(f.name(), dvType) == false) {
         // if this field doesn't exists we try to add it. if it exists and the DV type doesn't match we
         // get a consistent error message as if you try to do that during an indexing operation.
-        globalFieldNumberMap.addOrGet(f.name(), -1, dvType, 0, 0, f.name().equals(config.softDeletesField));
+        globalFieldNumberMap.addOrGet(f.name(), -1, dvType, 0, 0, 0, f.name().equals(config.softDeletesField));
         assert globalFieldNumberMap.contains(f.name(), dvType);
       }
       if (config.getIndexSortFields().contains(f.name())) {
@@ -2842,7 +2842,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.getPointDimensionCount(), fi.getPointNumBytes(), fi.isSoftDeletesField());
+              globalFieldNumberMap.addOrGet(fi.name, fi.number, fi.getDocValuesType(), fi.getPointDataDimensionCount(), fi.getPointIndexDimensionCount(), fi.getPointNumBytes(), fi.isSoftDeletesField());
             }
             infos.add(copySegmentAsIs(info, newSegName, context));
           }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/804afbfd/lucene/core/src/java/org/apache/lucene/index/IndexableFieldType.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexableFieldType.java b/lucene/core/src/java/org/apache/lucene/index/IndexableFieldType.java
index a797cf8..b2b2e77 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexableFieldType.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexableFieldType.java
@@ -98,9 +98,14 @@ public interface IndexableFieldType {
   public DocValuesType docValuesType();
 
   /**
-   * If this is positive, the field is indexed as a point.
+   * If this is positive (representing the number of point data dimensions), the field is indexed as a point.
    */
-  public int pointDimensionCount();
+  public int pointDataDimensionCount();
+
+  /**
+   * The number of dimensions used for the index key
+   */
+  public int pointIndexDimensionCount();
 
   /**
    * The number of bytes in each dimension's values.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/804afbfd/lucene/core/src/java/org/apache/lucene/index/PointValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/PointValues.java b/lucene/core/src/java/org/apache/lucene/index/PointValues.java
index df433d2..51b12d1 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PointValues.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PointValues.java
@@ -136,7 +136,7 @@ public abstract class PointValues {
       if (minValue == null) {
         minValue = leafMinValue.clone();
       } else {
-        final int numDimensions = values.getNumDimensions();
+        final int numDimensions = values.getNumIndexDimensions();
         final int numBytesPerDimension = values.getBytesPerDimension();
         for (int i = 0; i < numDimensions; ++i) {
           int offset = i * numBytesPerDimension;
@@ -167,7 +167,7 @@ public abstract class PointValues {
       if (maxValue == null) {
         maxValue = leafMaxValue.clone();
       } else {
-        final int numDimensions = values.getNumDimensions();
+        final int numDimensions = values.getNumIndexDimensions();
         final int numBytesPerDimension = values.getBytesPerDimension();
         for (int i = 0; i < numDimensions; ++i) {
           int offset = i * numBytesPerDimension;
@@ -233,8 +233,11 @@ public abstract class PointValues {
   /** Returns maximum value for each dimension, packed, or null if {@link #size} is <code>0</code> */
   public abstract byte[] getMaxPackedValue() throws IOException;
 
-  /** Returns how many dimensions were indexed */
-  public abstract int getNumDimensions() throws IOException;
+  /** Returns how many data dimensions are represented in the values */
+  public abstract int getNumDataDimensions() throws IOException;
+
+  /** Returns how many dimensions are used for the index */
+  public abstract int getNumIndexDimensions() throws IOException;
 
   /** Returns the number of bytes per dimension */
   public abstract int getBytesPerDimension() throws IOException;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/804afbfd/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java b/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
index 4aaf095..eb11ec3 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
@@ -43,7 +43,7 @@ class PointValuesWriter {
     this.bytes = new ByteBlockPool(docWriter.byteBlockAllocator);
     docIDs = new int[16];
     iwBytesUsed.addAndGet(16 * Integer.BYTES);
-    packedBytesLength = fieldInfo.getPointDimensionCount() * fieldInfo.getPointNumBytes();
+    packedBytesLength = fieldInfo.getPointDataDimensionCount() * fieldInfo.getPointNumBytes();
   }
 
   // TODO: if exactly the same value is added to exactly the same doc, should we dedup?
@@ -52,7 +52,7 @@ class PointValuesWriter {
       throw new IllegalArgumentException("field=" + fieldInfo.name + ": point value must not be null");
     }
     if (value.length != packedBytesLength) {
-      throw new IllegalArgumentException("field=" + fieldInfo.name + ": this field's value has length=" + value.length + " but should be " + (fieldInfo.getPointDimensionCount() * fieldInfo.getPointNumBytes()));
+      throw new IllegalArgumentException("field=" + fieldInfo.name + ": this field's value has length=" + value.length + " but should be " + (fieldInfo.getPointDataDimensionCount() * fieldInfo.getPointNumBytes()));
     }
 
     if (docIDs.length == numPoints) {
@@ -106,7 +106,12 @@ class PointValuesWriter {
       }
 
       @Override
-      public int getNumDimensions() {
+      public int getNumDataDimensions() {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public int getNumIndexDimensions() {
         throw new UnsupportedOperationException();
       }
 
@@ -229,8 +234,13 @@ class PointValuesWriter {
     }
 
     @Override
-    public int getNumDimensions() throws IOException {
-      return in.getNumDimensions();
+    public int getNumDataDimensions() throws IOException {
+      return in.getNumDataDimensions();
+    }
+
+    @Override
+    public int getNumIndexDimensions() throws IOException {
+      return in.getNumIndexDimensions();
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/804afbfd/lucene/core/src/java/org/apache/lucene/index/SortingLeafReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/SortingLeafReader.java b/lucene/core/src/java/org/apache/lucene/index/SortingLeafReader.java
index 55e4d20..d9ed94b 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SortingLeafReader.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SortingLeafReader.java
@@ -344,8 +344,13 @@ class SortingLeafReader extends FilterLeafReader {
     }
 
     @Override
-    public int getNumDimensions() throws IOException {
-      return in.getNumDimensions();
+    public int getNumDataDimensions() throws IOException {
+      return in.getNumDataDimensions();
+    }
+
+    @Override
+    public int getNumIndexDimensions() throws IOException {
+      return in.getNumIndexDimensions();
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/804afbfd/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java b/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
index 8aa71bf..b766146 100644
--- a/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
@@ -123,8 +123,8 @@ public abstract class PointInSetQuery extends Query {
           return null;
         }
 
-        if (values.getNumDimensions() != numDims) {
-          throw new IllegalArgumentException("field=\"" + field + "\" was indexed with numDims=" + values.getNumDimensions() + " but this query has numDims=" + numDims);
+        if (values.getNumIndexDimensions() != numDims) {
+          throw new IllegalArgumentException("field=\"" + field + "\" was indexed with numIndexDims=" + values.getNumIndexDimensions() + " but this query has numIndexDims=" + numDims);
         }
         if (values.getBytesPerDimension() != bytesPerDim) {
           throw new IllegalArgumentException("field=\"" + field + "\" was indexed with bytesPerDim=" + values.getBytesPerDimension() + " but this query has bytesPerDim=" + bytesPerDim);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/804afbfd/lucene/core/src/java/org/apache/lucene/search/PointRangeQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/PointRangeQuery.java b/lucene/core/src/java/org/apache/lucene/search/PointRangeQuery.java
index 34b8909..d34ea25 100644
--- a/lucene/core/src/java/org/apache/lucene/search/PointRangeQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/PointRangeQuery.java
@@ -234,8 +234,8 @@ public abstract class PointRangeQuery extends Query {
           return null;
         }
 
-        if (values.getNumDimensions() != numDims) {
-          throw new IllegalArgumentException("field=\"" + field + "\" was indexed with numDims=" + values.getNumDimensions() + " but this query has numDims=" + numDims);
+        if (values.getNumIndexDimensions() != numDims) {
+          throw new IllegalArgumentException("field=\"" + field + "\" was indexed with numIndexDimensions=" + values.getNumIndexDimensions() + " but this query has numDims=" + numDims);
         }
         if (bytesPerDim != values.getBytesPerDimension()) {
           throw new IllegalArgumentException("field=\"" + field + "\" was indexed with bytesPerDim=" + values.getBytesPerDimension() + " but this query has bytesPerDim=" + bytesPerDim);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/804afbfd/lucene/core/src/java/org/apache/lucene/util/bkd/BKDReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/bkd/BKDReader.java b/lucene/core/src/java/org/apache/lucene/util/bkd/BKDReader.java
index 194ac82..aa105b2 100644
--- a/lucene/core/src/java/org/apache/lucene/util/bkd/BKDReader.java
+++ b/lucene/core/src/java/org/apache/lucene/util/bkd/BKDReader.java
@@ -36,7 +36,8 @@ import org.apache.lucene.util.RamUsageEstimator;
 public final class BKDReader extends PointValues implements Accountable {
   // Packed array of byte[] holding all split values in the full binary tree:
   final int leafNodeOffset;
-  final int numDims;
+  final int numDataDims;
+  final int numIndexDims;
   final int bytesPerDim;
   final int numLeaves;
   final IndexInput in;
@@ -47,6 +48,7 @@ public final class BKDReader extends PointValues implements Accountable {
   final int docCount;
   final int version;
   protected final int packedBytesLength;
+  protected final int packedIndexBytesLength;
 
   // Used for 6.4.0+ index format:
   final byte[] packedIndex;
@@ -59,24 +61,30 @@ public final class BKDReader extends PointValues implements Accountable {
   /** Caller must pre-seek the provided {@link IndexInput} to the index location that {@link BKDWriter#finish} returned */
   public BKDReader(IndexInput in) throws IOException {
     version = CodecUtil.checkHeader(in, BKDWriter.CODEC_NAME, BKDWriter.VERSION_START, BKDWriter.VERSION_CURRENT);
-    numDims = in.readVInt();
+    numDataDims = in.readVInt();
+    if (version >= BKDWriter.VERSION_SELECTIVE_INDEXING) {
+      numIndexDims = in.readVInt();
+    } else {
+      numIndexDims = numDataDims;
+    }
     maxPointsInLeafNode = in.readVInt();
     bytesPerDim = in.readVInt();
-    bytesPerIndexEntry = numDims == 1 && version >= BKDWriter.VERSION_IMPLICIT_SPLIT_DIM_1D ? bytesPerDim : bytesPerDim + 1;
-    packedBytesLength = numDims * bytesPerDim;
+    bytesPerIndexEntry = numDataDims == 1 && version >= BKDWriter.VERSION_IMPLICIT_SPLIT_DIM_1D ? bytesPerDim : bytesPerDim + 1;
+    packedBytesLength = numDataDims * bytesPerDim;
+    packedIndexBytesLength = numIndexDims * bytesPerDim;
 
     // Read index:
     numLeaves = in.readVInt();
     assert numLeaves > 0;
     leafNodeOffset = numLeaves;
 
-    minPackedValue = new byte[packedBytesLength];
-    maxPackedValue = new byte[packedBytesLength];
+    minPackedValue = new byte[packedIndexBytesLength];
+    maxPackedValue = new byte[packedIndexBytesLength];
 
-    in.readBytes(minPackedValue, 0, packedBytesLength);
-    in.readBytes(maxPackedValue, 0, packedBytesLength);
+    in.readBytes(minPackedValue, 0, packedIndexBytesLength);
+    in.readBytes(maxPackedValue, 0, packedIndexBytesLength);
 
-    for(int dim=0;dim<numDims;dim++) {
+    for(int dim=0;dim<numIndexDims;dim++) {
       if (FutureArrays.compareUnsigned(minPackedValue, dim * bytesPerDim, dim * bytesPerDim + bytesPerDim, maxPackedValue, dim * bytesPerDim, dim * bytesPerDim + bytesPerDim) > 0) {
         throw new CorruptIndexException("minPackedValue " + new BytesRef(minPackedValue) + " is > maxPackedValue " + new BytesRef(maxPackedValue) + " for dim=" + dim, in);
       }
@@ -110,7 +118,7 @@ public final class BKDReader extends PointValues implements Accountable {
       // Possibly rotate the leaf block FPs, if the index not fully balanced binary tree (only happens
       // if it was created by BKDWriter.merge or OneDimWriter).  In this case the leaf nodes may straddle the two bottom
       // levels of the binary tree:
-      if (numDims == 1 && numLeaves > 1) {
+      if (numDataDims == 1 && numLeaves > 1) {
         int levelCount = 2;
         while (true) {
           if (numLeaves >= levelCount && numLeaves <= 2*levelCount) {
@@ -165,14 +173,14 @@ public final class BKDReader extends PointValues implements Accountable {
       splitPackedValueStack = new byte[treeDepth+1][];
       nodeID = 1;
       level = 1;
-      splitPackedValueStack[level] = new byte[packedBytesLength];
-    }      
+      splitPackedValueStack[level] = new byte[packedIndexBytesLength];
+    }
 
     public void pushLeft() {
       nodeID *= 2;
       level++;
       if (splitPackedValueStack[level] == null) {
-        splitPackedValueStack[level] = new byte[packedBytesLength];
+        splitPackedValueStack[level] = new byte[packedIndexBytesLength];
       }
     }
 
@@ -183,7 +191,7 @@ public final class BKDReader extends PointValues implements Accountable {
       nodeID = nodeID * 2 + 1;
       level++;
       if (splitPackedValueStack[level] == null) {
-        splitPackedValueStack[level] = new byte[packedBytesLength];
+        splitPackedValueStack[level] = new byte[packedIndexBytesLength];
       }
     }
 
@@ -304,7 +312,7 @@ public final class BKDReader extends PointValues implements Accountable {
       } else {
         leafBlockFP = -1;
         int address = nodeID * bytesPerIndexEntry;
-        if (numDims == 1) {
+        if (numIndexDims == 1) {
           splitDim = 0;
           if (version < BKDWriter.VERSION_IMPLICIT_SPLIT_DIM_1D) {
             // skip over wastefully encoded 0 splitDim:
@@ -367,10 +375,10 @@ public final class BKDReader extends PointValues implements Accountable {
       rightNodePositions = new int[treeDepth+1];
       splitValuesStack = new byte[treeDepth+1][];
       splitDims = new int[treeDepth+1];
-      negativeDeltas = new boolean[numDims*(treeDepth+1)];
+      negativeDeltas = new boolean[numIndexDims*(treeDepth+1)];
 
       in = new ByteArrayDataInput(packedIndex);
-      splitValuesStack[0] = new byte[packedBytesLength];
+      splitValuesStack[0] = new byte[packedIndexBytesLength];
       readNodeData(false);
       scratch = new BytesRef();
       scratch.length = bytesPerDim;
@@ -386,7 +394,7 @@ public final class BKDReader extends PointValues implements Accountable {
       index.leftNodePositions[level] = leftNodePositions[level];
       index.rightNodePositions[level] = rightNodePositions[level];
       index.splitValuesStack[index.level] = splitValuesStack[index.level].clone();
-      System.arraycopy(negativeDeltas, level*numDims, index.negativeDeltas, level*numDims, numDims);
+      System.arraycopy(negativeDeltas, level*numIndexDims, index.negativeDeltas, level*numIndexDims, numIndexDims);
       index.splitDims[level] = splitDims[level];
       return index;
     }
@@ -395,9 +403,9 @@ public final class BKDReader extends PointValues implements Accountable {
     public void pushLeft() {
       int nodePosition = leftNodePositions[level];
       super.pushLeft();
-      System.arraycopy(negativeDeltas, (level-1)*numDims, negativeDeltas, level*numDims, numDims);
+      System.arraycopy(negativeDeltas, (level-1)*numIndexDims, negativeDeltas, level*numIndexDims, numIndexDims);
       assert splitDim != -1;
-      negativeDeltas[level*numDims+splitDim] = true;
+      negativeDeltas[level*numIndexDims+splitDim] = true;
       in.setPosition(nodePosition);
       readNodeData(true);
     }
@@ -406,9 +414,9 @@ public final class BKDReader extends PointValues implements Accountable {
     public void pushRight() {
       int nodePosition = rightNodePositions[level];
       super.pushRight();
-      System.arraycopy(negativeDeltas, (level-1)*numDims, negativeDeltas, level*numDims, numDims);
+      System.arraycopy(negativeDeltas, (level-1)*numIndexDims, negativeDeltas, level*numIndexDims, numIndexDims);
       assert splitDim != -1;
-      negativeDeltas[level*numDims+splitDim] = false;
+      negativeDeltas[level*numIndexDims+splitDim] = false;
       in.setPosition(nodePosition);
       readNodeData(false);
     }
@@ -448,19 +456,19 @@ public final class BKDReader extends PointValues implements Accountable {
 
         // read split dim, prefix, firstDiffByteDelta encoded as int:
         int code = in.readVInt();
-        splitDim = code % numDims;
+        splitDim = code % numIndexDims;
         splitDims[level] = splitDim;
-        code /= numDims;
+        code /= numIndexDims;
         int prefix = code % (1+bytesPerDim);
         int suffix = bytesPerDim - prefix;
 
         if (splitValuesStack[level] == null) {
-          splitValuesStack[level] = new byte[packedBytesLength];
+          splitValuesStack[level] = new byte[packedIndexBytesLength];
         }
-        System.arraycopy(splitValuesStack[level-1], 0, splitValuesStack[level], 0, packedBytesLength);
+        System.arraycopy(splitValuesStack[level-1], 0, splitValuesStack[level], 0, packedIndexBytesLength);
         if (suffix > 0) {
           int firstDiffByteDelta = code / (1+bytesPerDim);
-          if (negativeDeltas[level*numDims + splitDim]) {
+          if (negativeDeltas[level*numIndexDims + splitDim]) {
             firstDiffByteDelta = -firstDiffByteDelta;
           }
           int oldByte = splitValuesStack[level][splitDim*bytesPerDim+prefix] & 0xFF;
@@ -497,7 +505,7 @@ public final class BKDReader extends PointValues implements Accountable {
   public static final class IntersectState {
     final IndexInput in;
     final int[] scratchDocIDs;
-    final byte[] scratchPackedValue1, scratchPackedValue2;
+    final byte[] scratchDataPackedValue, scratchMinIndexPackedValue, scratchMaxIndexPackedValue;
     final int[] commonPrefixLengths;
 
     final IntersectVisitor visitor;
@@ -505,6 +513,7 @@ public final class BKDReader extends PointValues implements Accountable {
 
     public IntersectState(IndexInput in, int numDims,
                           int packedBytesLength,
+                          int packedIndexBytesLength,
                           int maxPointsInLeafNode,
                           IntersectVisitor visitor,
                           IndexTree indexVisitor) {
@@ -512,8 +521,9 @@ public final class BKDReader extends PointValues implements Accountable {
       this.visitor = visitor;
       this.commonPrefixLengths = new int[numDims];
       this.scratchDocIDs = new int[maxPointsInLeafNode];
-      this.scratchPackedValue1 = new byte[packedBytesLength];
-      this.scratchPackedValue2 = new byte[packedBytesLength];
+      this.scratchDataPackedValue = new byte[packedBytesLength];
+      this.scratchMinIndexPackedValue = new byte[packedIndexBytesLength];
+      this.scratchMaxIndexPackedValue = new byte[packedIndexBytesLength];
       this.index = indexVisitor;
     }
   }
@@ -566,8 +576,9 @@ public final class BKDReader extends PointValues implements Accountable {
     } else {
       index = new LegacyIndexTree();
     }
-    return new IntersectState(in.clone(), numDims,
+    return new IntersectState(in.clone(), numDataDims,
                               packedBytesLength,
+                              packedIndexBytesLength,
                               maxPointsInLeafNode,
                               visitor,
                               index);
@@ -580,7 +591,7 @@ public final class BKDReader extends PointValues implements Accountable {
     int count = readDocIDs(state.in, index.getLeafBlockFP(), state.scratchDocIDs);
 
     // Again, this time reading values and checking with the visitor
-    visitDocValues(state.commonPrefixLengths, state.scratchPackedValue1, state.scratchPackedValue2, state.in, state.scratchDocIDs, count, state.visitor);
+    visitDocValues(state.commonPrefixLengths, state.scratchDataPackedValue, state.scratchMinIndexPackedValue, state.scratchMaxIndexPackedValue, state.in, state.scratchDocIDs, count, state.visitor);
   }
 
   private void visitDocIDs(IndexInput in, long blockFP, IntersectVisitor visitor) throws IOException {
@@ -613,16 +624,19 @@ public final class BKDReader extends PointValues implements Accountable {
     return count;
   }
 
-  void visitDocValues(int[] commonPrefixLengths, byte[] scratchPackedValue1, byte[] scratchPackedValue2, IndexInput in, int[] docIDs, int count, IntersectVisitor visitor) throws IOException {
+  void visitDocValues(int[] commonPrefixLengths, byte[] scratchDataPackedValue, byte[] scratchMinIndexPackedValue, byte[] scratchMaxIndexPackedValue,
+                      IndexInput in, int[] docIDs, int count, IntersectVisitor visitor) throws IOException {
 
 
-    readCommonPrefixes(commonPrefixLengths, scratchPackedValue1, in);
+    readCommonPrefixes(commonPrefixLengths, scratchDataPackedValue, in);
 
-    if (numDims != 1 && version >= BKDWriter.VERSION_LEAF_STORES_BOUNDS) {
-      byte[] minPackedValue = scratchPackedValue1;
-      byte[] maxPackedValue = scratchPackedValue2;
-      //Copy common prefixes before reading adjusted box
-      System.arraycopy(minPackedValue, 0, maxPackedValue, 0, packedBytesLength);
+    if (numIndexDims != 1 && version >= BKDWriter.VERSION_LEAF_STORES_BOUNDS) {
+      byte[] minPackedValue = scratchMinIndexPackedValue;
+      System.arraycopy(scratchDataPackedValue, 0, minPackedValue, 0, packedIndexBytesLength);
+      byte[] maxPackedValue = scratchMaxIndexPackedValue;
+      //Copy common prefixes before reading adjusted
+      // box
+      System.arraycopy(minPackedValue, 0, maxPackedValue, 0, packedIndexBytesLength);
       readMinMax(commonPrefixLengths, minPackedValue, maxPackedValue, in);
 
       // The index gives us range of values for each dimension, but the actual range of values
@@ -653,24 +667,24 @@ public final class BKDReader extends PointValues implements Accountable {
         : readCompressedDim(in);
 
     if (compressedDim == -1) {
-      visitRawDocValues(commonPrefixLengths, scratchPackedValue1, in, docIDs, count, visitor);
+      visitRawDocValues(commonPrefixLengths, scratchDataPackedValue, in, docIDs, count, visitor);
     } else {
-      visitCompressedDocValues(commonPrefixLengths, scratchPackedValue1, in, docIDs, count, visitor, compressedDim);
+      visitCompressedDocValues(commonPrefixLengths, scratchDataPackedValue, in, docIDs, count, visitor, compressedDim);
     }
   }
 
-    private void readMinMax(int[] commonPrefixLengths, byte[] minPackedValue, byte[] maxPackedValue, IndexInput in) throws IOException {
-      for (int dim = 0; dim < numDims; dim++) {
-        int prefix = commonPrefixLengths[dim];
-        in.readBytes(minPackedValue, dim * bytesPerDim + prefix, bytesPerDim - prefix);
-        in.readBytes(maxPackedValue, dim * bytesPerDim + prefix, bytesPerDim - prefix);
-      }
+  private void readMinMax(int[] commonPrefixLengths, byte[] minPackedValue, byte[] maxPackedValue, IndexInput in) throws IOException {
+    for (int dim = 0; dim < numIndexDims; dim++) {
+      int prefix = commonPrefixLengths[dim];
+      in.readBytes(minPackedValue, dim * bytesPerDim + prefix, bytesPerDim - prefix);
+      in.readBytes(maxPackedValue, dim * bytesPerDim + prefix, bytesPerDim - prefix);
     }
+  }
 
   // Just read suffixes for every dimension
   private void visitRawDocValues(int[] commonPrefixLengths, byte[] scratchPackedValue, IndexInput in, int[] docIDs, int count, IntersectVisitor visitor) throws IOException {
     for (int i = 0; i < count; ++i) {
-      for(int dim=0;dim<numDims;dim++) {
+      for(int dim=0;dim<numDataDims;dim++) {
         int prefix = commonPrefixLengths[dim];
         in.readBytes(scratchPackedValue, dim*bytesPerDim + prefix, bytesPerDim - prefix);
       }
@@ -688,7 +702,7 @@ public final class BKDReader 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<numDims;dim++) {
+        for(int dim=0;dim<numDataDims;dim++) {
           int prefix = commonPrefixLengths[dim];
           in.readBytes(scratchPackedValue, dim*bytesPerDim + prefix, bytesPerDim - prefix);
         }
@@ -703,14 +717,14 @@ public final class BKDReader extends PointValues implements Accountable {
 
   private int readCompressedDim(IndexInput in) throws IOException {
     int compressedDim = in.readByte();
-    if (compressedDim < -1 || compressedDim >= numDims) {
+    if (compressedDim < -1 || compressedDim >= numDataDims) {
       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<numDims;dim++) {
+    for(int dim=0;dim<numDataDims;dim++) {
       int prefix = in.readVInt();
       commonPrefixLengths[dim] = prefix;
       if (prefix > 0) {
@@ -747,15 +761,15 @@ public final class BKDReader extends PointValues implements Accountable {
         int count = readDocIDs(state.in, state.index.getLeafBlockFP(), state.scratchDocIDs);
 
         // Again, this time reading values and checking with the visitor
-        visitDocValues(state.commonPrefixLengths, state.scratchPackedValue1, state.scratchPackedValue2, state.in, state.scratchDocIDs, count, state.visitor);
+        visitDocValues(state.commonPrefixLengths, state.scratchDataPackedValue, state.scratchMinIndexPackedValue, state.scratchMaxIndexPackedValue, state.in, state.scratchDocIDs, count, state.visitor);
       }
 
     } else {
       
       // Non-leaf node: recurse on the split left and right nodes
       int splitDim = state.index.getSplitDim();
-      assert splitDim >= 0: "splitDim=" + splitDim;
-      assert splitDim < numDims;
+      assert splitDim >= 0: "splitDim=" + splitDim + ", numIndexDims=" + numIndexDims;
+      assert splitDim < numIndexDims: "splitDim=" + splitDim + ", numIndexDims=" + numIndexDims;
 
       byte[] splitPackedValue = state.index.getSplitPackedValue();
       BytesRef splitDimValue = state.index.getSplitDimValue();
@@ -763,11 +777,11 @@ public final class BKDReader extends PointValues implements Accountable {
       //System.out.println("  splitDimValue=" + splitDimValue + " splitDim=" + splitDim);
 
       // make sure cellMin <= splitValue <= cellMax:
-      assert FutureArrays.compareUnsigned(cellMinPacked, splitDim * bytesPerDim, splitDim * bytesPerDim + bytesPerDim, splitDimValue.bytes, splitDimValue.offset, splitDimValue.offset + bytesPerDim) <= 0: "bytesPerDim=" + bytesPerDim + " splitDim=" + splitDim + " numDims=" + numDims;
-      assert FutureArrays.compareUnsigned(cellMaxPacked, splitDim * bytesPerDim, splitDim * bytesPerDim + bytesPerDim, splitDimValue.bytes, splitDimValue.offset, splitDimValue.offset + bytesPerDim) >= 0: "bytesPerDim=" + bytesPerDim + " splitDim=" + splitDim + " numDims=" + numDims;
+      assert FutureArrays.compareUnsigned(cellMinPacked, splitDim * bytesPerDim, splitDim * bytesPerDim + bytesPerDim, splitDimValue.bytes, splitDimValue.offset, splitDimValue.offset + bytesPerDim) <= 0: "bytesPerDim=" + bytesPerDim + " splitDim=" + splitDim + " numIndexDims=" + numIndexDims + " numDataDims=" + numDataDims;
+      assert FutureArrays.compareUnsigned(cellMaxPacked, splitDim * bytesPerDim, splitDim * bytesPerDim + bytesPerDim, splitDimValue.bytes, splitDimValue.offset, splitDimValue.offset + bytesPerDim) >= 0: "bytesPerDim=" + bytesPerDim + " splitDim=" + splitDim + " numIndexDims=" + numIndexDims + " numDataDims=" + numDataDims;
 
       // Recurse on left sub-tree:
-      System.arraycopy(cellMaxPacked, 0, splitPackedValue, 0, packedBytesLength);
+      System.arraycopy(cellMaxPacked, 0, splitPackedValue, 0, packedIndexBytesLength);
       System.arraycopy(splitDimValue.bytes, splitDimValue.offset, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
       state.index.pushLeft();
       intersect(state, cellMinPacked, splitPackedValue);
@@ -777,7 +791,7 @@ public final class BKDReader extends PointValues implements Accountable {
       System.arraycopy(splitPackedValue, splitDim*bytesPerDim, splitDimValue.bytes, splitDimValue.offset, bytesPerDim);
 
       // Recurse on right sub-tree:
-      System.arraycopy(cellMinPacked, 0, splitPackedValue, 0, packedBytesLength);
+      System.arraycopy(cellMinPacked, 0, splitPackedValue, 0, packedIndexBytesLength);
       System.arraycopy(splitDimValue.bytes, splitDimValue.offset, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
       state.index.pushRight();
       intersect(state, splitPackedValue, cellMaxPacked);
@@ -808,8 +822,8 @@ public final class BKDReader extends PointValues implements Accountable {
       
       // Non-leaf node: recurse on the split left and right nodes
       int splitDim = state.index.getSplitDim();
-      assert splitDim >= 0: "splitDim=" + splitDim;
-      assert splitDim < numDims;
+      assert splitDim >= 0: "splitDim=" + splitDim + ", numIndexDims=" + numIndexDims;
+      assert splitDim < numIndexDims: "splitDim=" + splitDim + ", numIndexDims=" + numIndexDims;
 
       byte[] splitPackedValue = state.index.getSplitPackedValue();
       BytesRef splitDimValue = state.index.getSplitDimValue();
@@ -817,11 +831,11 @@ public final class BKDReader extends PointValues implements Accountable {
       //System.out.println("  splitDimValue=" + splitDimValue + " splitDim=" + splitDim);
 
       // make sure cellMin <= splitValue <= cellMax:
-      assert FutureArrays.compareUnsigned(cellMinPacked, splitDim * bytesPerDim, splitDim * bytesPerDim + bytesPerDim, splitDimValue.bytes, splitDimValue.offset, splitDimValue.offset + bytesPerDim) <= 0: "bytesPerDim=" + bytesPerDim + " splitDim=" + splitDim + " numDims=" + numDims;
-      assert FutureArrays.compareUnsigned(cellMaxPacked, splitDim * bytesPerDim, splitDim * bytesPerDim + bytesPerDim, splitDimValue.bytes, splitDimValue.offset, splitDimValue.offset + bytesPerDim) >= 0: "bytesPerDim=" + bytesPerDim + " splitDim=" + splitDim + " numDims=" + numDims;
+      assert FutureArrays.compareUnsigned(cellMinPacked, splitDim * bytesPerDim, splitDim * bytesPerDim + bytesPerDim, splitDimValue.bytes, splitDimValue.offset, splitDimValue.offset + bytesPerDim) <= 0: "bytesPerDim=" + bytesPerDim + " splitDim=" + splitDim + " numIndexDims=" + numIndexDims + " numDataDims=" + numDataDims;
+      assert FutureArrays.compareUnsigned(cellMaxPacked, splitDim * bytesPerDim, splitDim * bytesPerDim + bytesPerDim, splitDimValue.bytes, splitDimValue.offset, splitDimValue.offset + bytesPerDim) >= 0: "bytesPerDim=" + bytesPerDim + " splitDim=" + splitDim + " numIndexDims=" + numIndexDims + " numDataDims=" + numDataDims;
 
       // Recurse on left sub-tree:
-      System.arraycopy(cellMaxPacked, 0, splitPackedValue, 0, packedBytesLength);
+      System.arraycopy(cellMaxPacked, 0, splitPackedValue, 0, packedIndexBytesLength);
       System.arraycopy(splitDimValue.bytes, splitDimValue.offset, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
       state.index.pushLeft();
       final long leftCost = estimatePointCount(state, cellMinPacked, splitPackedValue);
@@ -831,7 +845,7 @@ public final class BKDReader extends PointValues implements Accountable {
       System.arraycopy(splitPackedValue, splitDim*bytesPerDim, splitDimValue.bytes, splitDimValue.offset, bytesPerDim);
 
       // Recurse on right sub-tree:
-      System.arraycopy(cellMinPacked, 0, splitPackedValue, 0, packedBytesLength);
+      System.arraycopy(cellMinPacked, 0, splitPackedValue, 0, packedIndexBytesLength);
       System.arraycopy(splitDimValue.bytes, splitDimValue.offset, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
       state.index.pushRight();
       final long rightCost = estimatePointCount(state, splitPackedValue, cellMaxPacked);
@@ -860,8 +874,13 @@ public final class BKDReader extends PointValues implements Accountable {
   }
 
   @Override
-  public int getNumDimensions() {
-    return numDims;
+  public int getNumDataDimensions() {
+    return numDataDims;
+  }
+
+  @Override
+  public int getNumIndexDimensions() {
+    return numIndexDims;
   }
 
   @Override