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:37 UTC

[1/6] lucene-solr:branch_7x: LUCENE-8496: revert fix SimpleTextBKDWriter.IntersectState to use numDataDims

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_7x bd16fdaf2 -> ac11c9e5b


LUCENE-8496: revert fix SimpleTextBKDWriter.IntersectState to use numDataDims


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/4c0bee84
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/4c0bee84
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/4c0bee84

Branch: refs/heads/branch_7x
Commit: 4c0bee8450b5439f796e8fc1c3c7aefd55233eda
Parents: bd16fda
Author: Nicholas Knize <nk...@gmail.com>
Authored: Wed Oct 10 11:59:21 2018 -0500
Committer: Nicholas Knize <nk...@gmail.com>
Committed: Wed Oct 10 11:59:21 2018 -0500

----------------------------------------------------------------------
 .../org/apache/lucene/codecs/simpletext/SimpleTextBKDWriter.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4c0bee84/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 9cacaa4..0e3d5d6 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
@@ -343,7 +343,7 @@ final class SimpleTextBKDWriter implements Closeable {
     public MergeReader(SimpleTextBKDReader bkd, MergeState.DocMap docMap) throws IOException {
       this.bkd = bkd;
       state = new SimpleTextBKDReader.IntersectState(bkd.in.clone(),
-                                                     bkd.numDataDims,
+                                                     bkd.numDims,
                                                      bkd.packedBytesLength,
                                                      bkd.maxPointsInLeafNode,
                                                      null);


[3/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

Posted by nk...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/test-framework/src/java/org/apache/lucene/index/AssertingLeafReader.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/AssertingLeafReader.java b/lucene/test-framework/src/java/org/apache/lucene/index/AssertingLeafReader.java
index e996684..71a3634 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/AssertingLeafReader.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/AssertingLeafReader.java
@@ -895,7 +895,7 @@ public class AssertingLeafReader extends FilterLeafReader {
 
     @Override
     public void intersect(IntersectVisitor visitor) throws IOException {
-      in.intersect(new AssertingIntersectVisitor(in.getNumDataDimensions(), in.getNumIndexDimensions(), in.getBytesPerDimension(), visitor));
+      in.intersect(new AssertingIntersectVisitor(in.getNumDimensions(), in.getBytesPerDimension(), visitor));
     }
 
     @Override
@@ -916,13 +916,8 @@ public class AssertingLeafReader extends FilterLeafReader {
     }
 
     @Override
-    public int getNumDataDimensions() throws IOException {
-      return in.getNumDataDimensions();
-    }
-
-    @Override
-    public int getNumIndexDimensions() throws IOException {
-      return in.getNumIndexDimensions();
+    public int getNumDimensions() throws IOException {
+      return in.getNumDimensions();
     }
 
     @Override
@@ -945,8 +940,7 @@ public class AssertingLeafReader extends FilterLeafReader {
   /** Validates in the 1D case that all points are visited in order, and point values are in bounds of the last cell checked */
   static class AssertingIntersectVisitor implements IntersectVisitor {
     final IntersectVisitor in;
-    final int numDataDims;
-    final int numIndexDims;
+    final int numDims;
     final int bytesPerDim;
     final byte[] lastDocValue;
     final byte[] lastMinPackedValue;
@@ -955,14 +949,13 @@ public class AssertingLeafReader extends FilterLeafReader {
     private int lastDocID = -1;
     private int docBudget;
 
-    AssertingIntersectVisitor(int numDataDims, int numIndexDims, int bytesPerDim, IntersectVisitor in) {
+    AssertingIntersectVisitor(int numDims, int bytesPerDim, IntersectVisitor in) {
       this.in = in;
-      this.numDataDims = numDataDims;
-      this.numIndexDims = numIndexDims;
+      this.numDims = numDims;
       this.bytesPerDim = bytesPerDim;
-      lastMaxPackedValue = new byte[numDataDims*bytesPerDim];
-      lastMinPackedValue = new byte[numDataDims*bytesPerDim];
-      if (numDataDims == 1) {
+      lastMaxPackedValue = new byte[numDims*bytesPerDim];
+      lastMinPackedValue = new byte[numDims*bytesPerDim];
+      if (numDims == 1) {
         lastDocValue = new byte[bytesPerDim];
       } else {
         lastDocValue = null;
@@ -986,14 +979,14 @@ public class AssertingLeafReader extends FilterLeafReader {
       assert lastCompareResult == PointValues.Relation.CELL_CROSSES_QUERY;
 
       // This doc's packed value should be contained in the last cell passed to compare:
-      for(int dim=0;dim<numIndexDims;dim++) {
-        assert FutureArrays.compareUnsigned(lastMinPackedValue, dim * bytesPerDim, dim * bytesPerDim + bytesPerDim, packedValue, dim * bytesPerDim, dim * bytesPerDim + bytesPerDim) <= 0: "dim=" + dim + " of " +  numDataDims + " value=" + new BytesRef(packedValue);
-        assert FutureArrays.compareUnsigned(lastMaxPackedValue, dim * bytesPerDim, dim * bytesPerDim + bytesPerDim, packedValue, dim * bytesPerDim, dim * bytesPerDim + bytesPerDim) >= 0: "dim=" + dim + " of " +  numDataDims + " value=" + new BytesRef(packedValue);
+      for(int dim=0;dim<numDims;dim++) {
+        assert FutureArrays.compareUnsigned(lastMinPackedValue, dim * bytesPerDim, dim * bytesPerDim + bytesPerDim, packedValue, dim * bytesPerDim, dim * bytesPerDim + bytesPerDim) <= 0: "dim=" + dim + " of " +  numDims + " value=" + new BytesRef(packedValue);
+        assert FutureArrays.compareUnsigned(lastMaxPackedValue, dim * bytesPerDim, dim * bytesPerDim + bytesPerDim, packedValue, dim * bytesPerDim, dim * bytesPerDim + bytesPerDim) >= 0: "dim=" + dim + " of " +  numDims + " value=" + new BytesRef(packedValue);
       }
 
       // TODO: we should assert that this "matches" whatever relation the last call to compare had returned
-      assert packedValue.length == numDataDims * bytesPerDim;
-      if (numDataDims == 1) {
+      assert packedValue.length == numDims * bytesPerDim;
+      if (numDims == 1) {
         int cmp = FutureArrays.compareUnsigned(lastDocValue, 0, bytesPerDim, packedValue, 0, bytesPerDim);
         if (cmp < 0) {
           // ok
@@ -1017,11 +1010,11 @@ public class AssertingLeafReader extends FilterLeafReader {
 
     @Override
     public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
-      for(int dim=0;dim<numIndexDims;dim++) {
+      for(int dim=0;dim<numDims;dim++) {
         assert FutureArrays.compareUnsigned(minPackedValue, dim * bytesPerDim, dim * bytesPerDim + bytesPerDim, maxPackedValue, dim * bytesPerDim, dim * bytesPerDim + bytesPerDim) <= 0;
       }
-      System.arraycopy(maxPackedValue, 0, lastMaxPackedValue, 0, numIndexDims*bytesPerDim);
-      System.arraycopy(minPackedValue, 0, lastMinPackedValue, 0, numIndexDims*bytesPerDim);
+      System.arraycopy(maxPackedValue, 0, lastMaxPackedValue, 0, numDims*bytesPerDim);
+      System.arraycopy(minPackedValue, 0, lastMinPackedValue, 0, numDims*bytesPerDim);
       lastCompareResult = in.compare(minPackedValue, maxPackedValue);
       return lastCompareResult;
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java
index a9e4e91..f449644 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java
@@ -323,7 +323,7 @@ abstract class BaseIndexFileFormatTestCase extends LuceneTestCase {
     FieldInfo proto = oneDocReader.getFieldInfos().fieldInfo("field");
     FieldInfo field = new FieldInfo(proto.name, proto.number, proto.hasVectors(), proto.omitsNorms(), proto.hasPayloads(), 
                                     proto.getIndexOptions(), proto.getDocValuesType(), proto.getDocValuesGen(), new HashMap<>(),
-                                    proto.getPointDataDimensionCount(), proto.getPointIndexDimensionCount(), proto.getPointNumBytes(), proto.isSoftDeletesField());
+                                    proto.getPointDimensionCount(), proto.getPointNumBytes(), proto.isSoftDeletesField());
 
     FieldInfos fieldInfos = new FieldInfos(new FieldInfo[] { field } );
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/test-framework/src/java/org/apache/lucene/index/BasePointsFormatTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BasePointsFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/BasePointsFormatTestCase.java
index 71a9f26..b2e3e4e 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/BasePointsFormatTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/BasePointsFormatTestCase.java
@@ -28,7 +28,6 @@ import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.document.BinaryPoint;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
-import org.apache.lucene.document.FieldType;
 import org.apache.lucene.document.IntPoint;
 import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.document.StringField;
@@ -521,14 +520,13 @@ public abstract class BasePointsFormatTestCase extends BaseIndexFileFormatTestCa
   private void doTestRandomBinary(int count) throws Exception {
     int numDocs = TestUtil.nextInt(random(), count, count*2);
     int numBytesPerDim = TestUtil.nextInt(random(), 2, PointValues.MAX_NUM_BYTES);
-    int numDataDims = TestUtil.nextInt(random(), 1, PointValues.MAX_DIMENSIONS);
-    int numIndexDims = TestUtil.nextInt(random(), 1, numDataDims);
+    int numDims = TestUtil.nextInt(random(), 1, PointValues.MAX_DIMENSIONS);
 
     byte[][][] docValues = new byte[numDocs][][];
 
     for(int docID=0;docID<numDocs;docID++) {
-      byte[][] values = new byte[numDataDims][];
-      for(int dim=0;dim<numDataDims;dim++) {
+      byte[][] values = new byte[numDims][];
+      for(int dim=0;dim<numDims;dim++) {
         values[dim] = new byte[numBytesPerDim];
         // TODO: sometimes test on a "small" volume too, so we test the high density cases, higher chance of boundary, etc. cases:
         random().nextBytes(values[dim]);
@@ -536,22 +534,17 @@ public abstract class BasePointsFormatTestCase extends BaseIndexFileFormatTestCa
       docValues[docID] = values;
     }
 
-    verify(docValues, null, numDataDims, numIndexDims, numBytesPerDim);
-  }
-
-  private void verify(byte[][][] docValues, int[] docIDs, int numDims, int numBytesPerDim) throws Exception {
-    verify(docValues, docIDs, numDims, numDims, numBytesPerDim);
+    verify(docValues, null, numDims, numBytesPerDim);
   }
 
   /** docIDs can be null, for the single valued case, else it maps value to docID, but all values for one doc must be adjacent */
-  private void verify(byte[][][] docValues, int[] docIDs, int numDataDims, int numIndexDims, int numBytesPerDim) throws Exception {
+  private void verify(byte[][][] docValues, int[] docIDs, int numDims, int numBytesPerDim) throws Exception {
     try (Directory dir = getDirectory(docValues.length)) {
       while (true) {
         try {
-          verify(dir, docValues, docIDs, numDataDims, numIndexDims, numBytesPerDim, false);
+          verify(dir, docValues, docIDs, numDims, numBytesPerDim, false);
           return;
         } catch (IllegalArgumentException iae) {
-          iae.printStackTrace();
           // This just means we got a too-small maxMB for the maxPointsInLeafNode; just retry
           assertTrue(iae.getMessage().contains("either increase maxMBSortInHeap or decrease maxPointsInLeafNode"));
         }
@@ -560,22 +553,9 @@ public abstract class BasePointsFormatTestCase extends BaseIndexFileFormatTestCa
   }
 
   private void verify(Directory dir, byte[][][] docValues, int[] ids, int numDims, int numBytesPerDim, boolean expectExceptions) throws Exception {
-    verify(dir, docValues, ids, numDims, numDims, numBytesPerDim, expectExceptions);
-  }
-
-  private byte[] flattenBinaryPoint(byte[][] value, int numDataDims, int numBytesPerDim) {
-    byte[] result = new byte[value.length * numBytesPerDim];
-    for (int d = 0; d < numDataDims; ++d) {
-      System.arraycopy(value[d], 0, result, d * numBytesPerDim, numBytesPerDim);
-    }
-    return result;
-  }
-
-  /** test selective indexing */
-  private void verify(Directory dir, byte[][][] docValues, int[] ids, int numDataDims, int numIndexDims, int numBytesPerDim, boolean expectExceptions) throws Exception {
     int numValues = docValues.length;
     if (VERBOSE) {
-      System.out.println("TEST: numValues=" + numValues + " numDataDims=" + numDataDims + " numIndexDims=" + numIndexDims + " numBytesPerDim=" + numBytesPerDim);
+      System.out.println("TEST: numValues=" + numValues + " numDims=" + numDims + " numBytesPerDim=" + numBytesPerDim);
     }
 
     // RandomIndexWriter is too slow:
@@ -598,10 +578,10 @@ public abstract class BasePointsFormatTestCase extends BaseIndexFileFormatTestCa
     DirectoryReader r = null;
 
     // Compute actual min/max values:
-    byte[][] expectedMinValues = new byte[numDataDims][];
-    byte[][] expectedMaxValues = new byte[numDataDims][];
+    byte[][] expectedMinValues = new byte[numDims][];
+    byte[][] expectedMaxValues = new byte[numDims][];
     for(int ord=0;ord<docValues.length;ord++) {
-      for(int dim=0;dim<numDataDims;dim++) {
+      for(int dim=0;dim<numDims;dim++) {
         if (ord == 0) {
           expectedMinValues[dim] = new byte[numBytesPerDim];
           System.arraycopy(docValues[ord][dim], 0, expectedMinValues[dim], 0, numBytesPerDim);
@@ -649,10 +629,6 @@ public abstract class BasePointsFormatTestCase extends BaseIndexFileFormatTestCa
 
     try {
 
-      FieldType fieldType = new FieldType();
-      fieldType.setDimensions(numDataDims, numIndexDims, numBytesPerDim);
-      fieldType.freeze();
-
       Document doc = null;
       int lastID = -1;
       for(int ord=0;ord<numValues;ord++) {
@@ -673,10 +649,7 @@ public abstract class BasePointsFormatTestCase extends BaseIndexFileFormatTestCa
           doc = new Document();
           doc.add(new NumericDocValuesField("id", id));
         }
-        // pack the binary point
-        byte[] val = flattenBinaryPoint(docValues[ord], numDataDims, numBytesPerDim);
-
-        doc.add(new BinaryPoint("field", val, fieldType));
+        doc.add(new BinaryPoint("field", docValues[ord]));
         lastID = id;
 
         if (random().nextInt(30) == 17) {
@@ -694,8 +667,7 @@ public abstract class BasePointsFormatTestCase extends BaseIndexFileFormatTestCa
         if (random().nextInt(30) == 17) {
           // randomly index some documents with this field, but we will delete them:
           Document xdoc = new Document();
-          val = flattenBinaryPoint(docValues[ord], numDataDims, numBytesPerDim);
-          xdoc.add(new BinaryPoint("field", val, fieldType));
+          xdoc.add(new BinaryPoint("field", docValues[ord]));
           xdoc.add(new StringField("nukeme", "yes", Field.Store.NO));
           if (useRealWriter) {
             w.w.addDocument(xdoc);
@@ -717,7 +689,7 @@ public abstract class BasePointsFormatTestCase extends BaseIndexFileFormatTestCa
 
         if (VERBOSE) {
           System.out.println("  ord=" + ord + " id=" + id);
-          for(int dim=0;dim<numDataDims;dim++) {
+          for(int dim=0;dim<numDims;dim++) {
             System.out.println("    dim=" + dim + " value=" + new BytesRef(docValues[ord][dim]));
           }
         }
@@ -759,10 +731,10 @@ public abstract class BasePointsFormatTestCase extends BaseIndexFileFormatTestCa
       Bits liveDocs = MultiFields.getLiveDocs(r);
 
       // Verify min/max values are correct:
-      byte[] minValues = new byte[numIndexDims*numBytesPerDim];
+      byte[] minValues = new byte[numDims*numBytesPerDim];
       Arrays.fill(minValues, (byte) 0xff);
 
-      byte[] maxValues = new byte[numIndexDims*numBytesPerDim];
+      byte[] maxValues = new byte[numDims*numBytesPerDim];
 
       for(LeafReaderContext ctx : r.leaves()) {
         PointValues dimValues = ctx.reader().getPointValues("field");
@@ -772,7 +744,7 @@ public abstract class BasePointsFormatTestCase extends BaseIndexFileFormatTestCa
 
         byte[] leafMinValues = dimValues.getMinPackedValue();
         byte[] leafMaxValues = dimValues.getMaxPackedValue();
-        for(int dim=0;dim<numIndexDims;dim++) {
+        for(int dim=0;dim<numDims;dim++) {
           if (FutureArrays.compareUnsigned(leafMinValues, dim * numBytesPerDim, dim * numBytesPerDim + numBytesPerDim, minValues, dim * numBytesPerDim, dim * numBytesPerDim + numBytesPerDim) < 0) {
             System.arraycopy(leafMinValues, dim*numBytesPerDim, minValues, dim*numBytesPerDim, numBytesPerDim);
           }
@@ -783,7 +755,7 @@ public abstract class BasePointsFormatTestCase extends BaseIndexFileFormatTestCa
       }
 
       byte[] scratch = new byte[numBytesPerDim];
-      for(int dim=0;dim<numIndexDims;dim++) {
+      for(int dim=0;dim<numDims;dim++) {
         System.arraycopy(minValues, dim*numBytesPerDim, scratch, 0, numBytesPerDim);
         //System.out.println("dim=" + dim + " expectedMin=" + new BytesRef(expectedMinValues[dim]) + " min=" + new BytesRef(scratch));
         assertTrue(Arrays.equals(expectedMinValues[dim], scratch));
@@ -799,9 +771,9 @@ public abstract class BasePointsFormatTestCase extends BaseIndexFileFormatTestCa
         }
 
         // Random N dims rect query:
-        byte[][] queryMin = new byte[numIndexDims][];
-        byte[][] queryMax = new byte[numIndexDims][];
-        for(int dim=0;dim<numIndexDims;dim++) {
+        byte[][] queryMin = new byte[numDims][];
+        byte[][] queryMax = new byte[numDims][];    
+        for(int dim=0;dim<numDims;dim++) {    
           queryMin[dim] = new byte[numBytesPerDim];
           random().nextBytes(queryMin[dim]);
           queryMax[dim] = new byte[numBytesPerDim];
@@ -814,7 +786,7 @@ public abstract class BasePointsFormatTestCase extends BaseIndexFileFormatTestCa
         }
 
         if (VERBOSE) {
-          for(int dim=0;dim<numIndexDims;dim++) {
+          for(int dim=0;dim<numDims;dim++) {
             System.out.println("  dim=" + dim + "\n    queryMin=" + new BytesRef(queryMin[dim]) + "\n    queryMax=" + new BytesRef(queryMax[dim]));
           }
         }
@@ -844,7 +816,7 @@ public abstract class BasePointsFormatTestCase extends BaseIndexFileFormatTestCa
                   return;
                 }
 
-                for(int dim=0;dim<numIndexDims;dim++) {
+                for(int dim=0;dim<numDims;dim++) {
                   //System.out.println("  dim=" + dim + " value=" + new BytesRef(packedValue, dim*numBytesPerDim, numBytesPerDim));
                   if (FutureArrays.compareUnsigned(packedValue, dim * numBytesPerDim, dim * numBytesPerDim + numBytesPerDim, queryMin[dim], 0, numBytesPerDim) < 0 ||
                       FutureArrays.compareUnsigned(packedValue, dim * numBytesPerDim, dim * numBytesPerDim + numBytesPerDim, queryMax[dim], 0, numBytesPerDim) > 0) {
@@ -861,7 +833,7 @@ public abstract class BasePointsFormatTestCase extends BaseIndexFileFormatTestCa
               public Relation compare(byte[] minPacked, byte[] maxPacked) {
                 boolean crosses = false;
                 //System.out.println("compare");
-                for(int dim=0;dim<numIndexDims;dim++) {
+                for(int dim=0;dim<numDims;dim++) {
                   if (FutureArrays.compareUnsigned(maxPacked, dim * numBytesPerDim, dim * numBytesPerDim + numBytesPerDim, queryMin[dim], 0, numBytesPerDim) < 0 ||
                       FutureArrays.compareUnsigned(minPacked, dim * numBytesPerDim, dim * numBytesPerDim + numBytesPerDim, queryMax[dim], 0, numBytesPerDim) > 0) {
                     //System.out.println("  query_outside_cell");
@@ -886,7 +858,7 @@ public abstract class BasePointsFormatTestCase extends BaseIndexFileFormatTestCa
         BitSet expected = new BitSet();
         for(int ord=0;ord<numValues;ord++) {
           boolean matches = true;
-          for(int dim=0;dim<numIndexDims;dim++) {
+          for(int dim=0;dim<numDims;dim++) {
             byte[] x = docValues[ord][dim];
             if (FutureArrays.compareUnsigned(x, 0, numBytesPerDim, queryMin[dim], 0, numBytesPerDim) < 0 ||
                 FutureArrays.compareUnsigned(x, 0, numBytesPerDim, queryMax[dim], 0, numBytesPerDim) > 0) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/test-framework/src/java/org/apache/lucene/index/MismatchedLeafReader.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/MismatchedLeafReader.java b/lucene/test-framework/src/java/org/apache/lucene/index/MismatchedLeafReader.java
index 691f19e..2c74677 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/MismatchedLeafReader.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/MismatchedLeafReader.java
@@ -76,8 +76,7 @@ public class MismatchedLeafReader extends FilterLeafReader {
                                         oldInfo.getDocValuesType(),  // docValuesType
                                         oldInfo.getDocValuesGen(),   // dvGen
                                         oldInfo.attributes(),        // attributes
-                                        oldInfo.getPointDataDimensionCount(),      // data dimension count
-                                        oldInfo.getPointIndexDimensionCount(),      // index dimension count
+                                        oldInfo.getPointDimensionCount(),      // dimension count
                                         oldInfo.getPointNumBytes(),  // dimension numBytes
                                         oldInfo.isSoftDeletesField()); // used as soft-deletes field
       shuffled.set(i, newInfo);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java b/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
index 78aa9c2..df28105 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
@@ -113,8 +113,7 @@ public class RandomCodec extends AssertingCodec {
             try (BKDWriter writer = new RandomlySplittingBKDWriter(writeState.segmentInfo.maxDoc(),
                                                                    writeState.directory,
                                                                    writeState.segmentInfo.name,
-                                                                   fieldInfo.getPointDataDimensionCount(),
-                                                                   fieldInfo.getPointIndexDimensionCount(),
+                                                                   fieldInfo.getPointDimensionCount(),
                                                                    fieldInfo.getPointNumBytes(),
                                                                    maxPointsInLeafNode,
                                                                    maxMBSortInHeap,
@@ -265,10 +264,10 @@ public class RandomCodec extends AssertingCodec {
 
     final Random random;
 
-    public RandomlySplittingBKDWriter(int maxDoc, Directory tempDir, String tempFileNamePrefix, int numDataDims, int numIndexDims,
+    public RandomlySplittingBKDWriter(int maxDoc, Directory tempDir, String tempFileNamePrefix, int numDims,
                                       int bytesPerDim, int maxPointsInLeafNode, double maxMBSortInHeap,
                                       long totalPointCount, boolean singleValuePerDoc, int randomSeed) throws IOException {
-      super(maxDoc, tempDir, tempFileNamePrefix, numDataDims, numIndexDims, bytesPerDim, maxPointsInLeafNode, maxMBSortInHeap, totalPointCount,
+      super(maxDoc, tempDir, tempFileNamePrefix, numDims, bytesPerDim, maxPointsInLeafNode, maxMBSortInHeap, totalPointCount,
             getRandomSingleValuePerDoc(singleValuePerDoc, randomSeed),
             getRandomLongOrds(totalPointCount, singleValuePerDoc, randomSeed),
             getRandomOfflineSorterBufferMB(randomSeed),
@@ -297,7 +296,7 @@ public class RandomCodec extends AssertingCodec {
     @Override
     protected int split(byte[] minPackedValue, byte[] maxPackedValue, int[] parentDims) {
       // BKD normally defaults by the widest dimension, to try to make as squarish cells as possible, but we just pick a random one ;)
-      return random.nextInt(numIndexDims);
+      return random.nextInt(numDims);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/test-framework/src/java/org/apache/lucene/index/RandomPostingsTester.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/RandomPostingsTester.java b/lucene/test-framework/src/java/org/apache/lucene/index/RandomPostingsTester.java
index b00bacf..0afab1c 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/RandomPostingsTester.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/RandomPostingsTester.java
@@ -122,7 +122,7 @@ public class RandomPostingsTester {
       fieldInfoArray[fieldUpto] = new FieldInfo(field, fieldUpto, false, false, true,
                                                 IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS,
                                                 DocValuesType.NONE, -1, new HashMap<>(),
-                                                0, 0, 0, false);
+                                                0, 0, false);
       fieldUpto++;
 
       SortedMap<BytesRef,SeedAndOrd> postings = new TreeMap<>();
@@ -638,7 +638,7 @@ public class RandomPostingsTester {
                                                    DocValuesType.NONE,
                                                    -1,
                                                    new HashMap<>(),
-                                                   0, 0, 0, false);
+                                                   0, 0, false);
     }
 
     FieldInfos newFieldInfos = new FieldInfos(newFieldInfoArray);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
index b8b9e29..f554bcb 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
@@ -2640,12 +2640,10 @@ public abstract class LuceneTestCase extends Assert {
     FieldInfos fieldInfos1 = MultiFields.getMergedFieldInfos(leftReader);
     FieldInfos fieldInfos2 = MultiFields.getMergedFieldInfos(rightReader);
     for(FieldInfo fieldInfo1 : fieldInfos1) {
-      if (fieldInfo1.getPointDataDimensionCount() != 0) {
+      if (fieldInfo1.getPointDimensionCount() != 0) {
         FieldInfo fieldInfo2 = fieldInfos2.fieldInfo(fieldInfo1.name);
-        // same data dimension count?
-        assertEquals(info, fieldInfo2.getPointDataDimensionCount(), fieldInfo2.getPointDataDimensionCount());
-        // same index dimension count?
-        assertEquals(info, fieldInfo2.getPointIndexDimensionCount(), fieldInfo2.getPointIndexDimensionCount());
+        // same dimension count?
+        assertEquals(info, fieldInfo2.getPointDimensionCount(), fieldInfo2.getPointDimensionCount());
         // same bytes per dimension?
         assertEquals(info, fieldInfo2.getPointNumBytes(), fieldInfo2.getPointNumBytes());
 
@@ -2657,12 +2655,10 @@ public abstract class LuceneTestCase extends Assert {
 
     // make sure FieldInfos2 doesn't have any point fields that FieldInfo1 didn't have
     for(FieldInfo fieldInfo2 : fieldInfos2) {
-      if (fieldInfo2.getPointDataDimensionCount() != 0) {
+      if (fieldInfo2.getPointDimensionCount() != 0) {
         FieldInfo fieldInfo1 = fieldInfos1.fieldInfo(fieldInfo2.name);
-        // same data dimension count?
-        assertEquals(info, fieldInfo2.getPointDataDimensionCount(), fieldInfo1.getPointDataDimensionCount());
-        // same index dimension count?
-        assertEquals(info, fieldInfo2.getPointIndexDimensionCount(), fieldInfo1.getPointIndexDimensionCount());
+        // same dimension count?
+        assertEquals(info, fieldInfo2.getPointDimensionCount(), fieldInfo1.getPointDimensionCount());
         // same bytes per dimension?
         assertEquals(info, fieldInfo2.getPointNumBytes(), fieldInfo1.getPointNumBytes());
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java b/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
index 811cf0d..0ea90fc 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
@@ -1070,7 +1070,7 @@ public final class TestUtil {
       final Field field1 = (Field) f;
       final Field field2;
       final DocValuesType dvType = field1.fieldType().docValuesType();
-      final int dimCount = field1.fieldType().pointDataDimensionCount();
+      final int dimCount = field1.fieldType().pointDimensionCount();
       if (dvType != DocValuesType.NONE) {
         switch(dvType) {
           case NUMERIC:

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java b/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
index f7006df..d434735 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
@@ -792,8 +792,7 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
               DocValuesType.NONE,
               fieldInfo.getDocValuesGen(),
               fieldInfo.attributes(),
-              fieldInfo.getPointDataDimensionCount(),
-              fieldInfo.getPointIndexDimensionCount(),
+              fieldInfo.getPointDimensionCount(),
               fieldInfo.getPointNumBytes(),
               fieldInfo.isSoftDeletesField());
           newInfos.add(f);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/solr/core/src/java/org/apache/solr/legacy/BBoxStrategy.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/legacy/BBoxStrategy.java b/solr/core/src/java/org/apache/solr/legacy/BBoxStrategy.java
index b2f239e..84f90ef 100644
--- a/solr/core/src/java/org/apache/solr/legacy/BBoxStrategy.java
+++ b/solr/core/src/java/org/apache/solr/legacy/BBoxStrategy.java
@@ -173,7 +173,7 @@ public class BBoxStrategy extends SpatialStrategy {
     if ((this.hasDocVals = fieldType.docValuesType() != DocValuesType.NONE)) {
       numQuads++;
     }
-    if ((this.hasPointVals = fieldType.pointDataDimensionCount() > 0)) {
+    if ((this.hasPointVals = fieldType.pointDimensionCount() > 0)) {
       numQuads++;
     }
     if (fieldType.indexOptions() != IndexOptions.NONE && fieldType instanceof LegacyFieldType && ((LegacyFieldType)fieldType).numericType() != null) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/solr/core/src/java/org/apache/solr/legacy/PointVectorStrategy.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/legacy/PointVectorStrategy.java b/solr/core/src/java/org/apache/solr/legacy/PointVectorStrategy.java
index 4df76ca..da48fcb 100644
--- a/solr/core/src/java/org/apache/solr/legacy/PointVectorStrategy.java
+++ b/solr/core/src/java/org/apache/solr/legacy/PointVectorStrategy.java
@@ -152,7 +152,7 @@ public class PointVectorStrategy extends SpatialStrategy {
     if ((this.hasDocVals = fieldType.docValuesType() != DocValuesType.NONE)) {
       numPairs++;
     }
-    if ((this.hasPointVals = fieldType.pointDataDimensionCount() > 0)) {
+    if ((this.hasPointVals = fieldType.pointDimensionCount() > 0)) {
       numPairs++;
     }
     if (fieldType.indexOptions() != IndexOptions.NONE && fieldType instanceof LegacyFieldType && ((LegacyFieldType)fieldType).numericType() != null) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/solr/core/src/java/org/apache/solr/schema/SchemaField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/SchemaField.java b/solr/core/src/java/org/apache/solr/schema/SchemaField.java
index e28629e..256cbae 100644
--- a/solr/core/src/java/org/apache/solr/schema/SchemaField.java
+++ b/solr/core/src/java/org/apache/solr/schema/SchemaField.java
@@ -415,12 +415,7 @@ public final class SchemaField extends FieldProperties implements IndexableField
   }
 
   @Override
-  public int pointDataDimensionCount() {
-    return 0;
-  }
-
-  @Override
-  public int pointIndexDimensionCount() {
+  public int pointDimensionCount() {
     return 0;
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java b/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
index d8b2297..1970a44 100644
--- a/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
+++ b/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
@@ -430,7 +430,7 @@ public class CollapsingQParserPlugin extends QParserPlugin {
                                       DocValuesType.NONE,
                                       fieldInfo.getDocValuesGen(),
                                       fieldInfo.attributes(),
-                                      0, 0, 0, fieldInfo.isSoftDeletesField());
+                                      0, 0, fieldInfo.isSoftDeletesField());
           newInfos.add(f);
 
         } else {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/solr/core/src/java/org/apache/solr/search/Insanity.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/Insanity.java b/solr/core/src/java/org/apache/solr/search/Insanity.java
index 7da7a5c..67d1320 100644
--- a/solr/core/src/java/org/apache/solr/search/Insanity.java
+++ b/solr/core/src/java/org/apache/solr/search/Insanity.java
@@ -67,7 +67,7 @@ public class Insanity {
         if (fi.name.equals(insaneField)) {
           filteredInfos.add(new FieldInfo(fi.name, fi.number, fi.hasVectors(), fi.omitsNorms(),
                                           fi.hasPayloads(), fi.getIndexOptions(), DocValuesType.NONE, -1, Collections.emptyMap(),
-                                          fi.getPointDataDimensionCount(), fi.getPointIndexDimensionCount(), fi.getPointNumBytes(), fi.isSoftDeletesField()));
+                                          fi.getPointDimensionCount(), fi.getPointNumBytes(), fi.isSoftDeletesField()));
         } else {
           filteredInfos.add(fi);
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/solr/core/src/java/org/apache/solr/uninverting/FieldCacheImpl.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/uninverting/FieldCacheImpl.java b/solr/core/src/java/org/apache/solr/uninverting/FieldCacheImpl.java
index 42c34ab..e6e8fda 100644
--- a/solr/core/src/java/org/apache/solr/uninverting/FieldCacheImpl.java
+++ b/solr/core/src/java/org/apache/solr/uninverting/FieldCacheImpl.java
@@ -592,11 +592,11 @@ public class FieldCacheImpl implements FieldCache {
       if (parser instanceof PointParser) {
         // points case
         // no points in this segment
-        if (info.getPointDataDimensionCount() == 0) {
+        if (info.getPointDimensionCount() == 0) {
           return DocValues.emptyNumeric();
         }
-        if (info.getPointDataDimensionCount() != 1) {
-          throw new IllegalStateException("Type mismatch: " + field + " was indexed with dimensions=" + info.getPointDataDimensionCount());
+        if (info.getPointDimensionCount() != 1) {
+          throw new IllegalStateException("Type mismatch: " + field + " was indexed with dimensions=" + info.getPointDimensionCount());
         }
         PointValues values = reader.getPointValues(field);
         // no actual points for this field (e.g. all points deleted)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/solr/core/src/java/org/apache/solr/uninverting/UninvertingReader.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/uninverting/UninvertingReader.java b/solr/core/src/java/org/apache/solr/uninverting/UninvertingReader.java
index e804635..176795d 100644
--- a/solr/core/src/java/org/apache/solr/uninverting/UninvertingReader.java
+++ b/solr/core/src/java/org/apache/solr/uninverting/UninvertingReader.java
@@ -237,12 +237,12 @@ public class UninvertingReader extends FilterLeafReader {
       DocValuesType type = fi.getDocValuesType();
       // fields which currently don't have docValues, but are uninvertable (indexed or points data present)
       if (type == DocValuesType.NONE &&
-          (fi.getIndexOptions() != IndexOptions.NONE || (fi.getPointNumBytes() > 0 && fi.getPointDataDimensionCount() == 1))) {
+          (fi.getIndexOptions() != IndexOptions.NONE || (fi.getPointNumBytes() > 0 && fi.getPointDimensionCount() == 1))) {
         Type t = mapping.apply(fi.name); // could definitely return null, thus still can't uninvert it
         if (t != null) {
           if (t == Type.INTEGER_POINT || t == Type.LONG_POINT || t == Type.FLOAT_POINT || t == Type.DOUBLE_POINT) {
             // type uses points
-            if (fi.getPointDataDimensionCount() == 0) {
+            if (fi.getPointDimensionCount() == 0) {
               continue;
             }
           } else {
@@ -284,7 +284,7 @@ public class UninvertingReader extends FilterLeafReader {
         wrap = true;
         newFieldInfos.add(new FieldInfo(fi.name, fi.number, fi.hasVectors(), fi.omitsNorms(),
             fi.hasPayloads(), fi.getIndexOptions(), type, fi.getDocValuesGen(), fi.attributes(),
-            fi.getPointDataDimensionCount(), fi.getPointIndexDimensionCount(), fi.getPointNumBytes(), fi.isSoftDeletesField()));
+            fi.getPointDimensionCount(), fi.getPointNumBytes(), fi.isSoftDeletesField()));
       } else {
         newFieldInfos.add(fi);
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/solr/core/src/test/org/apache/solr/uninverting/TestUninvertingReader.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/uninverting/TestUninvertingReader.java b/solr/core/src/test/org/apache/solr/uninverting/TestUninvertingReader.java
index 05a1254..9ec1234 100644
--- a/solr/core/src/test/org/apache/solr/uninverting/TestUninvertingReader.java
+++ b/solr/core/src/test/org/apache/solr/uninverting/TestUninvertingReader.java
@@ -379,14 +379,12 @@ public class TestUninvertingReader extends LuceneTestCase {
 
     FieldInfo intFInfo = fieldInfos.fieldInfo("int");
     assertEquals(DocValuesType.NUMERIC, intFInfo.getDocValuesType());
-    assertEquals(0, intFInfo.getPointDataDimensionCount());
-    assertEquals(0, intFInfo.getPointIndexDimensionCount());
+    assertEquals(0, intFInfo.getPointDimensionCount());
     assertEquals(0, intFInfo.getPointNumBytes());
 
     FieldInfo dintFInfo = fieldInfos.fieldInfo("dint");
     assertEquals(DocValuesType.NUMERIC, dintFInfo.getDocValuesType());
-    assertEquals(1, dintFInfo.getPointDataDimensionCount());
-    assertEquals(1, dintFInfo.getPointIndexDimensionCount());
+    assertEquals(1, dintFInfo.getPointDimensionCount());
     assertEquals(4, dintFInfo.getPointNumBytes());
 
     FieldInfo dvFInfo = fieldInfos.fieldInfo("dv");


[2/6] lucene-solr:branch_7x: LUCENE-8496: revert Fix BKDWriter to use writeField1Dim when numDataDims is set to 1

Posted by nk...@apache.org.
LUCENE-8496: revert Fix BKDWriter to use writeField1Dim when numDataDims is set to 1


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/7ff5298f
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/7ff5298f
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/7ff5298f

Branch: refs/heads/branch_7x
Commit: 7ff5298fef4e591c15744dba51bba9f09891f24b
Parents: 4c0bee8
Author: Nicholas Knize <nk...@gmail.com>
Authored: Wed Oct 10 12:00:12 2018 -0500
Committer: Nicholas Knize <nk...@gmail.com>
Committed: Wed Oct 10 12:00:12 2018 -0500

----------------------------------------------------------------------
 lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7ff5298f/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java b/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
index ff41137..2dd5e7f 100644
--- a/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
@@ -438,7 +438,7 @@ public class BKDWriter 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 (numDataDims == 1) {
+    if (numIndexDims == 1) {
       return writeField1Dim(out, fieldName, reader);
     } else {
       return writeFieldNDims(out, fieldName, reader);


[5/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

Posted by nk...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/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 9c85fa8..5c9a811 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.getPointDataDimensionCount()/2 != numDims) {
+    if (fieldInfo.getPointDimensionCount()/2 != numDims) {
       throw new IllegalArgumentException("field=\"" + field + "\" was indexed with numDims="
-          + fieldInfo.getPointDataDimensionCount()/2 + " but this query has numDims=" + numDims);
+          + fieldInfo.getPointDimensionCount()/2 + " but this query has numDims=" + numDims);
     }
   }
 

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/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 ca6ac7d..50aaa42 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.getPointDataDimensionCount() == 0) {
+    if (fi == null || fi.getPointDimensionCount() == 0) {
       // Field does not exist or does not index points
       return null;
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/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 dd2945e..1f7adbd 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.getPointDataDimensionCount() == 0) {
+            if (perField.fieldInfo.getPointDimensionCount() == 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.getPointDataDimensionCount() != 0) {
+          } else if (perField.fieldInfo.getPointDimensionCount() != 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.pointDataDimensionCount() != 0) {
+    if (fieldType.pointDimensionCount() != 0) {
       if (fp == null) {
         fp = getOrAddField(fieldName, fieldType, false);
       }
@@ -497,18 +497,17 @@ final class DefaultIndexingChain extends DocConsumer {
 
   /** Called from processDocument to index one field's point */
   private void indexPoint(PerField fp, IndexableField field) throws IOException {
-    int pointDataDimensionCount = field.fieldType().pointDataDimensionCount();
-    int pointIndexDimensionCount = field.fieldType().pointIndexDimensionCount();
+    int pointDimensionCount = field.fieldType().pointDimensionCount();
 
     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.getPointDataDimensionCount() == 0) {
-      fieldInfos.globalFieldNumbers.setDimensions(fp.fieldInfo.number, fp.fieldInfo.name, pointDataDimensionCount, pointIndexDimensionCount, dimensionNumBytes);
+    if (fp.fieldInfo.getPointDimensionCount() == 0) {
+      fieldInfos.globalFieldNumbers.setDimensions(fp.fieldInfo.number, fp.fieldInfo.name, pointDimensionCount, dimensionNumBytes);
     }
 
-    fp.fieldInfo.setPointDimensions(pointDataDimensionCount, pointIndexDimensionCount, dimensionNumBytes);
+    fp.fieldInfo.setPointDimensions(pointDimensionCount, dimensionNumBytes);
 
     if (fp.pointValuesWriter == null) {
       fp.pointValuesWriter = new PointValuesWriter(docWriter, fp.fieldInfo);

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

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/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 4410656..0ce5882 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.getPointDataDimensionCount(), fi.getPointIndexDimensionCount(), fi.getPointNumBytes(), fi.isSoftDeletesField());
+        map.addOrGet(fi.name, fi.number, fi.getDocValuesType(), fi.getPointDimensionCount(), 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, 0, f.name().equals(config.softDeletesField));
+        globalFieldNumberMap.addOrGet(f.name(), -1, dvType, 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.getPointDataDimensionCount(), fi.getPointIndexDimensionCount(), fi.getPointNumBytes(), fi.isSoftDeletesField());
+              globalFieldNumberMap.addOrGet(fi.name, fi.number, fi.getDocValuesType(), fi.getPointDimensionCount(), fi.getPointNumBytes(), fi.isSoftDeletesField());
             }
             infos.add(copySegmentAsIs(info, newSegName, context));
           }

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/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 51b12d1..df433d2 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.getNumIndexDimensions();
+        final int numDimensions = values.getNumDimensions();
         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.getNumIndexDimensions();
+        final int numDimensions = values.getNumDimensions();
         final int numBytesPerDimension = values.getBytesPerDimension();
         for (int i = 0; i < numDimensions; ++i) {
           int offset = i * numBytesPerDimension;
@@ -233,11 +233,8 @@ 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 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 how many dimensions were indexed */
+  public abstract int getNumDimensions() 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/ac11c9e5/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 eb11ec3..4aaf095 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.getPointDataDimensionCount() * fieldInfo.getPointNumBytes();
+    packedBytesLength = fieldInfo.getPointDimensionCount() * 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.getPointDataDimensionCount() * fieldInfo.getPointNumBytes()));
+      throw new IllegalArgumentException("field=" + fieldInfo.name + ": this field's value has length=" + value.length + " but should be " + (fieldInfo.getPointDimensionCount() * fieldInfo.getPointNumBytes()));
     }
 
     if (docIDs.length == numPoints) {
@@ -106,12 +106,7 @@ class PointValuesWriter {
       }
 
       @Override
-      public int getNumDataDimensions() {
-        throw new UnsupportedOperationException();
-      }
-
-      @Override
-      public int getNumIndexDimensions() {
+      public int getNumDimensions() {
         throw new UnsupportedOperationException();
       }
 
@@ -234,13 +229,8 @@ class PointValuesWriter {
     }
 
     @Override
-    public int getNumDataDimensions() throws IOException {
-      return in.getNumDataDimensions();
-    }
-
-    @Override
-    public int getNumIndexDimensions() throws IOException {
-      return in.getNumIndexDimensions();
+    public int getNumDimensions() throws IOException {
+      return in.getNumDimensions();
     }
 
     @Override

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/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 b766146..8aa71bf 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.getNumIndexDimensions() != numDims) {
-          throw new IllegalArgumentException("field=\"" + field + "\" was indexed with numIndexDims=" + values.getNumIndexDimensions() + " but this query has numIndexDims=" + numDims);
+        if (values.getNumDimensions() != numDims) {
+          throw new IllegalArgumentException("field=\"" + field + "\" was indexed with numDims=" + values.getNumDimensions() + " but this query has numDims=" + 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/ac11c9e5/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 d34ea25..34b8909 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.getNumIndexDimensions() != numDims) {
-          throw new IllegalArgumentException("field=\"" + field + "\" was indexed with numIndexDimensions=" + values.getNumIndexDimensions() + " but this query has numDims=" + numDims);
+        if (values.getNumDimensions() != numDims) {
+          throw new IllegalArgumentException("field=\"" + field + "\" was indexed with numDims=" + values.getNumDimensions() + " 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/ac11c9e5/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 5579616..194ac82 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,8 +36,7 @@ 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 numDataDims;
-  final int numIndexDims;
+  final int numDims;
   final int bytesPerDim;
   final int numLeaves;
   final IndexInput in;
@@ -48,7 +47,6 @@ 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;
@@ -61,30 +59,24 @@ 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);
-    numDataDims = in.readVInt();
-    if (version >= BKDWriter.VERSION_SELECTIVE_INDEXING) {
-      numIndexDims = in.readVInt();
-    } else {
-      numIndexDims = numDataDims;
-    }
+    numDims = in.readVInt();
     maxPointsInLeafNode = in.readVInt();
     bytesPerDim = in.readVInt();
-    bytesPerIndexEntry = numDataDims == 1 && version >= BKDWriter.VERSION_IMPLICIT_SPLIT_DIM_1D ? bytesPerDim : bytesPerDim + 1;
-    packedBytesLength = numDataDims * bytesPerDim;
-    packedIndexBytesLength = numIndexDims * bytesPerDim;
+    bytesPerIndexEntry = numDims == 1 && version >= BKDWriter.VERSION_IMPLICIT_SPLIT_DIM_1D ? bytesPerDim : bytesPerDim + 1;
+    packedBytesLength = numDims * bytesPerDim;
 
     // Read index:
     numLeaves = in.readVInt();
     assert numLeaves > 0;
     leafNodeOffset = numLeaves;
 
-    minPackedValue = new byte[packedIndexBytesLength];
-    maxPackedValue = new byte[packedIndexBytesLength];
+    minPackedValue = new byte[packedBytesLength];
+    maxPackedValue = new byte[packedBytesLength];
 
-    in.readBytes(minPackedValue, 0, packedIndexBytesLength);
-    in.readBytes(maxPackedValue, 0, packedIndexBytesLength);
+    in.readBytes(minPackedValue, 0, packedBytesLength);
+    in.readBytes(maxPackedValue, 0, packedBytesLength);
 
-    for(int dim=0;dim<numIndexDims;dim++) {
+    for(int dim=0;dim<numDims;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);
       }
@@ -118,7 +110,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 (numDataDims == 1 && numLeaves > 1) {
+      if (numDims == 1 && numLeaves > 1) {
         int levelCount = 2;
         while (true) {
           if (numLeaves >= levelCount && numLeaves <= 2*levelCount) {
@@ -174,13 +166,13 @@ public final class BKDReader extends PointValues implements Accountable {
       nodeID = 1;
       level = 1;
       splitPackedValueStack[level] = new byte[packedBytesLength];
-    }
+    }      
 
     public void pushLeft() {
       nodeID *= 2;
       level++;
       if (splitPackedValueStack[level] == null) {
-        splitPackedValueStack[level] = new byte[packedIndexBytesLength];
+        splitPackedValueStack[level] = new byte[packedBytesLength];
       }
     }
 
@@ -191,7 +183,7 @@ public final class BKDReader extends PointValues implements Accountable {
       nodeID = nodeID * 2 + 1;
       level++;
       if (splitPackedValueStack[level] == null) {
-        splitPackedValueStack[level] = new byte[packedIndexBytesLength];
+        splitPackedValueStack[level] = new byte[packedBytesLength];
       }
     }
 
@@ -312,7 +304,7 @@ public final class BKDReader extends PointValues implements Accountable {
       } else {
         leafBlockFP = -1;
         int address = nodeID * bytesPerIndexEntry;
-        if (numIndexDims == 1) {
+        if (numDims == 1) {
           splitDim = 0;
           if (version < BKDWriter.VERSION_IMPLICIT_SPLIT_DIM_1D) {
             // skip over wastefully encoded 0 splitDim:
@@ -375,7 +367,7 @@ 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[numDataDims*(treeDepth+1)];
+      negativeDeltas = new boolean[numDims*(treeDepth+1)];
 
       in = new ByteArrayDataInput(packedIndex);
       splitValuesStack[0] = new byte[packedBytesLength];
@@ -394,7 +386,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*numDataDims, index.negativeDeltas, level*numDataDims, numDataDims);
+      System.arraycopy(negativeDeltas, level*numDims, index.negativeDeltas, level*numDims, numDims);
       index.splitDims[level] = splitDims[level];
       return index;
     }
@@ -403,9 +395,9 @@ public final class BKDReader extends PointValues implements Accountable {
     public void pushLeft() {
       int nodePosition = leftNodePositions[level];
       super.pushLeft();
-      System.arraycopy(negativeDeltas, (level-1)*numDataDims, negativeDeltas, level*numDataDims, numDataDims);
+      System.arraycopy(negativeDeltas, (level-1)*numDims, negativeDeltas, level*numDims, numDims);
       assert splitDim != -1;
-      negativeDeltas[level*numDataDims+splitDim] = true;
+      negativeDeltas[level*numDims+splitDim] = true;
       in.setPosition(nodePosition);
       readNodeData(true);
     }
@@ -414,9 +406,9 @@ public final class BKDReader extends PointValues implements Accountable {
     public void pushRight() {
       int nodePosition = rightNodePositions[level];
       super.pushRight();
-      System.arraycopy(negativeDeltas, (level-1)*numDataDims, negativeDeltas, level*numDataDims, numDataDims);
+      System.arraycopy(negativeDeltas, (level-1)*numDims, negativeDeltas, level*numDims, numDims);
       assert splitDim != -1;
-      negativeDeltas[level*numDataDims+splitDim] = false;
+      negativeDeltas[level*numDims+splitDim] = false;
       in.setPosition(nodePosition);
       readNodeData(false);
     }
@@ -456,19 +448,19 @@ public final class BKDReader extends PointValues implements Accountable {
 
         // read split dim, prefix, firstDiffByteDelta encoded as int:
         int code = in.readVInt();
-        splitDim = code % numIndexDims;
+        splitDim = code % numDims;
         splitDims[level] = splitDim;
-        code /= numIndexDims;
+        code /= numDims;
         int prefix = code % (1+bytesPerDim);
         int suffix = bytesPerDim - prefix;
 
         if (splitValuesStack[level] == null) {
-          splitValuesStack[level] = new byte[packedIndexBytesLength];
+          splitValuesStack[level] = new byte[packedBytesLength];
         }
-        System.arraycopy(splitValuesStack[level-1], 0, splitValuesStack[level], 0, packedIndexBytesLength);
+        System.arraycopy(splitValuesStack[level-1], 0, splitValuesStack[level], 0, packedBytesLength);
         if (suffix > 0) {
           int firstDiffByteDelta = code / (1+bytesPerDim);
-          if (negativeDeltas[level*numIndexDims + splitDim]) {
+          if (negativeDeltas[level*numDims + splitDim]) {
             firstDiffByteDelta = -firstDiffByteDelta;
           }
           int oldByte = splitValuesStack[level][splitDim*bytesPerDim+prefix] & 0xFF;
@@ -505,7 +497,7 @@ public final class BKDReader extends PointValues implements Accountable {
   public static final class IntersectState {
     final IndexInput in;
     final int[] scratchDocIDs;
-    final byte[] scratchDataPackedValue, scratchMinIndexPackedValue, scratchMaxIndexPackedValue;
+    final byte[] scratchPackedValue1, scratchPackedValue2;
     final int[] commonPrefixLengths;
 
     final IntersectVisitor visitor;
@@ -513,7 +505,6 @@ public final class BKDReader extends PointValues implements Accountable {
 
     public IntersectState(IndexInput in, int numDims,
                           int packedBytesLength,
-                          int packedIndexBytesLength,
                           int maxPointsInLeafNode,
                           IntersectVisitor visitor,
                           IndexTree indexVisitor) {
@@ -521,9 +512,8 @@ public final class BKDReader extends PointValues implements Accountable {
       this.visitor = visitor;
       this.commonPrefixLengths = new int[numDims];
       this.scratchDocIDs = new int[maxPointsInLeafNode];
-      this.scratchDataPackedValue = new byte[packedBytesLength];
-      this.scratchMinIndexPackedValue = new byte[packedIndexBytesLength];
-      this.scratchMaxIndexPackedValue = new byte[packedIndexBytesLength];
+      this.scratchPackedValue1 = new byte[packedBytesLength];
+      this.scratchPackedValue2 = new byte[packedBytesLength];
       this.index = indexVisitor;
     }
   }
@@ -576,9 +566,8 @@ public final class BKDReader extends PointValues implements Accountable {
     } else {
       index = new LegacyIndexTree();
     }
-    return new IntersectState(in.clone(), numDataDims,
+    return new IntersectState(in.clone(), numDims,
                               packedBytesLength,
-                              packedIndexBytesLength,
                               maxPointsInLeafNode,
                               visitor,
                               index);
@@ -591,7 +580,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.scratchDataPackedValue, state.scratchMinIndexPackedValue, state.scratchMaxIndexPackedValue, state.in, state.scratchDocIDs, count, state.visitor);
+    visitDocValues(state.commonPrefixLengths, state.scratchPackedValue1, state.scratchPackedValue2, state.in, state.scratchDocIDs, count, state.visitor);
   }
 
   private void visitDocIDs(IndexInput in, long blockFP, IntersectVisitor visitor) throws IOException {
@@ -624,19 +613,16 @@ public final class BKDReader extends PointValues implements Accountable {
     return count;
   }
 
-  void visitDocValues(int[] commonPrefixLengths, byte[] scratchDataPackedValue, byte[] scratchMinIndexPackedValue, byte[] scratchMaxIndexPackedValue,
-                      IndexInput in, int[] docIDs, int count, IntersectVisitor visitor) throws IOException {
+  void visitDocValues(int[] commonPrefixLengths, byte[] scratchPackedValue1, byte[] scratchPackedValue2, IndexInput in, int[] docIDs, int count, IntersectVisitor visitor) throws IOException {
 
 
-    readCommonPrefixes(commonPrefixLengths, scratchDataPackedValue, in);
+    readCommonPrefixes(commonPrefixLengths, scratchPackedValue1, in);
 
-    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);
+    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);
       readMinMax(commonPrefixLengths, minPackedValue, maxPackedValue, in);
 
       // The index gives us range of values for each dimension, but the actual range of values
@@ -667,24 +653,24 @@ public final class BKDReader extends PointValues implements Accountable {
         : readCompressedDim(in);
 
     if (compressedDim == -1) {
-      visitRawDocValues(commonPrefixLengths, scratchDataPackedValue, in, docIDs, count, visitor);
+      visitRawDocValues(commonPrefixLengths, scratchPackedValue1, in, docIDs, count, visitor);
     } else {
-      visitCompressedDocValues(commonPrefixLengths, scratchDataPackedValue, in, docIDs, count, visitor, compressedDim);
+      visitCompressedDocValues(commonPrefixLengths, scratchPackedValue1, in, docIDs, count, visitor, compressedDim);
     }
   }
 
-  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);
+    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);
+      }
     }
-  }
 
   // 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<numDataDims;dim++) {
+      for(int dim=0;dim<numDims;dim++) {
         int prefix = commonPrefixLengths[dim];
         in.readBytes(scratchPackedValue, dim*bytesPerDim + prefix, bytesPerDim - prefix);
       }
@@ -702,7 +688,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<numDataDims;dim++) {
+        for(int dim=0;dim<numDims;dim++) {
           int prefix = commonPrefixLengths[dim];
           in.readBytes(scratchPackedValue, dim*bytesPerDim + prefix, bytesPerDim - prefix);
         }
@@ -717,14 +703,14 @@ public final class BKDReader extends PointValues implements Accountable {
 
   private int readCompressedDim(IndexInput in) throws IOException {
     int compressedDim = in.readByte();
-    if (compressedDim < -1 || compressedDim >= numDataDims) {
+    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) {
@@ -761,15 +747,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.scratchDataPackedValue, state.scratchMinIndexPackedValue, state.scratchMaxIndexPackedValue, state.in, state.scratchDocIDs, count, state.visitor);
+        visitDocValues(state.commonPrefixLengths, state.scratchPackedValue1, state.scratchPackedValue2, 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 + ", numIndexDims=" + numIndexDims;
-      assert splitDim < numIndexDims: "splitDim=" + splitDim + ", numIndexDims=" + numIndexDims;
+      assert splitDim >= 0: "splitDim=" + splitDim;
+      assert splitDim < numDims;
 
       byte[] splitPackedValue = state.index.getSplitPackedValue();
       BytesRef splitDimValue = state.index.getSplitDimValue();
@@ -777,11 +763,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 + " 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;
+      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;
 
       // Recurse on left sub-tree:
-      System.arraycopy(cellMaxPacked, 0, splitPackedValue, 0, packedIndexBytesLength);
+      System.arraycopy(cellMaxPacked, 0, splitPackedValue, 0, packedBytesLength);
       System.arraycopy(splitDimValue.bytes, splitDimValue.offset, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
       state.index.pushLeft();
       intersect(state, cellMinPacked, splitPackedValue);
@@ -791,7 +777,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, packedIndexBytesLength);
+      System.arraycopy(cellMinPacked, 0, splitPackedValue, 0, packedBytesLength);
       System.arraycopy(splitDimValue.bytes, splitDimValue.offset, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
       state.index.pushRight();
       intersect(state, splitPackedValue, cellMaxPacked);
@@ -822,8 +808,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 + ", numIndexDims=" + numIndexDims;
-      assert splitDim < numIndexDims: "splitDim=" + splitDim + ", numIndexDims=" + numIndexDims;
+      assert splitDim >= 0: "splitDim=" + splitDim;
+      assert splitDim < numDims;
 
       byte[] splitPackedValue = state.index.getSplitPackedValue();
       BytesRef splitDimValue = state.index.getSplitDimValue();
@@ -831,11 +817,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 + " 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;
+      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;
 
       // Recurse on left sub-tree:
-      System.arraycopy(cellMaxPacked, 0, splitPackedValue, 0, packedIndexBytesLength);
+      System.arraycopy(cellMaxPacked, 0, splitPackedValue, 0, packedBytesLength);
       System.arraycopy(splitDimValue.bytes, splitDimValue.offset, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
       state.index.pushLeft();
       final long leftCost = estimatePointCount(state, cellMinPacked, splitPackedValue);
@@ -845,7 +831,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, packedIndexBytesLength);
+      System.arraycopy(cellMinPacked, 0, splitPackedValue, 0, packedBytesLength);
       System.arraycopy(splitDimValue.bytes, splitDimValue.offset, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
       state.index.pushRight();
       final long rightCost = estimatePointCount(state, splitPackedValue, cellMaxPacked);
@@ -874,13 +860,8 @@ public final class BKDReader extends PointValues implements Accountable {
   }
 
   @Override
-  public int getNumDataDimensions() {
-    return numDataDims;
-  }
-
-  @Override
-  public int getNumIndexDimensions() {
-    return numIndexDims;
+  public int getNumDimensions() {
+    return numDims;
   }
 
   @Override


[4/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

Posted by nk...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java b/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
index 2dd5e7f..e47cbca 100644
--- a/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
@@ -89,8 +89,7 @@ public class BKDWriter implements Closeable {
   public static final int VERSION_IMPLICIT_SPLIT_DIM_1D = 3;
   public static final int VERSION_PACKED_INDEX = 4;
   public static final int VERSION_LEAF_STORES_BOUNDS = 5;
-  public static final int VERSION_SELECTIVE_INDEXING = 6;
-  public static final int VERSION_CURRENT = VERSION_SELECTIVE_INDEXING;
+  public static final int VERSION_CURRENT = VERSION_LEAF_STORES_BOUNDS;
 
   /** How many bytes each docs takes in the fixed-width offline format */
   private final int bytesPerDoc;
@@ -104,21 +103,15 @@ public class BKDWriter 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 TrackingDirectoryWrapper tempDir;
   final String tempFileNamePrefix;
   final double maxMBSortInHeap;
@@ -164,39 +157,37 @@ public class BKDWriter implements Closeable {
 
   private final int maxDoc;
 
-  public BKDWriter(int maxDoc, Directory tempDir, String tempFileNamePrefix, int numDataDims, int numIndexDims, int bytesPerDim,
+  public BKDWriter(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);
   }
 
-  protected BKDWriter(int maxDoc, Directory tempDir, String tempFileNamePrefix, int numDataDims, int numIndexDims, int bytesPerDim,
+  protected BKDWriter(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;
@@ -223,7 +214,7 @@ public class BKDWriter 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) {
@@ -236,14 +227,11 @@ public class BKDWriter 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);
@@ -292,10 +280,10 @@ public class BKDWriter 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);
@@ -340,9 +328,8 @@ public class BKDWriter implements Closeable {
     public MergeReader(BKDReader bkd, MergeState.DocMap docMap) throws IOException {
       this.bkd = bkd;
       state = new BKDReader.IntersectState(bkd.in.clone(),
-                                           bkd.numDataDims,
+                                           bkd.numDims,
                                            bkd.packedBytesLength,
-                                           bkd.packedIndexBytesLength,
                                            bkd.maxPointsInLeafNode,
                                            null,
                                            null);
@@ -363,7 +350,7 @@ public class BKDWriter implements Closeable {
           docsInBlock = bkd.readDocIDs(state.in, state.in.getFilePointer(), state.scratchDocIDs);
           assert docsInBlock > 0;
           docBlockUpto = 0;
-          bkd.visitDocValues(state.commonPrefixLengths, state.scratchDataPackedValue, state.scratchMinIndexPackedValue, state.scratchMaxIndexPackedValue, state.in, state.scratchDocIDs, docsInBlock, new IntersectVisitor() {
+          bkd.visitDocValues(state.commonPrefixLengths, state.scratchPackedValue1, state.scratchPackedValue2, state.in, state.scratchDocIDs, docsInBlock, new IntersectVisitor() {
             int i = 0;
 
             @Override
@@ -401,7 +388,7 @@ public class BKDWriter implements Closeable {
         if (mappedDocID != -1) {
           // Not deleted!
           docID = mappedDocID;
-          System.arraycopy(packedValues, index * bkd.packedBytesLength, state.scratchDataPackedValue, 0, bkd.packedBytesLength);
+          System.arraycopy(packedValues, index * bkd.packedBytesLength, state.scratchPackedValue1, 0, bkd.packedBytesLength);
           return true;
         }
       }
@@ -420,7 +407,7 @@ public class BKDWriter implements Closeable {
     public boolean lessThan(MergeReader a, MergeReader b) {
       assert a != b;
 
-      int cmp = FutureArrays.compareUnsigned(a.state.scratchDataPackedValue, 0, bytesPerDim, b.state.scratchDataPackedValue, 0, bytesPerDim);
+      int cmp = FutureArrays.compareUnsigned(a.state.scratchPackedValue1, 0, bytesPerDim, b.state.scratchPackedValue1, 0, bytesPerDim);
       if (cmp < 0) {
         return true;
       } else if (cmp > 0) {
@@ -438,7 +425,7 @@ public class BKDWriter 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);
@@ -481,7 +468,7 @@ public class BKDWriter 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);
@@ -494,12 +481,12 @@ public class BKDWriter implements Closeable {
       docsSeen.set(values.getDocID(i));
     }
 
-    final int[] parentSplits = new int[numIndexDims];
+    final int[] parentSplits = new int[numDims];
     build(1, numLeaves, values, 0, Math.toIntExact(pointCount), out,
           minPackedValue, maxPackedValue, parentSplits,
           splitPackedValues, leafBlockFPs,
           new int[maxPointsInLeafNode]);
-    assert Arrays.equals(parentSplits, new int[numIndexDims]);
+    assert Arrays.equals(parentSplits, new int[numDims]);
 
     long indexFP = out.getFilePointer();
     writeIndex(out, Math.toIntExact(countPerLeaf), leafBlockFPs, splitPackedValues);
@@ -509,7 +496,7 @@ public class BKDWriter 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);
 
@@ -562,7 +549,7 @@ public class BKDWriter implements Closeable {
       MergeReader reader = queue.top();
       // System.out.println("iter reader=" + reader);
 
-      oneDimWriter.add(reader.state.scratchDataPackedValue, reader.docID);
+      oneDimWriter.add(reader.state.scratchPackedValue1, reader.docID);
 
       if (reader.next()) {
         queue.updateTop();
@@ -589,8 +576,8 @@ public class BKDWriter implements Closeable {
     private 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");
@@ -667,9 +654,9 @@ public class BKDWriter 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;
 
@@ -844,7 +831,7 @@ public class BKDWriter implements Closeable {
   */
 
   private PointWriter sort(int dim) throws IOException {
-    assert dim >= 0 && dim < numDataDims;
+    assert dim >= 0 && dim < numDims;
 
     if (heapPointWriter != null) {
 
@@ -877,7 +864,7 @@ public class BKDWriter 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) {
@@ -962,7 +949,7 @@ public class BKDWriter implements Closeable {
     }
 
     LongBitSet ordBitSet;
-    if (numDataDims > 1) {
+    if (numDims > 1) {
       if (singleValuePerDoc) {
         ordBitSet = new LongBitSet(maxDoc);
       } else {
@@ -997,7 +984,7 @@ public class BKDWriter 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<>();
@@ -1005,9 +992,7 @@ public class BKDWriter implements Closeable {
     boolean success = false;
     try {
       //long t0 = System.nanoTime();
-      // even with selective indexing we create the sortedPointWriters so we can compress
-      // the leaf node data by common prefix
-      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();
@@ -1021,7 +1006,7 @@ public class BKDWriter implements Closeable {
         heapPointWriter = null;
       }
 
-      final int[] parentSplits = new int[numIndexDims];
+      final int[] parentSplits = new int[numDims];
       build(1, numLeaves, sortedPointWriters,
             ordBitSet, out,
             minPackedValue, maxPackedValue,
@@ -1029,7 +1014,7 @@ public class BKDWriter implements Closeable {
             splitPackedValues,
             leafBlockFPs,
             toCloseHeroically);
-      assert Arrays.equals(parentSplits, new int[numIndexDims]);
+      assert Arrays.equals(parentSplits, new int[numDims]);
 
       for(PathSlice slice : sortedPointWriters) {
         slice.writer.destroy();
@@ -1064,7 +1049,7 @@ public class BKDWriter implements Closeable {
     // Possibly rotate the leaf block FPs, if the index not fully balanced binary tree (only happens
     // if it was created by OneDimensionBKDWriter).  In this case the leaf nodes may straddle the two bottom
     // levels of the binary tree:
-    if (numIndexDims == 1 && numLeaves > 1) {
+    if (numDims == 1 && numLeaves > 1) {
       int levelCount = 2;
       while (true) {
         if (numLeaves >= levelCount && numLeaves <= 2*levelCount) {
@@ -1090,9 +1075,9 @@ public class BKDWriter implements Closeable {
 
     // This is the "file" we append the byte[] to:
     List<byte[]> blocks = new ArrayList<>();
-    byte[] lastSplitValues = new byte[bytesPerDim * numIndexDims];
+    byte[] lastSplitValues = new byte[bytesPerDim * numDims];
     //System.out.println("\npack index");
-    int totalSize = recursePackIndex(writeBuffer, leafBlockFPs, splitPackedValues, 0l, blocks, 1, lastSplitValues, new boolean[numIndexDims], false);
+    int totalSize = recursePackIndex(writeBuffer, leafBlockFPs, splitPackedValues, 0l, blocks, 1, lastSplitValues, new boolean[numDims], false);
 
     // Compact the byte[] blocks into single byte index:
     byte[] index = new byte[totalSize];
@@ -1144,7 +1129,7 @@ public class BKDWriter implements Closeable {
       if (isLeft == false) {
         leftBlockFP = getLeftMostLeafBlockFP(leafBlockFPs, nodeID);
         long delta = leftBlockFP - minBlockFP;
-        assert nodeID == 1 || delta > 0 : "expected nodeID=1 or delta > 0; got nodeID=" + nodeID + " and delta=" + delta;
+        assert nodeID == 1 || delta > 0;
         writeBuffer.writeVLong(delta);
       } else {
         // The left tree's left most leaf block FP is always the minimal FP:
@@ -1180,7 +1165,7 @@ public class BKDWriter implements Closeable {
       }
 
       // pack the prefix, splitDim and delta first diff byte into a single vInt:
-      int code = (firstDiffByteDelta * (1+bytesPerDim) + prefix) * numIndexDims + splitDim;
+      int code = (firstDiffByteDelta * (1+bytesPerDim) + prefix) * numDims + splitDim;
 
       //System.out.println("  code=" + code);
       //System.out.println("  splitValue=" + new BytesRef(splitPackedValues, address, bytesPerDim));
@@ -1266,15 +1251,14 @@ public class BKDWriter implements Closeable {
   private void writeIndex(IndexOutput out, int countPerLeaf, int numLeaves, byte[] packedIndex) throws IOException {
     
     CodecUtil.writeHeader(out, CODEC_NAME, VERSION_CURRENT);
-    out.writeVInt(numDataDims);
-    out.writeVInt(numIndexDims);
+    out.writeVInt(numDims);
     out.writeVInt(countPerLeaf);
     out.writeVInt(bytesPerDim);
 
     assert numLeaves > 0;
     out.writeVInt(numLeaves);
-    out.writeBytes(minPackedValue, 0, packedIndexBytesLength);
-    out.writeBytes(maxPackedValue, 0, packedIndexBytesLength);
+    out.writeBytes(minPackedValue, 0, packedBytesLength);
+    out.writeBytes(maxPackedValue, 0, packedBytesLength);
 
     out.writeVLong(pointCount);
     out.writeVInt(docsSeen.cardinality());
@@ -1294,7 +1278,7 @@ public class BKDWriter implements Closeable {
       // all values in this block are equal
       out.writeByte((byte) -1);
     } else {
-      if (numIndexDims != 1) {
+      if (numDims != 1) {
         writeActualBounds(out, commonPrefixLengths, count, packedValues);
       }
       assert commonPrefixLengths[sortedDim] < bytesPerDim;
@@ -1317,7 +1301,7 @@ public class BKDWriter implements Closeable {
   }
 
   private void writeActualBounds(DataOutput out, int[] commonPrefixLengths, int count, IntFunction<BytesRef> packedValues) throws IOException {
-    for (int dim = 0; dim < numIndexDims; ++dim) {
+    for (int dim = 0; dim < numDims; ++dim) {
       int commonPrefixLength = commonPrefixLengths[dim];
       int suffixLength = bytesPerDim - commonPrefixLength;
       if (suffixLength > 0) {
@@ -1355,7 +1339,7 @@ public class BKDWriter 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);
       }
@@ -1377,7 +1361,7 @@ public class BKDWriter implements Closeable {
   }
 
   private void writeCommonPrefixes(DataOutput out, int[] commonPrefixes, byte[] packedValue) throws IOException {
-    for(int dim=0;dim<numDataDims;dim++) {
+    for(int dim=0;dim<numDims;dim++) {
       out.writeVInt(commonPrefixes[dim]);
       //System.out.println(commonPrefixes[dim] + " of " + bytesPerDim);
       out.writeBytes(packedValue, dim*bytesPerDim, commonPrefixes[dim]);
@@ -1448,7 +1432,7 @@ public class BKDWriter implements Closeable {
       boolean result = reader.next();
       assert result: "rightCount=" + rightCount + " source.count=" + source.count + " source.writer=" + source.writer;
       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):
@@ -1463,7 +1447,7 @@ public class BKDWriter 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;
@@ -1491,7 +1475,7 @@ public class BKDWriter implements Closeable {
     for (int numSplits : parentSplits) {
       maxNumSplits = Math.max(maxNumSplits, numSplits);
     }
-    for (int dim = 0; dim < numIndexDims; ++dim) {
+    for (int dim = 0; dim < numDims; ++dim) {
       final int offset = dim * bytesPerDim;
       if (parentSplits[dim] < maxNumSplits / 2 &&
           FutureArrays.compareUnsigned(minPackedValue, offset, offset + bytesPerDim, maxPackedValue, offset, offset + bytesPerDim) != 0) {
@@ -1501,7 +1485,7 @@ public class BKDWriter implements Closeable {
 
     // 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);
@@ -1551,7 +1535,7 @@ public class BKDWriter 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]) {
@@ -1563,14 +1547,14 @@ public class BKDWriter 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));
@@ -1579,7 +1563,7 @@ public class BKDWriter 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) {
@@ -1650,8 +1634,8 @@ public class BKDWriter 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,
@@ -1685,7 +1669,7 @@ public class BKDWriter 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);
     }
@@ -1698,7 +1682,7 @@ public class BKDWriter 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
@@ -1786,7 +1770,7 @@ public class BKDWriter implements Closeable {
       // Inner node: partition/recurse
 
       int splitDim;
-      if (numIndexDims > 1) {
+      if (numDims > 1) {
         splitDim = split(minPackedValue, maxPackedValue, parentSplits);
       } else {
         splitDim = 0;
@@ -1807,24 +1791,24 @@ public class BKDWriter 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
@@ -1861,7 +1845,7 @@ public class BKDWriter implements Closeable {
             ordBitSet, out,
             minPackedValue, maxSplitPackedValue, parentSplits,
             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();
@@ -1874,7 +1858,7 @@ public class BKDWriter implements Closeable {
             ordBitSet, out,
             minSplitPackedValue, maxPackedValue, parentSplits,
             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();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/core/src/test/org/apache/lucene/document/TestFieldType.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/document/TestFieldType.java b/lucene/core/src/test/org/apache/lucene/document/TestFieldType.java
index b770e62..9214cb9 100644
--- a/lucene/core/src/test/org/apache/lucene/document/TestFieldType.java
+++ b/lucene/core/src/test/org/apache/lucene/document/TestFieldType.java
@@ -69,7 +69,7 @@ public class TestFieldType extends LuceneTestCase {
   public void testPointsToString() {
     FieldType ft = new FieldType();
     ft.setDimensions(1, Integer.BYTES);
-    assertEquals("pointDataDimensionCount=1,pointIndexDimensionCount=1,pointNumBytes=4", ft.toString());
+    assertEquals("pointDimensionCount=1,pointNumBytes=4", ft.toString());
   }
 
   private static Object randomValue(Class<?> clazz) {
@@ -85,20 +85,15 @@ public class TestFieldType extends LuceneTestCase {
 
   private static FieldType randomFieldType() throws Exception {
     // setDimensions handled special as values must be in-bounds.
-    Method setDimensionsMethodA = FieldType.class.getMethod("setDimensions", int.class, int.class);
-    Method setDimensionsMethodB = FieldType.class.getMethod("setDimensions", int.class, int.class, int.class);
+    Method setDimensionsMethod = FieldType.class.getMethod("setDimensions", int.class, int.class);
     FieldType ft = new FieldType();
     for (Method method : FieldType.class.getMethods()) {
       if (method.getName().startsWith("set")) {
         final Class<?>[] parameterTypes = method.getParameterTypes();
         final Object[] args = new Object[parameterTypes.length];
-        if (method.equals(setDimensionsMethodA)) {
+        if (method.equals(setDimensionsMethod)) {
           args[0] = 1 + random().nextInt(PointValues.MAX_DIMENSIONS);
           args[1] = 1 + random().nextInt(PointValues.MAX_NUM_BYTES);
-        } else if (method.equals(setDimensionsMethodB)) {
-          args[0] = 1 + random().nextInt(PointValues.MAX_DIMENSIONS);
-          args[1] = 1 + random().nextInt((Integer)args[0]);
-          args[2] = 1 + random().nextInt(PointValues.MAX_NUM_BYTES);
         } else {
           for (int i = 0; i < args.length; ++i) {
             args[i] = randomValue(parameterTypes[i]);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/core/src/test/org/apache/lucene/index/TestIndexableField.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexableField.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexableField.java
index ed40dcb..510a899 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexableField.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexableField.java
@@ -91,12 +91,7 @@ public class TestIndexableField extends LuceneTestCase {
       }
 
       @Override
-      public int pointDataDimensionCount() {
-        return 0;
-      }
-
-      @Override
-      public int pointIndexDimensionCount() {
+      public int pointDimensionCount() {
         return 0;
       }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/core/src/test/org/apache/lucene/index/TestPendingSoftDeletes.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestPendingSoftDeletes.java b/lucene/core/src/test/org/apache/lucene/index/TestPendingSoftDeletes.java
index c882a3b..7d03c7e 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestPendingSoftDeletes.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestPendingSoftDeletes.java
@@ -164,7 +164,7 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
     deletes.onNewReader(segmentReader, commitInfo);
     reader.close();
     writer.close();
-    FieldInfo fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, 0, Collections.emptyMap(), 0, 0, 0, true);
+    FieldInfo fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, 0, Collections.emptyMap(), 0, 0, true);
     List<Integer> docsDeleted = Arrays.asList(1, 3, 7, 8, DocIdSetIterator.NO_MORE_DOCS);
     List<DocValuesFieldUpdates> updates = Arrays.asList(singleUpdate(docsDeleted, 10, true));
     for (DocValuesFieldUpdates update : updates) {
@@ -185,7 +185,7 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
 
     docsDeleted = Arrays.asList(1, 2, DocIdSetIterator.NO_MORE_DOCS);
     updates = Arrays.asList(singleUpdate(docsDeleted, 10, true));
-    fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, 1, Collections.emptyMap(), 0, 0, 0, true);
+    fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, 1, Collections.emptyMap(), 0, 0, true);
     for (DocValuesFieldUpdates update : updates) {
       deletes.onDocValuesUpdate(fieldInfo, update.iterator());
     }
@@ -228,7 +228,7 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
     SegmentCommitInfo segmentInfo = segmentReader.getSegmentInfo();
     PendingDeletes deletes = newPendingDeletes(segmentInfo);
     deletes.onNewReader(segmentReader, segmentInfo);
-    FieldInfo fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, segmentInfo.getNextDocValuesGen(), Collections.emptyMap(), 0, 0, 0, true);
+    FieldInfo fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, segmentInfo.getNextDocValuesGen(), Collections.emptyMap(), 0, 0, true);
     List<Integer> docsDeleted = Arrays.asList(1, DocIdSetIterator.NO_MORE_DOCS);
     List<DocValuesFieldUpdates> updates = Arrays.asList(singleUpdate(docsDeleted, 3, true));
     for (DocValuesFieldUpdates update : updates) {
@@ -276,7 +276,7 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
     SegmentCommitInfo segmentInfo = segmentReader.getSegmentInfo();
     PendingDeletes deletes = newPendingDeletes(segmentInfo);
     deletes.onNewReader(segmentReader, segmentInfo);
-    FieldInfo fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, segmentInfo.getNextDocValuesGen(), Collections.emptyMap(), 0, 0, 0, true);
+    FieldInfo fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, segmentInfo.getNextDocValuesGen(), Collections.emptyMap(), 0, 0, true);
     List<DocValuesFieldUpdates> updates = Arrays.asList(singleUpdate(Arrays.asList(0, 1, DocIdSetIterator.NO_MORE_DOCS), 3, false));
     for (DocValuesFieldUpdates update : updates) {
       deletes.onDocValuesUpdate(fieldInfo, update.iterator());
@@ -295,7 +295,7 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
     assertEquals(0, deletes.numPendingDeletes());
 
     segmentInfo.advanceDocValuesGen();
-    fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, segmentInfo.getNextDocValuesGen(), Collections.emptyMap(), 0, 0, 0, true);
+    fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, segmentInfo.getNextDocValuesGen(), Collections.emptyMap(), 0, 0, true);
     updates = Arrays.asList(singleUpdate(Arrays.asList(1, DocIdSetIterator.NO_MORE_DOCS), 3, true));
     for (DocValuesFieldUpdates update : updates) {
       deletes.onDocValuesUpdate(fieldInfo, update.iterator());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java b/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java
index 087cd0b..7228f37 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java
@@ -73,7 +73,7 @@ public class TestPointValues extends LuceneTestCase {
     IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
       w.addDocument(doc);
     });
-    assertEquals("cannot change point data dimension count from 1 to 2 for field=\"dim\"", expected.getMessage());
+    assertEquals("cannot change point dimension count from 1 to 2 for field=\"dim\"", expected.getMessage());
     w.close();
     dir.close();
   }
@@ -91,7 +91,7 @@ public class TestPointValues extends LuceneTestCase {
     IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
       w.addDocument(doc2);
     });
-    assertEquals("cannot change point data dimension count from 1 to 2 for field=\"dim\"", expected.getMessage());
+    assertEquals("cannot change point dimension count from 1 to 2 for field=\"dim\"", expected.getMessage());
 
     w.close();
     dir.close();
@@ -111,7 +111,7 @@ public class TestPointValues extends LuceneTestCase {
     IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
       w.addDocument(doc2);
     });
-    assertEquals("cannot change point data dimension count from 1 to 2 for field=\"dim\"", expected.getMessage());
+    assertEquals("cannot change point dimension count from 1 to 2 for field=\"dim\"", expected.getMessage());
 
     w.close();
     dir.close();
@@ -133,7 +133,7 @@ public class TestPointValues extends LuceneTestCase {
     IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
       w2.addDocument(doc2);
     });
-    assertEquals("cannot change point data dimension count from 1 to 2 for field=\"dim\"", expected.getMessage());
+    assertEquals("cannot change point dimension count from 1 to 2 for field=\"dim\"", expected.getMessage());
 
     w2.close();
     dir.close();
@@ -156,7 +156,7 @@ public class TestPointValues extends LuceneTestCase {
     IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
       w2.addIndexes(new Directory[] {dir});
     });
-    assertEquals("cannot change point data dimension count from 2 to 1 for field=\"dim\"", expected.getMessage());
+    assertEquals("cannot change point dimension count from 2 to 1 for field=\"dim\"", expected.getMessage());
 
     IOUtils.close(w2, dir, dir2);
   }
@@ -179,7 +179,7 @@ public class TestPointValues extends LuceneTestCase {
     IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
         w2.addIndexes(new CodecReader[] {(CodecReader) getOnlyLeafReader(r)});
     });
-    assertEquals("cannot change point data dimension count from 2 to 1 for field=\"dim\"", expected.getMessage());
+    assertEquals("cannot change point dimension count from 2 to 1 for field=\"dim\"", expected.getMessage());
 
     IOUtils.close(r, w2, dir, dir2);
   }
@@ -203,7 +203,7 @@ public class TestPointValues extends LuceneTestCase {
     IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
       TestUtil.addIndexesSlowly(w2, r);
     });
-    assertEquals("cannot change point data dimension count from 2 to 1 for field=\"dim\"", expected.getMessage());
+    assertEquals("cannot change point dimension count from 2 to 1 for field=\"dim\"", expected.getMessage());
 
     IOUtils.close(r, w2, dir, dir2);
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java b/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
index ed57b82..0341b52 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
@@ -1095,7 +1095,7 @@ public class TestPointQueries extends LuceneTestCase {
     IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
       s.count(BinaryPoint.newRangeQuery("value", point, point));
     });
-    assertEquals("field=\"value\" was indexed with numIndexDimensions=1 but this query has numDims=2", expected.getMessage());
+    assertEquals("field=\"value\" was indexed with numDims=1 but this query has numDims=2", expected.getMessage());
 
     IOUtils.close(r, w, dir);
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/core/src/test/org/apache/lucene/util/TestDocIdSetBuilder.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/util/TestDocIdSetBuilder.java b/lucene/core/src/test/org/apache/lucene/util/TestDocIdSetBuilder.java
index a51b161..f87a73a 100644
--- a/lucene/core/src/test/org/apache/lucene/util/TestDocIdSetBuilder.java
+++ b/lucene/core/src/test/org/apache/lucene/util/TestDocIdSetBuilder.java
@@ -327,12 +327,7 @@ public class TestDocIdSetBuilder extends LuceneTestCase {
     }
 
     @Override
-    public int getNumDataDimensions() throws IOException {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public int getNumIndexDimensions() throws IOException {
+    public int getNumDimensions() throws IOException {
       throw new UnsupportedOperationException();
     }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/core/src/test/org/apache/lucene/util/bkd/Test2BBKDPoints.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/util/bkd/Test2BBKDPoints.java b/lucene/core/src/test/org/apache/lucene/util/bkd/Test2BBKDPoints.java
index 0d57bf8..e30168c 100644
--- a/lucene/core/src/test/org/apache/lucene/util/bkd/Test2BBKDPoints.java
+++ b/lucene/core/src/test/org/apache/lucene/util/bkd/Test2BBKDPoints.java
@@ -41,7 +41,7 @@ public class Test2BBKDPoints extends LuceneTestCase {
 
     final int numDocs = (Integer.MAX_VALUE / 26) + 100;
 
-    BKDWriter w = new BKDWriter(numDocs, dir, "_0", 1, 1, Long.BYTES,
+    BKDWriter w = new BKDWriter(numDocs, dir, "_0", 1, Long.BYTES,
                                 BKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE, BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP, 26L * numDocs, false);
     int counter = 0;
     byte[] packedBytes = new byte[Long.BYTES];
@@ -78,7 +78,7 @@ public class Test2BBKDPoints extends LuceneTestCase {
 
     final int numDocs = (Integer.MAX_VALUE / 26) + 100;
 
-    BKDWriter w = new BKDWriter(numDocs, dir, "_0", 2, 2, Long.BYTES,
+    BKDWriter w = new BKDWriter(numDocs, dir, "_0", 2, Long.BYTES,
                                 BKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE, BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP, 26L * numDocs, false);
     int counter = 0;
     byte[] packedBytes = new byte[2*Long.BYTES];

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/core/src/test/org/apache/lucene/util/bkd/TestBKD.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/util/bkd/TestBKD.java b/lucene/core/src/test/org/apache/lucene/util/bkd/TestBKD.java
index d75d785..cbd4d37 100644
--- a/lucene/core/src/test/org/apache/lucene/util/bkd/TestBKD.java
+++ b/lucene/core/src/test/org/apache/lucene/util/bkd/TestBKD.java
@@ -48,7 +48,7 @@ public class TestBKD extends LuceneTestCase {
 
   public void testBasicInts1D() throws Exception {
     try (Directory dir = getDirectory(100)) {
-      BKDWriter w = new BKDWriter(100, dir, "tmp", 1, 1, 4, 2, 1.0f, 100, true);
+      BKDWriter w = new BKDWriter(100, dir, "tmp", 1, 4, 2, 1.0f, 100, true);
       byte[] scratch = new byte[4];
       for(int docID=0;docID<100;docID++) {
         NumericUtils.intToSortableBytes(docID, scratch, 0);
@@ -121,13 +121,12 @@ public class TestBKD extends LuceneTestCase {
     int numDocs = atLeast(1000);
     try (Directory dir = getDirectory(numDocs)) {
       int numDims = TestUtil.nextInt(random(), 1, 5);
-      int numIndexDims = TestUtil.nextInt(random(), 1, numDims);
       int maxPointsInLeafNode = TestUtil.nextInt(random(), 50, 100);
       float maxMB = (float) 3.0 + (3*random().nextFloat());
-      BKDWriter w = new BKDWriter(numDocs, dir, "tmp", numDims, numIndexDims, 4, maxPointsInLeafNode, maxMB, numDocs, true);
+      BKDWriter w = new BKDWriter(numDocs, dir, "tmp", numDims, 4, maxPointsInLeafNode, maxMB, numDocs, true);
 
       if (VERBOSE) {
-        System.out.println("TEST: numDims=" + numDims + " numIndexDims=" + numIndexDims + " numDocs=" + numDocs);
+        System.out.println("TEST: numDims=" + numDims + " numDocs=" + numDocs);
       }
       int[][] docs = new int[numDocs][];
       byte[] scratch = new byte[4*numDims];
@@ -168,7 +167,7 @@ public class TestBKD extends LuceneTestCase {
 
         byte[] minPackedValue = r.getMinPackedValue();
         byte[] maxPackedValue = r.getMaxPackedValue();
-        for(int dim=0;dim<numIndexDims;dim++) {
+        for(int dim=0;dim<numDims;dim++) {
           assertEquals(minValue[dim], NumericUtils.sortableBytesToInt(minPackedValue, dim * Integer.BYTES));
           assertEquals(maxValue[dim], NumericUtils.sortableBytesToInt(maxPackedValue, dim * Integer.BYTES));
         }
@@ -182,7 +181,7 @@ public class TestBKD extends LuceneTestCase {
           // Random N dims rect query:
           int[] queryMin = new int[numDims];
           int[] queryMax = new int[numDims];    
-          for(int dim=0;dim<numIndexDims;dim++) {
+          for(int dim=0;dim<numDims;dim++) {
             queryMin[dim] = random().nextInt();
             queryMax[dim] = random().nextInt();
             if (queryMin[dim] > queryMax[dim]) {
@@ -203,7 +202,7 @@ public class TestBKD extends LuceneTestCase {
             @Override
             public void visit(int docID, byte[] packedValue) {
               //System.out.println("visit check docID=" + docID);
-              for(int dim=0;dim<numIndexDims;dim++) {
+              for(int dim=0;dim<numDims;dim++) {
                 int x = NumericUtils.sortableBytesToInt(packedValue, dim * Integer.BYTES);
                 if (x < queryMin[dim] || x > queryMax[dim]) {
                   //System.out.println("  no");
@@ -218,7 +217,7 @@ public class TestBKD extends LuceneTestCase {
             @Override
             public Relation compare(byte[] minPacked, byte[] maxPacked) {
               boolean crosses = false;
-              for(int dim=0;dim<numIndexDims;dim++) {
+              for(int dim=0;dim<numDims;dim++) {
                 int min = NumericUtils.sortableBytesToInt(minPacked, dim * Integer.BYTES);
                 int max = NumericUtils.sortableBytesToInt(maxPacked, dim * Integer.BYTES);
                 assert max >= min;
@@ -241,7 +240,7 @@ public class TestBKD extends LuceneTestCase {
           for(int docID=0;docID<numDocs;docID++) {
             int[] docValues = docs[docID];
             boolean expected = true;
-            for(int dim=0;dim<numIndexDims;dim++) {
+            for(int dim=0;dim<numDims;dim++) {
               int x = docValues[dim];
               if (x < queryMin[dim] || x > queryMax[dim]) {
                 expected = false;
@@ -265,7 +264,7 @@ public class TestBKD extends LuceneTestCase {
       int numDims = TestUtil.nextInt(random(), 1, 5);
       int maxPointsInLeafNode = TestUtil.nextInt(random(), 50, 100);
       float maxMB = (float) 3.0 + (3*random().nextFloat());
-      BKDWriter w = new BKDWriter(numDocs, dir, "tmp", numDims, numDims, numBytesPerDim, maxPointsInLeafNode, maxMB, numDocs, true);
+      BKDWriter w = new BKDWriter(numDocs, dir, "tmp", numDims, numBytesPerDim, maxPointsInLeafNode, maxMB, numDocs, true);
       BigInteger[][] docs = new BigInteger[numDocs][];
 
       byte[] scratch = new byte[numBytesPerDim*numDims];
@@ -381,14 +380,13 @@ public class TestBKD extends LuceneTestCase {
   public void testWithExceptions() throws Exception {
     int numDocs = atLeast(10000);
     int numBytesPerDim = TestUtil.nextInt(random(), 2, 30);
-    int numDataDims = TestUtil.nextInt(random(), 1, 5);
-    int numIndexDims = TestUtil.nextInt(random(), 1, numDataDims);
+    int numDims = TestUtil.nextInt(random(), 1, 5);
 
     byte[][][] docValues = new byte[numDocs][][];
 
     for(int docID=0;docID<numDocs;docID++) {
-      byte[][] values = new byte[numDataDims][];
-      for(int dim=0;dim<numDataDims;dim++) {
+      byte[][] values = new byte[numDims][];
+      for(int dim=0;dim<numDims;dim++) {
         values[dim] = new byte[numBytesPerDim];
         random().nextBytes(values[dim]);
       }
@@ -403,7 +401,7 @@ public class TestBKD extends LuceneTestCase {
       try {
         dir.setRandomIOExceptionRate(0.05);
         dir.setRandomIOExceptionRateOnOpen(0.05);
-        verify(dir, docValues, null, numDataDims, numIndexDims, numBytesPerDim, 50, maxMBHeap);
+        verify(dir, docValues, null, numDims, numBytesPerDim, 50, maxMBHeap);
       } catch (IllegalArgumentException iae) {
         // This just means we got a too-small maxMB for the maxPointsInLeafNode; just retry w/ more heap
         assertTrue(iae.getMessage().contains("either increase maxMBSortInHeap or decrease maxPointsInLeafNode"));
@@ -441,7 +439,7 @@ public class TestBKD extends LuceneTestCase {
   public void testTooLittleHeap() throws Exception { 
     try (Directory dir = getDirectory(0)) {
       IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
-        new BKDWriter(1, dir, "bkd", 1, 1, 16, 1000000, 0.001, 0, true);
+        new BKDWriter(1, dir, "bkd", 1, 16, 1000000, 0.001, 0, true);
       });
       assertTrue(expected.getMessage().contains("either increase maxMBSortInHeap or decrease maxPointsInLeafNode"));
     }
@@ -451,35 +449,33 @@ public class TestBKD extends LuceneTestCase {
     int numDocs = TestUtil.nextInt(random(), count, count*2);
     int numBytesPerDim = TestUtil.nextInt(random(), 2, 30);
 
-    int numDataDims = TestUtil.nextInt(random(), 1, 5);
-    int numIndexDims = TestUtil.nextInt(random(), 1, numDataDims);
+    int numDims = TestUtil.nextInt(random(), 1, 5);
 
     byte[][][] docValues = new byte[numDocs][][];
 
     for(int docID=0;docID<numDocs;docID++) {
-      byte[][] values = new byte[numDataDims][];
-      for(int dim=0;dim<numDataDims;dim++) {
+      byte[][] values = new byte[numDims][];
+      for(int dim=0;dim<numDims;dim++) {
         values[dim] = new byte[numBytesPerDim];
         random().nextBytes(values[dim]);
       }
       docValues[docID] = values;
     }
 
-    verify(docValues, null, numDataDims, numIndexDims, numBytesPerDim);
+    verify(docValues, null, numDims, numBytesPerDim);
   }
 
   public void testAllEqual() throws Exception {
     int numBytesPerDim = TestUtil.nextInt(random(), 2, 30);
-    int numDataDims = TestUtil.nextInt(random(), 1, 5);
-    int numIndexDims = TestUtil.nextInt(random(), 1, numDataDims);
+    int numDims = TestUtil.nextInt(random(), 1, 5);
 
     int numDocs = atLeast(1000);
     byte[][][] docValues = new byte[numDocs][][];
 
     for(int docID=0;docID<numDocs;docID++) {
       if (docID == 0) {
-        byte[][] values = new byte[numDataDims][];
-        for(int dim=0;dim<numDataDims;dim++) {
+        byte[][] values = new byte[numDims][];
+        for(int dim=0;dim<numDims;dim++) {
           values[dim] = new byte[numBytesPerDim];
           random().nextBytes(values[dim]);
         }
@@ -489,21 +485,20 @@ public class TestBKD extends LuceneTestCase {
       }
     }
 
-    verify(docValues, null, numDataDims, numIndexDims, numBytesPerDim);
+    verify(docValues, null, numDims, numBytesPerDim);
   }
 
   public void testOneDimEqual() throws Exception {
     int numBytesPerDim = TestUtil.nextInt(random(), 2, 30);
-    int numDataDims = TestUtil.nextInt(random(), 1, 5);
-    int numIndexDims = TestUtil.nextInt(random(), 1, numDataDims);
+    int numDims = TestUtil.nextInt(random(), 1, 5);
 
     int numDocs = atLeast(1000);
-    int theEqualDim = random().nextInt(numDataDims);
+    int theEqualDim = random().nextInt(numDims);
     byte[][][] docValues = new byte[numDocs][][];
 
     for(int docID=0;docID<numDocs;docID++) {
-      byte[][] values = new byte[numDataDims][];
-      for(int dim=0;dim<numDataDims;dim++) {
+      byte[][] values = new byte[numDims][];
+      for(int dim=0;dim<numDims;dim++) {
         values[dim] = new byte[numBytesPerDim];
         random().nextBytes(values[dim]);
       }
@@ -514,18 +509,17 @@ public class TestBKD extends LuceneTestCase {
     }
 
     // Use a small number of points in leaf blocks to trigger a lot of splitting
-    verify(docValues, null, numDataDims, numIndexDims, numBytesPerDim, TestUtil.nextInt(random(), 20, 50));
+    verify(docValues, null, numDims, numBytesPerDim, TestUtil.nextInt(random(), 20, 50));
   }
 
   // This triggers the logic that makes sure all dimensions get indexed
   // by looking at how many times each dim has been split
   public void testOneDimLowCard() throws Exception {
     int numBytesPerDim = TestUtil.nextInt(random(), 2, 30);
-    int numDataDims = TestUtil.nextInt(random(), 2, 5);
-    int numIndexDims = TestUtil.nextInt(random(), 2, numDataDims);
+    int numDims = TestUtil.nextInt(random(), 2, 5);
 
     int numDocs = atLeast(10000);
-    int theLowCardDim = random().nextInt(numDataDims);
+    int theLowCardDim = random().nextInt(numDims);
 
     byte[] value1 = new byte[numBytesPerDim];
     random().nextBytes(value1);
@@ -539,8 +533,8 @@ public class TestBKD extends LuceneTestCase {
     byte[][][] docValues = new byte[numDocs][][];
 
     for(int docID=0;docID<numDocs;docID++) {
-      byte[][] values = new byte[numDataDims][];
-      for(int dim=0;dim<numDataDims;dim++) {
+      byte[][] values = new byte[numDims][];
+      for(int dim=0;dim<numDims;dim++) {
         if (dim == theLowCardDim) {
           values[dim] = random().nextBoolean() ? value1 : value2;
         } else {
@@ -552,17 +546,16 @@ public class TestBKD extends LuceneTestCase {
     }
 
     // Use a small number of points in leaf blocks to trigger a lot of splitting
-    verify(docValues, null, numDataDims, numIndexDims, numBytesPerDim, TestUtil.nextInt(random(), 20, 50));
+    verify(docValues, null, numDims, numBytesPerDim, TestUtil.nextInt(random(), 20, 50));
   }
 
   // this should trigger run-length compression with lengths that are greater than 255
   public void testOneDimTwoValues() throws Exception {
     int numBytesPerDim = TestUtil.nextInt(random(), 2, 30);
-    int numDataDims = TestUtil.nextInt(random(), 1, 5);
-    int numIndexDims = TestUtil.nextInt(random(), 1, numDataDims);
+    int numDims = TestUtil.nextInt(random(), 1, 5);
 
     int numDocs = atLeast(1000);
-    int theDim = random().nextInt(numDataDims);
+    int theDim = random().nextInt(numDims);
     byte[] value1 = new byte[numBytesPerDim];
     random().nextBytes(value1);
     byte[] value2 = new byte[numBytesPerDim];
@@ -570,8 +563,8 @@ public class TestBKD extends LuceneTestCase {
     byte[][][] docValues = new byte[numDocs][][];
 
     for(int docID=0;docID<numDocs;docID++) {
-      byte[][] values = new byte[numDataDims][];
-      for(int dim=0;dim<numDataDims;dim++) {
+      byte[][] values = new byte[numDims][];
+      for(int dim=0;dim<numDims;dim++) {
         if (dim == theDim) {
           values[dim] = random().nextBoolean() ? value1 : value2;
         } else {
@@ -582,13 +575,12 @@ public class TestBKD extends LuceneTestCase {
       docValues[docID] = values;
     }
 
-    verify(docValues, null, numDataDims, numIndexDims, numBytesPerDim);
+    verify(docValues, null, numDims, numBytesPerDim);
   }
 
   public void testMultiValued() throws Exception {
     int numBytesPerDim = TestUtil.nextInt(random(), 2, 30);
-    int numDataDims = TestUtil.nextInt(random(), 1, 5);
-    int numIndexDims = TestUtil.nextInt(random(), 1, numDataDims);
+    int numDims = TestUtil.nextInt(random(), 1, 5);
 
     int numDocs = atLeast(1000);
     List<byte[][]> docValues = new ArrayList<>();
@@ -598,8 +590,8 @@ public class TestBKD extends LuceneTestCase {
       int numValuesInDoc = TestUtil.nextInt(random(), 1, 5);
       for(int ord=0;ord<numValuesInDoc;ord++) {
         docIDs.add(docID);
-        byte[][] values = new byte[numDataDims][];
-        for(int dim=0;dim<numDataDims;dim++) {
+        byte[][] values = new byte[numDims][];
+        for(int dim=0;dim<numDims;dim++) {
           values[dim] = new byte[numBytesPerDim];
           random().nextBytes(values[dim]);
         }
@@ -613,33 +605,33 @@ public class TestBKD extends LuceneTestCase {
       docIDsArray[i] = docIDs.get(i);
     }
 
-    verify(docValuesArray, docIDsArray, numDataDims, numIndexDims, numBytesPerDim);
+    verify(docValuesArray, docIDsArray, numDims, numBytesPerDim);
   }
 
   /** docIDs can be null, for the single valued case, else it maps value to docID */
-  private void verify(byte[][][] docValues, int[] docIDs, int numDataDims, int numIndexDims, int numBytesPerDim) throws Exception {
-    verify(docValues, docIDs, numDataDims, numIndexDims, numBytesPerDim, TestUtil.nextInt(random(), 50, 1000));
+  private void verify(byte[][][] docValues, int[] docIDs, int numDims, int numBytesPerDim) throws Exception {
+    verify(docValues, docIDs, numDims, numBytesPerDim, TestUtil.nextInt(random(), 50, 1000));
   }
 
-  private void verify(byte[][][] docValues, int[] docIDs, int numDataDims, int numIndexDims, int numBytesPerDim,
+  private void verify(byte[][][] docValues, int[] docIDs, int numDims, int numBytesPerDim,
       int maxPointsInLeafNode) throws Exception {
     try (Directory dir = getDirectory(docValues.length)) {
       double maxMB = (float) 3.0 + (3*random().nextDouble());
-      verify(dir, docValues, docIDs, numDataDims, numIndexDims, numBytesPerDim, maxPointsInLeafNode, maxMB);
+      verify(dir, docValues, docIDs, numDims, numBytesPerDim, maxPointsInLeafNode, maxMB);
     }
   }
 
-  private void verify(Directory dir, byte[][][] docValues, int[] docIDs, int numDataDims, int numIndexDims, int numBytesPerDim, int maxPointsInLeafNode, double maxMB) throws Exception {
+  private void verify(Directory dir, byte[][][] docValues, int[] docIDs, int numDims, int numBytesPerDim, int maxPointsInLeafNode, double maxMB) throws Exception {
     int numValues = docValues.length;
     if (VERBOSE) {
-      System.out.println("TEST: numValues=" + numValues + " numDataDims=" + numDataDims + " numIndexDims=" + numIndexDims + " numBytesPerDim=" + numBytesPerDim + " maxPointsInLeafNode=" + maxPointsInLeafNode + " maxMB=" + maxMB);
+      System.out.println("TEST: numValues=" + numValues + " numDims=" + numDims + " numBytesPerDim=" + numBytesPerDim + " maxPointsInLeafNode=" + maxPointsInLeafNode + " maxMB=" + maxMB);
     }
 
     List<Long> toMerge = null;
     List<MergeState.DocMap> docMaps = null;
     int seg = 0;
 
-    BKDWriter w = new BKDWriter(numValues, dir, "_" + seg, numDataDims, numIndexDims, numBytesPerDim, maxPointsInLeafNode, maxMB, docValues.length, false);
+    BKDWriter w = new BKDWriter(numValues, dir, "_" + seg, numDims, numBytesPerDim, maxPointsInLeafNode, maxMB, docValues.length, false);
     IndexOutput out = dir.createOutput("bkd", IOContext.DEFAULT);
     IndexInput in = null;
 
@@ -647,9 +639,9 @@ public class TestBKD extends LuceneTestCase {
 
     try {
 
-      byte[] scratch = new byte[numBytesPerDim*numDataDims];
+      byte[] scratch = new byte[numBytesPerDim*numDims];
       int lastDocIDBase = 0;
-      boolean useMerge = numDataDims == 1 && numValues >= 10 && random().nextBoolean();
+      boolean useMerge = numDims == 1 && numValues >= 10 && random().nextBoolean();
       int valuesInThisSeg;
       if (useMerge) {
         // Sometimes we will call merge with a single segment:
@@ -670,7 +662,7 @@ public class TestBKD extends LuceneTestCase {
         if (VERBOSE) {
           System.out.println("  ord=" + ord + " docID=" + docID + " lastDocIDBase=" + lastDocIDBase);
         }
-        for(int dim=0;dim<numDataDims;dim++) {
+        for(int dim=0;dim<numDims;dim++) {
           if (VERBOSE) {
             System.out.println("    " + dim + " -> " + new BytesRef(docValues[ord][dim]));
           }
@@ -699,7 +691,7 @@ public class TestBKD extends LuceneTestCase {
           seg++;
           maxPointsInLeafNode = TestUtil.nextInt(random(), 50, 1000);
           maxMB = (float) 3.0 + (3*random().nextDouble());
-          w = new BKDWriter(numValues, dir, "_" + seg, numDataDims, numIndexDims, numBytesPerDim, maxPointsInLeafNode, maxMB, docValues.length, false);
+          w = new BKDWriter(numValues, dir, "_" + seg, numDims, numBytesPerDim, maxPointsInLeafNode, maxMB, docValues.length, false);
           lastDocIDBase = docID;
         }
       }
@@ -720,7 +712,7 @@ public class TestBKD extends LuceneTestCase {
         out.close();
         in = dir.openInput("bkd", IOContext.DEFAULT);
         seg++;
-        w = new BKDWriter(numValues, dir, "_" + seg, numDataDims, numIndexDims, numBytesPerDim, maxPointsInLeafNode, maxMB, docValues.length, false);
+        w = new BKDWriter(numValues, dir, "_" + seg, numDims, numBytesPerDim, maxPointsInLeafNode, maxMB, docValues.length, false);
         List<BKDReader> readers = new ArrayList<>();
         for(long fp : toMerge) {
           in.seek(fp);
@@ -747,9 +739,9 @@ public class TestBKD extends LuceneTestCase {
         }
 
         // Random N dims rect query:
-        byte[][] queryMin = new byte[numDataDims][];
-        byte[][] queryMax = new byte[numDataDims][];
-        for(int dim=0;dim<numDataDims;dim++) {
+        byte[][] queryMin = new byte[numDims][];
+        byte[][] queryMax = new byte[numDims][];    
+        for(int dim=0;dim<numDims;dim++) {    
           queryMin[dim] = new byte[numBytesPerDim];
           random().nextBytes(queryMin[dim]);
           queryMax[dim] = new byte[numBytesPerDim];
@@ -772,7 +764,7 @@ public class TestBKD extends LuceneTestCase {
             @Override
             public void visit(int docID, byte[] packedValue) {
               //System.out.println("visit check docID=" + docID);
-              for(int dim=0;dim<numIndexDims;dim++) {
+              for(int dim=0;dim<numDims;dim++) {
                 if (FutureArrays.compareUnsigned(packedValue, dim * numBytesPerDim, dim * numBytesPerDim + numBytesPerDim, queryMin[dim], 0, numBytesPerDim) < 0 ||
                     FutureArrays.compareUnsigned(packedValue, dim * numBytesPerDim, dim * numBytesPerDim + numBytesPerDim, queryMax[dim], 0, numBytesPerDim) > 0) {
                   //System.out.println("  no");
@@ -787,7 +779,7 @@ public class TestBKD extends LuceneTestCase {
             @Override
             public Relation compare(byte[] minPacked, byte[] maxPacked) {
               boolean crosses = false;
-              for(int dim=0;dim<numIndexDims;dim++) {
+              for(int dim=0;dim<numDims;dim++) {
                 if (FutureArrays.compareUnsigned(maxPacked, dim * numBytesPerDim, dim * numBytesPerDim + numBytesPerDim, queryMin[dim], 0, numBytesPerDim) < 0 ||
                     FutureArrays.compareUnsigned(minPacked, dim * numBytesPerDim, dim * numBytesPerDim + numBytesPerDim, queryMax[dim], 0, numBytesPerDim) > 0) {
                   return Relation.CELL_OUTSIDE_QUERY;
@@ -808,7 +800,7 @@ public class TestBKD extends LuceneTestCase {
         BitSet expected = new BitSet();
         for(int ord=0;ord<numValues;ord++) {
           boolean matches = true;
-          for(int dim=0;dim<numIndexDims;dim++) {
+          for(int dim=0;dim<numDims;dim++) {
             byte[] x = docValues[ord][dim];
             if (FutureArrays.compareUnsigned(x, 0, numBytesPerDim, queryMin[dim], 0, numBytesPerDim) < 0 ||
                 FutureArrays.compareUnsigned(x, 0, numBytesPerDim, queryMax[dim], 0, numBytesPerDim) > 0) {
@@ -905,7 +897,7 @@ public class TestBKD extends LuceneTestCase {
       };
 
       CorruptIndexException e = expectThrows(CorruptIndexException.class, () -> {
-          verify(dir, docValues, null, numDims, numDims, numBytesPerDim, 50, 0.1);
+          verify(dir, docValues, null, numDims, numBytesPerDim, 50, 0.1);
         });
       assertTrue(e.getMessage().contains("checksum failed (hardware problem?)"));
     }
@@ -953,7 +945,7 @@ public class TestBKD extends LuceneTestCase {
       };
 
       Throwable t = expectThrows(CorruptIndexException.class, () -> {
-          verify(dir, docValues, null, numDims, numDims, numBytesPerDim, 50, 0.1);
+          verify(dir, docValues, null, numDims, numBytesPerDim, 50, 0.1);
         });
       assertCorruptionDetected(t);
     }
@@ -979,7 +971,7 @@ public class TestBKD extends LuceneTestCase {
   public void testTieBreakOrder() throws Exception {
     try (Directory dir = newDirectory()) {
       int numDocs = 10000;
-      BKDWriter w = new BKDWriter(numDocs+1, dir, "tmp", 1, 1, Integer.BYTES, 2, 0.01f, numDocs, true);
+      BKDWriter w = new BKDWriter(numDocs+1, dir, "tmp", 1, Integer.BYTES, 2, 0.01f, numDocs, true);
       for(int i=0;i<numDocs;i++) {
         w.add(new byte[Integer.BYTES], i);
       }
@@ -1020,7 +1012,7 @@ public class TestBKD extends LuceneTestCase {
       boolean singleValuePerDoc = false;
       boolean longOrds = true;
       int offlineSorterMaxTempFiles = TestUtil.nextInt(random(), 2, 20);
-      BKDWriter w = new BKDWriter(numDocs+1, dir, "tmp", 2, 2, Integer.BYTES, 2, 0.01f, numDocs,
+      BKDWriter w = new BKDWriter(numDocs+1, dir, "tmp", 2, Integer.BYTES, 2, 0.01f, numDocs,
                                   singleValuePerDoc, longOrds, 1, offlineSorterMaxTempFiles);
       byte[] buffer = new byte[2*Integer.BYTES];
       for(int i=0;i<numDocs;i++) {
@@ -1066,13 +1058,12 @@ public class TestBKD extends LuceneTestCase {
   // values as a LongPoint:
   public void testWastedLeadingBytes() throws Exception {
     int numDims = TestUtil.nextInt(random(), 1, PointValues.MAX_DIMENSIONS);
-    int numIndexDims = TestUtil.nextInt(random(), 1, numDims);
     int bytesPerDim = PointValues.MAX_NUM_BYTES;
     int bytesUsed = TestUtil.nextInt(random(), 1, 3);
 
     Directory dir = newFSDirectory(createTempDir());
     int numDocs = 100000;
-    BKDWriter w = new BKDWriter(numDocs+1, dir, "tmp", numDims, numIndexDims, bytesPerDim, 32, 1f, numDocs, true);
+    BKDWriter w = new BKDWriter(numDocs+1, dir, "tmp", numDims, bytesPerDim, 32, 1f, numDocs, true);
     byte[] tmp = new byte[bytesUsed];
     byte[] buffer = new byte[numDims * bytesPerDim];
     for(int i=0;i<numDocs;i++) {
@@ -1100,14 +1091,11 @@ public class TestBKD extends LuceneTestCase {
 
         @Override
         public void visit(int docID, byte[] packedValue) {
-          assert packedValue.length == numDims * bytesPerDim;
           visit(docID);
         }
 
         @Override
         public Relation compare(byte[] minPacked, byte[] maxPacked) {
-          assert minPacked.length == numIndexDims * bytesPerDim;
-          assert maxPacked.length == numIndexDims * bytesPerDim;
           if (random().nextInt(7) == 1) {
             return Relation.CELL_CROSSES_QUERY;
           } else {
@@ -1129,7 +1117,7 @@ public class TestBKD extends LuceneTestCase {
     final byte[] uniquePointValue = new byte[numBytesPerDim];
     random().nextBytes(uniquePointValue);
 
-    BKDWriter w = new BKDWriter(numValues, dir, "_temp", 1, 1, numBytesPerDim, maxPointsInLeafNode,
+    BKDWriter w = new BKDWriter(numValues, dir, "_temp", 1, numBytesPerDim, maxPointsInLeafNode,
         BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP, numValues, true);
     for (int i = 0; i < numValues; ++i) {
       if (i == numValues / 2) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/core/src/test/org/apache/lucene/util/bkd/TestMutablePointsReaderUtils.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/util/bkd/TestMutablePointsReaderUtils.java b/lucene/core/src/test/org/apache/lucene/util/bkd/TestMutablePointsReaderUtils.java
index da75b12..f38ab5a 100644
--- a/lucene/core/src/test/org/apache/lucene/util/bkd/TestMutablePointsReaderUtils.java
+++ b/lucene/core/src/test/org/apache/lucene/util/bkd/TestMutablePointsReaderUtils.java
@@ -232,12 +232,7 @@ public class TestMutablePointsReaderUtils extends LuceneTestCase {
     }
 
     @Override
-    public int getNumDataDimensions() throws IOException {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public int getNumIndexDimensions() throws IOException {
+    public int getNumDimensions() throws IOException {
       throw new UnsupportedOperationException();
     }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/highlighter/src/java/org/apache/lucene/search/highlight/TermVectorLeafReader.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/highlight/TermVectorLeafReader.java b/lucene/highlighter/src/java/org/apache/lucene/search/highlight/TermVectorLeafReader.java
index 42f7aec..1eef95f 100644
--- a/lucene/highlighter/src/java/org/apache/lucene/search/highlight/TermVectorLeafReader.java
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/highlight/TermVectorLeafReader.java
@@ -81,7 +81,7 @@ public class TermVectorLeafReader extends LeafReader {
     }
     FieldInfo fieldInfo = new FieldInfo(field, 0,
                                         true, true, terms.hasPayloads(),
-                                        indexOptions, DocValuesType.NONE, -1, Collections.emptyMap(), 0, 0, 0, false);
+                                        indexOptions, DocValuesType.NONE, -1, Collections.emptyMap(), 0, 0, false);
     fieldInfos = new FieldInfos(new FieldInfo[]{fieldInfo});
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/join/src/java/org/apache/lucene/search/join/PointInSetIncludingScoreQuery.java
----------------------------------------------------------------------
diff --git a/lucene/join/src/java/org/apache/lucene/search/join/PointInSetIncludingScoreQuery.java b/lucene/join/src/java/org/apache/lucene/search/join/PointInSetIncludingScoreQuery.java
index c534d1f..a7f6099 100644
--- a/lucene/join/src/java/org/apache/lucene/search/join/PointInSetIncludingScoreQuery.java
+++ b/lucene/join/src/java/org/apache/lucene/search/join/PointInSetIncludingScoreQuery.java
@@ -146,8 +146,8 @@ abstract class PointInSetIncludingScoreQuery extends Query {
         if (fieldInfo == null) {
           return null;
         }
-        if (fieldInfo.getPointDataDimensionCount() != 1) {
-          throw new IllegalArgumentException("field=\"" + field + "\" was indexed with numDims=" + fieldInfo.getPointDataDimensionCount() + " but this query has numDims=1");
+        if (fieldInfo.getPointDimensionCount() != 1) {
+          throw new IllegalArgumentException("field=\"" + field + "\" was indexed with numDims=" + fieldInfo.getPointDimensionCount() + " but this query has numDims=1");
         }
         if (fieldInfo.getPointNumBytes() != bytesPerDim) {
           throw new IllegalArgumentException("field=\"" + field + "\" was indexed with bytesPerDim=" + fieldInfo.getPointNumBytes() + " but this query has bytesPerDim=" + bytesPerDim);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
----------------------------------------------------------------------
diff --git a/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java b/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
index 32707ab..c51b104 100644
--- a/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
+++ b/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
@@ -409,7 +409,7 @@ public class MemoryIndex {
       storeDocValues(info, docValuesType, docValuesValue);
     }
 
-    if (field.fieldType().pointDataDimensionCount() > 0) {
+    if (field.fieldType().pointDimensionCount() > 0) {
       storePointValues(info, field.binaryValue());
     }
 
@@ -485,9 +485,9 @@ public class MemoryIndex {
     if (info == null) {
       fields.put(fieldName, info = new Info(createFieldInfo(fieldName, fields.size(), fieldType), byteBlockPool));
     }
-    if (fieldType.pointDataDimensionCount() != info.fieldInfo.getPointDataDimensionCount()) {
-      if (fieldType.pointDataDimensionCount() > 0)
-        info.fieldInfo.setPointDimensions(fieldType.pointDataDimensionCount(), fieldType.pointIndexDimensionCount(), fieldType.pointNumBytes());
+    if (fieldType.pointDimensionCount() != info.fieldInfo.getPointDimensionCount()) {
+      if (fieldType.pointDimensionCount() > 0)
+        info.fieldInfo.setPointDimensions(fieldType.pointDimensionCount(), fieldType.pointNumBytes());
     }
     if (fieldType.docValuesType() != info.fieldInfo.getDocValuesType()) {
       if (fieldType.docValuesType() != DocValuesType.NONE)
@@ -500,7 +500,7 @@ public class MemoryIndex {
     IndexOptions indexOptions = storeOffsets ? IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS : IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
     return new FieldInfo(fieldName, ord, fieldType.storeTermVectors(), fieldType.omitNorms(), storePayloads,
         indexOptions, fieldType.docValuesType(), -1, Collections.emptyMap(),
-        fieldType.pointDataDimensionCount(), fieldType.pointIndexDimensionCount(), fieldType.pointNumBytes(), false);
+        fieldType.pointDimensionCount(), fieldType.pointNumBytes(), false);
   }
 
   private void storePointValues(Info info, BytesRef pointValue) {
@@ -519,8 +519,7 @@ public class MemoryIndex {
       info.fieldInfo = new FieldInfo(
           info.fieldInfo.name, info.fieldInfo.number, info.fieldInfo.hasVectors(), info.fieldInfo.hasPayloads(),
           info.fieldInfo.hasPayloads(), info.fieldInfo.getIndexOptions(), docValuesType, -1, info.fieldInfo.attributes(),
-          info.fieldInfo.getPointDataDimensionCount(), info.fieldInfo.getPointIndexDimensionCount(), info.fieldInfo.getPointNumBytes(),
-          info.fieldInfo.isSoftDeletesField()
+          info.fieldInfo.getPointDimensionCount(), info.fieldInfo.getPointNumBytes(), info.fieldInfo.isSoftDeletesField()
       );
     } else if (existingDocValuesType != docValuesType) {
       throw new IllegalArgumentException("Can't add [" + docValuesType + "] doc values field [" + fieldName + "], because [" + existingDocValuesType + "] doc values field already exists");
@@ -867,7 +866,7 @@ public class MemoryIndex {
         if (pointValues != null) {
           assert pointValues[0].bytes.length == pointValues[0].length : "BytesRef should wrap a precise byte[], BytesRef.deepCopyOf() should take care of this";
 
-          final int numDimensions = fieldInfo.getPointDataDimensionCount();
+          final int numDimensions = fieldInfo.getPointDimensionCount();
           final int numBytesPerDimension = fieldInfo.getPointNumBytes();
           if (numDimensions == 1) {
             // PointInSetQuery.MergePointVisitor expects values to be visited in increasing order,
@@ -1568,13 +1567,8 @@ public class MemoryIndex {
       }
 
       @Override
-      public int getNumDataDimensions() throws IOException {
-        return info.fieldInfo.getPointDataDimensionCount();
-      }
-
-      @Override
-      public int getNumIndexDimensions() throws IOException {
-        return info.fieldInfo.getPointDataDimensionCount();
+      public int getNumDimensions() throws IOException {
+        return info.fieldInfo.getPointDimensionCount();
       }
 
       @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/sandbox/src/java/org/apache/lucene/document/BigIntegerPoint.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/BigIntegerPoint.java b/lucene/sandbox/src/java/org/apache/lucene/document/BigIntegerPoint.java
index 82eeb3f..4dc50d1 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/BigIntegerPoint.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/BigIntegerPoint.java
@@ -62,8 +62,8 @@ public class BigIntegerPoint extends Field {
 
   /** Change the values of this field */
   public void setBigIntegerValues(BigInteger... 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);
   }
@@ -75,8 +75,8 @@ public class BigIntegerPoint 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 == BYTES;
@@ -119,7 +119,7 @@ public class BigIntegerPoint 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/sandbox/src/java/org/apache/lucene/document/HalfFloatPoint.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/HalfFloatPoint.java b/lucene/sandbox/src/java/org/apache/lucene/document/HalfFloatPoint.java
index 15d8240..1f599a6 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/HalfFloatPoint.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/HalfFloatPoint.java
@@ -213,8 +213,8 @@ public final class HalfFloatPoint 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);
   }
@@ -226,8 +226,8 @@ public final class HalfFloatPoint 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 == BYTES;
@@ -270,7 +270,7 @@ public final class HalfFloatPoint 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/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java
index 2cdef86..5029d0e 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java
@@ -148,7 +148,7 @@ public class BBoxStrategy extends SpatialStrategy {
     if ((this.hasDocVals = fieldType.docValuesType() != DocValuesType.NONE)) {
       numQuads++;
     }
-    if ((this.hasPointVals = fieldType.pointDataDimensionCount() > 0)) {
+    if ((this.hasPointVals = fieldType.pointDimensionCount() > 0)) {
       numQuads++;
     }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/PointVectorStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/PointVectorStrategy.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/PointVectorStrategy.java
index cc7f6ef..0dd3c09 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/PointVectorStrategy.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/PointVectorStrategy.java
@@ -138,7 +138,7 @@ public class PointVectorStrategy extends SpatialStrategy {
     if ((this.hasDocVals = fieldType.docValuesType() != DocValuesType.NONE)) {
       numPairs++;
     }
-    if ((this.hasPointVals = fieldType.pointDataDimensionCount() > 0)) {
+    if ((this.hasPointVals = fieldType.pointDimensionCount() > 0)) {
       numPairs++;
     }
     this.fieldsLen = numPairs * 2;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointsFormat.java
index 4943b99..ff2e1b6 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointsFormat.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointsFormat.java
@@ -131,8 +131,8 @@ public final class AssertingPointsFormat extends PointsFormat {
     
     @Override
     public void writeField(FieldInfo fieldInfo, PointsReader values) throws IOException {
-      if (fieldInfo.getPointDataDimensionCount() == 0) {
-        throw new IllegalArgumentException("writing field=\"" + fieldInfo.name + "\" but pointDataDimensionalCount is 0");
+      if (fieldInfo.getPointDimensionCount() == 0) {
+        throw new IllegalArgumentException("writing field=\"" + fieldInfo.name + "\" but pointDimensionalCount is 0");
       }
       in.writeField(fieldInfo, values);
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac11c9e5/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyPointsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyPointsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyPointsFormat.java
index 681edbd..486d81c 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyPointsFormat.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyPointsFormat.java
@@ -155,19 +155,11 @@ class CrankyPointsFormat extends PointsFormat {
         }
 
         @Override
-        public int getNumDataDimensions() throws IOException {
+        public int getNumDimensions() throws IOException {
           if (random.nextInt(100) == 0) {
             throw new IOException("Fake IOException");
           }
-          return delegate.getNumDataDimensions();
-        }
-
-        @Override
-        public int getNumIndexDimensions() throws IOException {
-          if (random.nextInt(100) == 0) {
-            throw new IOException("Fake IOException");
-          }
-          return delegate.getNumIndexDimensions();
+          return delegate.getNumDimensions();
         }
 
         @Override


[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

Posted by nk...@apache.org.
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));
     }