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

[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

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