You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by iv...@apache.org on 2020/09/08 08:24:44 UTC

[lucene-solr] branch branch_8x updated: LUCENE-9292: Refactor BKD point configuration into its own class (#1697) (#1840)

This is an automated email from the ASF dual-hosted git repository.

ivera pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/branch_8x by this push:
     new fcb7e13  LUCENE-9292: Refactor BKD point configuration into its own class (#1697) (#1840)
fcb7e13 is described below

commit fcb7e13f63576fb1941a183710f4d8b44718c237
Author: Ignacio Vera <iv...@apache.org>
AuthorDate: Tue Sep 8 10:24:28 2020 +0200

    LUCENE-9292: Refactor BKD point configuration into its own class (#1697) (#1840)
---
 lucene/CHANGES.txt                                 |   4 +-
 .../codecs/lucene60/Lucene60PointsWriter.java      |  23 +-
 .../codecs/lucene60/TestLucene60PointsFormat.java  |   4 +-
 .../codecs/simpletext/SimpleTextBKDWriter.java     | 386 ++++++-------
 .../codecs/simpletext/SimpleTextPointsWriter.java  |  12 +-
 .../codecs/lucene86/Lucene86PointsWriter.java      |  23 +-
 .../java/org/apache/lucene/index/PointValues.java  |  33 +-
 .../java/org/apache/lucene/util/bkd/BKDConfig.java |  90 ++++
 .../apache/lucene/util/bkd/BKDRadixSelector.java   | 207 ++++---
 .../java/org/apache/lucene/util/bkd/BKDReader.java | 212 ++++----
 .../java/org/apache/lucene/util/bkd/BKDWriter.java | 597 ++++++++++-----------
 .../apache/lucene/util/bkd/HeapPointReader.java    |  18 +-
 .../apache/lucene/util/bkd/HeapPointWriter.java    |  53 +-
 .../lucene/util/bkd/MutablePointsReaderUtils.java  |  47 +-
 .../apache/lucene/util/bkd/OfflinePointReader.java |  36 +-
 .../apache/lucene/util/bkd/OfflinePointWriter.java |  14 +-
 .../codecs/lucene86/TestLucene86PointsFormat.java  |   4 +-
 .../lucene/search/TestLatLonPointQueries.java      |   4 +-
 .../org/apache/lucene/search/TestPointQueries.java |   8 +-
 .../apache/lucene/util/bkd/Test2BBKDPoints.java    |   8 +-
 .../test/org/apache/lucene/util/bkd/TestBKD.java   |  31 +-
 .../lucene/util/bkd/TestBKDRadixSelector.java      | 231 ++++----
 .../apache/lucene/util/bkd/TestBKDRadixSort.java   | 112 ++--
 .../util/bkd/TestMutablePointsReaderUtils.java     | 101 ++--
 .../java/org/apache/lucene/index/RandomCodec.java  |  19 +-
 25 files changed, 1120 insertions(+), 1157 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 2a6a3ae..7b52d37 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -73,7 +73,9 @@ Documentation
 
 Other
 ---------------------
-(No changes)
+
+* LUCENE-9292: Refactor BKD point configuration into its own class. (Ignacio Vera)
+
 
 ======================= Lucene 8.6.2 =======================
 
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java
index 06e9653..078d5b2 100644
--- a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java
@@ -38,6 +38,7 @@ import org.apache.lucene.index.PointValues.Relation;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.bkd.BKDConfig;
 import org.apache.lucene.util.bkd.BKDReader;
 import org.apache.lucene.util.bkd.BKDWriter;
 
@@ -82,7 +83,7 @@ public class Lucene60PointsWriter extends PointsWriter implements Closeable {
 
   /** Uses the defaults values for {@code maxPointsInLeafNode} (1024) and {@code maxMBSortInHeap} (16.0) */
   public Lucene60PointsWriter(SegmentWriteState writeState) throws IOException {
-    this(writeState, BKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE, BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP);
+    this(writeState, BKDConfig.DEFAULT_MAX_POINTS_IN_LEAF_NODE, BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP);
   }
 
   @Override
@@ -90,13 +91,15 @@ public class Lucene60PointsWriter extends PointsWriter implements Closeable {
 
     PointValues values = reader.getValues(fieldInfo.name);
 
+    BKDConfig config = new BKDConfig(fieldInfo.getPointDimensionCount(),
+                                     fieldInfo.getPointIndexDimensionCount(),
+                                     fieldInfo.getPointNumBytes(),
+                                     maxPointsInLeafNode);
+
     try (BKDWriter writer = new BKDWriter(writeState.segmentInfo.maxDoc(),
                                           writeState.directory,
                                           writeState.segmentInfo.name,
-                                          fieldInfo.getPointDimensionCount(),
-                                          fieldInfo.getPointIndexDimensionCount(),
-                                          fieldInfo.getPointNumBytes(),
-                                          maxPointsInLeafNode,
+                                          config,
                                           maxMBSortInHeap,
                                           values.size())) {
 
@@ -173,6 +176,11 @@ public class Lucene60PointsWriter extends PointsWriter implements Closeable {
             }
           }
 
+          BKDConfig config = new BKDConfig(fieldInfo.getPointDimensionCount(),
+                                           fieldInfo.getPointIndexDimensionCount(),
+                                           fieldInfo.getPointNumBytes(),
+                                           maxPointsInLeafNode);
+
           //System.out.println("MERGE: field=" + fieldInfo.name);
           // Optimize the 1D case to use BKDWriter.merge, which does a single merge sort of the
           // already sorted incoming segments, instead of trying to sort all points again as if
@@ -180,10 +188,7 @@ public class Lucene60PointsWriter extends PointsWriter implements Closeable {
           try (BKDWriter writer = new BKDWriter(writeState.segmentInfo.maxDoc(),
                                                 writeState.directory,
                                                 writeState.segmentInfo.name,
-                                                fieldInfo.getPointDimensionCount(),
-                                                fieldInfo.getPointIndexDimensionCount(),
-                                                fieldInfo.getPointNumBytes(),
-                                                maxPointsInLeafNode,
+                                                config,
                                                 maxMBSortInHeap,
                                                 totMaxSize)) {
             List<BKDReader> bkdReaders = new ArrayList<>();
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene60/TestLucene60PointsFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene60/TestLucene60PointsFormat.java
index 35262a0..8b8fec8 100644
--- a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene60/TestLucene60PointsFormat.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene60/TestLucene60PointsFormat.java
@@ -37,7 +37,7 @@ import org.apache.lucene.index.PointValues.Relation;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.FutureArrays;
 import org.apache.lucene.util.TestUtil;
-import org.apache.lucene.util.bkd.BKDWriter;
+import org.apache.lucene.util.bkd.BKDConfig;
 
 /**
  * Tests Lucene60PointsFormat
@@ -48,7 +48,7 @@ public class TestLucene60PointsFormat extends BasePointsFormatTestCase {
   
   public TestLucene60PointsFormat() {
     codec = new Lucene84RWCodec();
-    maxPointsInLeafNode = BKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE;
+    maxPointsInLeafNode = BKDConfig.DEFAULT_MAX_POINTS_IN_LEAF_NODE;
   }
 
   @Override
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 1ab79ba..326dc9b 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
@@ -39,6 +39,7 @@ import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.FutureArrays;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.NumericUtils;
+import org.apache.lucene.util.bkd.BKDConfig;
 import org.apache.lucene.util.bkd.BKDRadixSelector;
 import org.apache.lucene.util.bkd.BKDWriter;
 import org.apache.lucene.util.bkd.HeapPointWriter;
@@ -87,35 +88,12 @@ final class SimpleTextBKDWriter implements Closeable {
   public static final int VERSION_IMPLICIT_SPLIT_DIM_1D = 3;
   public static final int VERSION_CURRENT = VERSION_IMPLICIT_SPLIT_DIM_1D;
 
-  /** How many bytes each docs takes in the fixed-width offline format */
-  private final int bytesPerDoc;
-
-  /** Default maximum number of point in each leaf block */
-  public static final int DEFAULT_MAX_POINTS_IN_LEAF_NODE = 1024;
-
   /** Default maximum heap to use, before spilling to (slower) disk */
   public static final float DEFAULT_MAX_MB_SORT_IN_HEAP = 16.0f;
 
-  /** Maximum number of dimensions (2 * max index dimensions) */
-  public static final int MAX_DIMS = 16;
-
-  /** Maximum number of dimensions */
-  public static final int MAX_INDEX_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;
+  protected final BKDConfig config;
 
-  /** How many bytes each value in each dimension takes. */
-  protected final int bytesPerDim;
-
-  /** numDims * bytesPerDim */
-  protected final int packedBytesLength;
-
-  /** numIndexDims * bytesPerDim */
-  protected final int packedIndexBytesLength;
 
   final BytesRefBuilder scratch = new BytesRefBuilder();
 
@@ -136,7 +114,7 @@ final class SimpleTextBKDWriter implements Closeable {
   private boolean finished;
 
   private IndexOutput tempInput;
-  protected final int maxPointsInLeafNode;
+
   private final int maxPointsSortInHeap;
 
   /** Minimum per-dim values, packed */
@@ -153,63 +131,39 @@ final class SimpleTextBKDWriter implements Closeable {
   private final int maxDoc;
 
 
-  public SimpleTextBKDWriter(int maxDoc, Directory tempDir, String tempFileNamePrefix, int numDataDims, int numIndexDims, int bytesPerDim,
-                              int maxPointsInLeafNode, double maxMBSortInHeap, long totalPointCount) throws IOException {
-    verifyParams(numDataDims, numIndexDims, maxPointsInLeafNode, maxMBSortInHeap, totalPointCount);
+  public SimpleTextBKDWriter(int maxDoc, Directory tempDir, String tempFileNamePrefix, BKDConfig config, double maxMBSortInHeap, long totalPointCount) throws IOException {
+    verifyParams(maxMBSortInHeap, totalPointCount);
+    this.config = config;
     // 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.bytesPerDim = bytesPerDim;
+
     this.totalPointCount = totalPointCount;
     this.maxDoc = maxDoc;
     docsSeen = new FixedBitSet(maxDoc);
-    packedBytesLength = numDataDims * bytesPerDim;
-    packedIndexBytesLength = numIndexDims * bytesPerDim;
 
-    scratchDiff = new byte[bytesPerDim];
-    scratch1 = new byte[packedBytesLength];
-    scratch2 = new byte[packedBytesLength];
-    commonPrefixLengths = new int[numDataDims];
 
-    minPackedValue = new byte[packedIndexBytesLength];
-    maxPackedValue = new byte[packedIndexBytesLength];
+    scratchDiff = new byte[config.bytesPerDim];
+    scratch1 = new byte[config.packedBytesLength];
+    scratch2 = new byte[config.packedBytesLength];
+    commonPrefixLengths = new int[config.numDims];
 
-    // dimensional values (numDims * bytesPerDim) +  docID (int)
-    bytesPerDoc = packedBytesLength + Integer.BYTES;
+    minPackedValue = new byte[config.packedIndexBytesLength];
+    maxPackedValue = new byte[config.packedIndexBytesLength];
 
     // Maximum number of points we hold in memory at any time
-    maxPointsSortInHeap = (int) ((maxMBSortInHeap * 1024 * 1024) / (bytesPerDoc * numDataDims));
+    maxPointsSortInHeap = (int) ((maxMBSortInHeap * 1024 * 1024) / (config.bytesPerDoc * config.numDims));
 
     // Finally, we must be able to hold at least the leaf node in heap during build:
-    if (maxPointsSortInHeap < maxPointsInLeafNode) {
-      throw new IllegalArgumentException("maxMBSortInHeap=" + maxMBSortInHeap + " only allows for maxPointsSortInHeap=" + maxPointsSortInHeap + ", but this is less than maxPointsInLeafNode=" + maxPointsInLeafNode + "; either increase maxMBSortInHeap or decrease maxPointsInLeafNode");
+    if (maxPointsSortInHeap < config.maxPointsInLeafNode) {
+      throw new IllegalArgumentException("maxMBSortInHeap=" + maxMBSortInHeap + " only allows for maxPointsSortInHeap=" + maxPointsSortInHeap + ", but this is less than config.maxPointsInLeafNode=" + config.maxPointsInLeafNode + "; either increase maxMBSortInHeap or decrease config.maxPointsInLeafNode");
     }
 
     this.maxMBSortInHeap = maxMBSortInHeap;
   }
 
-  public static void verifyParams(int numDims, int numIndexDims, 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 (numDims < 1 || numDims > MAX_DIMS) {
-      throw new IllegalArgumentException("numDims must be 1 .. " + MAX_DIMS + " (got: " + numDims + ")");
-    }
-    if (numIndexDims < 1 || numIndexDims > MAX_INDEX_DIMS) {
-      throw new IllegalArgumentException("numIndexDims must be 1 .. " + MAX_INDEX_DIMS + " (got: " + numIndexDims + ")");
-    }
-    if (numIndexDims > numDims) {
-      throw new IllegalArgumentException("numIndexDims cannot exceed numDims (" + numDims + ") (got: " + numIndexDims + ")");
-    }
-    if (maxPointsInLeafNode <= 0) {
-      throw new IllegalArgumentException("maxPointsInLeafNode must be > 0; got " + maxPointsInLeafNode);
-    }
-    if (maxPointsInLeafNode > ArrayUtil.MAX_ARRAY_LENGTH) {
-      throw new IllegalArgumentException("maxPointsInLeafNode must be <= ArrayUtil.MAX_ARRAY_LENGTH (= " + ArrayUtil.MAX_ARRAY_LENGTH + "); got " + maxPointsInLeafNode);
-    }
+  public static void verifyParams(double maxMBSortInHeap, long totalPointCount) {
     if (maxMBSortInHeap < 0.0) {
       throw new IllegalArgumentException("maxMBSortInHeap must be >= 0.0 (got: " + maxMBSortInHeap + ")");
     }
@@ -219,8 +173,8 @@ final class SimpleTextBKDWriter implements Closeable {
   }
 
   public void add(byte[] packedValue, int docID) throws IOException {
-    if (packedValue.length != packedBytesLength) {
-      throw new IllegalArgumentException("packedValue should be length=" + packedBytesLength + " (got: " + packedValue.length + ")");
+    if (packedValue.length != config.packedBytesLength) {
+      throw new IllegalArgumentException("packedValue should be length=" + config.packedBytesLength + " (got: " + packedValue.length + ")");
     }
     if (pointCount >= totalPointCount) {
       throw new IllegalStateException("totalPointCount=" + totalPointCount + " was passed when we were created, but we just hit " + (pointCount + 1) + " values");
@@ -229,21 +183,21 @@ final class SimpleTextBKDWriter implements Closeable {
       assert pointWriter == null : "Point writer is already initialized";
       //total point count is an estimation but the final point count must be equal or lower to that number.
       if (totalPointCount > maxPointsSortInHeap) {
-        pointWriter = new OfflinePointWriter(tempDir, tempFileNamePrefix, packedBytesLength, "spill", 0);
+        pointWriter = new OfflinePointWriter(config, tempDir, tempFileNamePrefix, "spill", 0);
         tempInput = ((OfflinePointWriter)pointWriter).out;
       } else {
-        pointWriter = new HeapPointWriter(Math.toIntExact(totalPointCount), packedBytesLength);
+        pointWriter = new HeapPointWriter(config, Math.toIntExact(totalPointCount));
       }
-      System.arraycopy(packedValue, 0, minPackedValue, 0, packedIndexBytesLength);
-      System.arraycopy(packedValue, 0, maxPackedValue, 0, packedIndexBytesLength);
+      System.arraycopy(packedValue, 0, minPackedValue, 0, config.packedIndexBytesLength);
+      System.arraycopy(packedValue, 0, maxPackedValue, 0, config.packedIndexBytesLength);
     } else {
-      for(int dim=0;dim<numIndexDims;dim++) {
-        int offset = dim*bytesPerDim;
-        if (FutureArrays.compareUnsigned(packedValue, offset, offset + bytesPerDim, minPackedValue, offset, offset + bytesPerDim) < 0) {
-          System.arraycopy(packedValue, offset, minPackedValue, offset, bytesPerDim);
+      for(int dim=0;dim<config.numIndexDims;dim++) {
+        int offset = dim*config.bytesPerDim;
+        if (FutureArrays.compareUnsigned(packedValue, offset, offset + config.bytesPerDim, minPackedValue, offset, offset + config.bytesPerDim) < 0) {
+          System.arraycopy(packedValue, offset, minPackedValue, offset, config.bytesPerDim);
         }
-        if (FutureArrays.compareUnsigned(packedValue, offset, offset + bytesPerDim, maxPackedValue, offset, offset + bytesPerDim) > 0) {
-          System.arraycopy(packedValue, offset, maxPackedValue, offset, bytesPerDim);
+        if (FutureArrays.compareUnsigned(packedValue, offset, offset + config.bytesPerDim, maxPackedValue, offset, offset + config.bytesPerDim) > 0) {
+          System.arraycopy(packedValue, offset, maxPackedValue, offset, config.bytesPerDim);
         }
       }
     }
@@ -258,12 +212,12 @@ final class SimpleTextBKDWriter implements Closeable {
   }
 
   /** Write a field from a {@link MutablePointValues}. This way of writing
-   *  points is faster than regular writes with {@link BKDWriter#add} since
+   *  points is faster than regular writes with {@link SimpleTextBKDWriter#add} since
    *  there is opportunity for reordering points before writing them to
    *  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 (config.numIndexDims == 1) {
       return writeField1Dim(out, fieldName, reader);
     } else {
       return writeFieldNDims(out, fieldName, reader);
@@ -289,7 +243,7 @@ final class SimpleTextBKDWriter implements Closeable {
     long countPerLeaf = pointCount = values.size();
     long innerNodeCount = 1;
 
-    while (countPerLeaf > maxPointsInLeafNode) {
+    while (countPerLeaf > config.maxPointsInLeafNode) {
       countPerLeaf = (countPerLeaf+1)/2;
       innerNodeCount *= 2;
     }
@@ -298,7 +252,7 @@ final class SimpleTextBKDWriter implements Closeable {
 
     checkMaxLeafNodeCount(numLeaves);
 
-    final byte[] splitPackedValues = new byte[numLeaves * (bytesPerDim + 1)];
+    final byte[] splitPackedValues = new byte[numLeaves * (config.bytesPerDim + 1)];
     final long[] leafBlockFPs = new long[numLeaves];
 
     // compute the min/max for this slice
@@ -306,13 +260,13 @@ 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++) {
-        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);
+      for(int dim=0;dim<config.numIndexDims;dim++) {
+        int offset = dim*config.bytesPerDim;
+        if (FutureArrays.compareUnsigned(scratchBytesRef1.bytes, scratchBytesRef1.offset + offset, scratchBytesRef1.offset + offset + config.bytesPerDim, minPackedValue, offset, offset + config.bytesPerDim) < 0) {
+          System.arraycopy(scratchBytesRef1.bytes, scratchBytesRef1.offset + offset, minPackedValue, offset, config.bytesPerDim);
         }
-        if (FutureArrays.compareUnsigned(scratchBytesRef1.bytes, scratchBytesRef1.offset + offset, scratchBytesRef1.offset + offset + bytesPerDim, maxPackedValue, offset, offset + bytesPerDim) > 0) {
-          System.arraycopy(scratchBytesRef1.bytes, scratchBytesRef1.offset + offset, maxPackedValue, offset, bytesPerDim);
+        if (FutureArrays.compareUnsigned(scratchBytesRef1.bytes, scratchBytesRef1.offset + offset, scratchBytesRef1.offset + offset + config.bytesPerDim, maxPackedValue, offset, offset + config.bytesPerDim) > 0) {
+          System.arraycopy(scratchBytesRef1.bytes, scratchBytesRef1.offset + offset, maxPackedValue, offset, config.bytesPerDim);
         }
       }
 
@@ -320,8 +274,8 @@ final class SimpleTextBKDWriter implements Closeable {
     }
 
     build(1, numLeaves, values, 0, Math.toIntExact(pointCount), out,
-        minPackedValue, maxPackedValue, splitPackedValues, leafBlockFPs,
-        new int[maxPointsInLeafNode]);
+            minPackedValue, maxPackedValue, splitPackedValues, leafBlockFPs,
+            new int[config.maxPointsInLeafNode]);
 
     long indexFP = out.getFilePointer();
     writeIndex(out, leafBlockFPs, splitPackedValues);
@@ -332,7 +286,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(config, maxDoc, reader, 0, Math.toIntExact(reader.size()));
 
     final OneDimensionBKDWriter oneDimWriter = new OneDimensionBKDWriter(out);
 
@@ -362,14 +316,14 @@ final class SimpleTextBKDWriter implements Closeable {
     final IndexOutput out;
     final List<Long> leafBlockFPs = new ArrayList<>();
     final List<byte[]> leafBlockStartValues = new ArrayList<>();
-    final byte[] leafValues = new byte[maxPointsInLeafNode * packedBytesLength];
-    final int[] leafDocs = new int[maxPointsInLeafNode];
+    final byte[] leafValues = new byte[config.maxPointsInLeafNode * config.packedBytesLength];
+    final int[] leafDocs = new int[config.maxPointsInLeafNode];
     long valueCount;
     int leafCount;
 
     OneDimensionBKDWriter(IndexOutput out) {
-      if (numIndexDims != 1) {
-        throw new UnsupportedOperationException("numIndexDims must be 1 but got " + numIndexDims);
+      if (config.numIndexDims != 1) {
+        throw new UnsupportedOperationException("config.numIndexDims must be 1 but got " + config.numIndexDims);
       }
       if (pointCount != 0) {
         throw new IllegalStateException("cannot mix add and merge");
@@ -385,7 +339,7 @@ final class SimpleTextBKDWriter implements Closeable {
 
       this.out = out;
 
-      lastPackedValue = new byte[packedBytesLength];
+      lastPackedValue = new byte[config.packedBytesLength];
     }
 
     // for asserts
@@ -394,9 +348,9 @@ final class SimpleTextBKDWriter implements Closeable {
 
     void add(byte[] packedValue, int docID) throws IOException {
       assert valueInOrder(valueCount + leafCount,
-          0, lastPackedValue, packedValue, 0, docID, lastDocID);
+              0, lastPackedValue, packedValue, 0, docID, lastDocID);
 
-      System.arraycopy(packedValue, 0, leafValues, leafCount * packedBytesLength, packedBytesLength);
+      System.arraycopy(packedValue, 0, leafValues, leafCount * config.packedBytesLength, config.packedBytesLength);
       leafDocs[leafCount] = docID;
       docsSeen.set(docID);
       leafCount++;
@@ -405,7 +359,7 @@ final class SimpleTextBKDWriter implements Closeable {
         throw new IllegalStateException("totalPointCount=" + totalPointCount + " was passed when we were created, but we just hit " + pointCount + " values");
       }
 
-      if (leafCount == maxPointsInLeafNode) {
+      if (leafCount == config.maxPointsInLeafNode) {
         // We write a block once we hit exactly the max count ... this is different from
         // when we flush a new segment, where we write between max/2 and max per leaf block,
         // so merged segments will behave differently from newly flushed segments:
@@ -434,7 +388,7 @@ final class SimpleTextBKDWriter implements Closeable {
 
       //System.out.println("BKDW: now rotate numInnerNodes=" + numInnerNodes + " leafBlockStarts=" + leafBlockStartValues.size());
 
-      byte[] index = new byte[(1+numInnerNodes) * (1+bytesPerDim)];
+      byte[] index = new byte[(1+numInnerNodes) * (1+config.bytesPerDim)];
       rotateToTree(1, 0, numInnerNodes, index, leafBlockStartValues);
       long[] arr = new long[leafBlockFPs.size()];
       for(int i=0;i<leafBlockFPs.size();i++) {
@@ -447,24 +401,24 @@ 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, config.packedIndexBytesLength);
       }
-      System.arraycopy(leafValues, (leafCount - 1) * packedBytesLength, maxPackedValue, 0, packedIndexBytesLength);
+      System.arraycopy(leafValues, (leafCount - 1) * config.packedBytesLength, maxPackedValue, 0, config.packedIndexBytesLength);
 
       valueCount += leafCount;
 
       if (leafBlockFPs.size() > 0) {
         // Save the first (minimum) value in each leaf block except the first, to build the split value index in the end:
-        leafBlockStartValues.add(ArrayUtil.copyOfSubArray(leafValues, 0, packedBytesLength));
+        leafBlockStartValues.add(ArrayUtil.copyOfSubArray(leafValues, 0, config.packedBytesLength));
       }
       leafBlockFPs.add(out.getFilePointer());
       checkMaxLeafNodeCount(leafBlockFPs.size());
 
-      Arrays.fill(commonPrefixLengths, bytesPerDim);
+      Arrays.fill(commonPrefixLengths, config.bytesPerDim);
       // Find per-dim common prefix:
-      for(int dim=0;dim<numDataDims;dim++) {
-        int offset1 = dim * bytesPerDim;
-        int offset2 = (leafCount - 1) * packedBytesLength + offset1;
+      for(int dim=0;dim<config.numDims;dim++) {
+        int offset1 = dim * config.bytesPerDim;
+        int offset2 = (leafCount - 1) * config.packedBytesLength + offset1;
         for(int j=0;j<commonPrefixLengths[dim];j++) {
           if (leafValues[offset1+j] != leafValues[offset2+j]) {
             commonPrefixLengths[dim] = j;
@@ -479,19 +433,19 @@ final class SimpleTextBKDWriter implements Closeable {
         final BytesRef scratch = new BytesRef();
 
         {
-          scratch.length = packedBytesLength;
+          scratch.length = config.packedBytesLength;
           scratch.bytes = leafValues;
         }
 
         @Override
         public BytesRef apply(int i) {
-          scratch.offset = packedBytesLength * i;
+          scratch.offset = config.packedBytesLength * i;
           return scratch;
         }
       };
-      assert valuesInOrderAndBounds(leafCount, 0, ArrayUtil.copyOfSubArray(leafValues, 0, packedBytesLength),
-          ArrayUtil.copyOfSubArray(leafValues, (leafCount - 1) * packedBytesLength, leafCount * packedBytesLength),
-          packedValues, leafDocs, 0);
+      assert valuesInOrderAndBounds(leafCount, 0, ArrayUtil.copyOfSubArray(leafValues, 0, config.packedBytesLength),
+              ArrayUtil.copyOfSubArray(leafValues, (leafCount - 1) * config.packedBytesLength, leafCount * config.packedBytesLength),
+              packedValues, leafDocs, 0);
       writeLeafBlockPackedValues(out, commonPrefixLengths, leafCount, 0, packedValues);
     }
 
@@ -499,12 +453,12 @@ final class SimpleTextBKDWriter implements Closeable {
 
   // TODO: there must be a simpler way?
   private void rotateToTree(int nodeID, int offset, int count, byte[] index, List<byte[]> leafBlockStartValues) {
-    //System.out.println("ROTATE: nodeID=" + nodeID + " offset=" + offset + " count=" + count + " bpd=" + bytesPerDim + " index.length=" + index.length);
+    //System.out.println("ROTATE: nodeID=" + nodeID + " offset=" + offset + " count=" + count + " bpd=" + config.bytesPerDim + " index.length=" + index.length);
     if (count == 1) {
       // Leaf index node
       //System.out.println("  leaf index node");
       //System.out.println("  index[" + nodeID + "] = blockStartValues[" + offset + "]");
-      System.arraycopy(leafBlockStartValues.get(offset), 0, index, nodeID*(1+bytesPerDim)+1, bytesPerDim);
+      System.arraycopy(leafBlockStartValues.get(offset), 0, index, nodeID*(1+config.bytesPerDim)+1, config.bytesPerDim);
     } else if (count > 1) {
       // Internal index node: binary partition of count
       int countAtLevel = 1;
@@ -525,7 +479,7 @@ final class SimpleTextBKDWriter implements Closeable {
           System.out.println("  rootOffset=" + rootOffset);
           */
 
-          System.arraycopy(leafBlockStartValues.get(rootOffset), 0, index, nodeID*(1+bytesPerDim)+1, bytesPerDim);
+          System.arraycopy(leafBlockStartValues.get(rootOffset), 0, index, nodeID*(1+config.bytesPerDim)+1, config.bytesPerDim);
           //System.out.println("  index[" + nodeID + "] = blockStartValues[" + rootOffset + "]");
 
           // TODO: we could optimize/specialize, when we know it's simply fully balanced binary tree
@@ -547,8 +501,8 @@ final class SimpleTextBKDWriter implements Closeable {
   }
 
   private void checkMaxLeafNodeCount(int numLeaves) {
-    if ((1+bytesPerDim) * (long) numLeaves > ArrayUtil.MAX_ARRAY_LENGTH) {
-      throw new IllegalStateException("too many nodes; increase maxPointsInLeafNode (currently " + maxPointsInLeafNode + ") and reindex");
+    if ((1+config.bytesPerDim) * (long) numLeaves > ArrayUtil.MAX_ARRAY_LENGTH) {
+      throw new IllegalStateException("too many nodes; increase config.maxPointsInLeafNode (currently " + config.maxPointsInLeafNode + ") and reindex");
     }
   }
 
@@ -581,7 +535,7 @@ final class SimpleTextBKDWriter implements Closeable {
     long countPerLeaf = pointCount;
     long innerNodeCount = 1;
 
-    while (countPerLeaf > maxPointsInLeafNode) {
+    while (countPerLeaf > config.maxPointsInLeafNode) {
       countPerLeaf = (countPerLeaf+1)/2;
       innerNodeCount *= 2;
     }
@@ -594,24 +548,24 @@ final class SimpleTextBKDWriter implements Closeable {
     // step of the recursion to recompute the split dim:
 
     // Indexed by nodeID, but first (root) nodeID is 1.  We do 1+ because the lead byte at each recursion says which dim we split on.
-    byte[] splitPackedValues = new byte[Math.toIntExact(numLeaves*(1+bytesPerDim))];
+    byte[] splitPackedValues = new byte[Math.toIntExact(numLeaves*(1+config.bytesPerDim))];
 
     // +1 because leaf count is power of 2 (e.g. 8), and innerNodeCount is power of 2 minus 1 (e.g. 7)
     long[] leafBlockFPs = new long[numLeaves];
 
     // Make sure the math above "worked":
-    assert pointCount / numLeaves <= maxPointsInLeafNode: "pointCount=" + pointCount + " numLeaves=" + numLeaves + " maxPointsInLeafNode=" + maxPointsInLeafNode;
+    assert pointCount / numLeaves <= config.maxPointsInLeafNode: "pointCount=" + pointCount + " numLeaves=" + numLeaves + " config.maxPointsInLeafNode=" + config.maxPointsInLeafNode;
 
     //We re-use the selector so we do not need to create an object every time.
-    BKDRadixSelector radixSelector = new BKDRadixSelector(numDataDims, numIndexDims, bytesPerDim, maxPointsSortInHeap, tempDir, tempFileNamePrefix);
+    BKDRadixSelector radixSelector = new BKDRadixSelector(config, maxPointsSortInHeap, tempDir, tempFileNamePrefix);
 
     boolean success = false;
     try {
 
 
       build(1, numLeaves, points, out,
-          radixSelector, minPackedValue, maxPackedValue,
-            splitPackedValues, leafBlockFPs, new int[maxPointsInLeafNode]);
+              radixSelector, minPackedValue, maxPackedValue,
+              splitPackedValues, leafBlockFPs, new int[config.maxPointsInLeafNode]);
 
 
       // If no exception, we should have cleaned everything up:
@@ -637,19 +591,19 @@ 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);
+    writeInt(out, config.numDims);
     newline(out);
 
     write(out, NUM_INDEX_DIMS);
-    writeInt(out, numIndexDims);
+    writeInt(out, config.numIndexDims);
     newline(out);
 
     write(out, BYTES_PER_DIM);
-    writeInt(out, bytesPerDim);
+    writeInt(out, config.bytesPerDim);
     newline(out);
 
     write(out, MAX_LEAF_POINTS);
-    writeInt(out, maxPointsInLeafNode);
+    writeInt(out, config.maxPointsInLeafNode);
     newline(out);
 
     write(out, INDEX_COUNT);
@@ -680,8 +634,8 @@ final class SimpleTextBKDWriter implements Closeable {
       newline(out);
     }
 
-    assert (splitPackedValues.length % (1 + bytesPerDim)) == 0;
-    int count = splitPackedValues.length / (1 + bytesPerDim);
+    assert (splitPackedValues.length % (1 + config.bytesPerDim)) == 0;
+    int count = splitPackedValues.length / (1 + config.bytesPerDim);
     assert count == leafBlockFPs.length;
 
     write(out, SPLIT_COUNT);
@@ -690,10 +644,10 @@ final class SimpleTextBKDWriter implements Closeable {
 
     for(int i=0;i<count;i++) {
       write(out, SPLIT_DIM);
-      writeInt(out, splitPackedValues[i * (1 + bytesPerDim)] & 0xff);
+      writeInt(out, splitPackedValues[i * (1 + config.bytesPerDim)] & 0xff);
       newline(out);
       write(out, SPLIT_VALUE);
-      br = new BytesRef(splitPackedValues, 1+(i * (1+bytesPerDim)), bytesPerDim);
+      br = new BytesRef(splitPackedValues, 1+(i * (1+config.bytesPerDim)), config.bytesPerDim);
       write(out, br.toString());
       newline(out);
     }
@@ -723,11 +677,11 @@ final class SimpleTextBKDWriter implements Closeable {
   private void writeLeafBlockPackedValuesRange(IndexOutput out, int[] commonPrefixLengths, int start, int end, IntFunction<BytesRef> packedValues) throws IOException {
     for (int i = start; i < end; ++i) {
       BytesRef ref = packedValues.apply(i);
-      assert ref.length == packedBytesLength;
+      assert ref.length == config.packedBytesLength;
 
-      for(int dim=0;dim<numDataDims;dim++) {
+      for(int dim=0;dim<config.numDims;dim++) {
         int prefix = commonPrefixLengths[dim];
-        out.writeBytes(ref.bytes, ref.offset + dim*bytesPerDim + prefix, bytesPerDim-prefix);
+        out.writeBytes(ref.bytes, ref.offset + dim*config.bytesPerDim + prefix, config.bytesPerDim-prefix);
       }
     }
   }
@@ -781,12 +735,12 @@ 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++) {
-      int offset = bytesPerDim*dim;
-      if (FutureArrays.compareUnsigned(packedValue.bytes, packedValue.offset + offset, packedValue.offset + offset + bytesPerDim, minPackedValue, offset, offset + bytesPerDim) < 0) {
+    for(int dim=0;dim<config.numIndexDims;dim++) {
+      int offset = config.bytesPerDim*dim;
+      if (FutureArrays.compareUnsigned(packedValue.bytes, packedValue.offset + offset, packedValue.offset + offset + config.bytesPerDim, minPackedValue, offset, offset + config.bytesPerDim) < 0) {
         return false;
       }
-      if (FutureArrays.compareUnsigned(packedValue.bytes, packedValue.offset + offset, packedValue.offset + offset + bytesPerDim, maxPackedValue, offset, offset + bytesPerDim) > 0) {
+      if (FutureArrays.compareUnsigned(packedValue.bytes, packedValue.offset + offset, packedValue.offset + offset + config.bytesPerDim, maxPackedValue, offset, offset + config.bytesPerDim) > 0) {
         return false;
       }
     }
@@ -797,10 +751,10 @@ 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++) {
-      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);
+    for(int dim=0;dim<config.numIndexDims;dim++) {
+      NumericUtils.subtract(config.bytesPerDim, dim, maxPackedValue, minPackedValue, scratchDiff);
+      if (splitDim == -1 || FutureArrays.compareUnsigned(scratchDiff, 0, config.bytesPerDim, scratch1, 0, config.bytesPerDim) > 0) {
+        System.arraycopy(scratchDiff, 0, scratch1, 0, config.bytesPerDim);
         splitDim = dim;
       }
     }
@@ -813,7 +767,7 @@ final class SimpleTextBKDWriter implements Closeable {
   private HeapPointWriter switchToHeap(PointWriter source) throws IOException {
     int count = Math.toIntExact(source.count());
     try (PointReader reader = source.getReader(0, count);
-        HeapPointWriter writer = new HeapPointWriter(count, packedBytesLength)) {
+         HeapPointWriter writer = new HeapPointWriter(config, count)) {
       for(int i=0;i<count;i++) {
         boolean hasNext = reader.next();
         assert hasNext;
@@ -827,25 +781,25 @@ final class SimpleTextBKDWriter implements Closeable {
 
   /* Recursively reorders the provided reader and writes the bkd-tree on the fly. */
   private void build(int nodeID, int leafNodeOffset,
-      MutablePointValues reader, int from, int to,
-      IndexOutput out,
-      byte[] minPackedValue, byte[] maxPackedValue,
-      byte[] splitPackedValues,
-      long[] leafBlockFPs,
-      int[] spareDocIds) throws IOException {
+                     MutablePointValues reader, int from, int to,
+                     IndexOutput out,
+                     byte[] minPackedValue, byte[] maxPackedValue,
+                     byte[] splitPackedValues,
+                     long[] leafBlockFPs,
+                     int[] spareDocIds) throws IOException {
 
     if (nodeID >= leafNodeOffset) {
       // leaf node
       final int count = to - from;
-      assert count <= maxPointsInLeafNode;
+      assert count <= config.maxPointsInLeafNode;
 
       // Compute common prefixes
-      Arrays.fill(commonPrefixLengths, bytesPerDim);
+      Arrays.fill(commonPrefixLengths, config.bytesPerDim);
       reader.getValue(from, scratchBytesRef1);
       for (int i = from + 1; i < to; ++i) {
         reader.getValue(i, scratchBytesRef2);
-        for (int dim=0;dim<numDataDims;dim++) {
-          final int offset = dim * bytesPerDim;
+        for (int dim=0;dim<config.numDims;dim++) {
+          final int offset = dim * config.bytesPerDim;
           for(int j=0;j<commonPrefixLengths[dim];j++) {
             if (scratchBytesRef1.bytes[scratchBytesRef1.offset+offset+j] != scratchBytesRef2.bytes[scratchBytesRef2.offset+offset+j]) {
               commonPrefixLengths[dim] = j;
@@ -856,23 +810,23 @@ 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) {
-        if (commonPrefixLengths[dim] < bytesPerDim) {
+      FixedBitSet[] usedBytes = new FixedBitSet[config.numDims];
+      for (int dim = 0; dim < config.numDims; ++dim) {
+        if (commonPrefixLengths[dim] < config.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<config.numDims;dim++) {
           if (usedBytes[dim] != null) {
-            byte b = reader.getByteAt(i, dim * bytesPerDim + commonPrefixLengths[dim]);
+            byte b = reader.getByteAt(i, dim * config.bytesPerDim + commonPrefixLengths[dim]);
             usedBytes[dim].set(Byte.toUnsignedInt(b));
           }
         }
       }
       int sortedDim = 0;
       int sortedDimCardinality = Integer.MAX_VALUE;
-      for (int dim = 0; dim < numDataDims; ++dim) {
+      for (int dim = 0; dim < config.numDims; ++dim) {
         if (usedBytes[dim] != null) {
           final int cardinality = usedBytes[dim].cardinality();
           if (cardinality < sortedDimCardinality) {
@@ -883,8 +837,8 @@ final class SimpleTextBKDWriter implements Closeable {
       }
 
       // sort by sortedDim
-      MutablePointsReaderUtils.sortByDim(numDataDims, numIndexDims, sortedDim, bytesPerDim, commonPrefixLengths,
-                                         reader, from, to, scratchBytesRef1, scratchBytesRef2);
+      MutablePointsReaderUtils.sortByDim(config, sortedDim, commonPrefixLengths,
+              reader, from, to, scratchBytesRef1, scratchBytesRef2);
 
       // Save the block file pointer:
       leafBlockFPs[nodeID - leafNodeOffset] = out.getFilePointer();
@@ -898,7 +852,7 @@ final class SimpleTextBKDWriter implements Closeable {
 
       // Write the common prefixes:
       reader.getValue(from, scratchBytesRef1);
-      System.arraycopy(scratchBytesRef1.bytes, scratchBytesRef1.offset, scratch1, 0, packedBytesLength);
+      System.arraycopy(scratchBytesRef1.bytes, scratchBytesRef1.offset, scratch1, 0, config.packedBytesLength);
 
       // Write the full values:
       IntFunction<BytesRef> packedValues = new IntFunction<BytesRef>() {
@@ -909,7 +863,7 @@ final class SimpleTextBKDWriter implements Closeable {
         }
       };
       assert valuesInOrderAndBounds(count, sortedDim, minPackedValue, maxPackedValue, packedValues,
-          docIDs, 0);
+              docIDs, 0);
       writeLeafBlockPackedValues(out, commonPrefixLengths, count, sortedDim, packedValues);
 
     } else {
@@ -919,34 +873,34 @@ final class SimpleTextBKDWriter implements Closeable {
       final int splitDim = split(minPackedValue, maxPackedValue);
       final int mid = (from + to + 1) >>> 1;
 
-      int commonPrefixLen = bytesPerDim;
-      for (int i = 0; i < bytesPerDim; ++i) {
-        if (minPackedValue[splitDim * bytesPerDim + i] != maxPackedValue[splitDim * bytesPerDim + i]) {
+      int commonPrefixLen = config.bytesPerDim;
+      for (int i = 0; i < config.bytesPerDim; ++i) {
+        if (minPackedValue[splitDim * config.bytesPerDim + i] != maxPackedValue[splitDim * config.bytesPerDim + i]) {
           commonPrefixLen = i;
           break;
         }
       }
-      MutablePointsReaderUtils.partition(numDataDims, numIndexDims, maxDoc, splitDim, bytesPerDim, commonPrefixLen,
-          reader, from, to, mid, scratchBytesRef1, scratchBytesRef2);
+      MutablePointsReaderUtils.partition(config, maxDoc, splitDim, commonPrefixLen,
+              reader, from, to, mid, scratchBytesRef1, scratchBytesRef2);
 
       // set the split value
-      final int address = nodeID * (1+bytesPerDim);
+      final int address = nodeID * (1+config.bytesPerDim);
       splitPackedValues[address] = (byte) splitDim;
       reader.getValue(mid, scratchBytesRef1);
-      System.arraycopy(scratchBytesRef1.bytes, scratchBytesRef1.offset + splitDim * bytesPerDim, splitPackedValues, address + 1, bytesPerDim);
+      System.arraycopy(scratchBytesRef1.bytes, scratchBytesRef1.offset + splitDim * config.bytesPerDim, splitPackedValues, address + 1, config.bytesPerDim);
 
-      byte[] minSplitPackedValue = ArrayUtil.copyOfSubArray(minPackedValue, 0, packedIndexBytesLength);
-      byte[] maxSplitPackedValue = ArrayUtil.copyOfSubArray(maxPackedValue, 0, packedIndexBytesLength);
-      System.arraycopy(scratchBytesRef1.bytes, scratchBytesRef1.offset + splitDim * bytesPerDim,
-          minSplitPackedValue, splitDim * bytesPerDim, bytesPerDim);
-      System.arraycopy(scratchBytesRef1.bytes, scratchBytesRef1.offset + splitDim * bytesPerDim,
-          maxSplitPackedValue, splitDim * bytesPerDim, bytesPerDim);
+      byte[] minSplitPackedValue = ArrayUtil.copyOfSubArray(minPackedValue, 0, config.packedIndexBytesLength);
+      byte[] maxSplitPackedValue = ArrayUtil.copyOfSubArray(maxPackedValue, 0, config.packedIndexBytesLength);
+      System.arraycopy(scratchBytesRef1.bytes, scratchBytesRef1.offset + splitDim * config.bytesPerDim,
+              minSplitPackedValue, splitDim * config.bytesPerDim, config.bytesPerDim);
+      System.arraycopy(scratchBytesRef1.bytes, scratchBytesRef1.offset + splitDim * config.bytesPerDim,
+              maxSplitPackedValue, splitDim * config.bytesPerDim, config.bytesPerDim);
 
       // recurse
       build(nodeID * 2, leafNodeOffset, reader, from, mid, out,
-          minPackedValue, maxSplitPackedValue, splitPackedValues, leafBlockFPs, spareDocIds);
+              minPackedValue, maxSplitPackedValue, splitPackedValues, leafBlockFPs, spareDocIds);
       build(nodeID * 2 + 1, leafNodeOffset, reader, mid, to, out,
-          minSplitPackedValue, maxPackedValue, splitPackedValues, leafBlockFPs, spareDocIds);
+              minSplitPackedValue, maxPackedValue, splitPackedValues, leafBlockFPs, spareDocIds);
     }
   }
 
@@ -981,17 +935,17 @@ final class SimpleTextBKDWriter implements Closeable {
 
       int sortedDim = 0;
       int sortedDimCardinality = Integer.MAX_VALUE;
-      FixedBitSet[] usedBytes = new FixedBitSet[numDataDims];
-      for (int dim = 0; dim < numDataDims; ++dim) {
-        if (commonPrefixLengths[dim] < bytesPerDim) {
+      FixedBitSet[] usedBytes = new FixedBitSet[config.numDims];
+      for (int dim = 0; dim < config.numDims; ++dim) {
+        if (commonPrefixLengths[dim] < config.bytesPerDim) {
           usedBytes[dim] = new FixedBitSet(256);
         }
       }
       //Find the dimension to compress
-      for (int dim = 0; dim < numDataDims; dim++) {
+      for (int dim = 0; dim < config.numDims; dim++) {
         int prefix = commonPrefixLengths[dim];
-        if (prefix < bytesPerDim) {
-          int offset = dim * bytesPerDim;
+        if (prefix < config.bytesPerDim) {
+          int offset = dim * config.bytesPerDim;
           for (int i = 0; i < heapSource.count(); ++i) {
             PointValue value = heapSource.getPackedValueSlice(i);
             BytesRef packedValue = value.packedValue();
@@ -1032,7 +986,7 @@ final class SimpleTextBKDWriter implements Closeable {
         final BytesRef scratch = new BytesRef();
 
         {
-          scratch.length = packedBytesLength;
+          scratch.length = config.packedBytesLength;
         }
 
         @Override
@@ -1042,14 +996,14 @@ final class SimpleTextBKDWriter implements Closeable {
         }
       };
       assert valuesInOrderAndBounds(count, sortedDim, minPackedValue, maxPackedValue, packedValues,
-          docIDs, 0);
+              docIDs, 0);
       writeLeafBlockPackedValues(out, commonPrefixLengths, count, sortedDim, packedValues);
 
     } else {
       // Inner node: partition/recurse
 
       int splitDim;
-      if (numIndexDims > 1) {
+      if (config.numIndexDims > 1) {
         splitDim = split(minPackedValue, maxPackedValue);
       } else {
         splitDim = 0;
@@ -1061,54 +1015,54 @@ final class SimpleTextBKDWriter implements Closeable {
       long rightCount = points.count / 2;
       long leftCount = points.count - rightCount;
 
-      int commonPrefixLen = FutureArrays.mismatch(minPackedValue, splitDim * bytesPerDim,
-          splitDim * bytesPerDim + bytesPerDim, maxPackedValue, splitDim * bytesPerDim,
-          splitDim * bytesPerDim + bytesPerDim);
+      int commonPrefixLen = FutureArrays.mismatch(minPackedValue, splitDim * config.bytesPerDim,
+              splitDim * config.bytesPerDim + config.bytesPerDim, maxPackedValue, splitDim * config.bytesPerDim,
+              splitDim * config.bytesPerDim + config.bytesPerDim);
       if (commonPrefixLen == -1) {
-        commonPrefixLen = bytesPerDim;
+        commonPrefixLen = config.bytesPerDim;
       }
 
       BKDRadixSelector.PathSlice[] pathSlices = new BKDRadixSelector.PathSlice[2];
 
       byte[] splitValue =  radixSelector.select(points, pathSlices, points.start, points.start + points.count,  points.start + leftCount, splitDim, commonPrefixLen);
 
-      int address = nodeID * (1 + bytesPerDim);
+      int address = nodeID * (1 + config.bytesPerDim);
       splitPackedValues[address] = (byte) splitDim;
-      System.arraycopy(splitValue, 0, splitPackedValues, address + 1, bytesPerDim);
+      System.arraycopy(splitValue, 0, splitPackedValues, address + 1, config.bytesPerDim);
 
-      byte[] minSplitPackedValue = new byte[packedIndexBytesLength];
-      System.arraycopy(minPackedValue, 0, minSplitPackedValue, 0, packedIndexBytesLength);
+      byte[] minSplitPackedValue = new byte[config.packedIndexBytesLength];
+      System.arraycopy(minPackedValue, 0, minSplitPackedValue, 0, config.packedIndexBytesLength);
 
-      byte[] maxSplitPackedValue = new byte[packedIndexBytesLength];
-      System.arraycopy(maxPackedValue, 0, maxSplitPackedValue, 0, packedIndexBytesLength);
+      byte[] maxSplitPackedValue = new byte[config.packedIndexBytesLength];
+      System.arraycopy(maxPackedValue, 0, maxSplitPackedValue, 0, config.packedIndexBytesLength);
 
-      System.arraycopy(splitValue, 0, minSplitPackedValue, splitDim * bytesPerDim, bytesPerDim);
-      System.arraycopy(splitValue, 0, maxSplitPackedValue, splitDim * bytesPerDim, bytesPerDim);
+      System.arraycopy(splitValue, 0, minSplitPackedValue, splitDim * config.bytesPerDim, config.bytesPerDim);
+      System.arraycopy(splitValue, 0, maxSplitPackedValue, splitDim * config.bytesPerDim, config.bytesPerDim);
 
       // Recurse on left tree:
       build(2*nodeID, leafNodeOffset, pathSlices[0], out, radixSelector,
-            minPackedValue, maxSplitPackedValue, splitPackedValues, leafBlockFPs, spareDocIds);
+              minPackedValue, maxSplitPackedValue, splitPackedValues, leafBlockFPs, spareDocIds);
 
       // TODO: we could "tail recurse" here?  have our parent discard its refs as we recurse right?
       // Recurse on right tree:
       build(2*nodeID+1, leafNodeOffset, pathSlices[1], out, radixSelector,
-            minSplitPackedValue, maxPackedValue, splitPackedValues, leafBlockFPs, spareDocIds);
+              minSplitPackedValue, maxPackedValue, splitPackedValues, leafBlockFPs, spareDocIds);
     }
   }
 
   private void computeCommonPrefixLength(HeapPointWriter heapPointWriter, byte[] commonPrefix) {
-    Arrays.fill(commonPrefixLengths, bytesPerDim);
+    Arrays.fill(commonPrefixLengths, config.bytesPerDim);
     PointValue value = heapPointWriter.getPackedValueSlice(0);
     BytesRef packedValue = value.packedValue();
-    for (int dim = 0; dim < numDataDims; dim++) {
-      System.arraycopy(packedValue.bytes, packedValue.offset + dim * bytesPerDim, commonPrefix, dim * bytesPerDim, bytesPerDim);
+    for (int dim = 0; dim < config.numDims; dim++) {
+      System.arraycopy(packedValue.bytes, packedValue.offset + dim * config.bytesPerDim, commonPrefix, dim * config.bytesPerDim, config.bytesPerDim);
     }
     for (int i = 1; i < heapPointWriter.count(); i++) {
       value = heapPointWriter.getPackedValueSlice(i);
       packedValue = value.packedValue();
-      for (int dim = 0; dim < numDataDims; dim++) {
+      for (int dim = 0; dim < config.numDims; dim++) {
         if (commonPrefixLengths[dim] != 0) {
-          int j = FutureArrays.mismatch(commonPrefix, dim * bytesPerDim, dim * bytesPerDim + commonPrefixLengths[dim], packedValue.bytes, packedValue.offset + dim * bytesPerDim, packedValue.offset + dim * bytesPerDim + commonPrefixLengths[dim]);
+          int j = FutureArrays.mismatch(commonPrefix, dim * config.bytesPerDim, dim * config.bytesPerDim + commonPrefixLengths[dim], packedValue.bytes, packedValue.offset + dim * config.bytesPerDim, packedValue.offset + dim * config.bytesPerDim + commonPrefixLengths[dim]);
           if (j != -1) {
             commonPrefixLengths[dim] = j;
           }
@@ -1119,14 +1073,14 @@ final class SimpleTextBKDWriter implements Closeable {
 
   // only called from assert
   private boolean valuesInOrderAndBounds(int count, int sortedDim, byte[] minPackedValue, byte[] maxPackedValue,
-      IntFunction<BytesRef> values, int[] docs, int docsOffset) throws IOException {
-    byte[] lastPackedValue = new byte[packedBytesLength];
+                                         IntFunction<BytesRef> values, int[] docs, int docsOffset) throws IOException {
+    byte[] lastPackedValue = new byte[config.packedBytesLength];
     int lastDoc = -1;
     for (int i=0;i<count;i++) {
       BytesRef packedValue = values.apply(i);
-      assert packedValue.length == packedBytesLength;
+      assert packedValue.length == config.packedBytesLength;
       assert valueInOrder(i, sortedDim, lastPackedValue, packedValue.bytes, packedValue.offset,
-          docs[docsOffset + i], lastDoc);
+              docs[docsOffset + i], lastDoc);
       lastDoc = docs[docsOffset + i];
 
       // Make sure this value does in fact fall within this leaf cell:
@@ -1137,25 +1091,25 @@ final class SimpleTextBKDWriter implements Closeable {
 
   // only called from assert
   private boolean valueInOrder(long ord, int sortedDim, byte[] lastPackedValue, byte[] packedValue, int packedValueOffset,
-      int doc, int lastDoc) {
-    int dimOffset = sortedDim * bytesPerDim;
+                               int doc, int lastDoc) {
+    int dimOffset = sortedDim * config.bytesPerDim;
     if (ord > 0) {
-      int cmp = FutureArrays.compareUnsigned(lastPackedValue, dimOffset, dimOffset + bytesPerDim, packedValue, packedValueOffset + dimOffset, packedValueOffset + dimOffset + bytesPerDim);
+      int cmp = FutureArrays.compareUnsigned(lastPackedValue, dimOffset, dimOffset + config.bytesPerDim, packedValue, packedValueOffset + dimOffset, packedValueOffset + dimOffset + config.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, config.packedBytesLength) + " ord=" + ord + " sortedDim=" + sortedDim);
       }
-      if (cmp == 0  && numDataDims > numIndexDims) {
-        int dataOffset = numIndexDims * bytesPerDim;
-        cmp = FutureArrays.compareUnsigned(lastPackedValue, dataOffset, packedBytesLength, packedValue, packedValueOffset + dataOffset, packedValueOffset + packedBytesLength);
+      if (cmp == 0  && config.numDims > config.numIndexDims) {
+        int dataOffset = config.numIndexDims * config.bytesPerDim;
+        cmp = FutureArrays.compareUnsigned(lastPackedValue, dataOffset, config.packedBytesLength, packedValue, packedValueOffset + dataOffset, packedValueOffset + config.packedBytesLength);
         if (cmp > 0) {
-          throw new AssertionError("data values out of order: last value=" + new BytesRef(lastPackedValue) + " current value=" + new BytesRef(packedValue, packedValueOffset, packedBytesLength) + " ord=" + ord);
+          throw new AssertionError("data values out of order: last value=" + new BytesRef(lastPackedValue) + " current value=" + new BytesRef(packedValue, packedValueOffset, config.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);
       }
     }
-    System.arraycopy(packedValue, packedValueOffset, lastPackedValue, 0, packedBytesLength);
+    System.arraycopy(packedValue, packedValueOffset, lastPackedValue, 0, config.packedBytesLength);
     return true;
   }
 
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 cddbeb3..8155829 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
@@ -32,6 +32,7 @@ import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.bkd.BKDConfig;
 
 class SimpleTextPointsWriter extends PointsWriter {
 
@@ -72,14 +73,17 @@ class SimpleTextPointsWriter extends PointsWriter {
 
     PointValues values = reader.getValues(fieldInfo.name);
 
+
+    BKDConfig config = new BKDConfig(fieldInfo.getPointDimensionCount(),
+        fieldInfo.getPointIndexDimensionCount(),
+        fieldInfo.getPointNumBytes(),
+        BKDConfig.DEFAULT_MAX_POINTS_IN_LEAF_NODE);
+
     // We use our own fork of the BKDWriter to customize how it writes the index and blocks to disk:
     try (SimpleTextBKDWriter writer = new SimpleTextBKDWriter(writeState.segmentInfo.maxDoc(),
                                                               writeState.directory,
                                                               writeState.segmentInfo.name,
-                                                              fieldInfo.getPointDimensionCount(),
-                                                              fieldInfo.getPointIndexDimensionCount(),
-                                                              fieldInfo.getPointNumBytes(),
-                                                              SimpleTextBKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE,
+                                                              config,
                                                               SimpleTextBKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP,
                                                               values.size())) {
 
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene86/Lucene86PointsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene86/Lucene86PointsWriter.java
index 6fe3571..f6fc48a 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene86/Lucene86PointsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene86/Lucene86PointsWriter.java
@@ -36,6 +36,7 @@ import org.apache.lucene.index.PointValues.Relation;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.bkd.BKDConfig;
 import org.apache.lucene.util.bkd.BKDReader;
 import org.apache.lucene.util.bkd.BKDWriter;
 
@@ -98,7 +99,7 @@ public class Lucene86PointsWriter extends PointsWriter implements Closeable {
 
   /** Uses the defaults values for {@code maxPointsInLeafNode} (1024) and {@code maxMBSortInHeap} (16.0) */
   public Lucene86PointsWriter(SegmentWriteState writeState) throws IOException {
-    this(writeState, BKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE, BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP);
+    this(writeState, BKDConfig.DEFAULT_MAX_POINTS_IN_LEAF_NODE, BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP);
   }
 
   @Override
@@ -106,13 +107,15 @@ public class Lucene86PointsWriter extends PointsWriter implements Closeable {
 
     PointValues values = reader.getValues(fieldInfo.name);
 
+    BKDConfig config = new BKDConfig(fieldInfo.getPointDimensionCount(),
+                                     fieldInfo.getPointIndexDimensionCount(),
+                                     fieldInfo.getPointNumBytes(),
+                                     maxPointsInLeafNode);
+
     try (BKDWriter writer = new BKDWriter(writeState.segmentInfo.maxDoc(),
                                           writeState.directory,
                                           writeState.segmentInfo.name,
-                                          fieldInfo.getPointDimensionCount(),
-                                          fieldInfo.getPointIndexDimensionCount(),
-                                          fieldInfo.getPointNumBytes(),
-                                          maxPointsInLeafNode,
+                                          config,
                                           maxMBSortInHeap,
                                           values.size())) {
 
@@ -189,6 +192,11 @@ public class Lucene86PointsWriter extends PointsWriter implements Closeable {
             }
           }
 
+          BKDConfig config = new BKDConfig(fieldInfo.getPointDimensionCount(),
+                                           fieldInfo.getPointIndexDimensionCount(),
+                                           fieldInfo.getPointNumBytes(),
+                                           maxPointsInLeafNode);
+
           //System.out.println("MERGE: field=" + fieldInfo.name);
           // Optimize the 1D case to use BKDWriter.merge, which does a single merge sort of the
           // already sorted incoming segments, instead of trying to sort all points again as if
@@ -196,10 +204,7 @@ public class Lucene86PointsWriter extends PointsWriter implements Closeable {
           try (BKDWriter writer = new BKDWriter(writeState.segmentInfo.maxDoc(),
                                                 writeState.directory,
                                                 writeState.segmentInfo.name,
-                                                fieldInfo.getPointDimensionCount(),
-                                                fieldInfo.getPointIndexDimensionCount(),
-                                                fieldInfo.getPointNumBytes(),
-                                                maxPointsInLeafNode,
+                                                config,
                                                 maxMBSortInHeap,
                                                 totMaxSize)) {
             List<BKDReader> bkdReaders = new ArrayList<>();
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 ba24616..57bb3a9 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PointValues.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PointValues.java
@@ -29,17 +29,18 @@ import org.apache.lucene.document.LatLonPoint;
 import org.apache.lucene.document.LongPoint;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.util.FutureArrays;
-import org.apache.lucene.util.bkd.BKDWriter;
+import org.apache.lucene.util.bkd.BKDConfig;
 
-/** 
+/**
  * Access to indexed numeric values.
  * <p>
- * Points represent numeric values and are indexed differently than ordinary text. Instead of an inverted index, 
- * points are indexed with datastructures such as <a href="https://en.wikipedia.org/wiki/K-d_tree">KD-trees</a>. 
- * These structures are optimized for operations such as <i>range</i>, <i>distance</i>, <i>nearest-neighbor</i>, 
- * and <i>point-in-polygon</i> queries. 
- * <h1>Basic Point Types</h1>
- * <table summary="Basic point types in Java and Lucene">
+ * Points represent numeric values and are indexed differently than ordinary text. Instead of an inverted index,
+ * points are indexed with datastructures such as <a href="https://en.wikipedia.org/wiki/K-d_tree">KD-trees</a>.
+ * These structures are optimized for operations such as <i>range</i>, <i>distance</i>, <i>nearest-neighbor</i>,
+ * and <i>point-in-polygon</i> queries.
+ * <h2>Basic Point Types</h2>
+ * <table>
+ *   <caption>Basic point types in Java and Lucene</caption>
  *   <tr><th>Java type</th><th>Lucene class</th></tr>
  *   <tr><td>{@code int}</td><td>{@link IntPoint}</td></tr>
  *   <tr><td>{@code long}</td><td>{@link LongPoint}</td></tr>
@@ -51,9 +52,9 @@ import org.apache.lucene.util.bkd.BKDWriter;
  * </table>
  * * in the <i>lucene-sandbox</i> jar<br>
  * <p>
- * Basic Lucene point types behave like their java peers: for example {@link IntPoint} represents a signed 32-bit 
+ * Basic Lucene point types behave like their java peers: for example {@link IntPoint} represents a signed 32-bit
  * {@link Integer}, supporting values ranging from {@link Integer#MIN_VALUE} to {@link Integer#MAX_VALUE}, ordered
- * consistent with {@link Integer#compareTo(Integer)}. In addition to indexing support, point classes also contain 
+ * consistent with {@link Integer#compareTo(Integer)}. In addition to indexing support, point classes also contain
  * static methods (such as {@link IntPoint#newRangeQuery(String, int, int)}) for creating common queries. For example:
  * <pre class="prettyprint">
  *   // add year 1970 to document
@@ -65,9 +66,9 @@ import org.apache.lucene.util.bkd.BKDWriter;
  *   Query query = IntPoint.newRangeQuery("year", 1960, 1980);
  *   TopDocs docs = searcher.search(query, ...);
  * </pre>
- * <h1>Geospatial Point Types</h1>
+ * <h2>Geospatial Point Types</h2>
  * Although basic point types such as {@link DoublePoint} support points in multi-dimensional space too, Lucene has
- * specialized classes for location data. These classes are optimized for location data: they are more space-efficient and 
+ * specialized classes for location data. These classes are optimized for location data: they are more space-efficient and
  * support special operations such as <i>distance</i> and <i>polygon</i> queries. There are currently two implementations:
  * <br>
  * <ol>
@@ -75,8 +76,8 @@ import org.apache.lucene.util.bkd.BKDWriter;
  *   <li><a href="{@docRoot}/../spatial3d/org/apache/lucene/spatial3d/Geo3DPoint.html">Geo3DPoint</a>* in <i>lucene-spatial3d</i>: indexes {@code (latitude,longitude)} as {@code (x,y,z)} in three-dimensional space.
  * </ol>
  * * does <b>not</b> support altitude, 3D here means "uses three dimensions under-the-hood"<br>
- * <h1>Advanced usage</h1>
- * Custom structures can be created on top of single- or multi- dimensional basic types, on top of 
+ * <h2>Advanced usage</h2>
+ * Custom structures can be created on top of single- or multi- dimensional basic types, on top of
  * {@link BinaryPoint} for more flexibility, or via custom {@link Field} subclasses.
  *
  *  @lucene.experimental */
@@ -86,10 +87,10 @@ public abstract class PointValues {
   public static final int MAX_NUM_BYTES = 16;
 
   /** Maximum number of dimensions */
-  public static final int MAX_DIMENSIONS = BKDWriter.MAX_DIMS;
+  public static final int MAX_DIMENSIONS = BKDConfig.MAX_DIMS;
 
   /** Maximum number of index dimensions */
-  public static final int MAX_INDEX_DIMENSIONS = BKDWriter.MAX_INDEX_DIMS;
+  public static final int MAX_INDEX_DIMENSIONS = BKDConfig.MAX_INDEX_DIMS;
 
   /** Return the cumulated number of points across all leaves of the given
    * {@link IndexReader}. Leaves that do not have points for the given field
diff --git a/lucene/core/src/java/org/apache/lucene/util/bkd/BKDConfig.java b/lucene/core/src/java/org/apache/lucene/util/bkd/BKDConfig.java
new file mode 100644
index 0000000..c53f3b6
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/util/bkd/BKDConfig.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.util.bkd;
+
+import org.apache.lucene.util.ArrayUtil;
+
+/**
+ * Basic parameters for indexing points on the BKD tree.
+ */
+public final class BKDConfig {
+
+  /** Default maximum number of point in each leaf block */
+  public static final int DEFAULT_MAX_POINTS_IN_LEAF_NODE = 512;
+
+  /** Maximum number of index dimensions (2 * max index dimensions) */
+  public static final int MAX_DIMS = 16;
+
+  /** Maximum number of index dimensions */
+  public static final int MAX_INDEX_DIMS = 8;
+
+  /** How many dimensions we are storing at the leaf (data) nodes */
+  public final int numDims;
+
+  /** How many dimensions we are indexing in the internal nodes */
+  public final int numIndexDims;
+
+  /** How many bytes each value in each dimension takes. */
+  public final int bytesPerDim;
+
+  /** max points allowed on a Leaf block */
+  public final int maxPointsInLeafNode;
+
+  /** numDataDims * bytesPerDim */
+  public final int packedBytesLength;
+
+  /** numIndexDims * bytesPerDim */
+  public final int packedIndexBytesLength;
+
+  /** packedBytesLength plus docID size */
+  public final int bytesPerDoc;
+
+  public BKDConfig(final int numDims, final int numIndexDims, final int bytesPerDim, final int maxPointsInLeafNode) {
+    verifyParams(numDims, numIndexDims, bytesPerDim, maxPointsInLeafNode);
+    this.numDims = numDims;
+    this.numIndexDims = numIndexDims;
+    this.bytesPerDim = bytesPerDim;
+    this.maxPointsInLeafNode = maxPointsInLeafNode;
+    this.packedIndexBytesLength = numIndexDims * bytesPerDim;
+    this.packedBytesLength = numDims * bytesPerDim;
+    // dimensional values (numDims * bytesPerDim) + docID (int)
+    this.bytesPerDoc = this.packedBytesLength + Integer.BYTES;
+  }
+
+  private static void verifyParams(final int numDims, final int numIndexDims, final int bytesPerDim, final int maxPointsInLeafNode) {
+    // Check inputs are on bounds
+    if (numDims < 1 || numDims > MAX_DIMS) {
+      throw new IllegalArgumentException("numDims must be 1 .. " + MAX_DIMS + " (got: " + numDims + ")");
+    }
+    if (numIndexDims < 1 || numIndexDims > MAX_INDEX_DIMS) {
+      throw new IllegalArgumentException("numIndexDims must be 1 .. " + MAX_INDEX_DIMS + " (got: " + numIndexDims + ")");
+    }
+    if (numIndexDims > numDims) {
+      throw new IllegalArgumentException("numIndexDims cannot exceed numDims (" + numDims + ") (got: " + numIndexDims + ")");
+    }
+    if (bytesPerDim <= 0) {
+      throw new IllegalArgumentException("bytesPerDim must be > 0; got " + bytesPerDim);
+    }
+    if (maxPointsInLeafNode <= 0) {
+      throw new IllegalArgumentException("maxPointsInLeafNode must be > 0; got " + maxPointsInLeafNode);
+    }
+    if (maxPointsInLeafNode > ArrayUtil.MAX_ARRAY_LENGTH) {
+      throw new IllegalArgumentException("maxPointsInLeafNode must be <= ArrayUtil.MAX_ARRAY_LENGTH (= " + ArrayUtil.MAX_ARRAY_LENGTH + "); got " + maxPointsInLeafNode);
+    }
+  }
+}
\ No newline at end of file
diff --git a/lucene/core/src/java/org/apache/lucene/util/bkd/BKDRadixSelector.java b/lucene/core/src/java/org/apache/lucene/util/bkd/BKDRadixSelector.java
index 4ade104..ec3ddee 100644
--- a/lucene/core/src/java/org/apache/lucene/util/bkd/BKDRadixSelector.java
+++ b/lucene/core/src/java/org/apache/lucene/util/bkd/BKDRadixSelector.java
@@ -36,57 +36,46 @@ import org.apache.lucene.util.Sorter;
  *  @lucene.internal
  * */
 public final class BKDRadixSelector {
-  //size of the histogram
+  // size of the histogram
   private static final int HISTOGRAM_SIZE = 256;
-  //size of the online buffer: 8 KB
+  // size of the online buffer: 8 KB
   private static final int MAX_SIZE_OFFLINE_BUFFER = 1024 * 8;
-  //histogram array
+  // histogram array
   private final long[] histogram;
-  //bytes per dimension
-  private final int bytesPerDim;
-  // number of bytes to be sorted: bytesPerDim + Integer.BYTES
+  // number of bytes to be sorted: config.bytesPerDim + Integer.BYTES
   private final int bytesSorted;
-  //data dimensions size
-  private final int packedBytesLength;
-  // data dimensions plus docID size
-  private final int packedBytesDocIDLength;
-  //flag to when we are moving to sort on heap
+  // flag to when we are moving to sort on heap
   private final int maxPointsSortInHeap;
-  //reusable buffer
+  // reusable buffer
   private final byte[] offlineBuffer;
-  //holder for partition points
+  // holder for partition points
   private final int[] partitionBucket;
   // scratch array to hold temporary data
   private final byte[] scratch;
-  //Directory to create new Offline writer
+  // Directory to create new Offline writer
   private final Directory tempDir;
   // prefix for temp files
   private final String tempFileNamePrefix;
-  // data and index dimensions
-  private final int numDataDims, numIndexDims;
-
+  // BKD tree configuration
+  private final BKDConfig config;
 
   /**
    * Sole constructor.
    */
-  public BKDRadixSelector(int numDataDims, int numIndexDims, int bytesPerDim, int maxPointsSortInHeap, Directory tempDir, String tempFileNamePrefix) {
-    this.bytesPerDim = bytesPerDim;
-    this.numDataDims = numDataDims;
-    this.numIndexDims = numIndexDims;
-    this.packedBytesLength = numDataDims * bytesPerDim;
-    this.packedBytesDocIDLength = packedBytesLength + Integer.BYTES;
+  public BKDRadixSelector(BKDConfig config, int maxPointsSortInHeap, Directory tempDir, String tempFileNamePrefix) {
+    this.config = config;
+    this.maxPointsSortInHeap = maxPointsSortInHeap;
+    this.tempDir = tempDir;
+    this.tempFileNamePrefix = tempFileNamePrefix;
     // Selection and sorting is done in a given dimension. In case the value of the dimension are equal
     // between two points we tie break first using the data-only dimensions and if those are still equal
     // we tie-break on the docID. Here we account for all bytes used in the process.
-    this.bytesSorted = bytesPerDim  + (numDataDims - numIndexDims) * bytesPerDim + Integer.BYTES;
-    this.maxPointsSortInHeap = maxPointsSortInHeap;
-    int numberOfPointsOffline  = MAX_SIZE_OFFLINE_BUFFER / packedBytesDocIDLength;
-    this.offlineBuffer = new byte[numberOfPointsOffline * packedBytesDocIDLength];
+    this.bytesSorted = config.bytesPerDim + (config.numDims - config.numIndexDims) * config.bytesPerDim + Integer.BYTES;
+    final int numberOfPointsOffline = MAX_SIZE_OFFLINE_BUFFER / config.bytesPerDoc;
+    this.offlineBuffer = new byte[numberOfPointsOffline * config.bytesPerDoc];
     this.partitionBucket = new int[bytesSorted];
     this.histogram = new long[HISTOGRAM_SIZE];
     this.scratch = new byte[bytesSorted];
-    this.tempDir = tempDir;
-    this.tempFileNamePrefix = tempFileNamePrefix;
   }
 
   /**
@@ -109,7 +98,7 @@ public final class BKDRadixSelector {
 
     assert partitionSlices.length > 1 : "[partition alices] must be > 1, got " + partitionSlices.length;
 
-    //If we are on heap then we just select on heap
+    // If we are on heap then we just select on heap
     if (points.writer instanceof HeapPointWriter) {
       byte[] partition = heapRadixSelect((HeapPointWriter) points.writer, dim, Math.toIntExact(from), Math.toIntExact(to),  Math.toIntExact(partitionPoint), dimCommonPrefix);
       partitionSlices[0] = new PathSlice(points.writer, from, partitionPoint - from);
@@ -137,18 +126,18 @@ public final class BKDRadixSelector {
   }
 
   private int findCommonPrefixAndHistogram(OfflinePointWriter points, long from, long to, int dim, int dimCommonPrefix) throws IOException{
-    //find common prefix
+    // find common prefix
     int commonPrefixPosition = bytesSorted;
-    final int offset = dim * bytesPerDim;
+    final int offset = dim * config.bytesPerDim;
     try (OfflinePointReader reader = points.getReader(from, to - from, offlineBuffer)) {
       assert commonPrefixPosition > dimCommonPrefix;
       reader.next();
       PointValue pointValue = reader.pointValue();
       BytesRef packedValueDocID = pointValue.packedValueDocIDBytes();
       // copy dimension
-      System.arraycopy(packedValueDocID.bytes, packedValueDocID.offset + offset, scratch, 0, bytesPerDim);
+      System.arraycopy(packedValueDocID.bytes, packedValueDocID.offset + offset, scratch, 0, config.bytesPerDim);
       // copy data dimensions and docID
-      System.arraycopy(packedValueDocID.bytes, packedValueDocID.offset + numIndexDims * bytesPerDim, scratch, bytesPerDim, (numDataDims - numIndexDims) * bytesPerDim + Integer.BYTES);
+      System.arraycopy(packedValueDocID.bytes, packedValueDocID.offset + config.packedIndexBytesLength, scratch, config.bytesPerDim, (config.numDims - config.numIndexDims) * config.bytesPerDim + Integer.BYTES);
 
       for (long i = from + 1; i < to; i++) {
         reader.next();
@@ -164,20 +153,20 @@ public final class BKDRadixSelector {
           }
           break;
         } else {
-          //check common prefix and adjust histogram
-          final int startIndex = (dimCommonPrefix > bytesPerDim) ? bytesPerDim : dimCommonPrefix;
-          final int endIndex = (commonPrefixPosition > bytesPerDim) ? bytesPerDim : commonPrefixPosition;
+          // Check common prefix and adjust histogram
+          final int startIndex = (dimCommonPrefix > config.bytesPerDim) ? config.bytesPerDim : dimCommonPrefix;
+          final int endIndex = (commonPrefixPosition > config.bytesPerDim) ? config.bytesPerDim : commonPrefixPosition;
           packedValueDocID = pointValue.packedValueDocIDBytes();
           int j = FutureArrays.mismatch(scratch, startIndex, endIndex, packedValueDocID.bytes, packedValueDocID.offset + offset + startIndex, packedValueDocID.offset + offset + endIndex);
           if (j == -1) {
-            if (commonPrefixPosition > bytesPerDim) {
-              //tie-break on data dimensions + docID
-              final int startTieBreak = numIndexDims * bytesPerDim;
-              final int endTieBreak = startTieBreak + commonPrefixPosition - bytesPerDim;
-              int k = FutureArrays.mismatch(scratch, bytesPerDim, commonPrefixPosition,
-                  packedValueDocID.bytes, packedValueDocID.offset + startTieBreak, packedValueDocID.offset + endTieBreak);
+            if (commonPrefixPosition > config.bytesPerDim) {
+              // Tie-break on data dimensions + docID
+              final int startTieBreak = config.packedIndexBytesLength;
+              final int endTieBreak = startTieBreak + commonPrefixPosition - config.bytesPerDim;
+              int k = FutureArrays.mismatch(scratch, config.bytesPerDim, commonPrefixPosition,
+                      packedValueDocID.bytes, packedValueDocID.offset + startTieBreak, packedValueDocID.offset + endTieBreak);
               if (k != -1) {
-                commonPrefixPosition = bytesPerDim + k;
+                commonPrefixPosition = config.bytesPerDim + k;
                 Arrays.fill(histogram, 0);
                 histogram[scratch[commonPrefixPosition] & 0xff] = i - from;
               }
@@ -194,7 +183,7 @@ public final class BKDRadixSelector {
       }
     }
 
-    //build partition buckets up to commonPrefix
+    // Build partition buckets up to commonPrefix
     for (int i = 0; i < commonPrefixPosition; i++) {
       partitionBucket[i] = scratch[i] & 0xff;
     }
@@ -203,22 +192,22 @@ public final class BKDRadixSelector {
 
   private int getBucket(int offset, int commonPrefixPosition, PointValue pointValue) {
     int bucket;
-    if (commonPrefixPosition < bytesPerDim) {
+    if (commonPrefixPosition < config.bytesPerDim) {
       BytesRef packedValue = pointValue.packedValue();
       bucket = packedValue.bytes[packedValue.offset + offset + commonPrefixPosition] & 0xff;
     } else {
       BytesRef packedValueDocID = pointValue.packedValueDocIDBytes();
-      bucket = packedValueDocID.bytes[packedValueDocID.offset + numIndexDims * bytesPerDim + commonPrefixPosition - bytesPerDim] & 0xff;
+      bucket = packedValueDocID.bytes[packedValueDocID.offset + config.packedIndexBytesLength + commonPrefixPosition - config.bytesPerDim] & 0xff;
     }
     return bucket;
   }
 
   private byte[] buildHistogramAndPartition(OfflinePointWriter points, PointWriter left, PointWriter right,
                                             long from, long to, long partitionPoint, int iteration,  int baseCommonPrefix, int dim) throws IOException {
-    //find common prefix from baseCommonPrefix and build histogram
+    // Find common prefix from baseCommonPrefix and build histogram
     int commonPrefix = findCommonPrefixAndHistogram(points, from, to, dim, baseCommonPrefix);
 
-    //if all equals we just partition the points
+    // If all equals we just partition the points
     if (commonPrefix == bytesSorted) {
       offlinePartition(points, left, right, null, from, to, dim, commonPrefix - 1, partitionPoint);
       return partitionPointFromCommonPrefix();
@@ -227,7 +216,7 @@ public final class BKDRadixSelector {
     long leftCount = 0;
     long rightCount = 0;
 
-    //Count left points and record the partition point
+    // Count left points and record the partition point
     for(int i = 0; i < HISTOGRAM_SIZE; i++) {
       long size = histogram[i];
       if (leftCount + size > partitionPoint - from) {
@@ -236,7 +225,7 @@ public final class BKDRadixSelector {
       }
       leftCount += size;
     }
-    //Count right points
+    // Count right points
     for(int i = partitionBucket[commonPrefix] + 1; i < HISTOGRAM_SIZE; i++) {
       rightCount += histogram[i];
     }
@@ -244,17 +233,17 @@ public final class BKDRadixSelector {
     long delta = histogram[partitionBucket[commonPrefix]];
     assert leftCount + rightCount + delta == to - from : (leftCount + rightCount + delta) + " / " + (to - from);
 
-    //special case when points are equal except last byte, we can just tie-break
+    // Special case when points are equal except last byte, we can just tie-break
     if (commonPrefix == bytesSorted - 1) {
       long tieBreakCount =(partitionPoint - from - leftCount);
       offlinePartition(points, left,  right, null, from, to, dim, commonPrefix, tieBreakCount);
       return partitionPointFromCommonPrefix();
     }
 
-    //create the delta points writer
+    // Create the delta points writer
     PointWriter deltaPoints;
     try (PointWriter tempDeltaPoints = getDeltaPointWriter(left, right, delta, iteration)) {
-      //divide the points. This actually destroys the current writer
+      // Divide the points. This actually destroys the current writer
       offlinePartition(points, left, right, tempDeltaPoints, from, to, dim, commonPrefix, 0);
       deltaPoints = tempDeltaPoints;
     }
@@ -271,7 +260,7 @@ public final class BKDRadixSelector {
   private void offlinePartition(OfflinePointWriter points, PointWriter left, PointWriter right, PointWriter deltaPoints,
                                 long from, long to, int dim, int bytePosition, long numDocsTiebreak) throws IOException {
     assert bytePosition == bytesSorted -1 || deltaPoints != null;
-    int offset =  dim * bytesPerDim;
+    int offset =  dim * config.bytesPerDim;
     long tiebreakCounter = 0;
     try (OfflinePointReader reader = points.getReader(from, to - from, offlineBuffer)) {
       while (reader.next()) {
@@ -297,13 +286,13 @@ public final class BKDRadixSelector {
         }
       }
     }
-    //Delete original file
+    // Delete original file
     points.destroy();
   }
 
   private byte[] partitionPointFromCommonPrefix() {
-    byte[] partition = new byte[bytesPerDim];
-    for (int i = 0; i < bytesPerDim; i++) {
+    byte[] partition = new byte[config.bytesPerDim];
+    for (int i = 0; i < config.bytesPerDim; i++) {
       partition[i] = (byte)partitionBucket[i];
     }
     return partition;
@@ -323,9 +312,9 @@ public final class BKDRadixSelector {
   }
 
   private byte[] heapRadixSelect(HeapPointWriter points, int dim, int from, int to, int partitionPoint, int commonPrefixLength) {
-    final int dimOffset = dim * bytesPerDim + commonPrefixLength;
-    final int dimCmpBytes = bytesPerDim - commonPrefixLength;
-    final int dataOffset = numIndexDims * bytesPerDim - dimCmpBytes;
+    final int dimOffset = dim * config.bytesPerDim + commonPrefixLength;
+    final int dimCmpBytes = config.bytesPerDim - commonPrefixLength;
+    final int dataOffset = config.packedIndexBytesLength - dimCmpBytes;
     new RadixSelector(bytesSorted - commonPrefixLength) {
 
       @Override
@@ -338,21 +327,20 @@ public final class BKDRadixSelector {
         assert k >= 0 : "negative prefix " + k;
         if (k  < dimCmpBytes) {
           // dim bytes
-          return points.block[i * packedBytesDocIDLength + dimOffset + k] & 0xff;
+          return points.block[i * config.bytesPerDoc + dimOffset + k] & 0xff;
         } else {
           // data bytes
-          return points.block[i * packedBytesDocIDLength + dataOffset + k] & 0xff;
+          return points.block[i * config.bytesPerDoc + dataOffset + k] & 0xff;
         }
       }
 
       @Override
       protected Selector getFallbackSelector(int d) {
         final int skypedBytes = d + commonPrefixLength;
-        final int dimStart = dim * bytesPerDim + skypedBytes;
-        final int dimEnd =  dim * bytesPerDim + bytesPerDim;
-        final int dataOffset = numIndexDims * bytesPerDim;
+        final int dimStart = dim * config.bytesPerDim + skypedBytes;
+        final int dimEnd =  dim * config.bytesPerDim + config.bytesPerDim;
         // data length is composed by the data dimensions plus the docID
-        final int dataLength = (numDataDims - numIndexDims) * bytesPerDim + Integer.BYTES;
+        final int dataLength = (config.numDims - config.numIndexDims) * config.bytesPerDim + Integer.BYTES;
         return new IntroSelector() {
 
           @Override
@@ -362,55 +350,55 @@ public final class BKDRadixSelector {
 
           @Override
           protected void setPivot(int i) {
-            if (skypedBytes < bytesPerDim) {
-              System.arraycopy(points.block, i * packedBytesDocIDLength + dim * bytesPerDim, scratch, 0, bytesPerDim);
+            if (skypedBytes < config.bytesPerDim) {
+              System.arraycopy(points.block, i * config.bytesPerDoc + dim * config.bytesPerDim, scratch, 0, config.bytesPerDim);
             }
-            System.arraycopy(points.block, i * packedBytesDocIDLength + dataOffset, scratch, bytesPerDim, dataLength);
+            System.arraycopy(points.block, i * config.bytesPerDoc + config.packedIndexBytesLength, scratch, config.bytesPerDim, dataLength);
           }
 
           @Override
           protected int compare(int i, int j) {
-            if (skypedBytes < bytesPerDim) {
-              int iOffset = i * packedBytesDocIDLength;
-              int jOffset = j * packedBytesDocIDLength;
+            if (skypedBytes < config.bytesPerDim) {
+              int iOffset = i * config.bytesPerDoc;
+              int jOffset = j * config.bytesPerDoc;
               int cmp = FutureArrays.compareUnsigned(points.block, iOffset + dimStart, iOffset + dimEnd, points.block, jOffset + dimStart, jOffset + dimEnd);
               if (cmp != 0) {
                 return cmp;
               }
             }
-            int iOffset = i * packedBytesDocIDLength + dataOffset;
-            int jOffset = j * packedBytesDocIDLength + dataOffset;
+            int iOffset = i * config.bytesPerDoc + config.packedIndexBytesLength;
+            int jOffset = j * config.bytesPerDoc + config.packedIndexBytesLength;
             return FutureArrays.compareUnsigned(points.block, iOffset, iOffset + dataLength, points.block, jOffset, jOffset + dataLength);
           }
 
           @Override
           protected int comparePivot(int j) {
-            if (skypedBytes < bytesPerDim) {
-              int jOffset = j * packedBytesDocIDLength;
-              int cmp = FutureArrays.compareUnsigned(scratch, skypedBytes, bytesPerDim, points.block, jOffset + dimStart, jOffset + dimEnd);
+            if (skypedBytes < config.bytesPerDim) {
+              int jOffset = j * config.bytesPerDoc;
+              int cmp = FutureArrays.compareUnsigned(scratch, skypedBytes, config.bytesPerDim, points.block, jOffset + dimStart, jOffset + dimEnd);
               if (cmp != 0) {
                 return cmp;
               }
             }
-            int jOffset = j * packedBytesDocIDLength + dataOffset;
-            return FutureArrays.compareUnsigned(scratch, bytesPerDim, bytesPerDim + dataLength, points.block, jOffset, jOffset + dataLength);
+            int jOffset = j * config.bytesPerDoc + config.packedIndexBytesLength;
+            return FutureArrays.compareUnsigned(scratch, config.bytesPerDim, config.bytesPerDim + dataLength, points.block, jOffset, jOffset + dataLength);
           }
         };
       }
     }.select(from, to, partitionPoint);
 
-    byte[] partition = new byte[bytesPerDim];
+    byte[] partition = new byte[config.bytesPerDim];
     PointValue pointValue = points.getPackedValueSlice(partitionPoint);
     BytesRef packedValue = pointValue.packedValue();
-    System.arraycopy(packedValue.bytes, packedValue.offset + dim * bytesPerDim, partition, 0, bytesPerDim);
+    System.arraycopy(packedValue.bytes, packedValue.offset + dim * config.bytesPerDim, partition, 0, config.bytesPerDim);
     return partition;
   }
 
   /** Sort the heap writer by the specified dim. It is used to sort the leaves of the tree */
   public void heapRadixSort(final HeapPointWriter points, int from, int to, int dim, int commonPrefixLength) {
-    final int dimOffset = dim * bytesPerDim + commonPrefixLength;
-    final int dimCmpBytes = bytesPerDim - commonPrefixLength;
-    final int dataOffset = numIndexDims * bytesPerDim - dimCmpBytes;
+    final int dimOffset = dim * config.bytesPerDim + commonPrefixLength;
+    final int dimCmpBytes = config.bytesPerDim - commonPrefixLength;
+    final int dataOffset = config.packedIndexBytesLength - dimCmpBytes;
     new MSBRadixSorter(bytesSorted - commonPrefixLength) {
 
       @Override
@@ -418,10 +406,10 @@ public final class BKDRadixSelector {
         assert k >= 0 : "negative prefix " + k;
         if (k  < dimCmpBytes) {
           // dim bytes
-          return points.block[i * packedBytesDocIDLength + dimOffset + k] & 0xff;
+          return points.block[i * config.bytesPerDoc + dimOffset + k] & 0xff;
         } else {
           // data bytes
-          return points.block[i * packedBytesDocIDLength + dataOffset + k] & 0xff;
+          return points.block[i * config.bytesPerDoc + dataOffset + k] & 0xff;
         }
       }
 
@@ -433,11 +421,10 @@ public final class BKDRadixSelector {
       @Override
       protected Sorter getFallbackSorter(int k) {
         final int skypedBytes = k + commonPrefixLength;
-        final int dimStart = dim * bytesPerDim + skypedBytes;
-        final int dimEnd =  dim * bytesPerDim + bytesPerDim;
-        final int dataOffset = numIndexDims * bytesPerDim;
+        final int dimStart = dim * config.bytesPerDim + skypedBytes;
+        final int dimEnd =  dim * config.bytesPerDim + config.bytesPerDim;
         // data length is composed by the data dimensions plus the docID
-        final int dataLength = (numDataDims - numIndexDims) * bytesPerDim + Integer.BYTES;
+        final int dataLength = (config.numDims - config.numIndexDims) * config.bytesPerDim + Integer.BYTES;
         return new IntroSorter() {
 
           @Override
@@ -447,38 +434,38 @@ public final class BKDRadixSelector {
 
           @Override
           protected void setPivot(int i) {
-            if (skypedBytes < bytesPerDim) {
-              System.arraycopy(points.block, i * packedBytesDocIDLength + dim * bytesPerDim, scratch, 0, bytesPerDim);
+            if (skypedBytes < config.bytesPerDim) {
+              System.arraycopy(points.block, i * config.bytesPerDoc + dim * config.bytesPerDim, scratch, 0, config.bytesPerDim);
             }
-            System.arraycopy(points.block, i * packedBytesDocIDLength + dataOffset, scratch, bytesPerDim, dataLength);
+            System.arraycopy(points.block, i * config.bytesPerDoc + config.packedIndexBytesLength, scratch, config.bytesPerDim, dataLength);
           }
 
           @Override
           protected int compare(int i, int j) {
-            if (skypedBytes < bytesPerDim) {
-              int iOffset = i * packedBytesDocIDLength;
-              int jOffset = j * packedBytesDocIDLength;
+            if (skypedBytes < config.bytesPerDim) {
+              int iOffset = i * config.bytesPerDoc;
+              int jOffset = j * config.bytesPerDoc;
               int cmp = FutureArrays.compareUnsigned(points.block, iOffset + dimStart, iOffset + dimEnd, points.block, jOffset + dimStart, jOffset + dimEnd);
               if (cmp != 0) {
                 return cmp;
               }
             }
-            int iOffset = i * packedBytesDocIDLength + dataOffset;
-            int jOffset = j * packedBytesDocIDLength + dataOffset;
+            int iOffset = i * config.bytesPerDoc + config.packedIndexBytesLength;
+            int jOffset = j * config.bytesPerDoc + config.packedIndexBytesLength;
             return FutureArrays.compareUnsigned(points.block, iOffset, iOffset + dataLength, points.block, jOffset, jOffset + dataLength);
           }
 
           @Override
           protected int comparePivot(int j) {
-            if (skypedBytes < bytesPerDim) {
-              int jOffset = j * packedBytesDocIDLength;
-              int cmp = FutureArrays.compareUnsigned(scratch, skypedBytes, bytesPerDim, points.block, jOffset + dimStart, jOffset + dimEnd);
+            if (skypedBytes < config.bytesPerDim) {
+              int jOffset = j * config.bytesPerDoc;
+              int cmp = FutureArrays.compareUnsigned(scratch, skypedBytes, config.bytesPerDim, points.block, jOffset + dimStart, jOffset + dimEnd);
               if (cmp != 0) {
                 return cmp;
               }
             }
-            int jOffset = j * packedBytesDocIDLength + dataOffset;
-            return FutureArrays.compareUnsigned(scratch, bytesPerDim, bytesPerDim + dataLength, points.block, jOffset, jOffset + dataLength);
+            int jOffset = j * config.bytesPerDoc + config.packedIndexBytesLength;
+            return FutureArrays.compareUnsigned(scratch, config.bytesPerDim, config.bytesPerDim + dataLength, points.block, jOffset, jOffset + dataLength);
           }
         };
       }
@@ -487,9 +474,9 @@ public final class BKDRadixSelector {
 
   private PointWriter getDeltaPointWriter(PointWriter left, PointWriter right, long delta, int iteration) throws IOException {
     if (delta <= getMaxPointsSortInHeap(left, right)) {
-      return  new HeapPointWriter(Math.toIntExact(delta), packedBytesLength);
+      return  new HeapPointWriter(config, Math.toIntExact(delta));
     } else {
-      return new OfflinePointWriter(tempDir, tempFileNamePrefix, packedBytesLength, "delta" + iteration, delta);
+      return new OfflinePointWriter(config, tempDir, tempFileNamePrefix, "delta" + iteration, delta);
     }
   }
 
@@ -506,13 +493,13 @@ public final class BKDRadixSelector {
   }
 
   PointWriter getPointWriter(long count, String desc) throws IOException {
-    //As we recurse, we hold two on-heap point writers at any point. Therefore the
-    //max size for these objects is half of the total points we can have on-heap.
+    // As we recurse, we hold two on-heap point writers at any point. Therefore the
+    // max size for these objects is half of the total points we can have on-heap.
     if (count <= maxPointsSortInHeap / 2) {
       int size = Math.toIntExact(count);
-      return new HeapPointWriter(size, packedBytesLength);
+      return new HeapPointWriter(config, size);
     } else {
-      return new OfflinePointWriter(tempDir, tempFileNamePrefix, packedBytesLength, desc, count);
+      return new OfflinePointWriter(config, tempDir, tempFileNamePrefix, desc, count);
     }
   }
 
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 05a1aa6..c0aa525 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,19 +36,14 @@ public final class BKDReader extends PointValues {
 
   // Packed array of byte[] holding all split values in the full binary tree:
   final int leafNodeOffset;
-  final int numDataDims;
-  final int numIndexDims;
-  final int bytesPerDim;
+  final BKDConfig config;
   final int numLeaves;
   final IndexInput in;
-  final int maxPointsInLeafNode;
   final byte[] minPackedValue;
   final byte[] maxPackedValue;
   final long pointCount;
   final int docCount;
   final int version;
-  protected final int packedBytesLength;
-  protected final int packedIndexBytesLength;
   final long minLeafBlockFP;
 
   final IndexInput packedIndex;
@@ -57,34 +52,34 @@ public final class BKDReader extends PointValues {
    * BKD tree is always stored off-heap. */
   public BKDReader(IndexInput metaIn, IndexInput indexIn, IndexInput dataIn) throws IOException {
     version = CodecUtil.checkHeader(metaIn, BKDWriter.CODEC_NAME, BKDWriter.VERSION_START, BKDWriter.VERSION_CURRENT);
-    numDataDims = metaIn.readVInt();
+    final int numDims = metaIn.readVInt();
+    final int numIndexDims;
     if (version >= BKDWriter.VERSION_SELECTIVE_INDEXING) {
       numIndexDims = metaIn.readVInt();
     } else {
-      numIndexDims = numDataDims;
+      numIndexDims = numDims;
     }
-    maxPointsInLeafNode = metaIn.readVInt();
-    bytesPerDim = metaIn.readVInt();
-    packedBytesLength = numDataDims * bytesPerDim;
-    packedIndexBytesLength = numIndexDims * bytesPerDim;
+    final int maxPointsInLeafNode = metaIn.readVInt();
+    final int bytesPerDim = metaIn.readVInt();
+    config = new BKDConfig(numDims, numIndexDims, bytesPerDim, maxPointsInLeafNode);
 
     // Read index:
     numLeaves = metaIn.readVInt();
     assert numLeaves > 0;
     leafNodeOffset = numLeaves;
 
-    minPackedValue = new byte[packedIndexBytesLength];
-    maxPackedValue = new byte[packedIndexBytesLength];
+    minPackedValue = new byte[config.packedIndexBytesLength];
+    maxPackedValue = new byte[config.packedIndexBytesLength];
 
-    metaIn.readBytes(minPackedValue, 0, packedIndexBytesLength);
-    metaIn.readBytes(maxPackedValue, 0, packedIndexBytesLength);
+    metaIn.readBytes(minPackedValue, 0, config.packedIndexBytesLength);
+    metaIn.readBytes(maxPackedValue, 0, config.packedIndexBytesLength);
 
-    for(int dim=0;dim<numIndexDims;dim++) {
-      if (FutureArrays.compareUnsigned(minPackedValue, dim * bytesPerDim, dim * bytesPerDim + bytesPerDim, maxPackedValue, dim * bytesPerDim, dim * bytesPerDim + bytesPerDim) > 0) {
+    for(int dim=0;dim<config.numIndexDims;dim++) {
+      if (FutureArrays.compareUnsigned(minPackedValue, dim * config.bytesPerDim, dim * config.bytesPerDim + config.bytesPerDim, maxPackedValue, dim * config.bytesPerDim, dim * config.bytesPerDim + config.bytesPerDim) > 0) {
         throw new CorruptIndexException("minPackedValue " + new BytesRef(minPackedValue) + " is > maxPackedValue " + new BytesRef(maxPackedValue) + " for dim=" + dim, metaIn);
       }
     }
-    
+
     pointCount = metaIn.readVLong();
     docCount = metaIn.readVInt();
 
@@ -144,24 +139,24 @@ public final class BKDReader extends PointValues {
       splitPackedValueStack = new byte[treeDepth+1][];
       this.nodeID = nodeID;
       this.level = level;
-      splitPackedValueStack[level] = new byte[packedIndexBytesLength];
+      splitPackedValueStack[level] = new byte[config.packedIndexBytesLength];
       leafBlockFPStack = new long[treeDepth+1];
       rightNodePositions = new int[treeDepth+1];
       splitValuesStack = new byte[treeDepth+1][];
       splitDims = new int[treeDepth+1];
-      negativeDeltas = new boolean[numIndexDims*(treeDepth+1)];
+      negativeDeltas = new boolean[config.numIndexDims*(treeDepth+1)];
       this.in = in;
-      splitValuesStack[0] = new byte[packedIndexBytesLength];
+      splitValuesStack[0] = new byte[config.packedIndexBytesLength];
       scratch = new BytesRef();
-      scratch.length = bytesPerDim;
-    }      
+      scratch.length = config.bytesPerDim;
+    }
 
     public void pushLeft() {
       nodeID *= 2;
       level++;
       readNodeData(true);
     }
-    
+
     /** Clone, but you are not allowed to pop up past the point where the clone happened. */
     @Override
     public IndexTree clone() {
@@ -171,11 +166,11 @@ public final class BKDReader extends PointValues {
       index.leafBlockFPStack[level] = leafBlockFPStack[level];
       index.rightNodePositions[level] = rightNodePositions[level];
       index.splitValuesStack[index.level] = splitValuesStack[index.level].clone();
-      System.arraycopy(negativeDeltas, level*numIndexDims, index.negativeDeltas, level*numIndexDims, numIndexDims);
+      System.arraycopy(negativeDeltas, level*config.numIndexDims, index.negativeDeltas, level*config.numIndexDims, config.numIndexDims);
       index.splitDims[level] = splitDims[level];
       return index;
     }
-    
+
     public void pushRight() {
       final int nodePosition = rightNodePositions[level];
       assert nodePosition >= in.getFilePointer() : "nodePosition = " + nodePosition + " < currentPosition=" + in.getFilePointer();
@@ -213,7 +208,7 @@ public final class BKDReader extends PointValues {
       assert splitPackedValueStack[level] != null: "level=" + level;
       return splitPackedValueStack[level];
     }
-                                                       
+
     /** Only valid after pushLeft or pushRight, not pop! */
     public int getSplitDim() {
       assert isLeafNode() == false;
@@ -224,10 +219,10 @@ public final class BKDReader extends PointValues {
     public BytesRef getSplitDimValue() {
       assert isLeafNode() == false;
       scratch.bytes = splitValuesStack[level];
-      scratch.offset = splitDim * bytesPerDim;
+      scratch.offset = splitDim * config.bytesPerDim;
       return scratch;
     }
-    
+
     /** Only valid after pushLeft or pushRight, not pop! */
     public long getLeafBlockFP() {
       assert isLeafNode(): "nodeID=" + nodeID + " is not a leaf";
@@ -271,11 +266,11 @@ public final class BKDReader extends PointValues {
 
     private void readNodeData(boolean isLeft) {
       if (splitPackedValueStack[level] == null) {
-        splitPackedValueStack[level] = new byte[packedIndexBytesLength];
+        splitPackedValueStack[level] = new byte[config.packedIndexBytesLength];
       }
-      System.arraycopy(negativeDeltas, (level-1)*numIndexDims, negativeDeltas, level*numIndexDims, numIndexDims);
+      System.arraycopy(negativeDeltas, (level-1)*config.numIndexDims, negativeDeltas, level*config.numIndexDims, config.numIndexDims);
       assert splitDim != -1;
-      negativeDeltas[level*numIndexDims+splitDim] = isLeft;
+      negativeDeltas[level*config.numIndexDims+splitDim] = isLeft;
 
       try {
         leafBlockFPStack[level] = leafBlockFPStack[level - 1];
@@ -291,24 +286,24 @@ public final class BKDReader extends PointValues {
 
           // read split dim, prefix, firstDiffByteDelta encoded as int:
           int code = in.readVInt();
-          splitDim = code % numIndexDims;
+          splitDim = code % config.numIndexDims;
           splitDims[level] = splitDim;
-          code /= numIndexDims;
-          int prefix = code % (1 + bytesPerDim);
-          int suffix = bytesPerDim - prefix;
+          code /= config.numIndexDims;
+          int prefix = code % (1 + config.bytesPerDim);
+          int suffix = config.bytesPerDim - prefix;
 
           if (splitValuesStack[level] == null) {
-            splitValuesStack[level] = new byte[packedIndexBytesLength];
+            splitValuesStack[level] = new byte[config.packedIndexBytesLength];
           }
-          System.arraycopy(splitValuesStack[level - 1], 0, splitValuesStack[level], 0, packedIndexBytesLength);
+          System.arraycopy(splitValuesStack[level - 1], 0, splitValuesStack[level], 0, config.packedIndexBytesLength);
           if (suffix > 0) {
-            int firstDiffByteDelta = code / (1 + bytesPerDim);
-            if (negativeDeltas[level * numIndexDims + splitDim]) {
+            int firstDiffByteDelta = code / (1 + config.bytesPerDim);
+            if (negativeDeltas[level * config.numIndexDims + splitDim]) {
               firstDiffByteDelta = -firstDiffByteDelta;
             }
-            int oldByte = splitValuesStack[level][splitDim * bytesPerDim + prefix] & 0xFF;
-            splitValuesStack[level][splitDim * bytesPerDim + prefix] = (byte) (oldByte + firstDiffByteDelta);
-            in.readBytes(splitValuesStack[level], splitDim * bytesPerDim + prefix + 1, suffix - 1);
+            int oldByte = splitValuesStack[level][splitDim * config.bytesPerDim + prefix] & 0xFF;
+            splitValuesStack[level][splitDim * config.bytesPerDim + prefix] = (byte) (oldByte + firstDiffByteDelta);
+            in.readBytes(splitValuesStack[level], splitDim * config.bytesPerDim + prefix + 1, suffix - 1);
           } else {
             // our split value is == last split value in this dim, which can happen when there are many duplicate values
           }
@@ -347,19 +342,17 @@ public final class BKDReader extends PointValues {
     final IntersectVisitor visitor;
     public final IndexTree index;
 
-    public IntersectState(IndexInput in, int numDims,
-                          int packedBytesLength,
-                          int packedIndexBytesLength,
-                          int maxPointsInLeafNode,
-                          IntersectVisitor visitor,
-                          IndexTree indexVisitor) {
+    public  IntersectState(IndexInput in,
+                           BKDConfig config,
+                           IntersectVisitor visitor,
+                           IndexTree indexVisitor) {
       this.in = in;
       this.visitor = visitor;
-      this.commonPrefixLengths = new int[numDims];
-      this.scratchIterator = new BKDReaderDocIDSetIterator(maxPointsInLeafNode);
-      this.scratchDataPackedValue = new byte[packedBytesLength];
-      this.scratchMinIndexPackedValue = new byte[packedIndexBytesLength];
-      this.scratchMaxIndexPackedValue = new byte[packedIndexBytesLength];
+      this.commonPrefixLengths = new int[config.numDims];
+      this.scratchIterator = new BKDReaderDocIDSetIterator(config.maxPointsInLeafNode);
+      this.scratchDataPackedValue = new byte[config.packedBytesLength];
+      this.scratchMinIndexPackedValue = new byte[config.packedIndexBytesLength];
+      this.scratchMaxIndexPackedValue = new byte[config.packedIndexBytesLength];
       this.index = indexVisitor;
     }
   }
@@ -379,7 +372,7 @@ public final class BKDReader extends PointValues {
     //System.out.println("R: addAll nodeID=" + nodeID);
 
     if (grown == false) {
-      final long maxPointCount = (long) maxPointsInLeafNode * state.index.getNumLeaves();
+      final long maxPointCount = (long) config.maxPointsInLeafNode * state.index.getNumLeaves();
       if (maxPointCount <= Integer.MAX_VALUE) { // could be >MAX_VALUE if there are more than 2B points in total
         state.visitor.grow((int) maxPointCount);
         grown = true;
@@ -407,12 +400,7 @@ public final class BKDReader extends PointValues {
   /** Create a new {@link IntersectState} */
   public IntersectState getIntersectState(IntersectVisitor visitor) {
     IndexTree index = new IndexTree();
-    return new IntersectState(in.clone(), numDataDims,
-                              packedBytesLength,
-                              packedIndexBytesLength,
-                              maxPointsInLeafNode,
-                              visitor,
-                              index);
+    return new IntersectState(in.clone(), config, visitor, index);
   }
 
   /** Visits all docIDs and packed values in a single leaf block */
@@ -457,15 +445,15 @@ public final class BKDReader extends PointValues {
   }
 
   void visitDocValuesNoCardinality(int[] commonPrefixLengths, byte[] scratchDataPackedValue, byte[] scratchMinIndexPackedValue, byte[] scratchMaxIndexPackedValue,
-                      IndexInput in, BKDReaderDocIDSetIterator scratchIterator, int count, IntersectVisitor visitor) throws IOException {
+                                   IndexInput in, BKDReaderDocIDSetIterator scratchIterator, int count, IntersectVisitor visitor) throws IOException {
     readCommonPrefixes(commonPrefixLengths, scratchDataPackedValue, in);
 
-    if (numIndexDims != 1 && version >= BKDWriter.VERSION_LEAF_STORES_BOUNDS) {
+    if (config.numIndexDims != 1 && version >= BKDWriter.VERSION_LEAF_STORES_BOUNDS) {
       byte[] minPackedValue = scratchMinIndexPackedValue;
-      System.arraycopy(scratchDataPackedValue, 0, minPackedValue, 0, packedIndexBytesLength);
+      System.arraycopy(scratchDataPackedValue, 0, minPackedValue, 0, config.packedIndexBytesLength);
       byte[] maxPackedValue = scratchMaxIndexPackedValue;
       // Copy common prefixes before reading adjusted box
-      System.arraycopy(minPackedValue, 0, maxPackedValue, 0, packedIndexBytesLength);
+      System.arraycopy(minPackedValue, 0, maxPackedValue, 0, config.packedIndexBytesLength);
       readMinMax(commonPrefixLengths, minPackedValue, maxPackedValue, in);
 
       // The index gives us range of values for each dimension, but the actual range of values
@@ -510,12 +498,12 @@ public final class BKDReader extends PointValues {
       visitor.grow(count);
       visitUniqueRawDocValues(scratchDataPackedValue, scratchIterator, count, visitor);
     } else {
-      if (numIndexDims != 1) {
+      if (config.numIndexDims != 1) {
         byte[] minPackedValue = scratchMinIndexPackedValue;
-        System.arraycopy(scratchDataPackedValue, 0, minPackedValue, 0, packedIndexBytesLength);
+        System.arraycopy(scratchDataPackedValue, 0, minPackedValue, 0, config.packedIndexBytesLength);
         byte[] maxPackedValue = scratchMaxIndexPackedValue;
         // Copy common prefixes before reading adjusted box
-        System.arraycopy(minPackedValue, 0, maxPackedValue, 0, packedIndexBytesLength);
+        System.arraycopy(minPackedValue, 0, maxPackedValue, 0, config.packedIndexBytesLength);
         readMinMax(commonPrefixLengths, minPackedValue, maxPackedValue, in);
 
         // The index gives us range of values for each dimension, but the actual range of values
@@ -550,10 +538,10 @@ public final class BKDReader extends PointValues {
   }
 
   private void readMinMax(int[] commonPrefixLengths, byte[] minPackedValue, byte[] maxPackedValue, IndexInput in) throws IOException {
-    for (int dim = 0; dim < numIndexDims; dim++) {
+    for (int dim = 0; dim < config.numIndexDims; dim++) {
       int prefix = commonPrefixLengths[dim];
-      in.readBytes(minPackedValue, dim * bytesPerDim + prefix, bytesPerDim - prefix);
-      in.readBytes(maxPackedValue, dim * bytesPerDim + prefix, bytesPerDim - prefix);
+      in.readBytes(minPackedValue, dim * config.bytesPerDim + prefix, config.bytesPerDim - prefix);
+      in.readBytes(maxPackedValue, dim * config.bytesPerDim + prefix, config.bytesPerDim - prefix);
     }
   }
 
@@ -562,9 +550,9 @@ public final class BKDReader extends PointValues {
     int i;
     for (i = 0; i < count;) {
       int length = in.readVInt();
-      for(int dim = 0; dim < numDataDims; dim++) {
+      for(int dim = 0; dim < config.numDims; dim++) {
         int prefix = commonPrefixLengths[dim];
-        in.readBytes(scratchPackedValue, dim*bytesPerDim + prefix, bytesPerDim - prefix);
+        in.readBytes(scratchPackedValue, dim*config.bytesPerDim + prefix, config.bytesPerDim - prefix);
       }
       scratchIterator.reset(i, length);
       visitor.visit(scratchIterator, scratchPackedValue);
@@ -584,16 +572,16 @@ public final class BKDReader extends PointValues {
   private void visitCompressedDocValues(int[] commonPrefixLengths, byte[] scratchPackedValue, IndexInput in, BKDReaderDocIDSetIterator scratchIterator, int count, IntersectVisitor visitor, int compressedDim) throws IOException {
     // the byte at `compressedByteOffset` is compressed using run-length compression,
     // other suffix bytes are stored verbatim
-    final int compressedByteOffset = compressedDim * bytesPerDim + commonPrefixLengths[compressedDim];
+    final int compressedByteOffset = compressedDim * config.bytesPerDim + commonPrefixLengths[compressedDim];
     commonPrefixLengths[compressedDim]++;
     int i;
     for (i = 0; i < count; ) {
       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 < config.numDims; dim++) {
           int prefix = commonPrefixLengths[dim];
-          in.readBytes(scratchPackedValue, dim*bytesPerDim + prefix, bytesPerDim - prefix);
+          in.readBytes(scratchPackedValue, dim*config.bytesPerDim + prefix, config.bytesPerDim - prefix);
         }
         visitor.visit(scratchIterator.docIDs[i+j], scratchPackedValue);
       }
@@ -606,18 +594,18 @@ public final class BKDReader extends PointValues {
 
   private int readCompressedDim(IndexInput in) throws IOException {
     int compressedDim = in.readByte();
-    if (compressedDim < -2 || compressedDim >= numDataDims || (version < BKDWriter.VERSION_LOW_CARDINALITY_LEAVES && compressedDim == -2)) {
+    if (compressedDim < -2 || compressedDim >= config.numDims || (version < BKDWriter.VERSION_LOW_CARDINALITY_LEAVES && compressedDim == -2)) {
       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<config.numDims;dim++) {
       int prefix = in.readVInt();
       commonPrefixLengths[dim] = prefix;
       if (prefix > 0) {
-        in.readBytes(scratchPackedValue, dim*bytesPerDim, prefix);
+        in.readBytes(scratchPackedValue, dim*config.bytesPerDim, prefix);
       }
       //System.out.println("R: " + dim + " of " + numDims + " prefix=" + prefix);
     }
@@ -628,7 +616,7 @@ public final class BKDReader extends PointValues {
     /*
     System.out.println("\nR: intersect nodeID=" + state.index.getNodeID());
     for(int dim=0;dim<numDims;dim++) {
-      System.out.println("  dim=" + dim + "\n    cellMin=" + new BytesRef(cellMinPacked, dim*bytesPerDim, bytesPerDim) + "\n    cellMax=" + new BytesRef(cellMaxPacked, dim*bytesPerDim, bytesPerDim));
+      System.out.println("  dim=" + dim + "\n    cellMin=" + new BytesRef(cellMinPacked, dim*config.bytesPerDim, config.bytesPerDim) + "\n    cellMax=" + new BytesRef(cellMaxPacked, dim*config.bytesPerDim, config.bytesPerDim));
     }
     */
 
@@ -641,9 +629,9 @@ public final class BKDReader extends PointValues {
       addAll(state, false);
       // The cell crosses the shape boundary, or the cell fully contains the query, so we fall through and do full filtering:
     } else if (state.index.isLeafNode()) {
-      
+
       // TODO: we can assert that the first value here in fact matches what the index claimed?
-      
+
       // In the unbalanced case it's possible the left most node only has one child:
       if (state.index.nodeExists()) {
         // Leaf node; scan and filter all points in this block:
@@ -654,34 +642,34 @@ public final class BKDReader extends PointValues {
       }
 
     } 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 + ", config.numIndexDims=" + config.numIndexDims;
+      assert splitDim < config.numIndexDims: "splitDim=" + splitDim + ", config.numIndexDims=" + config.numIndexDims;
 
       byte[] splitPackedValue = state.index.getSplitPackedValue();
       BytesRef splitDimValue = state.index.getSplitDimValue();
-      assert splitDimValue.length == bytesPerDim;
+      assert splitDimValue.length == config.bytesPerDim;
       //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 * config.bytesPerDim, splitDim * config.bytesPerDim + config.bytesPerDim, splitDimValue.bytes, splitDimValue.offset, splitDimValue.offset + config.bytesPerDim) <= 0: "config.bytesPerDim=" + config.bytesPerDim + " splitDim=" + splitDim + " config.numIndexDims=" + config.numIndexDims + " config.numDims=" + config.numDims;
+      assert FutureArrays.compareUnsigned(cellMaxPacked, splitDim * config.bytesPerDim, splitDim * config.bytesPerDim + config.bytesPerDim, splitDimValue.bytes, splitDimValue.offset, splitDimValue.offset + config.bytesPerDim) >= 0: "config.bytesPerDim=" + config.bytesPerDim + " splitDim=" + splitDim + " config.numIndexDims=" + config.numIndexDims + " config.numDims=" + config.numDims;
 
       // Recurse on left sub-tree:
-      System.arraycopy(cellMaxPacked, 0, splitPackedValue, 0, packedIndexBytesLength);
-      System.arraycopy(splitDimValue.bytes, splitDimValue.offset, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
+      System.arraycopy(cellMaxPacked, 0, splitPackedValue, 0, config.packedIndexBytesLength);
+      System.arraycopy(splitDimValue.bytes, splitDimValue.offset, splitPackedValue, splitDim*config.bytesPerDim, config.bytesPerDim);
       state.index.pushLeft();
       intersect(state, cellMinPacked, splitPackedValue);
       state.index.pop();
 
       // Restore the split dim value since it may have been overwritten while recursing:
-      System.arraycopy(splitPackedValue, splitDim*bytesPerDim, splitDimValue.bytes, splitDimValue.offset, bytesPerDim);
+      System.arraycopy(splitPackedValue, splitDim*config.bytesPerDim, splitDimValue.bytes, splitDimValue.offset, config.bytesPerDim);
 
       // Recurse on right sub-tree:
-      System.arraycopy(cellMinPacked, 0, splitPackedValue, 0, packedIndexBytesLength);
-      System.arraycopy(splitDimValue.bytes, splitDimValue.offset, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
+      System.arraycopy(cellMinPacked, 0, splitPackedValue, 0, config.packedIndexBytesLength);
+      System.arraycopy(splitDimValue.bytes, splitDimValue.offset, splitPackedValue, splitDim*config.bytesPerDim, config.bytesPerDim);
       state.index.pushRight();
       intersect(state, splitPackedValue, cellMaxPacked);
       state.index.pop();
@@ -693,7 +681,7 @@ public final class BKDReader extends PointValues {
     /*
     System.out.println("\nR: intersect nodeID=" + state.index.getNodeID());
     for(int dim=0;dim<numDims;dim++) {
-      System.out.println("  dim=" + dim + "\n    cellMin=" + new BytesRef(cellMinPacked, dim*bytesPerDim, bytesPerDim) + "\n    cellMax=" + new BytesRef(cellMaxPacked, dim*bytesPerDim, bytesPerDim));
+      System.out.println("  dim=" + dim + "\n    cellMin=" + new BytesRef(cellMinPacked, dim*config.bytesPerDim, config.bytesPerDim) + "\n    cellMax=" + new BytesRef(cellMaxPacked, dim*config.bytesPerDim, config.bytesPerDim));
     }
     */
 
@@ -703,39 +691,39 @@ public final class BKDReader extends PointValues {
       // This cell is fully outside of the query shape: stop recursing
       return 0L;
     } else if (r == Relation.CELL_INSIDE_QUERY) {
-      return (long) maxPointsInLeafNode * state.index.getNumLeaves();
+      return (long) config.maxPointsInLeafNode * state.index.getNumLeaves();
     } else if (state.index.isLeafNode()) {
       // Assume half the points matched
-      return (maxPointsInLeafNode + 1) / 2;
+      return (config.maxPointsInLeafNode + 1) / 2;
     } 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 + ", config.numIndexDims=" + config.numIndexDims;
+      assert splitDim < config.numIndexDims: "splitDim=" + splitDim + ", config.numIndexDims=" + config.numIndexDims;
 
       byte[] splitPackedValue = state.index.getSplitPackedValue();
       BytesRef splitDimValue = state.index.getSplitDimValue();
-      assert splitDimValue.length == bytesPerDim;
+      assert splitDimValue.length == config.bytesPerDim;
       //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 * config.bytesPerDim, splitDim * config.bytesPerDim + config.bytesPerDim, splitDimValue.bytes, splitDimValue.offset, splitDimValue.offset + config.bytesPerDim) <= 0: "config.bytesPerDim=" + config.bytesPerDim + " splitDim=" + splitDim + " config.numIndexDims=" + config.numIndexDims + " config.numDims=" + config.numDims;
+      assert FutureArrays.compareUnsigned(cellMaxPacked, splitDim * config.bytesPerDim, splitDim * config.bytesPerDim + config.bytesPerDim, splitDimValue.bytes, splitDimValue.offset, splitDimValue.offset + config.bytesPerDim) >= 0: "config.bytesPerDim=" + config.bytesPerDim + " splitDim=" + splitDim + " config.numIndexDims=" + config.numIndexDims + " config.numDims=" + config.numDims;
 
       // Recurse on left sub-tree:
-      System.arraycopy(cellMaxPacked, 0, splitPackedValue, 0, packedIndexBytesLength);
-      System.arraycopy(splitDimValue.bytes, splitDimValue.offset, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
+      System.arraycopy(cellMaxPacked, 0, splitPackedValue, 0, config.packedIndexBytesLength);
+      System.arraycopy(splitDimValue.bytes, splitDimValue.offset, splitPackedValue, splitDim*config.bytesPerDim, config.bytesPerDim);
       state.index.pushLeft();
       final long leftCost = estimatePointCount(state, cellMinPacked, splitPackedValue);
       state.index.pop();
 
       // Restore the split dim value since it may have been overwritten while recursing:
-      System.arraycopy(splitPackedValue, splitDim*bytesPerDim, splitDimValue.bytes, splitDimValue.offset, bytesPerDim);
+      System.arraycopy(splitPackedValue, splitDim*config.bytesPerDim, splitDimValue.bytes, splitDimValue.offset, config.bytesPerDim);
 
       // Recurse on right sub-tree:
-      System.arraycopy(cellMinPacked, 0, splitPackedValue, 0, packedIndexBytesLength);
-      System.arraycopy(splitDimValue.bytes, splitDimValue.offset, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
+      System.arraycopy(cellMinPacked, 0, splitPackedValue, 0, config.packedIndexBytesLength);
+      System.arraycopy(splitDimValue.bytes, splitDimValue.offset, splitPackedValue, splitDim*config.bytesPerDim, config.bytesPerDim);
       state.index.pushRight();
       final long rightCost = estimatePointCount(state, splitPackedValue, cellMaxPacked);
       state.index.pop();
@@ -755,17 +743,17 @@ public final class BKDReader extends PointValues {
 
   @Override
   public int getNumDimensions() {
-    return numDataDims;
+    return config.numDims;
   }
 
   @Override
   public int getNumIndexDimensions() {
-    return numIndexDims;
+    return config.numIndexDims;
   }
 
   @Override
   public int getBytesPerDimension() {
-    return bytesPerDim;
+    return config.bytesPerDim;
   }
 
   @Override
@@ -798,7 +786,7 @@ public final class BKDReader extends PointValues {
 
     @Override
     public int docID() {
-     return docID;
+      return docID;
     }
 
     private void  reset(int offset, int length) {
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 dc24381..0a7f815 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
@@ -29,13 +29,12 @@ import org.apache.lucene.codecs.MutablePointValues;
 import org.apache.lucene.index.MergeState;
 import org.apache.lucene.index.PointValues.IntersectVisitor;
 import org.apache.lucene.index.PointValues.Relation;
+import org.apache.lucene.store.ByteBuffersDataOutput;
 import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.GrowableByteArrayDataOutput;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.store.TrackingDirectoryWrapper;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
@@ -59,20 +58,20 @@ import org.apache.lucene.util.PriorityQueue;
 /**
  *  Recursively builds a block KD-tree to assign all incoming points in N-dim space to smaller
  *  and smaller N-dim rectangles (cells) until the number of points in a given
- *  rectangle is &lt;= <code>maxPointsInLeafNode</code>.  The tree is
+ *  rectangle is &lt;= <code>config.maxPointsInLeafNode</code>.  The tree is
  *  partially balanced, which means the leaf nodes will have
- *  the requested <code>maxPointsInLeafNode</code> except one that might have less.
+ *  the requested <code>config.maxPointsInLeafNode</code> except one that might have less.
  *  Leaf nodes may straddle the two bottom levels of the binary tree.
  *  Values that fall exactly on a cell boundary may be in either cell.
  *
  *  <p>The number of dimensions can be 1 to 8, but every byte[] value is fixed length.
  *
  *  <p>This consumes heap during writing: it allocates a <code>Long[numLeaves]</code>,
- *  a <code>byte[numLeaves*(1+bytesPerDim)]</code> and then uses up to the specified
+ *  a <code>byte[numLeaves*(1+config.bytesPerDim)]</code> and then uses up to the specified
  *  {@code maxMBSortInHeap} heap space for writing.
  *
  *  <p>
- *  <b>NOTE</b>: This can write at most Integer.MAX_VALUE * <code>maxPointsInLeafNode</code> / bytesPerDim
+ *  <b>NOTE</b>: This can write at most Integer.MAX_VALUE * <code>config.maxPointsInLeafNode</code> / config.bytesPerDim
  *  total points.
  *
  * @lucene.experimental */
@@ -88,38 +87,14 @@ public class BKDWriter implements Closeable {
   public static final int VERSION_META_FILE = 9;
   public static final int VERSION_CURRENT = VERSION_META_FILE;
 
-  /** How many bytes each docs takes in the fixed-width offline format */
-  private final int bytesPerDoc;
-
-  /** Default maximum number of point in each leaf block */
-  public static final int DEFAULT_MAX_POINTS_IN_LEAF_NODE = 512;
-
-  /** Default maximum heap to use, before spilling to (slower) disk */
-  public static final float DEFAULT_MAX_MB_SORT_IN_HEAP = 16.0f;
-
-  /** Maximum number of index dimensions (2 * max index dimensions) */
-  public static final int MAX_DIMS = 16;
-
-  /** Maximum number of index dimensions */
-  public static final int MAX_INDEX_DIMS = 8;
 
   /** Number of splits before we compute the exact bounding box of an inner node. */
   private static final int SPLITS_BEFORE_EXACT_BOUNDS = 4;
+  /** Default maximum heap to use, before spilling to (slower) disk */
+  public static final float DEFAULT_MAX_MB_SORT_IN_HEAP = 16.0f;
 
-  /** 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 bytes each value in each dimension takes. */
-  protected final int bytesPerDim;
-
-  /** numDataDims * bytesPerDim */
-  protected final int packedBytesLength;
-
-  /** numIndexDims * bytesPerDim */
-  protected final int packedIndexBytesLength;
+  /** BKD tree configuration */
+  protected final BKDConfig config;
 
   final TrackingDirectoryWrapper tempDir;
   final String tempFileNamePrefix;
@@ -138,7 +113,6 @@ public class BKDWriter implements Closeable {
   private boolean finished;
 
   private IndexOutput tempInput;
-  protected final int maxPointsInLeafNode;
   private final int maxPointsSortInHeap;
 
   /** Minimum per-dim values, packed */
@@ -154,63 +128,42 @@ public class BKDWriter implements Closeable {
 
   private final int maxDoc;
 
-  public BKDWriter(int maxDoc, Directory tempDir, String tempFileNamePrefix, int numDataDims, int numIndexDims, int bytesPerDim,
-                   int maxPointsInLeafNode, double maxMBSortInHeap, long totalPointCount) throws IOException {
-    verifyParams(numDataDims, numIndexDims, maxPointsInLeafNode, maxMBSortInHeap, totalPointCount);
+  public BKDWriter(int maxDoc, Directory tempDir, String tempFileNamePrefix, BKDConfig config,
+                   double maxMBSortInHeap, long totalPointCount) {
+    verifyParams(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.bytesPerDim = bytesPerDim;
+    this.maxMBSortInHeap = maxMBSortInHeap;
+
     this.totalPointCount = totalPointCount;
     this.maxDoc = maxDoc;
-    docsSeen = new FixedBitSet(maxDoc);
-    packedBytesLength = numDataDims * bytesPerDim;
-    packedIndexBytesLength = numIndexDims * bytesPerDim;
 
-    scratchDiff = new byte[bytesPerDim];
-    scratch1 = new byte[packedBytesLength];
-    scratch2 = new byte[packedBytesLength];
-    commonPrefixLengths = new int[numDataDims];
+    this.config = config;
+
+    docsSeen = new FixedBitSet(maxDoc);
 
-    minPackedValue = new byte[packedIndexBytesLength];
-    maxPackedValue = new byte[packedIndexBytesLength];
+    scratchDiff = new byte[config.bytesPerDim];
+    scratch1 = new byte[config.packedBytesLength];
+    scratch2 = new byte[config.packedBytesLength];
+    commonPrefixLengths = new int[config.numDims];
 
-    // dimensional values (numDims * bytesPerDim) + docID (int)
-    bytesPerDoc = packedBytesLength + Integer.BYTES;
+    minPackedValue = new byte[config.packedIndexBytesLength];
+    maxPackedValue = new byte[config.packedIndexBytesLength];
 
     // Maximum number of points we hold in memory at any time
-    maxPointsSortInHeap = (int) ((maxMBSortInHeap * 1024 * 1024) / (bytesPerDoc));
+    maxPointsSortInHeap = (int) ((maxMBSortInHeap * 1024 * 1024) / (config.bytesPerDoc));
 
     // Finally, we must be able to hold at least the leaf node in heap during build:
-    if (maxPointsSortInHeap < maxPointsInLeafNode) {
-      throw new IllegalArgumentException("maxMBSortInHeap=" + maxMBSortInHeap + " only allows for maxPointsSortInHeap=" + maxPointsSortInHeap + ", but this is less than maxPointsInLeafNode=" + maxPointsInLeafNode + "; either increase maxMBSortInHeap or decrease maxPointsInLeafNode");
+    if (maxPointsSortInHeap < config.maxPointsInLeafNode) {
+      throw new IllegalArgumentException("maxMBSortInHeap=" + maxMBSortInHeap + " only allows for maxPointsSortInHeap="
+              + maxPointsSortInHeap + ", but this is less than maxPointsInLeafNode=" + config.maxPointsInLeafNode + "; "
+              + "either increase maxMBSortInHeap or decrease maxPointsInLeafNode");
     }
-
-    this.maxMBSortInHeap = maxMBSortInHeap;
   }
 
-  public static void verifyParams(int numDims, int numIndexDims, 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 (numDims < 1 || numDims > MAX_DIMS) {
-      throw new IllegalArgumentException("numDims must be 1 .. " + MAX_DIMS + " (got: " + numDims + ")");
-    }
-    if (numIndexDims < 1 || numIndexDims > MAX_INDEX_DIMS) {
-      throw new IllegalArgumentException("numIndexDims must be 1 .. " + MAX_INDEX_DIMS + " (got: " + numIndexDims + ")");
-    }
-    if (numIndexDims > numDims) {
-      throw new IllegalArgumentException("numIndexDims cannot exceed numDims (" + numDims + ") (got: " + numIndexDims + ")");
-    }
-    if (maxPointsInLeafNode <= 0) {
-      throw new IllegalArgumentException("maxPointsInLeafNode must be > 0; got " + maxPointsInLeafNode);
-    }
-    if (maxPointsInLeafNode > ArrayUtil.MAX_ARRAY_LENGTH) {
-      throw new IllegalArgumentException("maxPointsInLeafNode must be <= ArrayUtil.MAX_ARRAY_LENGTH (= " + ArrayUtil.MAX_ARRAY_LENGTH + "); got " + maxPointsInLeafNode);
-    }
+  private static void verifyParams(double maxMBSortInHeap, long totalPointCount) {
     if (maxMBSortInHeap < 0.0) {
       throw new IllegalArgumentException("maxMBSortInHeap must be >= 0.0 (got: " + maxMBSortInHeap + ")");
     }
@@ -221,35 +174,33 @@ public class BKDWriter implements Closeable {
 
   private void initPointWriter() throws IOException {
     assert pointWriter == null : "Point writer is already initialized";
-    //total point count is an estimation but the final point count must be equal or lower to that number.
+    // Total point count is an estimation but the final point count must be equal or lower to that number.
     if (totalPointCount > maxPointsSortInHeap) {
-      pointWriter = new OfflinePointWriter(tempDir, tempFileNamePrefix, packedBytesLength, "spill", 0);
+      pointWriter = new OfflinePointWriter(config, tempDir, tempFileNamePrefix, "spill", 0);
       tempInput = ((OfflinePointWriter)pointWriter).out;
     } else {
-      pointWriter = new HeapPointWriter(Math.toIntExact(totalPointCount), packedBytesLength);
+      pointWriter = new HeapPointWriter(config, Math.toIntExact(totalPointCount));
     }
   }
 
-
   public void add(byte[] packedValue, int docID) throws IOException {
-    if (packedValue.length != packedBytesLength) {
-      throw new IllegalArgumentException("packedValue should be length=" + packedBytesLength + " (got: " + packedValue.length + ")");
+    if (packedValue.length != config.packedBytesLength) {
+      throw new IllegalArgumentException("packedValue should be length=" + config.packedBytesLength + " (got: " + packedValue.length + ")");
     }
     if (pointCount >= totalPointCount) {
       throw new IllegalStateException("totalPointCount=" + totalPointCount + " was passed when we were created, but we just hit " + (pointCount + 1) + " values");
     }
     if (pointCount == 0) {
       initPointWriter();
-      System.arraycopy(packedValue, 0, minPackedValue, 0, packedIndexBytesLength);
-      System.arraycopy(packedValue, 0, maxPackedValue, 0, packedIndexBytesLength);
+      System.arraycopy(packedValue, 0, minPackedValue, 0, config.packedIndexBytesLength);
+      System.arraycopy(packedValue, 0, maxPackedValue, 0, config.packedIndexBytesLength);
     } else {
-      for(int dim=0;dim<numIndexDims;dim++) {
-        int offset = dim*bytesPerDim;
-        if (FutureArrays.compareUnsigned(packedValue, offset, offset + bytesPerDim, minPackedValue, offset, offset + bytesPerDim) < 0) {
-          System.arraycopy(packedValue, offset, minPackedValue, offset, bytesPerDim);
-        }
-        if (FutureArrays.compareUnsigned(packedValue, offset, offset + bytesPerDim, maxPackedValue, offset, offset + bytesPerDim) > 0) {
-          System.arraycopy(packedValue, offset, maxPackedValue, offset, bytesPerDim);
+      for(int dim=0;dim<config.numIndexDims;dim++) {
+        int offset = dim*config.bytesPerDim;
+        if (FutureArrays.compareUnsigned(packedValue, offset, offset + config.bytesPerDim, minPackedValue, offset, offset + config.bytesPerDim) < 0) {
+          System.arraycopy(packedValue, offset, minPackedValue, offset, config.bytesPerDim);
+        } else if (FutureArrays.compareUnsigned(packedValue, offset, offset + config.bytesPerDim, maxPackedValue, offset, offset + config.bytesPerDim) > 0) {
+          System.arraycopy(packedValue, offset, maxPackedValue, offset, config.bytesPerDim);
         }
       }
     }
@@ -280,15 +231,12 @@ 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.packedBytesLength,
-              bkd.packedIndexBytesLength,
-              bkd.maxPointsInLeafNode,
+              bkd.config,
               null,
               null);
       this.docMap = docMap;
       state.in.seek(bkd.getMinLeafBlockFP());
-      this.packedValues = new byte[bkd.maxPointsInLeafNode * bkd.packedBytesLength];
+      this.packedValues = new byte[bkd.config.maxPointsInLeafNode * bkd.config.packedBytesLength];
     }
 
     public boolean next() throws IOException {
@@ -314,7 +262,7 @@ public class BKDWriter implements Closeable {
             @Override
             public void visit(int docID, byte[] packedValue) {
               assert docID == state.scratchIterator.docIDs[i];
-              System.arraycopy(packedValue, 0, packedValues, i * bkd.packedBytesLength, bkd.packedBytesLength);
+              System.arraycopy(packedValue, 0, packedValues, i * bkd.config.packedBytesLength, bkd.config.packedBytesLength);
               i++;
             }
 
@@ -341,7 +289,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.config.packedBytesLength, state.scratchDataPackedValue, 0, bkd.config.packedBytesLength);
           return true;
         }
       }
@@ -394,7 +342,7 @@ public class BKDWriter implements Closeable {
    *  disk. This method does not use transient disk in order to reorder points.
    */
   public Runnable writeField(IndexOutput metaOut, IndexOutput indexOut, IndexOutput dataOut, String fieldName, MutablePointValues reader) throws IOException {
-    if (numDataDims == 1) {
+    if (config.numDims == 1) {
       return writeField1Dim(metaOut, indexOut, dataOut, fieldName, reader);
     } else {
       return writeFieldNDims(metaOut, indexOut, dataOut, fieldName, reader);
@@ -406,17 +354,17 @@ public class BKDWriter implements Closeable {
       return;
     }
     values.getValue(from, scratch);
-    System.arraycopy(scratch.bytes, scratch.offset, minPackedValue, 0, packedIndexBytesLength);
-    System.arraycopy(scratch.bytes, scratch.offset, maxPackedValue, 0, packedIndexBytesLength);
+    System.arraycopy(scratch.bytes, scratch.offset, minPackedValue, 0, config.packedIndexBytesLength);
+    System.arraycopy(scratch.bytes, scratch.offset, maxPackedValue, 0, config.packedIndexBytesLength);
     for (int i = from + 1 ; i < to; ++i) {
       values.getValue(i, scratch);
-      for(int dim = 0; dim < numIndexDims; dim++) {
-        final int startOffset = dim * bytesPerDim;
-        final int endOffset = startOffset + bytesPerDim;
+      for(int dim = 0; dim < config.numIndexDims; dim++) {
+        final int startOffset = dim * config.bytesPerDim;
+        final int endOffset = startOffset + config.bytesPerDim;
         if (FutureArrays.compareUnsigned(scratch.bytes, scratch.offset + startOffset, scratch.offset + endOffset, minPackedValue, startOffset, endOffset) < 0) {
-          System.arraycopy(scratch.bytes, scratch.offset + startOffset, minPackedValue, startOffset, bytesPerDim);
+          System.arraycopy(scratch.bytes, scratch.offset + startOffset, minPackedValue, startOffset, config.bytesPerDim);
         } else if (FutureArrays.compareUnsigned(scratch.bytes, scratch.offset + startOffset, scratch.offset + endOffset, maxPackedValue, startOffset, endOffset) > 0) {
-          System.arraycopy(scratch.bytes, scratch.offset + startOffset, maxPackedValue, startOffset, bytesPerDim);
+          System.arraycopy(scratch.bytes, scratch.offset + startOffset, maxPackedValue, startOffset, config.bytesPerDim);
         }
       }
     }
@@ -439,12 +387,12 @@ public class BKDWriter implements Closeable {
 
     pointCount = values.size();
 
-    final int numLeaves = Math.toIntExact((pointCount + maxPointsInLeafNode - 1) / maxPointsInLeafNode);
+    final int numLeaves = Math.toIntExact((pointCount + config.maxPointsInLeafNode - 1) / config.maxPointsInLeafNode);
     final int numSplits = numLeaves - 1;
 
     checkMaxLeafNodeCount(numLeaves);
 
-    final byte[] splitPackedValues = new byte[numSplits * bytesPerDim];
+    final byte[] splitPackedValues = new byte[numSplits * config.bytesPerDim];
     final byte[] splitDimensionValues = new byte[numSplits];
     final long[] leafBlockFPs = new long[numLeaves];
 
@@ -455,14 +403,14 @@ public class BKDWriter implements Closeable {
     }
 
     final long dataStartFP = dataOut.getFilePointer();
-    final int[] parentSplits = new int[numIndexDims];
+    final int[] parentSplits = new int[config.numIndexDims];
     build(0, numLeaves, values, 0, Math.toIntExact(pointCount), dataOut,
             minPackedValue.clone(), maxPackedValue.clone(), parentSplits,
             splitPackedValues, splitDimensionValues, leafBlockFPs,
-            new int[maxPointsInLeafNode]);
-    assert Arrays.equals(parentSplits, new int[numIndexDims]);
+            new int[config.maxPointsInLeafNode]);
+    assert Arrays.equals(parentSplits, new int[config.numIndexDims]);
 
-    scratchBytesRef1.length = bytesPerDim;
+    scratchBytesRef1.length = config.bytesPerDim;
     scratchBytesRef1.bytes = splitPackedValues;
 
     BKDTreeLeafNodes leafNodes  = new BKDTreeLeafNodes() {
@@ -473,7 +421,7 @@ public class BKDWriter implements Closeable {
 
       @Override
       public BytesRef getSplitValue(int index) {
-        scratchBytesRef1.offset = index * bytesPerDim;
+        scratchBytesRef1.offset = index * config.bytesPerDim;
         return scratchBytesRef1;
       }
 
@@ -490,7 +438,7 @@ public class BKDWriter implements Closeable {
 
     return () -> {
       try {
-        writeIndex(metaOut, indexOut, maxPointsInLeafNode, leafNodes, dataStartFP);
+        writeIndex(metaOut, indexOut, config.maxPointsInLeafNode, leafNodes, dataStartFP);
       } catch (IOException e) {
         throw new UncheckedIOException(e);
       }
@@ -500,7 +448,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 Runnable writeField1Dim(IndexOutput metaOut, IndexOutput indexOut, IndexOutput dataOut, String fieldName, MutablePointValues reader) throws IOException {
-    MutablePointsReaderUtils.sort(maxDoc, packedIndexBytesLength, reader, 0, Math.toIntExact(reader.size()));
+    MutablePointsReaderUtils.sort(config, maxDoc, reader, 0, Math.toIntExact(reader.size()));
 
     final OneDimensionBKDWriter oneDimWriter = new OneDimensionBKDWriter(metaOut, indexOut, dataOut);
 
@@ -512,7 +460,7 @@ public class BKDWriter implements Closeable {
       }
 
       @Override
-      public void visit(int docID) throws IOException {
+      public void visit(int docID) {
         throw new IllegalStateException();
       }
 
@@ -531,7 +479,7 @@ public class BKDWriter implements Closeable {
   public Runnable merge(IndexOutput metaOut, IndexOutput indexOut, IndexOutput dataOut, List<MergeState.DocMap> docMaps, List<BKDReader> readers) throws IOException {
     assert docMaps == null || readers.size() == docMaps.size();
 
-    BKDMergeQueue queue = new BKDMergeQueue(bytesPerDim, readers.size());
+    BKDMergeQueue queue = new BKDMergeQueue(config.bytesPerDim, readers.size());
 
     for(int i=0;i<readers.size();i++) {
       BKDReader bkd = readers.get(i);
@@ -566,8 +514,8 @@ public class BKDWriter implements Closeable {
     return oneDimWriter.finish();
   }
 
-  // reused when writing leaf blocks
-  private final GrowableByteArrayDataOutput scratchOut = new GrowableByteArrayDataOutput(32*1024);
+  // Reused when writing leaf blocks
+  private final ByteBuffersDataOutput scratchOut = ByteBuffersDataOutput.newResettableInstance();
 
   private class OneDimensionBKDWriter {
 
@@ -575,15 +523,15 @@ public class BKDWriter implements Closeable {
     final long dataStartFP;
     final List<Long> leafBlockFPs = new ArrayList<>();
     final List<byte[]> leafBlockStartValues = new ArrayList<>();
-    final byte[] leafValues = new byte[maxPointsInLeafNode * packedBytesLength];
-    final int[] leafDocs = new int[maxPointsInLeafNode];
+    final byte[] leafValues = new byte[config.maxPointsInLeafNode * config.packedBytesLength];
+    final int[] leafDocs = new int[config.maxPointsInLeafNode];
     private long valueCount;
     private int leafCount;
     private int leafCardinality;
 
     OneDimensionBKDWriter(IndexOutput metaOut, IndexOutput indexOut, IndexOutput dataOut) {
-      if (numIndexDims != 1) {
-        throw new UnsupportedOperationException("numIndexDims must be 1 but got " + numIndexDims);
+      if (config.numIndexDims != 1) {
+        throw new UnsupportedOperationException("config.numIndexDims must be 1 but got " + config.numIndexDims);
       }
       if (pointCount != 0) {
         throw new IllegalStateException("cannot mix add and merge");
@@ -602,7 +550,7 @@ public class BKDWriter implements Closeable {
       this.dataOut = dataOut;
       this.dataStartFP = dataOut.getFilePointer();
 
-      lastPackedValue = new byte[packedBytesLength];
+      lastPackedValue = new byte[config.packedBytesLength];
     }
 
     // for asserts
@@ -610,13 +558,13 @@ public class BKDWriter implements Closeable {
     private int lastDocID;
 
     void add(byte[] packedValue, int docID) throws IOException {
-      assert valueInOrder(valueCount + leafCount,
+      assert valueInOrder(config, valueCount + leafCount,
               0, lastPackedValue, packedValue, 0, docID, lastDocID);
 
-      if (leafCount == 0 || FutureArrays.mismatch(leafValues, (leafCount - 1) * bytesPerDim, leafCount * bytesPerDim, packedValue, 0, bytesPerDim) != -1) {
+      if (leafCount == 0 || FutureArrays.mismatch(leafValues, (leafCount - 1) * config.bytesPerDim, leafCount * config.bytesPerDim, packedValue, 0, config.bytesPerDim) != -1) {
         leafCardinality++;
       }
-      System.arraycopy(packedValue, 0, leafValues, leafCount * packedBytesLength, packedBytesLength);
+      System.arraycopy(packedValue, 0, leafValues, leafCount * config.packedBytesLength, config.packedBytesLength);
       leafDocs[leafCount] = docID;
       docsSeen.set(docID);
       leafCount++;
@@ -625,7 +573,7 @@ public class BKDWriter implements Closeable {
         throw new IllegalStateException("totalPointCount=" + totalPointCount + " was passed when we were created, but we just hit " + (valueCount + leafCount) + " values");
       }
 
-      if (leafCount == maxPointsInLeafNode) {
+      if (leafCount == config.maxPointsInLeafNode) {
         // We write a block once we hit exactly the max count ... this is different from
         // when we write N > 1 dimensional points where we write between max/2 and max per leaf block
         writeLeafBlock(leafCardinality);
@@ -649,7 +597,7 @@ public class BKDWriter implements Closeable {
 
       pointCount = valueCount;
 
-      scratchBytesRef1.length = bytesPerDim;
+      scratchBytesRef1.length = config.bytesPerDim;
       scratchBytesRef1.offset = 0;
       assert leafBlockStartValues.size() + 1 == leafBlockFPs.size();
       BKDTreeLeafNodes leafNodes = new BKDTreeLeafNodes() {
@@ -676,7 +624,7 @@ public class BKDWriter implements Closeable {
       };
       return () -> {
         try {
-          writeIndex(metaOut, indexOut, maxPointsInLeafNode, leafNodes, dataStartFP);
+          writeIndex(metaOut, indexOut, config.maxPointsInLeafNode, leafNodes, dataStartFP);
         } catch (IOException e) {
           throw new UncheckedIOException(e);
         }
@@ -686,47 +634,47 @@ public class BKDWriter implements Closeable {
     private void writeLeafBlock(int leafCardinality) throws IOException {
       assert leafCount != 0;
       if (valueCount == 0) {
-        System.arraycopy(leafValues, 0, minPackedValue, 0, packedIndexBytesLength);
+        System.arraycopy(leafValues, 0, minPackedValue, 0, config.packedIndexBytesLength);
       }
-      System.arraycopy(leafValues, (leafCount - 1) * packedBytesLength, maxPackedValue, 0, packedIndexBytesLength);
+      System.arraycopy(leafValues, (leafCount - 1) * config.packedBytesLength, maxPackedValue, 0, config.packedIndexBytesLength);
 
       valueCount += leafCount;
 
       if (leafBlockFPs.size() > 0) {
         // Save the first (minimum) value in each leaf block except the first, to build the split value index in the end:
-        leafBlockStartValues.add(ArrayUtil.copyOfSubArray(leafValues, 0, packedBytesLength));
+        leafBlockStartValues.add(ArrayUtil.copyOfSubArray(leafValues, 0, config.packedBytesLength));
       }
       leafBlockFPs.add(dataOut.getFilePointer());
       checkMaxLeafNodeCount(leafBlockFPs.size());
 
       // Find per-dim common prefix:
-      int offset = (leafCount - 1) * packedBytesLength;
-      int prefix = FutureArrays.mismatch(leafValues, 0, bytesPerDim, leafValues, offset, offset + bytesPerDim);
+      int offset = (leafCount - 1) * config.packedBytesLength;
+      int prefix = FutureArrays.mismatch(leafValues, 0, config.bytesPerDim, leafValues, offset, offset + config.bytesPerDim);
       if (prefix == -1) {
-        prefix = bytesPerDim;
+        prefix = config.bytesPerDim;
       }
 
       commonPrefixLengths[0] = prefix;
 
-      assert scratchOut.getPosition() == 0;
+      assert scratchOut.size() == 0;
       writeLeafBlockDocs(scratchOut, leafDocs, 0, leafCount);
       writeCommonPrefixes(scratchOut, commonPrefixLengths, leafValues);
 
-      scratchBytesRef1.length = packedBytesLength;
+      scratchBytesRef1.length = config.packedBytesLength;
       scratchBytesRef1.bytes = leafValues;
 
       final IntFunction<BytesRef> packedValues = new IntFunction<BytesRef>() {
         @Override
         public BytesRef apply(int i) {
-          scratchBytesRef1.offset = packedBytesLength * i;
+          scratchBytesRef1.offset = config.packedBytesLength * i;
           return scratchBytesRef1;
         }
       };
-      assert valuesInOrderAndBounds(leafCount, 0, ArrayUtil.copyOfSubArray(leafValues, 0, packedBytesLength),
-              ArrayUtil.copyOfSubArray(leafValues, (leafCount - 1) * packedBytesLength, leafCount * packedBytesLength),
+      assert valuesInOrderAndBounds(config, leafCount, 0, ArrayUtil.copyOfSubArray(leafValues, 0, config.packedBytesLength),
+              ArrayUtil.copyOfSubArray(leafValues, (leafCount - 1) * config.packedBytesLength, leafCount * config.packedBytesLength),
               packedValues, leafDocs, 0);
       writeLeafBlockPackedValues(scratchOut, commonPrefixLengths, leafCount, 0, packedValues, leafCardinality);
-      dataOut.writeBytes(scratchOut.getBytes(), 0, scratchOut.getPosition());
+      scratchOut.copyTo(dataOut);
       scratchOut.reset();
     }
   }
@@ -759,7 +707,7 @@ public class BKDWriter implements Closeable {
       int count = 0;
       while (r.next()) {
         byte[] v = r.packedValue();
-        System.out.println("      " + count + ": " + new BytesRef(v, dim*bytesPerDim, bytesPerDim));
+        System.out.println("      " + count + ": " + new BytesRef(v, dim*config.bytesPerDim, config.bytesPerDim));
         count++;
         if (count == slice.count) {
           break;
@@ -770,8 +718,8 @@ public class BKDWriter implements Closeable {
   */
 
   private void checkMaxLeafNodeCount(int numLeaves) {
-    if (bytesPerDim * (long) numLeaves > ArrayUtil.MAX_ARRAY_LENGTH) {
-      throw new IllegalStateException("too many nodes; increase maxPointsInLeafNode (currently " + maxPointsInLeafNode + ") and reindex");
+    if (config.bytesPerDim * (long) numLeaves > ArrayUtil.MAX_ARRAY_LENGTH) {
+      throw new IllegalStateException("too many nodes; increase config.maxPointsInLeafNode (currently " + config.maxPointsInLeafNode + ") and reindex");
     }
   }
 
@@ -800,8 +748,7 @@ public class BKDWriter implements Closeable {
     tempInput = null;
     pointWriter = null;
 
-
-    final int numLeaves = Math.toIntExact((pointCount + maxPointsInLeafNode - 1) / maxPointsInLeafNode);
+    final int numLeaves = Math.toIntExact((pointCount + config.maxPointsInLeafNode - 1) / config.maxPointsInLeafNode);
     final int numSplits = numLeaves - 1;
 
     checkMaxLeafNodeCount(numLeaves);
@@ -810,23 +757,23 @@ public class BKDWriter implements Closeable {
     // step of the recursion to recompute the split dim:
 
     // Indexed by nodeID, but first (root) nodeID is 1.  We do 1+ because the lead byte at each recursion says which dim we split on.
-    byte[] splitPackedValues = new byte[Math.toIntExact(numSplits*bytesPerDim)];
+    byte[] splitPackedValues = new byte[Math.toIntExact(numSplits*config.bytesPerDim)];
     byte[] splitDimensionValues = new byte[numSplits];
 
     // +1 because leaf count is power of 2 (e.g. 8), and innerNodeCount is power of 2 minus 1 (e.g. 7)
     long[] leafBlockFPs = new long[numLeaves];
 
     // Make sure the math above "worked":
-    assert pointCount / numLeaves <= maxPointsInLeafNode: "pointCount=" + pointCount + " numLeaves=" + numLeaves + " maxPointsInLeafNode=" + maxPointsInLeafNode;
+    assert pointCount / numLeaves <= config.maxPointsInLeafNode: "pointCount=" + pointCount + " numLeaves=" + numLeaves + " config.maxPointsInLeafNode=" + config.maxPointsInLeafNode;
 
     //We re-use the selector so we do not need to create an object every time.
-    BKDRadixSelector radixSelector = new BKDRadixSelector(numDataDims, numIndexDims, bytesPerDim, maxPointsSortInHeap, tempDir, tempFileNamePrefix);
+    BKDRadixSelector radixSelector = new BKDRadixSelector(config, maxPointsSortInHeap, tempDir, tempFileNamePrefix);
 
     final long dataStartFP = dataOut.getFilePointer();
     boolean success = false;
     try {
 
-      final int[] parentSplits = new int[numIndexDims];
+      final int[] parentSplits = new int[config.numIndexDims];
       build(0, numLeaves, points,
               dataOut, radixSelector,
               minPackedValue.clone(), maxPackedValue.clone(),
@@ -834,8 +781,8 @@ public class BKDWriter implements Closeable {
               splitPackedValues,
               splitDimensionValues,
               leafBlockFPs,
-              new int[maxPointsInLeafNode]);
-      assert Arrays.equals(parentSplits, new int[numIndexDims]);
+              new int[config.maxPointsInLeafNode]);
+      assert Arrays.equals(parentSplits, new int[config.numIndexDims]);
 
       // If no exception, we should have cleaned everything up:
       assert tempDir.getCreatedFiles().isEmpty();
@@ -850,7 +797,7 @@ public class BKDWriter implements Closeable {
     }
 
     scratchBytesRef1.bytes = splitPackedValues;
-    scratchBytesRef1.length = bytesPerDim;
+    scratchBytesRef1.length = config.bytesPerDim;
     BKDTreeLeafNodes leafNodes  = new BKDTreeLeafNodes() {
       @Override
       public long getLeafLP(int index) {
@@ -859,7 +806,7 @@ public class BKDWriter implements Closeable {
 
       @Override
       public BytesRef getSplitValue(int index) {
-        scratchBytesRef1.offset = index * bytesPerDim;
+        scratchBytesRef1.offset = index * config.bytesPerDim;
         return scratchBytesRef1;
       }
 
@@ -877,7 +824,7 @@ public class BKDWriter implements Closeable {
     return () -> {
       // Write index:
       try {
-        writeIndex(metaOut, indexOut, maxPointsInLeafNode, leafNodes, dataStartFP);
+        writeIndex(metaOut, indexOut, config.maxPointsInLeafNode, leafNodes, dataStartFP);
       } catch (IOException e) {
         throw new UncheckedIOException(e);
       }
@@ -887,13 +834,13 @@ public class BKDWriter implements Closeable {
   /** Packs the two arrays, representing a semi-balanced binary tree, into a compact byte[] structure. */
   private byte[] packIndex(BKDTreeLeafNodes leafNodes) throws IOException {
     /** Reused while packing the index */
-    RAMOutputStream writeBuffer = new RAMOutputStream();
+    ByteBuffersDataOutput writeBuffer = ByteBuffersDataOutput.newResettableInstance();
 
     // This is the "file" we append the byte[] to:
     List<byte[]> blocks = new ArrayList<>();
-    byte[] lastSplitValues = new byte[bytesPerDim * numIndexDims];
+    byte[] lastSplitValues = new byte[config.bytesPerDim * config.numIndexDims];
     //System.out.println("\npack index");
-    int totalSize = recursePackIndex(writeBuffer, leafNodes, 0l, blocks, lastSplitValues, new boolean[numIndexDims], false,
+    int totalSize = recursePackIndex(writeBuffer, leafNodes, 0l, blocks, lastSplitValues, new boolean[config.numIndexDims], false,
             0, leafNodes.numLeaves());
 
     // Compact the byte[] blocks into single byte index:
@@ -909,19 +856,17 @@ public class BKDWriter implements Closeable {
   }
 
   /** Appends the current contents of writeBuffer as another block on the growing in-memory file */
-  private int appendBlock(RAMOutputStream writeBuffer, List<byte[]> blocks) throws IOException {
-    int pos = Math.toIntExact(writeBuffer.getFilePointer());
-    byte[] bytes = new byte[pos];
-    writeBuffer.writeTo(bytes, 0);
+  private int appendBlock(ByteBuffersDataOutput writeBuffer, List<byte[]> blocks) {
+    byte[] block = writeBuffer.toArrayCopy();
+    blocks.add(block);
     writeBuffer.reset();
-    blocks.add(bytes);
-    return pos;
+    return block.length;
   }
 
   /**
    * lastSplitValues is per-dimension split value previously seen; we use this to prefix-code the split byte[] on each inner node
    */
-  private int recursePackIndex(RAMOutputStream writeBuffer, BKDTreeLeafNodes leafNodes, long minBlockFP, List<byte[]> blocks,
+  private int recursePackIndex(ByteBuffersDataOutput writeBuffer, BKDTreeLeafNodes leafNodes, long minBlockFP, List<byte[]> blocks,
                                byte[] lastSplitValues, boolean[] negativeDeltas, boolean isLeft, int leavesOffset, int numLeaves) throws IOException {
     if (numLeaves == 1) {
       if (isLeft) {
@@ -954,21 +899,21 @@ public class BKDWriter implements Closeable {
       BytesRef splitValue = leafNodes.getSplitValue(splitOffset);
       int address = splitValue.offset;
 
-      //System.out.println("recursePack inner nodeID=" + nodeID + " splitDim=" + splitDim + " splitValue=" + new BytesRef(splitPackedValues, address, bytesPerDim));
+      //System.out.println("recursePack inner nodeID=" + nodeID + " splitDim=" + splitDim + " splitValue=" + new BytesRef(splitPackedValues, address, config.bytesPerDim));
 
       // find common prefix with last split value in this dim:
-      int prefix = FutureArrays.mismatch(splitValue.bytes, address, address + bytesPerDim, lastSplitValues,
-              splitDim * bytesPerDim, splitDim * bytesPerDim + bytesPerDim);
+      int prefix = FutureArrays.mismatch(splitValue.bytes, address, address + config.bytesPerDim, lastSplitValues,
+              splitDim * config.bytesPerDim, splitDim * config.bytesPerDim + config.bytesPerDim);
       if (prefix == -1) {
-        prefix = bytesPerDim;
+        prefix = config.bytesPerDim;
       }
 
-      //System.out.println("writeNodeData nodeID=" + nodeID + " splitDim=" + splitDim + " numDims=" + numDims + " bytesPerDim=" + bytesPerDim + " prefix=" + prefix);
+      //System.out.println("writeNodeData nodeID=" + nodeID + " splitDim=" + splitDim + " numDims=" + numDims + " config.bytesPerDim=" + config.bytesPerDim + " prefix=" + prefix);
 
       int firstDiffByteDelta;
-      if (prefix < bytesPerDim) {
-        //System.out.println("  delta byte cur=" + Integer.toHexString(splitPackedValues[address+prefix]&0xFF) + " prev=" + Integer.toHexString(lastSplitValues[splitDim * bytesPerDim + prefix]&0xFF) + " negated?=" + negativeDeltas[splitDim]);
-        firstDiffByteDelta = (splitValue.bytes[address+prefix]&0xFF) - (lastSplitValues[splitDim * bytesPerDim + prefix]&0xFF);
+      if (prefix < config.bytesPerDim) {
+        //System.out.println("  delta byte cur=" + Integer.toHexString(splitPackedValues[address+prefix]&0xFF) + " prev=" + Integer.toHexString(lastSplitValues[splitDim * config.bytesPerDim + prefix]&0xFF) + " negated?=" + negativeDeltas[splitDim]);
+        firstDiffByteDelta = (splitValue.bytes[address+prefix]&0xFF) - (lastSplitValues[splitDim * config.bytesPerDim + prefix]&0xFF);
         if (negativeDeltas[splitDim]) {
           firstDiffByteDelta = -firstDiffByteDelta;
         }
@@ -979,15 +924,15 @@ 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+config.bytesPerDim) + prefix) * config.numIndexDims + splitDim;
 
       //System.out.println("  code=" + code);
-      //System.out.println("  splitValue=" + new BytesRef(splitPackedValues, address, bytesPerDim));
+      //System.out.println("  splitValue=" + new BytesRef(splitPackedValues, address, config.bytesPerDim));
 
       writeBuffer.writeVInt(code);
 
       // write the split value, prefix coded vs. our parent's split value:
-      int suffix = bytesPerDim - prefix;
+      int suffix = config.bytesPerDim - prefix;
       byte[] savSplitValue = new byte[suffix];
       if (suffix > 1) {
         writeBuffer.writeBytes(splitValue.bytes, address+prefix+1, suffix-1);
@@ -995,10 +940,10 @@ public class BKDWriter implements Closeable {
 
       byte[] cmp = lastSplitValues.clone();
 
-      System.arraycopy(lastSplitValues, splitDim * bytesPerDim + prefix, savSplitValue, 0, suffix);
+      System.arraycopy(lastSplitValues, splitDim * config.bytesPerDim + prefix, savSplitValue, 0, suffix);
 
       // copy our split value into lastSplitValues for our children to prefix-code against
-      System.arraycopy(splitValue.bytes, address+prefix, lastSplitValues, splitDim * bytesPerDim + prefix, suffix);
+      System.arraycopy(splitValue.bytes, address+prefix, lastSplitValues, splitDim * config.bytesPerDim + prefix, suffix);
 
       int numBytes = appendBlock(writeBuffer, blocks);
 
@@ -1020,9 +965,7 @@ public class BKDWriter implements Closeable {
         assert leftNumBytes == 0: "leftNumBytes=" + leftNumBytes;
       }
 
-      int numBytes2 = Math.toIntExact(writeBuffer.getFilePointer());
-      byte[] bytes2 = new byte[numBytes2];
-      writeBuffer.writeTo(bytes2, 0);
+      byte[] bytes2 = writeBuffer.toArrayCopy();
       writeBuffer.reset();
       // replace our placeholder:
       blocks.set(idxSav, bytes2);
@@ -1034,7 +977,7 @@ public class BKDWriter implements Closeable {
       negativeDeltas[splitDim] = savNegativeDelta;
 
       // restore lastSplitValues to what caller originally passed us:
-      System.arraycopy(savSplitValue, 0, lastSplitValues, splitDim * bytesPerDim + prefix, suffix);
+      System.arraycopy(savSplitValue, 0, lastSplitValues, splitDim * config.bytesPerDim + prefix, suffix);
 
       assert Arrays.equals(lastSplitValues, cmp);
 
@@ -1046,18 +989,18 @@ public class BKDWriter implements Closeable {
     byte[] packedIndex = packIndex(leafNodes);
     writeIndex(metaOut, indexOut, countPerLeaf, leafNodes.numLeaves(), packedIndex, dataStartFP);
   }
-  
+
   private void writeIndex(IndexOutput metaOut, IndexOutput indexOut, int countPerLeaf, int numLeaves, byte[] packedIndex, long dataStartFP) throws IOException {
     CodecUtil.writeHeader(metaOut, CODEC_NAME, VERSION_CURRENT);
-    metaOut.writeVInt(numDataDims);
-    metaOut.writeVInt(numIndexDims);
+    metaOut.writeVInt(config.numDims);
+    metaOut.writeVInt(config.numIndexDims);
     metaOut.writeVInt(countPerLeaf);
-    metaOut.writeVInt(bytesPerDim);
+    metaOut.writeVInt(config.bytesPerDim);
 
     assert numLeaves > 0;
     metaOut.writeVInt(numLeaves);
-    metaOut.writeBytes(minPackedValue, 0, packedIndexBytesLength);
-    metaOut.writeBytes(maxPackedValue, 0, packedIndexBytesLength);
+    metaOut.writeBytes(minPackedValue, 0, config.packedIndexBytesLength);
+    metaOut.writeBytes(maxPackedValue, 0, config.packedIndexBytesLength);
 
     metaOut.writeVLong(pointCount);
     metaOut.writeVInt(docsSeen.cardinality());
@@ -1071,20 +1014,20 @@ public class BKDWriter implements Closeable {
   }
 
   private void writeLeafBlockDocs(DataOutput out, int[] docIDs, int start, int count) throws IOException {
-    assert count > 0: "maxPointsInLeafNode=" + maxPointsInLeafNode;
+    assert count > 0: "config.maxPointsInLeafNode=" + config.maxPointsInLeafNode;
     out.writeVInt(count);
     DocIdsWriter.writeDocIds(docIDs, start, count, out);
   }
 
   private void writeLeafBlockPackedValues(DataOutput out, int[] commonPrefixLengths, int count, int sortedDim, IntFunction<BytesRef> packedValues, int leafCardinality) throws IOException {
     int prefixLenSum = Arrays.stream(commonPrefixLengths).sum();
-    if (prefixLenSum == packedBytesLength) {
+    if (prefixLenSum == config.packedBytesLength) {
       // all values in this block are equal
       out.writeByte((byte) -1);
     } else {
-      assert commonPrefixLengths[sortedDim] < bytesPerDim;
+      assert commonPrefixLengths[sortedDim] < config.bytesPerDim;
       // estimate if storing the values with cardinality is cheaper than storing all values.
-      int compressedByteOffset = sortedDim * bytesPerDim + commonPrefixLengths[sortedDim];
+      int compressedByteOffset = sortedDim * config.bytesPerDim + commonPrefixLengths[sortedDim];
       int highCardinalityCost;
       int lowCardinalityCost;
       if (count == leafCardinality) {
@@ -1102,9 +1045,9 @@ public class BKDWriter implements Closeable {
           i += runLen;
         }
         // Add cost of runLen compression
-        highCardinalityCost = count * (packedBytesLength - prefixLenSum - 1) + 2 * numRunLens;
+        highCardinalityCost = count * (config.packedBytesLength - prefixLenSum - 1) + 2 * numRunLens;
         // +1 is the byte needed for storing the cardinality
-        lowCardinalityCost = leafCardinality * (packedBytesLength - prefixLenSum + 1);
+        lowCardinalityCost = leafCardinality * (config.packedBytesLength - prefixLenSum + 1);
       }
       if (lowCardinalityCost <= highCardinalityCost) {
         out.writeByte((byte) -2);
@@ -1117,38 +1060,38 @@ public class BKDWriter implements Closeable {
   }
 
   private void writeLowCardinalityLeafBlockPackedValues(DataOutput out, int[] commonPrefixLengths, int count, IntFunction<BytesRef> packedValues) throws IOException {
-    if (numIndexDims != 1) {
+    if (config.numIndexDims != 1) {
       writeActualBounds(out, commonPrefixLengths, count, packedValues);
     }
     BytesRef value = packedValues.apply(0);
-    System.arraycopy(value.bytes, value.offset, scratch1, 0, packedBytesLength);
+    System.arraycopy(value.bytes, value.offset, scratch1, 0, config.packedBytesLength);
     int cardinality = 1;
     for (int i = 1; i < count; i++) {
       value = packedValues.apply(i);
-      for(int dim = 0; dim < numDataDims; dim++) {
-        final int start = dim * bytesPerDim + commonPrefixLengths[dim];
-        final int end = dim * bytesPerDim + bytesPerDim;
+      for(int dim = 0; dim < config.numDims; dim++) {
+        final int start = dim * config.bytesPerDim + commonPrefixLengths[dim];
+        final int end = dim * config.bytesPerDim + config.bytesPerDim;
         if (FutureArrays.mismatch(value.bytes, value.offset + start, value.offset + end, scratch1, start, end) != -1) {
           out.writeVInt(cardinality);
-          for (int j = 0; j < numDataDims; j++) {
-            out.writeBytes(scratch1, j * bytesPerDim + commonPrefixLengths[j], bytesPerDim - commonPrefixLengths[j]);
+          for (int j = 0; j < config.numDims; j++) {
+            out.writeBytes(scratch1, j * config.bytesPerDim + commonPrefixLengths[j], config.bytesPerDim - commonPrefixLengths[j]);
           }
-          System.arraycopy(value.bytes, value.offset, scratch1, 0, packedBytesLength);
+          System.arraycopy(value.bytes, value.offset, scratch1, 0, config.packedBytesLength);
           cardinality = 1;
           break;
-        } else if (dim == numDataDims - 1){
+        } else if (dim == config.numDims - 1){
           cardinality++;
         }
       }
     }
     out.writeVInt(cardinality);
-    for (int i = 0; i < numDataDims; i++) {
-      out.writeBytes(scratch1, i * bytesPerDim + commonPrefixLengths[i], bytesPerDim - commonPrefixLengths[i]);
+    for (int i = 0; i < config.numDims; i++) {
+      out.writeBytes(scratch1, i * config.bytesPerDim + commonPrefixLengths[i], config.bytesPerDim - commonPrefixLengths[i]);
     }
   }
 
   private void writeHighCardinalityLeafBlockPackedValues(DataOutput out, int[] commonPrefixLengths, int count, int sortedDim, IntFunction<BytesRef> packedValues, int compressedByteOffset) throws IOException {
-    if (numIndexDims != 1) {
+    if (config.numIndexDims != 1) {
       writeActualBounds(out, commonPrefixLengths, count, packedValues);
     }
     commonPrefixLengths[sortedDim]++;
@@ -1167,11 +1110,11 @@ 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 < config.numIndexDims; ++dim) {
       int commonPrefixLength = commonPrefixLengths[dim];
-      int suffixLength = bytesPerDim - commonPrefixLength;
+      int suffixLength = config.bytesPerDim - commonPrefixLength;
       if (suffixLength > 0) {
-        BytesRef[] minMax = computeMinMax(count, packedValues, dim * bytesPerDim + commonPrefixLength, suffixLength);
+        BytesRef[] minMax = computeMinMax(count, packedValues, dim * config.bytesPerDim + commonPrefixLength, suffixLength);
         BytesRef min = minMax[0];
         BytesRef max = minMax[1];
         out.writeBytes(min.bytes, min.offset, min.length);
@@ -1203,11 +1146,11 @@ public class BKDWriter implements Closeable {
   private void writeLeafBlockPackedValuesRange(DataOutput out, int[] commonPrefixLengths, int start, int end, IntFunction<BytesRef> packedValues) throws IOException {
     for (int i = start; i < end; ++i) {
       BytesRef ref = packedValues.apply(i);
-      assert ref.length == packedBytesLength;
+      assert ref.length == config.packedBytesLength;
 
-      for(int dim=0;dim<numDataDims;dim++) {
+      for(int dim=0;dim<config.numDims;dim++) {
         int prefix = commonPrefixLengths[dim];
-        out.writeBytes(ref.bytes, ref.offset + dim*bytesPerDim + prefix, bytesPerDim-prefix);
+        out.writeBytes(ref.bytes, ref.offset + dim*config.bytesPerDim + prefix, config.bytesPerDim-prefix);
       }
     }
   }
@@ -1227,10 +1170,10 @@ 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<config.numDims;dim++) {
       out.writeVInt(commonPrefixes[dim]);
-      //System.out.println(commonPrefixes[dim] + " of " + bytesPerDim);
-      out.writeBytes(packedValue, dim*bytesPerDim, commonPrefixes[dim]);
+      //System.out.println(commonPrefixes[dim] + " of " + config.bytesPerDim);
+      out.writeBytes(packedValue, dim*config.bytesPerDim, commonPrefixes[dim]);
     }
   }
 
@@ -1271,21 +1214,6 @@ public class BKDWriter implements Closeable {
     throw IOUtils.rethrowAlways(priorException);
   }
 
-  /** Called only in assert */
-  private boolean valueInBounds(BytesRef packedValue, byte[] minPackedValue, byte[] maxPackedValue) {
-    for(int dim=0;dim<numIndexDims;dim++) {
-      int offset = bytesPerDim*dim;
-      if (FutureArrays.compareUnsigned(packedValue.bytes, packedValue.offset + offset, packedValue.offset + offset + bytesPerDim, minPackedValue, offset, offset + bytesPerDim) < 0) {
-        return false;
-      }
-      if (FutureArrays.compareUnsigned(packedValue.bytes, packedValue.offset + offset, packedValue.offset + offset + bytesPerDim, maxPackedValue, offset, offset + bytesPerDim) > 0) {
-        return false;
-      }
-    }
-
-    return true;
-  }
-
   /**
    * Pick the next dimension to split.
    * @param minPackedValue the min values for all dimensions
@@ -1301,20 +1229,20 @@ public class BKDWriter implements Closeable {
     for (int numSplits : parentSplits) {
       maxNumSplits = Math.max(maxNumSplits, numSplits);
     }
-    for (int dim = 0; dim < numIndexDims; ++dim) {
-      final int offset = dim * bytesPerDim;
+    for (int dim = 0; dim < config.numIndexDims; ++dim) {
+      final int offset = dim * config.bytesPerDim;
       if (parentSplits[dim] < maxNumSplits / 2 &&
-              FutureArrays.compareUnsigned(minPackedValue, offset, offset + bytesPerDim, maxPackedValue, offset, offset + bytesPerDim) != 0) {
+              FutureArrays.compareUnsigned(minPackedValue, offset, offset + config.bytesPerDim, maxPackedValue, offset, offset + config.bytesPerDim) != 0) {
         return dim;
       }
     }
 
     // Find which dim has the largest span so we can split on it:
     int splitDim = -1;
-    for(int dim=0;dim<numIndexDims;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);
+    for(int dim=0;dim<config.numIndexDims;dim++) {
+      NumericUtils.subtract(config.bytesPerDim, dim, maxPackedValue, minPackedValue, scratchDiff);
+      if (splitDim == -1 || FutureArrays.compareUnsigned(scratchDiff, 0, config.bytesPerDim, scratch1, 0, config.bytesPerDim) > 0) {
+        System.arraycopy(scratchDiff, 0, scratch1, 0, config.bytesPerDim);
         splitDim = dim;
       }
     }
@@ -1327,7 +1255,7 @@ public class BKDWriter implements Closeable {
   private HeapPointWriter switchToHeap(PointWriter source) throws IOException {
     int count = Math.toIntExact(source.count());
     try (PointReader reader = source.getReader(0, source.count());
-         HeapPointWriter writer = new HeapPointWriter(count, packedBytesLength)) {
+         HeapPointWriter writer = new HeapPointWriter(config, count)) {
       for(int i=0;i<count;i++) {
         boolean hasNext = reader.next();
         assert hasNext;
@@ -1355,15 +1283,15 @@ public class BKDWriter implements Closeable {
     if (numLeaves == 1) {
       // leaf node
       final int count = to - from;
-      assert count <= maxPointsInLeafNode;
+      assert count <= config.maxPointsInLeafNode;
 
       // Compute common prefixes
-      Arrays.fill(commonPrefixLengths, bytesPerDim);
+      Arrays.fill(commonPrefixLengths, config.bytesPerDim);
       reader.getValue(from, scratchBytesRef1);
       for (int i = from + 1; i < to; ++i) {
         reader.getValue(i, scratchBytesRef2);
-        for (int dim=0;dim<numDataDims;dim++) {
-          final int offset = dim * bytesPerDim;
+        for (int dim=0;dim<config.numDims;dim++) {
+          final int offset = dim * config.bytesPerDim;
           int dimensionPrefixLength = commonPrefixLengths[dim];
           commonPrefixLengths[dim] = FutureArrays.mismatch(scratchBytesRef1.bytes, scratchBytesRef1.offset + offset,
                   scratchBytesRef1.offset + offset + dimensionPrefixLength,
@@ -1376,23 +1304,23 @@ 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) {
-        if (commonPrefixLengths[dim] < bytesPerDim) {
+      FixedBitSet[] usedBytes = new FixedBitSet[config.numDims];
+      for (int dim = 0; dim < config.numDims; ++dim) {
+        if (commonPrefixLengths[dim] < config.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<config.numDims;dim++) {
           if (usedBytes[dim] != null) {
-            byte b = reader.getByteAt(i, dim * bytesPerDim + commonPrefixLengths[dim]);
+            byte b = reader.getByteAt(i, dim * config.bytesPerDim + commonPrefixLengths[dim]);
             usedBytes[dim].set(Byte.toUnsignedInt(b));
           }
         }
       }
       int sortedDim = 0;
       int sortedDimCardinality = Integer.MAX_VALUE;
-      for (int dim = 0; dim < numDataDims; ++dim) {
+      for (int dim = 0; dim < config.numDims; ++dim) {
         if (usedBytes[dim] != null) {
           final int cardinality = usedBytes[dim].cardinality();
           if (cardinality < sortedDimCardinality) {
@@ -1403,7 +1331,7 @@ public class BKDWriter implements Closeable {
       }
 
       // sort by sortedDim
-      MutablePointsReaderUtils.sortByDim(numDataDims, numIndexDims, sortedDim, bytesPerDim, commonPrefixLengths,
+      MutablePointsReaderUtils.sortByDim(config, sortedDim, commonPrefixLengths,
               reader, from, to, scratchBytesRef1, scratchBytesRef2);
 
       BytesRef comparator = scratchBytesRef1;
@@ -1412,9 +1340,9 @@ public class BKDWriter implements Closeable {
       int leafCardinality = 1;
       for (int i = from + 1; i < to; ++i) {
         reader.getValue(i, collector);
-        for (int dim =0; dim < numDataDims; dim++) {
-          final int start = dim * bytesPerDim + commonPrefixLengths[dim];
-          final int end = dim * bytesPerDim + bytesPerDim;
+        for (int dim =0; dim < config.numDims; dim++) {
+          final int start = dim * config.bytesPerDim + commonPrefixLengths[dim];
+          final int end = dim * config.bytesPerDim + config.bytesPerDim;
           if (FutureArrays.mismatch(collector.bytes, collector.offset + start, collector.offset + end,
                   comparator.bytes, comparator.offset + start, comparator.offset + end) != -1) {
             leafCardinality++;
@@ -1428,7 +1356,7 @@ public class BKDWriter implements Closeable {
       // Save the block file pointer:
       leafBlockFPs[leavesOffset] = out.getFilePointer();
 
-      assert scratchOut.getPosition() == 0;
+      assert scratchOut.size() == 0;
 
       // Write doc IDs
       int[] docIDs = spareDocIds;
@@ -1440,7 +1368,7 @@ public class BKDWriter implements Closeable {
 
       // Write the common prefixes:
       reader.getValue(from, scratchBytesRef1);
-      System.arraycopy(scratchBytesRef1.bytes, scratchBytesRef1.offset, scratch1, 0, packedBytesLength);
+      System.arraycopy(scratchBytesRef1.bytes, scratchBytesRef1.offset, scratch1, 0, config.packedBytesLength);
       writeCommonPrefixes(scratchOut, commonPrefixLengths, scratch1);
 
       // Write the full values:
@@ -1451,25 +1379,23 @@ public class BKDWriter implements Closeable {
           return scratchBytesRef1;
         }
       };
-      assert valuesInOrderAndBounds(count, sortedDim, minPackedValue, maxPackedValue, packedValues,
+      assert valuesInOrderAndBounds(config, count, sortedDim, minPackedValue, maxPackedValue, packedValues,
               docIDs, 0);
       writeLeafBlockPackedValues(scratchOut, commonPrefixLengths, count, sortedDim, packedValues, leafCardinality);
-
-      out.writeBytes(scratchOut.getBytes(), 0, scratchOut.getPosition());
+      scratchOut.copyTo(out);
       scratchOut.reset();
-
     } else {
       // inner node
 
       final int splitDim;
       // compute the split dimension and partition around it
-      if (numIndexDims == 1) {
+      if (config.numIndexDims == 1) {
         splitDim = 0;
       } else {
         // for dimensions > 2 we recompute the bounds for the current inner node to help the algorithm choose best
         // split dimensions. Because it is an expensive operation, the frequency we recompute the bounds is given
         // by SPLITS_BEFORE_EXACT_BOUNDS.
-        if (numLeaves != leafBlockFPs.length && numIndexDims > 2 && Arrays.stream(parentSplits).sum() % SPLITS_BEFORE_EXACT_BOUNDS == 0) {
+        if (numLeaves != leafBlockFPs.length && config.numIndexDims > 2 && Arrays.stream(parentSplits).sum() % SPLITS_BEFORE_EXACT_BOUNDS == 0) {
           computePackedValueBounds(reader, from, to, minPackedValue, maxPackedValue, scratchBytesRef1);
         }
         splitDim = split(minPackedValue, maxPackedValue, parentSplits);
@@ -1478,32 +1404,32 @@ public class BKDWriter implements Closeable {
       // How many leaves will be in the left tree:
       int numLeftLeafNodes = getNumLeftLeafNodes(numLeaves);
       // How many points will be in the left tree:
-      final int mid = from + numLeftLeafNodes * maxPointsInLeafNode;
+      final int mid = from + numLeftLeafNodes * config.maxPointsInLeafNode;
 
-      int commonPrefixLen = FutureArrays.mismatch(minPackedValue, splitDim * bytesPerDim,
-              splitDim * bytesPerDim + bytesPerDim, maxPackedValue, splitDim * bytesPerDim,
-              splitDim * bytesPerDim + bytesPerDim);
+      int commonPrefixLen = FutureArrays.mismatch(minPackedValue, splitDim * config.bytesPerDim,
+              splitDim * config.bytesPerDim + config.bytesPerDim, maxPackedValue, splitDim * config.bytesPerDim,
+              splitDim * config.bytesPerDim + config.bytesPerDim);
       if (commonPrefixLen == -1) {
-        commonPrefixLen = bytesPerDim;
+        commonPrefixLen = config.bytesPerDim;
       }
 
-      MutablePointsReaderUtils.partition(numDataDims, numIndexDims, maxDoc, splitDim, bytesPerDim, commonPrefixLen,
+      MutablePointsReaderUtils.partition(config, maxDoc, splitDim, commonPrefixLen,
               reader, from, to, mid, scratchBytesRef1, scratchBytesRef2);
 
       final int rightOffset = leavesOffset + numLeftLeafNodes;
       final int splitOffset = rightOffset - 1;
       // set the split value
-      final int address = splitOffset * bytesPerDim;
+      final int address = splitOffset * config.bytesPerDim;
       splitDimensionValues[splitOffset] = (byte) splitDim;
       reader.getValue(mid, scratchBytesRef1);
-      System.arraycopy(scratchBytesRef1.bytes, scratchBytesRef1.offset + splitDim * bytesPerDim, splitPackedValues, address, bytesPerDim);
+      System.arraycopy(scratchBytesRef1.bytes, scratchBytesRef1.offset + splitDim * config.bytesPerDim, splitPackedValues, address, config.bytesPerDim);
 
-      byte[] minSplitPackedValue = ArrayUtil.copyOfSubArray(minPackedValue, 0, packedIndexBytesLength);
-      byte[] maxSplitPackedValue = ArrayUtil.copyOfSubArray(maxPackedValue, 0, packedIndexBytesLength);
-      System.arraycopy(scratchBytesRef1.bytes, scratchBytesRef1.offset + splitDim * bytesPerDim,
-              minSplitPackedValue, splitDim * bytesPerDim, bytesPerDim);
-      System.arraycopy(scratchBytesRef1.bytes, scratchBytesRef1.offset + splitDim * bytesPerDim,
-              maxSplitPackedValue, splitDim * bytesPerDim, bytesPerDim);
+      byte[] minSplitPackedValue = ArrayUtil.copyOfSubArray(minPackedValue, 0, config.packedIndexBytesLength);
+      byte[] maxSplitPackedValue = ArrayUtil.copyOfSubArray(maxPackedValue, 0, config.packedIndexBytesLength);
+      System.arraycopy(scratchBytesRef1.bytes, scratchBytesRef1.offset + splitDim * config.bytesPerDim,
+              minSplitPackedValue, splitDim * config.bytesPerDim, config.bytesPerDim);
+      System.arraycopy(scratchBytesRef1.bytes, scratchBytesRef1.offset + splitDim * config.bytesPerDim,
+              maxSplitPackedValue, splitDim * config.bytesPerDim, config.bytesPerDim);
 
       // recurse
       parentSplits[splitDim]++;
@@ -1517,23 +1443,24 @@ public class BKDWriter implements Closeable {
     }
   }
 
+
   private void computePackedValueBounds(BKDRadixSelector.PathSlice slice, byte[] minPackedValue, byte[] maxPackedValue) throws IOException {
     try (PointReader reader = slice.writer.getReader(slice.start, slice.count)) {
       if (reader.next() == false) {
         return;
       }
       BytesRef value = reader.pointValue().packedValue();
-      System.arraycopy(value.bytes, value.offset, minPackedValue, 0, packedIndexBytesLength);
-      System.arraycopy(value.bytes, value.offset, maxPackedValue, 0, packedIndexBytesLength);
+      System.arraycopy(value.bytes, value.offset, minPackedValue, 0, config.packedIndexBytesLength);
+      System.arraycopy(value.bytes, value.offset, maxPackedValue, 0, config.packedIndexBytesLength);
       while (reader.next()) {
         value = reader.pointValue().packedValue();
-        for(int dim = 0; dim < numIndexDims; dim++) {
-          final int startOffset = dim * bytesPerDim;
-          final int endOffset = startOffset + bytesPerDim;
+        for(int dim = 0; dim < config.numIndexDims; dim++) {
+          final int startOffset = dim * config.bytesPerDim;
+          final int endOffset = startOffset + config.bytesPerDim;
           if (FutureArrays.compareUnsigned(value.bytes, value.offset + startOffset, value.offset + endOffset, minPackedValue, startOffset, endOffset) < 0) {
-            System.arraycopy(value.bytes, value.offset + startOffset, minPackedValue, startOffset, bytesPerDim);
+            System.arraycopy(value.bytes, value.offset + startOffset, minPackedValue, startOffset, config.bytesPerDim);
           } else if (FutureArrays.compareUnsigned(value.bytes, value.offset + startOffset, value.offset + endOffset, maxPackedValue, startOffset, endOffset) > 0) {
-            System.arraycopy(value.bytes, value.offset + startOffset, maxPackedValue, startOffset, bytesPerDim);
+            System.arraycopy(value.bytes, value.offset + startOffset, maxPackedValue, startOffset, config.bytesPerDim);
           }
         }
       }
@@ -1573,17 +1500,17 @@ public class BKDWriter implements Closeable {
 
       int sortedDim = 0;
       int sortedDimCardinality = Integer.MAX_VALUE;
-      FixedBitSet[] usedBytes = new FixedBitSet[numDataDims];
-      for (int dim = 0; dim < numDataDims; ++dim) {
-        if (commonPrefixLengths[dim] < bytesPerDim) {
+      FixedBitSet[] usedBytes = new FixedBitSet[config.numDims];
+      for (int dim = 0; dim < config.numDims; ++dim) {
+        if (commonPrefixLengths[dim] < config.bytesPerDim) {
           usedBytes[dim] = new FixedBitSet(256);
         }
       }
       //Find the dimension to compress
-      for (int dim = 0; dim < numDataDims; dim++) {
+      for (int dim = 0; dim < config.numDims; dim++) {
         int prefix = commonPrefixLengths[dim];
-        if (prefix < bytesPerDim) {
-          int offset = dim * bytesPerDim;
+        if (prefix < config.bytesPerDim) {
+          int offset = dim * config.bytesPerDim;
           for (int i = from; i < to; ++i) {
             PointValue value = heapSource.getPackedValueSlice(i);
             BytesRef packedValue = value.packedValue();
@@ -1601,7 +1528,7 @@ public class BKDWriter implements Closeable {
       // sort the chosen dimension
       radixSelector.heapRadixSort(heapSource, from, to, sortedDim, commonPrefixLengths[sortedDim]);
       // compute cardinality
-      int leafCardinality = heapSource.computeCardinality(from ,to, numDataDims, bytesPerDim, commonPrefixLengths);
+      int leafCardinality = heapSource.computeCardinality(from, to, commonPrefixLengths);
 
       // Save the block file pointer:
       leafBlockFPs[leavesOffset] = out.getFilePointer();
@@ -1630,7 +1557,7 @@ public class BKDWriter implements Closeable {
         final BytesRef scratch = new BytesRef();
 
         {
-          scratch.length = packedBytesLength;
+          scratch.length = config.packedBytesLength;
         }
 
         @Override
@@ -1639,7 +1566,7 @@ public class BKDWriter implements Closeable {
           return value.packedValue();
         }
       };
-      assert valuesInOrderAndBounds(count, sortedDim, minPackedValue, maxPackedValue, packedValues,
+      assert valuesInOrderAndBounds(config, count, sortedDim, minPackedValue, maxPackedValue, packedValues,
               docIDs, 0);
       writeLeafBlockPackedValues(out, commonPrefixLengths, count, sortedDim, packedValues, leafCardinality);
 
@@ -1647,13 +1574,13 @@ public class BKDWriter implements Closeable {
       // Inner node: partition/recurse
 
       final int splitDim;
-      if (numIndexDims == 1) {
+      if (config.numIndexDims == 1) {
         splitDim = 0;
       } else {
         // for dimensions > 2 we recompute the bounds for the current inner node to help the algorithm choose best
         // split dimensions. Because it is an expensive operation, the frequency we recompute the bounds is given
         // by SPLITS_BEFORE_EXACT_BOUNDS.
-        if (numLeaves != leafBlockFPs.length && numIndexDims > 2 && Arrays.stream(parentSplits).sum() % SPLITS_BEFORE_EXACT_BOUNDS == 0) {
+        if (numLeaves != leafBlockFPs.length && config.numIndexDims > 2 && Arrays.stream(parentSplits).sum() % SPLITS_BEFORE_EXACT_BOUNDS == 0) {
           computePackedValueBounds(points, minPackedValue, maxPackedValue);
         }
         splitDim = split(minPackedValue, maxPackedValue, parentSplits);
@@ -1662,17 +1589,17 @@ public class BKDWriter implements Closeable {
       assert numLeaves <= leafBlockFPs.length : "numLeaves=" + numLeaves + " leafBlockFPs.length=" + leafBlockFPs.length;
 
       // How many leaves will be in the left tree:
-      int numLeftLeafNodes = getNumLeftLeafNodes(numLeaves);
+      final int numLeftLeafNodes = getNumLeftLeafNodes(numLeaves);
       // How many points will be in the left tree:
-      final long leftCount = numLeftLeafNodes * maxPointsInLeafNode;
+      final long leftCount = numLeftLeafNodes * config.maxPointsInLeafNode;
 
       BKDRadixSelector.PathSlice[] slices = new BKDRadixSelector.PathSlice[2];
 
-      int commonPrefixLen = FutureArrays.mismatch(minPackedValue, splitDim * bytesPerDim,
-              splitDim * bytesPerDim + bytesPerDim, maxPackedValue, splitDim * bytesPerDim,
-              splitDim * bytesPerDim + bytesPerDim);
+      int commonPrefixLen = FutureArrays.mismatch(minPackedValue, splitDim * config.bytesPerDim,
+              splitDim * config.bytesPerDim + config.bytesPerDim, maxPackedValue, splitDim * config.bytesPerDim,
+              splitDim * config.bytesPerDim + config.bytesPerDim);
       if (commonPrefixLen == -1) {
-        commonPrefixLen = bytesPerDim;
+        commonPrefixLen = config.bytesPerDim;
       }
 
       byte[] splitValue = radixSelector.select(points, slices, points.start, points.start + points.count,  points.start + leftCount, splitDim, commonPrefixLen);
@@ -1681,17 +1608,17 @@ public class BKDWriter implements Closeable {
       final int splitValueOffset = rightOffset - 1;
 
       splitDimensionValues[splitValueOffset] = (byte) splitDim;
-      int address = splitValueOffset * bytesPerDim;
-      System.arraycopy(splitValue, 0, splitPackedValues, address, bytesPerDim);
+      int address = splitValueOffset * config.bytesPerDim;
+      System.arraycopy(splitValue, 0, splitPackedValues, address, config.bytesPerDim);
 
-      byte[] minSplitPackedValue = new byte[packedIndexBytesLength];
-      System.arraycopy(minPackedValue, 0, minSplitPackedValue, 0, packedIndexBytesLength);
+      byte[] minSplitPackedValue = new byte[config.packedIndexBytesLength];
+      System.arraycopy(minPackedValue, 0, minSplitPackedValue, 0, config.packedIndexBytesLength);
 
-      byte[] maxSplitPackedValue = new byte[packedIndexBytesLength];
-      System.arraycopy(maxPackedValue, 0, maxSplitPackedValue, 0, packedIndexBytesLength);
+      byte[] maxSplitPackedValue = new byte[config.packedIndexBytesLength];
+      System.arraycopy(maxPackedValue, 0, maxSplitPackedValue, 0, config.packedIndexBytesLength);
 
-      System.arraycopy(splitValue, 0, minSplitPackedValue, splitDim * bytesPerDim, bytesPerDim);
-      System.arraycopy(splitValue, 0, maxSplitPackedValue, splitDim * bytesPerDim, bytesPerDim);
+      System.arraycopy(splitValue, 0, minSplitPackedValue, splitDim * config.bytesPerDim, config.bytesPerDim);
+      System.arraycopy(splitValue, 0, maxSplitPackedValue, splitDim * config.bytesPerDim, config.bytesPerDim);
 
       parentSplits[splitDim]++;
       // Recurse on left tree:
@@ -1709,18 +1636,18 @@ public class BKDWriter implements Closeable {
   }
 
   private void computeCommonPrefixLength(HeapPointWriter heapPointWriter, byte[] commonPrefix, int from, int to) {
-    Arrays.fill(commonPrefixLengths, bytesPerDim);
+    Arrays.fill(commonPrefixLengths, config.bytesPerDim);
     PointValue value = heapPointWriter.getPackedValueSlice(from);
     BytesRef packedValue = value.packedValue();
-    for (int dim = 0; dim < numDataDims; dim++) {
-      System.arraycopy(packedValue.bytes, packedValue.offset + dim * bytesPerDim, commonPrefix, dim * bytesPerDim, bytesPerDim);
+    for (int dim = 0; dim < config.numDims; dim++) {
+      System.arraycopy(packedValue.bytes, packedValue.offset + dim * config.bytesPerDim, commonPrefix, dim * config.bytesPerDim, config.bytesPerDim);
     }
     for (int i = from + 1; i < to; i++) {
       value =  heapPointWriter.getPackedValueSlice(i);
       packedValue = value.packedValue();
-      for (int dim = 0; dim < numDataDims; dim++) {
+      for (int dim = 0; dim < config.numDims; dim++) {
         if (commonPrefixLengths[dim] != 0) {
-          int j = FutureArrays.mismatch(commonPrefix, dim * bytesPerDim, dim * bytesPerDim + commonPrefixLengths[dim], packedValue.bytes, packedValue.offset + dim * bytesPerDim, packedValue.offset + dim * bytesPerDim + commonPrefixLengths[dim]);
+          int j = FutureArrays.mismatch(commonPrefix, dim * config.bytesPerDim, dim * config.bytesPerDim + commonPrefixLengths[dim], packedValue.bytes, packedValue.offset + dim * config.bytesPerDim, packedValue.offset + dim * config.bytesPerDim + commonPrefixLengths[dim]);
           if (j != -1) {
             commonPrefixLengths[dim] = j;
           }
@@ -1730,44 +1657,58 @@ public class BKDWriter implements Closeable {
   }
 
   // only called from assert
-  private boolean valuesInOrderAndBounds(int count, int sortedDim, byte[] minPackedValue, byte[] maxPackedValue,
-                                         IntFunction<BytesRef> values, int[] docs, int docsOffset) throws IOException {
-    byte[] lastPackedValue = new byte[packedBytesLength];
+  private static boolean valuesInOrderAndBounds(BKDConfig config, int count, int sortedDim, byte[] minPackedValue, byte[] maxPackedValue,
+                                                IntFunction<BytesRef> values, int[] docs, int docsOffset) {
+    byte[] lastPackedValue = new byte[config.packedBytesLength];
     int lastDoc = -1;
     for (int i=0;i<count;i++) {
       BytesRef packedValue = values.apply(i);
-      assert packedValue.length == packedBytesLength;
-      assert valueInOrder(i, sortedDim, lastPackedValue, packedValue.bytes, packedValue.offset,
+      assert packedValue.length == config.packedBytesLength;
+      assert valueInOrder(config, i, sortedDim, lastPackedValue, packedValue.bytes, packedValue.offset,
               docs[docsOffset + i], lastDoc);
       lastDoc = docs[docsOffset + i];
 
       // Make sure this value does in fact fall within this leaf cell:
-      assert valueInBounds(packedValue, minPackedValue, maxPackedValue);
+      assert valueInBounds(config, packedValue, minPackedValue, maxPackedValue);
     }
     return true;
   }
 
   // only called from assert
-  private boolean valueInOrder(long ord, int sortedDim, byte[] lastPackedValue, byte[] packedValue, int packedValueOffset,
-                               int doc, int lastDoc) {
-    int dimOffset = sortedDim * bytesPerDim;
+  private static boolean valueInOrder(BKDConfig config, long ord, int sortedDim, byte[] lastPackedValue, byte[] packedValue, int packedValueOffset,
+                                      int doc, int lastDoc) {
+    int dimOffset = sortedDim * config.bytesPerDim;
     if (ord > 0) {
-      int cmp = FutureArrays.compareUnsigned(lastPackedValue, dimOffset, dimOffset + bytesPerDim, packedValue, packedValueOffset + dimOffset, packedValueOffset + dimOffset + bytesPerDim);
+      int cmp = FutureArrays.compareUnsigned(lastPackedValue, dimOffset, dimOffset + config.bytesPerDim, packedValue, packedValueOffset + dimOffset, packedValueOffset + dimOffset + config.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);
+        throw new AssertionError("values out of order: last value=" + new BytesRef(lastPackedValue) + " current value=" + new BytesRef(packedValue, packedValueOffset, config.packedBytesLength) + " ord=" + ord);
       }
-      if (cmp == 0  && numDataDims > numIndexDims) {
-        int dataOffset = numIndexDims * bytesPerDim;
-        cmp = FutureArrays.compareUnsigned(lastPackedValue, dataOffset, packedBytesLength, packedValue, packedValueOffset + dataOffset, packedValueOffset + packedBytesLength);
+      if (cmp == 0  && config.numDims > config.numIndexDims) {
+        cmp = FutureArrays.compareUnsigned(lastPackedValue, config.packedIndexBytesLength, config.packedBytesLength, packedValue, packedValueOffset + config.packedIndexBytesLength, packedValueOffset + config.packedBytesLength);
         if (cmp > 0) {
-          throw new AssertionError("data values out of order: last value=" + new BytesRef(lastPackedValue) + " current value=" + new BytesRef(packedValue, packedValueOffset, packedBytesLength) + " ord=" + ord);
+          throw new AssertionError("data values out of order: last value=" + new BytesRef(lastPackedValue) + " current value=" + new BytesRef(packedValue, packedValueOffset, config.packedBytesLength) + " ord=" + ord);
         }
       }
       if (cmp == 0 && doc < lastDoc) {
         throw new AssertionError("docs out of order: last doc=" + lastDoc + " current doc=" + doc + " ord=" + ord);
       }
     }
-    System.arraycopy(packedValue, packedValueOffset, lastPackedValue, 0, packedBytesLength);
+    System.arraycopy(packedValue, packedValueOffset, lastPackedValue, 0, config.packedBytesLength);
+    return true;
+  }
+
+  // only called from assert
+  private static boolean valueInBounds(BKDConfig config, BytesRef packedValue, byte[] minPackedValue, byte[] maxPackedValue) {
+    for(int dim=0;dim<config.numIndexDims;dim++) {
+      int offset = config.bytesPerDim*dim;
+      if (FutureArrays.compareUnsigned(packedValue.bytes, packedValue.offset + offset, packedValue.offset + offset + config.bytesPerDim, minPackedValue, offset, offset + config.bytesPerDim) < 0) {
+        return false;
+      }
+      if (FutureArrays.compareUnsigned(packedValue.bytes, packedValue.offset + offset, packedValue.offset + offset + config.bytesPerDim, maxPackedValue, offset, offset + config.bytesPerDim) > 0) {
+        return false;
+      }
+    }
+
     return true;
   }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/util/bkd/HeapPointReader.java b/lucene/core/src/java/org/apache/lucene/util/bkd/HeapPointReader.java
index 4cee1ef..6701f42 100644
--- a/lucene/core/src/java/org/apache/lucene/util/bkd/HeapPointReader.java
+++ b/lucene/core/src/java/org/apache/lucene/util/bkd/HeapPointReader.java
@@ -26,19 +26,17 @@ import org.apache.lucene.util.BytesRef;
 public final class HeapPointReader implements PointReader {
   private int curRead;
   final byte[] block;
-  final int packedBytesLength;
-  final int packedBytesDocIDLength;
+  final BKDConfig config;
   final int end;
   private final HeapPointValue pointValue;
 
-  public HeapPointReader(byte[] block, int packedBytesLength, int start, int end) {
+  public HeapPointReader(BKDConfig config, byte[] block, int start, int end) {
     this.block = block;
     curRead = start-1;
     this.end = end;
-    this.packedBytesLength = packedBytesLength;
-    this.packedBytesDocIDLength = packedBytesLength + Integer.BYTES;
+    this.config = config;
     if (start < end) {
-      this.pointValue = new HeapPointValue(block, packedBytesLength);
+      this.pointValue = new HeapPointValue(config, block);
     } else {
       //no values
       this.pointValue = null;
@@ -53,7 +51,7 @@ public final class HeapPointReader implements PointReader {
 
   @Override
   public PointValue pointValue() {
-    pointValue.setOffset(curRead * packedBytesDocIDLength);
+    pointValue.setOffset(curRead * config.bytesPerDoc);
     return pointValue;
   }
 
@@ -70,10 +68,10 @@ public final class HeapPointReader implements PointReader {
     final BytesRef packedValueDocID;
     final int packedValueLength;
 
-    HeapPointValue(byte[] value, int packedValueLength) {
-      this.packedValueLength = packedValueLength;
+    HeapPointValue(BKDConfig config, byte[] value) {
+      this.packedValueLength = config.packedBytesLength;
       this.packedValue = new BytesRef(value, 0, packedValueLength);
-      this.packedValueDocID = new BytesRef(value, 0, packedValueLength + Integer.BYTES);
+      this.packedValueDocID = new BytesRef(value, 0, config.bytesPerDoc);
     }
 
     /**
diff --git a/lucene/core/src/java/org/apache/lucene/util/bkd/HeapPointWriter.java b/lucene/core/src/java/org/apache/lucene/util/bkd/HeapPointWriter.java
index cd1b8ac..54652d0 100644
--- a/lucene/core/src/java/org/apache/lucene/util/bkd/HeapPointWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/util/bkd/HeapPointWriter.java
@@ -27,8 +27,7 @@ import org.apache.lucene.util.FutureArrays;
 public final class HeapPointWriter implements PointWriter {
   public final byte[] block;
   final int size;
-  final int packedBytesLength;
-  final int packedBytesDocIDLength;
+  final BKDConfig config;
   private final byte[] scratch;
   private int nextWrite;
   private boolean closed;
@@ -36,16 +35,15 @@ public final class HeapPointWriter implements PointWriter {
   private HeapPointReader.HeapPointValue pointValue;
 
 
-  public HeapPointWriter(int size, int packedBytesLength) {
-    this.packedBytesDocIDLength = packedBytesLength + Integer.BYTES;
-    this.packedBytesLength = packedBytesLength;
-    this.block = new byte[packedBytesDocIDLength * size];
+  public HeapPointWriter(BKDConfig config, int size) {
+    this.config = config;
+    this.block = new byte[config.bytesPerDoc * size];
     this.size = size;
-    this.scratch = new byte[packedBytesDocIDLength];
+    this.scratch = new byte[config.bytesPerDoc];
     if (size > 0) {
-      pointValue = new HeapPointReader.HeapPointValue(block, packedBytesLength);
+      pointValue = new HeapPointReader.HeapPointValue(config, block);
     } else {
-      //no values
+      // no values
       pointValue =  null;
     }
   }
@@ -53,17 +51,17 @@ public final class HeapPointWriter implements PointWriter {
   /** Returns a reference, in <code>result</code>, to the byte[] slice holding this value */
   public PointValue getPackedValueSlice(int index) {
     assert index < nextWrite : "nextWrite=" + (nextWrite) + " vs index=" + index;
-    pointValue.setOffset(index * packedBytesDocIDLength);
+    pointValue.setOffset(index * config.bytesPerDoc);
     return pointValue;
   }
 
   @Override
   public void append(byte[] packedValue, int docID) {
     assert closed == false : "point writer is already closed";
-    assert packedValue.length == packedBytesLength : "[packedValue] must have length [" + packedBytesLength + "] but was [" + packedValue.length + "]";
+    assert packedValue.length == config.packedBytesLength : "[packedValue] must have length [" + config.packedBytesLength + "] but was [" + packedValue.length + "]";
     assert nextWrite < size : "nextWrite=" + (nextWrite + 1) + " vs size=" + size;
-    System.arraycopy(packedValue, 0, block, nextWrite * packedBytesDocIDLength, packedBytesLength);
-    int position = nextWrite * packedBytesDocIDLength + packedBytesLength;
+    System.arraycopy(packedValue, 0, block, nextWrite * config.bytesPerDoc, config.packedBytesLength);
+    int position = nextWrite * config.bytesPerDoc + config.packedBytesLength;
     block[position] = (byte) (docID >> 24);
     block[++position] = (byte) (docID >> 16);
     block[++position] = (byte) (docID >> 8);
@@ -76,33 +74,32 @@ public final class HeapPointWriter implements PointWriter {
     assert closed == false : "point writer is already closed";
     assert nextWrite < size : "nextWrite=" + (nextWrite + 1) + " vs size=" + size;
     BytesRef packedValueDocID = pointValue.packedValueDocIDBytes();
-    assert packedValueDocID.length == packedBytesDocIDLength : "[packedValue] must have length [" + (packedBytesDocIDLength) + "] but was [" + packedValueDocID.length + "]";
-    System.arraycopy(packedValueDocID.bytes, packedValueDocID.offset, block, nextWrite * packedBytesDocIDLength, packedBytesDocIDLength);
+    assert packedValueDocID.length == config.bytesPerDoc : "[packedValue] must have length [" + (config.bytesPerDoc) + "] but was [" + packedValueDocID.length + "]";
+    System.arraycopy(packedValueDocID.bytes, packedValueDocID.offset, block, nextWrite * config.bytesPerDoc, config.bytesPerDoc);
     nextWrite++;
   }
 
   public void swap(int i, int j) {
 
-    int indexI = i * packedBytesDocIDLength;
-    int indexJ = j * packedBytesDocIDLength;
+    int indexI = i * config.bytesPerDoc;
+    int indexJ = j * config.bytesPerDoc;
 
     // scratch1 = values[i]
-    System.arraycopy(block, indexI, scratch, 0, packedBytesDocIDLength);
+    System.arraycopy(block, indexI, scratch, 0, config.bytesPerDoc);
     // values[i] = values[j]
-    System.arraycopy(block, indexJ, block, indexI, packedBytesDocIDLength);
+    System.arraycopy(block, indexJ, block, indexI, config.bytesPerDoc);
     // values[j] = scratch1
-    System.arraycopy(scratch, 0, block, indexJ, packedBytesDocIDLength);
+    System.arraycopy(scratch, 0, block, indexJ, config.bytesPerDoc);
   }
 
-  public int computeCardinality(int from, int to, int numDataDims, int bytesPerDim, int[] commonPrefixLengths) {
-    assert packedBytesLength == numDataDims * bytesPerDim;
+  public int computeCardinality(int from, int to, int[] commonPrefixLengths) {
     int leafCardinality = 1;
     for (int i = from + 1; i < to; i++) {
-      for (int dim = 0; dim < numDataDims; dim++) {
-        final int start = dim * bytesPerDim + commonPrefixLengths[dim];
-        final int end = dim * bytesPerDim + bytesPerDim;
-        if (FutureArrays.mismatch(block, i * packedBytesDocIDLength + start, i * packedBytesDocIDLength + end,
-            block, (i - 1) * packedBytesDocIDLength  + start, (i - 1) * packedBytesDocIDLength + end) != -1) {
+      for (int dim = 0; dim < config.numDims; dim++) {
+        final int start = dim * config.bytesPerDim + commonPrefixLengths[dim];
+        final int end = dim * config.bytesPerDim + config.bytesPerDim;
+        if (FutureArrays.mismatch(block, i * config.bytesPerDoc + start, i * config.bytesPerDoc + end,
+                block, (i - 1) * config.bytesPerDoc  + start, (i - 1) * config.bytesPerDoc + end) != -1) {
           leafCardinality++;
           break;
         }
@@ -121,7 +118,7 @@ public final class HeapPointWriter implements PointWriter {
     assert closed : "point writer is still open and trying to get a reader";
     assert start + length <= size: "start=" + start + " length=" + length + " docIDs.length=" + size;
     assert start + length <= nextWrite: "start=" + start + " length=" + length + " nextWrite=" + nextWrite;
-    return new HeapPointReader(block, packedBytesLength, (int) start, Math.toIntExact(start+length));
+    return new HeapPointReader(config, block, (int) start, Math.toIntExact(start+length));
   }
 
   @Override
diff --git a/lucene/core/src/java/org/apache/lucene/util/bkd/MutablePointsReaderUtils.java b/lucene/core/src/java/org/apache/lucene/util/bkd/MutablePointsReaderUtils.java
index 45e695b..3d54e69 100644
--- a/lucene/core/src/java/org/apache/lucene/util/bkd/MutablePointsReaderUtils.java
+++ b/lucene/core/src/java/org/apache/lucene/util/bkd/MutablePointsReaderUtils.java
@@ -34,10 +34,10 @@ public final class MutablePointsReaderUtils {
   MutablePointsReaderUtils() {}
 
   /** Sort the given {@link MutablePointValues} based on its packed value then doc ID. */
-  public static void sort(int maxDoc, int packedBytesLength,
+  public static void sort(BKDConfig config, int maxDoc,
                           MutablePointValues reader, int from, int to) {
     final int bitsPerDocId = PackedInts.bitsRequired(maxDoc - 1);
-    new MSBRadixSorter(packedBytesLength + (bitsPerDocId + 7) / 8) {
+    new MSBRadixSorter(config.packedBytesLength + (bitsPerDocId + 7) / 8) {
 
       @Override
       protected void swap(int i, int j) {
@@ -46,10 +46,10 @@ public final class MutablePointsReaderUtils {
 
       @Override
       protected int byteAt(int i, int k) {
-        if (k < packedBytesLength) {
+        if (k < config.packedBytesLength) {
           return Byte.toUnsignedInt(reader.getByteAt(i, k));
         } else {
-          final int shift = bitsPerDocId - ((k - packedBytesLength + 1) << 3);
+          final int shift = bitsPerDocId - ((k - config.packedBytesLength + 1) << 3);
           return (reader.getDocID(i) >>> Math.max(0, shift)) & 0xff;
         }
       }
@@ -75,10 +75,10 @@ public final class MutablePointsReaderUtils {
 
           @Override
           protected int comparePivot(int j) {
-            if (k < packedBytesLength) {
+            if (k < config.packedBytesLength) {
               reader.getValue(j, scratch);
-              int cmp = FutureArrays.compareUnsigned(pivot.bytes, pivot.offset + k, pivot.offset + k + packedBytesLength - k,
-                  scratch.bytes, scratch.offset + k, scratch.offset + k + packedBytesLength - k);
+              int cmp = FutureArrays.compareUnsigned(pivot.bytes, pivot.offset + k, pivot.offset + k + config.packedBytesLength - k,
+                      scratch.bytes, scratch.offset + k, scratch.offset + k + config.packedBytesLength - k);
               if (cmp != 0) {
                 return cmp;
               }
@@ -92,14 +92,12 @@ public final class MutablePointsReaderUtils {
   }
 
   /** Sort points on the given dimension. */
-  public static void sortByDim(int numDataDim, int numIndexDim, int sortedDim, int bytesPerDim, int[] commonPrefixLengths,
+  public static void sortByDim(BKDConfig config, int sortedDim, int[] commonPrefixLengths,
                                MutablePointValues reader, int from, int to,
                                BytesRef scratch1, BytesRef scratch2) {
 
-    final int start = sortedDim * bytesPerDim + commonPrefixLengths[sortedDim];
-    final int dimEnd =  sortedDim * bytesPerDim + bytesPerDim;
-    final int dataStart = numIndexDim * bytesPerDim;
-    final int dataEnd = dataStart + (numDataDim - numIndexDim) * bytesPerDim;
+    final int start = sortedDim * config.bytesPerDim + commonPrefixLengths[sortedDim];
+    final int dimEnd =  sortedDim * config.bytesPerDim + config.bytesPerDim;
     // No need for a fancy radix sort here, this is called on the leaves only so
     // there are not many values to sort
     new IntroSorter() {
@@ -122,10 +120,10 @@ public final class MutablePointsReaderUtils {
       protected int comparePivot(int j) {
         reader.getValue(j, scratch2);
         int cmp = FutureArrays.compareUnsigned(pivot.bytes, pivot.offset + start, pivot.offset + dimEnd, scratch2.bytes,
-            scratch2.offset + start, scratch2.offset + dimEnd);
+                scratch2.offset + start, scratch2.offset + dimEnd);
         if (cmp == 0) {
-          cmp = FutureArrays.compareUnsigned(pivot.bytes, pivot.offset + dataStart, pivot.offset + dataEnd,
-              scratch2.bytes, scratch2.offset + dataStart, scratch2.offset + dataEnd);
+          cmp = FutureArrays.compareUnsigned(pivot.bytes, pivot.offset + config.packedIndexBytesLength, pivot.offset + config.packedBytesLength,
+                  scratch2.bytes, scratch2.offset + config.packedIndexBytesLength, scratch2.offset + config.packedBytesLength);
           if (cmp == 0) {
             cmp = pivotDoc - reader.getDocID(j);
           }
@@ -138,20 +136,19 @@ public final class MutablePointsReaderUtils {
   /** Partition points around {@code mid}. All values on the left must be less
    *  than or equal to it and all values on the right must be greater than or
    *  equal to it. */
-  public static void partition(int numDataDim, int numIndexDim, int maxDoc, int splitDim, int bytesPerDim, int commonPrefixLen,
+  public static void partition(BKDConfig config, int maxDoc, int splitDim, int commonPrefixLen,
                                MutablePointValues reader, int from, int to, int mid,
                                BytesRef scratch1, BytesRef scratch2) {
-    final int dimOffset = splitDim * bytesPerDim + commonPrefixLen;
-    final int dimCmpBytes = bytesPerDim - commonPrefixLen;
-    final int dataOffset = numIndexDim * bytesPerDim;
-    final int dataCmpBytes = (numDataDim - numIndexDim) * bytesPerDim + dimCmpBytes;
+    final int dimOffset = splitDim * config.bytesPerDim + commonPrefixLen;
+    final int dimCmpBytes = config.bytesPerDim - commonPrefixLen;
+    final int dataCmpBytes = (config.numDims - config.numIndexDims) * config.bytesPerDim + dimCmpBytes;
     final int bitsPerDocId = PackedInts.bitsRequired(maxDoc - 1);
     new RadixSelector(dataCmpBytes + (bitsPerDocId + 7) / 8) {
 
       @Override
       protected Selector getFallbackSelector(int k) {
-        final int dataStart = (k < dimCmpBytes) ? dataOffset : dataOffset + k - dimCmpBytes;
-        final int dataEnd = numDataDim * bytesPerDim;
+        final int dataStart = (k < dimCmpBytes) ? config.packedIndexBytesLength : config.packedIndexBytesLength + k - dimCmpBytes;
+        final int dataEnd = config.numDims * config.bytesPerDim;
         return new IntroSelector() {
 
           final BytesRef pivot = scratch1;
@@ -173,7 +170,7 @@ public final class MutablePointsReaderUtils {
             if (k < dimCmpBytes) {
               reader.getValue(j, scratch2);
               int cmp = FutureArrays.compareUnsigned(pivot.bytes, pivot.offset + dimOffset + k, pivot.offset + dimOffset + dimCmpBytes,
-                  scratch2.bytes, scratch2.offset + dimOffset + k, scratch2.offset + dimOffset + dimCmpBytes);
+                      scratch2.bytes, scratch2.offset + dimOffset + k, scratch2.offset + dimOffset + dimCmpBytes);
               if (cmp != 0) {
                 return cmp;
               }
@@ -181,7 +178,7 @@ public final class MutablePointsReaderUtils {
             if (k < dataCmpBytes) {
               reader.getValue(j, scratch2);
               int cmp = FutureArrays.compareUnsigned(pivot.bytes, pivot.offset + dataStart, pivot.offset + dataEnd,
-                  scratch2.bytes, scratch2.offset + dataStart, scratch2.offset + dataEnd);
+                      scratch2.bytes, scratch2.offset + dataStart, scratch2.offset + dataEnd);
               if (cmp != 0) {
                 return cmp;
               }
@@ -201,7 +198,7 @@ public final class MutablePointsReaderUtils {
         if (k < dimCmpBytes) {
           return Byte.toUnsignedInt(reader.getByteAt(i, dimOffset + k));
         } else if (k < dataCmpBytes) {
-          return Byte.toUnsignedInt(reader.getByteAt(i, dataOffset + k - dimCmpBytes));
+          return Byte.toUnsignedInt(reader.getByteAt(i, config.packedIndexBytesLength + k - dimCmpBytes));
         } else {
           final int shift = bitsPerDocId - ((k - dataCmpBytes + 1) << 3);
           return (reader.getDocID(i) >>> Math.max(0, shift)) & 0xff;
diff --git a/lucene/core/src/java/org/apache/lucene/util/bkd/OfflinePointReader.java b/lucene/core/src/java/org/apache/lucene/util/bkd/OfflinePointReader.java
index 030bd71..0b256c0 100644
--- a/lucene/core/src/java/org/apache/lucene/util/bkd/OfflinePointReader.java
+++ b/lucene/core/src/java/org/apache/lucene/util/bkd/OfflinePointReader.java
@@ -37,32 +37,30 @@ public final class OfflinePointReader implements PointReader {
   final IndexInput in;
   byte[] onHeapBuffer;
   int offset;
-  final int bytesPerDoc;
   private boolean checked;
-  private final int packedValueLength;
+  private final BKDConfig config;
   private int pointsInBuffer;
   private final int maxPointOnHeap;
   // File name we are reading
   final String name;
   private final OfflinePointValue pointValue;
 
-  public OfflinePointReader(Directory tempDir, String tempFileName, int packedBytesLength, long start, long length, byte[] reusableBuffer) throws IOException {
-    this.bytesPerDoc = packedBytesLength + Integer.BYTES;
-    this.packedValueLength = packedBytesLength;
+  public OfflinePointReader(BKDConfig config, Directory tempDir, String tempFileName, long start, long length, byte[] reusableBuffer) throws IOException {
+    this.config = config;
 
-    if ((start + length) * bytesPerDoc + CodecUtil.footerLength() > tempDir.fileLength(tempFileName)) {
-      throw new IllegalArgumentException("requested slice is beyond the length of this file: start=" + start + " length=" + length + " bytesPerDoc=" + bytesPerDoc + " fileLength=" + tempDir.fileLength(tempFileName) + " tempFileName=" + tempFileName);
+    if ((start + length) * config.bytesPerDoc + CodecUtil.footerLength() > tempDir.fileLength(tempFileName)) {
+      throw new IllegalArgumentException("requested slice is beyond the length of this file: start=" + start + " length=" + length + " bytesPerDoc=" + config.bytesPerDoc + " fileLength=" + tempDir.fileLength(tempFileName) + " tempFileName=" + tempFileName);
     }
     if (reusableBuffer == null) {
       throw new IllegalArgumentException("[reusableBuffer] cannot be null");
     }
-    if (reusableBuffer.length < bytesPerDoc) {
-      throw new IllegalArgumentException("Length of [reusableBuffer] must be bigger than " + bytesPerDoc);
+    if (reusableBuffer.length < config.bytesPerDoc) {
+      throw new IllegalArgumentException("Length of [reusableBuffer] must be bigger than " + config.bytesPerDoc);
     }
 
-    this.maxPointOnHeap =  reusableBuffer.length / bytesPerDoc;
+    this.maxPointOnHeap =  reusableBuffer.length / config.bytesPerDoc;
     // Best-effort checksumming:
-    if (start == 0 && length*bytesPerDoc == tempDir.fileLength(tempFileName) - CodecUtil.footerLength()) {
+    if (start == 0 && length*config.bytesPerDoc == tempDir.fileLength(tempFileName) - CodecUtil.footerLength()) {
       // If we are going to read the entire file, e.g. because BKDWriter is now
       // partitioning it, we open with checksums:
       in = tempDir.openChecksumInput(tempFileName, IOContext.READONCE);
@@ -76,11 +74,11 @@ public final class OfflinePointReader implements PointReader {
 
     name = tempFileName;
 
-    long seekFP = start * bytesPerDoc;
+    long seekFP = start * config.bytesPerDoc;
     in.seek(seekFP);
     countLeft = length;
     this.onHeapBuffer = reusableBuffer;
-    this.pointValue = new OfflinePointValue(onHeapBuffer, packedValueLength);
+    this.pointValue = new OfflinePointValue(config, onHeapBuffer);
   }
 
   @Override
@@ -93,11 +91,11 @@ public final class OfflinePointReader implements PointReader {
       }
       try {
         if (countLeft > maxPointOnHeap) {
-          in.readBytes(onHeapBuffer, 0, maxPointOnHeap * bytesPerDoc);
+          in.readBytes(onHeapBuffer, 0, maxPointOnHeap * config.bytesPerDoc);
           pointsInBuffer = maxPointOnHeap - 1;
           countLeft -= maxPointOnHeap;
         } else {
-          in.readBytes(onHeapBuffer, 0, (int) countLeft * bytesPerDoc);
+          in.readBytes(onHeapBuffer, 0, (int) countLeft * config.bytesPerDoc);
           pointsInBuffer = Math.toIntExact(countLeft - 1);
           countLeft = 0;
         }
@@ -108,7 +106,7 @@ public final class OfflinePointReader implements PointReader {
       }
     } else {
       this.pointsInBuffer--;
-      this.offset += bytesPerDoc;
+      this.offset += config.bytesPerDoc;
     }
     return true;
   }
@@ -141,10 +139,10 @@ public final class OfflinePointReader implements PointReader {
     final BytesRef packedValueDocID;
     final int packedValueLength;
 
-    OfflinePointValue(byte[] value, int packedValueLength) {
-      this.packedValueLength = packedValueLength;
+    OfflinePointValue(BKDConfig config, byte[] value) {
+      this.packedValueLength = config.packedBytesLength;
       this.packedValue = new BytesRef(value, 0, packedValueLength);
-      this.packedValueDocID = new BytesRef(value, 0, packedValueLength + Integer.BYTES);
+      this.packedValueDocID = new BytesRef(value, 0, config.bytesPerDoc);
     }
 
     /**
diff --git a/lucene/core/src/java/org/apache/lucene/util/bkd/OfflinePointWriter.java b/lucene/core/src/java/org/apache/lucene/util/bkd/OfflinePointWriter.java
index d9fa86f..01ae136 100644
--- a/lucene/core/src/java/org/apache/lucene/util/bkd/OfflinePointWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/util/bkd/OfflinePointWriter.java
@@ -34,25 +34,25 @@ public final class OfflinePointWriter implements PointWriter {
   final Directory tempDir;
   public final IndexOutput out;
   public final String name;
-  final int packedBytesLength;
+  final BKDConfig config;
   long count;
   private boolean closed;
   final long expectedCount;
 
   /** Create a new writer with an unknown number of incoming points */
-  public OfflinePointWriter(Directory tempDir, String tempFileNamePrefix, int packedBytesLength,
+  public OfflinePointWriter(BKDConfig config, Directory tempDir, String tempFileNamePrefix,
                             String desc, long expectedCount) throws IOException {
     this.out = tempDir.createTempOutput(tempFileNamePrefix, "bkd_" + desc, IOContext.DEFAULT);
     this.name = out.getName();
     this.tempDir = tempDir;
-    this.packedBytesLength = packedBytesLength;
+    this.config = config;
     this.expectedCount = expectedCount;
   }
 
   @Override
   public void append(byte[] packedValue, int docID) throws IOException {
     assert closed == false : "Point writer is already closed";
-    assert packedValue.length == packedBytesLength : "[packedValue] must have length [" + packedBytesLength + "] but was [" + packedValue.length + "]";
+    assert packedValue.length == config.packedBytesLength : "[packedValue] must have length [" + config.packedBytesLength + "] but was [" + packedValue.length + "]";
     out.writeBytes(packedValue, 0, packedValue.length);
     out.writeInt(docID);
     count++;
@@ -63,7 +63,7 @@ public final class OfflinePointWriter implements PointWriter {
   public void append(PointValue pointValue) throws IOException {
     assert closed == false : "Point writer is already closed";
     BytesRef packedValueDocID = pointValue.packedValueDocIDBytes();
-    assert packedValueDocID.length == packedBytesLength + Integer.BYTES : "[packedValue and docID] must have length [" + (packedBytesLength + Integer.BYTES) + "] but was [" + packedValueDocID.length + "]";
+    assert packedValueDocID.length == config.bytesPerDoc : "[packedValue and docID] must have length [" + (config.bytesPerDoc) + "] but was [" + packedValueDocID.length + "]";
     out.writeBytes(packedValueDocID.bytes, packedValueDocID.offset, packedValueDocID.length);
     count++;
     assert expectedCount == 0 || count <= expectedCount : "expectedCount=" + expectedCount + " vs count=" + count;
@@ -71,7 +71,7 @@ public final class OfflinePointWriter implements PointWriter {
 
   @Override
   public PointReader getReader(long start, long length) throws IOException {
-    byte[] buffer  = new byte[packedBytesLength + Integer.BYTES];
+    byte[] buffer  = new byte[config.bytesPerDoc];
     return getReader(start, length,  buffer);
   }
 
@@ -79,7 +79,7 @@ public final class OfflinePointWriter implements PointWriter {
     assert closed: "point writer is still open and trying to get a reader";
     assert start + length <= count: "start=" + start + " length=" + length + " count=" + count;
     assert expectedCount == 0 || count == expectedCount;
-    return new OfflinePointReader(tempDir, name, packedBytesLength, start, length, reusableBuffer);
+    return new OfflinePointReader(config, tempDir, name, start, length, reusableBuffer);
   }
 
   @Override
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene86/TestLucene86PointsFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene86/TestLucene86PointsFormat.java
index f46b632..dd7018c 100644
--- a/lucene/core/src/test/org/apache/lucene/codecs/lucene86/TestLucene86PointsFormat.java
+++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene86/TestLucene86PointsFormat.java
@@ -41,7 +41,7 @@ import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.FutureArrays;
 import org.apache.lucene.util.TestUtil;
-import org.apache.lucene.util.bkd.BKDWriter;
+import org.apache.lucene.util.bkd.BKDConfig;
 
 public class TestLucene86PointsFormat extends BasePointsFormatTestCase {
 
@@ -79,7 +79,7 @@ public class TestLucene86PointsFormat extends BasePointsFormatTestCase {
     } else {
       // standard issue
       codec = defaultCodec;
-      maxPointsInLeafNode = BKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE;
+      maxPointsInLeafNode = BKDConfig.DEFAULT_MAX_POINTS_IN_LEAF_NODE;
     }
   }
 
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestLatLonPointQueries.java b/lucene/core/src/test/org/apache/lucene/search/TestLatLonPointQueries.java
index f0e9612..d3fce00 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestLatLonPointQueries.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestLatLonPointQueries.java
@@ -27,7 +27,7 @@ import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.bkd.BKDWriter;
+import org.apache.lucene.util.bkd.BKDConfig;
 
 public class TestLatLonPointQueries extends BaseGeoPointTestCase {
 
@@ -62,7 +62,7 @@ public class TestLatLonPointQueries extends BaseGeoPointTestCase {
   }
 
   public void testDistanceQueryWithInvertedIntersection() throws IOException {
-    final int numMatchingDocs = atLeast(10 * BKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE);
+    final int numMatchingDocs = atLeast(10 * BKDConfig.DEFAULT_MAX_POINTS_IN_LEAF_NODE);
 
     try (Directory dir = newDirectory()) {
 
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 9bbeb50..4eac831 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
@@ -69,7 +69,7 @@ import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.NumericUtils;
 import org.apache.lucene.util.TestUtil;
-import org.apache.lucene.util.bkd.BKDWriter;
+import org.apache.lucene.util.bkd.BKDConfig;
 import org.junit.BeforeClass;
 
 @LuceneTestCase.SuppressCodecs("SimpleText")
@@ -2181,7 +2181,7 @@ public class TestPointQueries extends LuceneTestCase {
     Directory dir = newDirectory();
     IndexWriter w = new IndexWriter(dir, newIndexWriterConfig());
     final int numDims = TestUtil.nextInt(random(), 1, 3);
-    final int numDocs = atLeast(10 * BKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE); // we need multiple leaves to enable this optimization
+    final int numDocs = atLeast(10 * BKDConfig.DEFAULT_MAX_POINTS_IN_LEAF_NODE); // we need multiple leaves to enable this optimization
     for (int i = 0; i < numDocs; ++i) {
       Document doc = new Document();
       int[] values = new int[numDims];
@@ -2202,9 +2202,9 @@ public class TestPointQueries extends LuceneTestCase {
     assertEquals(high[0] - low[0] + 1, searcher.count(IntPoint.newRangeQuery("f", low, high)));
     Arrays.fill(high, numDocs - 1);
     assertEquals(high[0] - low[0] + 1, searcher.count(IntPoint.newRangeQuery("f", low, high)));
-    Arrays.fill(low, BKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE + 1);
+    Arrays.fill(low, BKDConfig.DEFAULT_MAX_POINTS_IN_LEAF_NODE + 1);
     assertEquals(high[0] - low[0] + 1, searcher.count(IntPoint.newRangeQuery("f", low, high)));
-    Arrays.fill(high, numDocs - BKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE);
+    Arrays.fill(high, numDocs - BKDConfig.DEFAULT_MAX_POINTS_IN_LEAF_NODE);
     assertEquals(high[0] - low[0] + 1, searcher.count(IntPoint.newRangeQuery("f", low, high)));
 
     r.close();
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 1a10252..3583ce2 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,8 +41,8 @@ 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.DEFAULT_MAX_POINTS_IN_LEAF_NODE, BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP, 26L * numDocs);
+    BKDWriter w = new BKDWriter(numDocs, dir, "_0", new BKDConfig(1, 1, Long.BYTES, BKDConfig.DEFAULT_MAX_POINTS_IN_LEAF_NODE),
+                                BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP, 26L * numDocs);
     int counter = 0;
     byte[] packedBytes = new byte[Long.BYTES];
     for (int docID = 0; docID < numDocs; docID++) {
@@ -80,8 +80,8 @@ 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.DEFAULT_MAX_POINTS_IN_LEAF_NODE, BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP, 26L * numDocs);
+    BKDWriter w = new BKDWriter(numDocs, dir, "_0", new BKDConfig(2, 2, Long.BYTES, BKDConfig.DEFAULT_MAX_POINTS_IN_LEAF_NODE),
+                                BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP, 26L * numDocs);
     int counter = 0;
     byte[] packedBytes = new byte[2*Long.BYTES];
     for (int docID = 0; docID < numDocs; docID++) {
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 f7e1b55..f6eacc6 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
@@ -51,7 +51,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);
+      BKDWriter w = new BKDWriter(100, dir, "tmp", new BKDConfig(1, 1, 4, 2), 1.0f, 100);
       byte[] scratch = new byte[4];
       for(int docID=0;docID<100;docID++) {
         NumericUtils.intToSortableBytes(docID, scratch, 0);
@@ -129,7 +129,7 @@ public class TestBKD extends LuceneTestCase {
       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);
+      BKDWriter w = new BKDWriter(numDocs, dir, "tmp", new BKDConfig(numDims, numIndexDims, 4, maxPointsInLeafNode), maxMB, numDocs);
 
       if (VERBOSE) {
         System.out.println("TEST: numDims=" + numDims + " numIndexDims=" + numIndexDims + " numDocs=" + numDocs);
@@ -272,7 +272,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);
+      BKDWriter w = new BKDWriter(numDocs, dir, "tmp", new BKDConfig(numDims, numDims, numBytesPerDim, maxPointsInLeafNode), maxMB, numDocs);
       BigInteger[][] docs = new BigInteger[numDocs][];
 
       byte[] scratch = new byte[numBytesPerDim*numDims];
@@ -450,7 +450,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);
+        new BKDWriter(1, dir, "bkd", new BKDConfig(1, 1, 16, 1000000), 0.001, 0);
       });
       assertTrue(expected.getMessage().contains("either increase maxMBSortInHeap or decrease maxPointsInLeafNode"));
     }
@@ -708,7 +708,7 @@ public class TestBKD extends LuceneTestCase {
         maxDocs = random().nextLong();
       }
     }
-    BKDWriter w = new BKDWriter(numValues, dir, "_" + seg, numDataDims, numIndexDims, numBytesPerDim, maxPointsInLeafNode, maxMB, maxDocs);
+    BKDWriter w = new BKDWriter(numValues, dir, "_" + seg, new BKDConfig(numDataDims, numIndexDims, numBytesPerDim, maxPointsInLeafNode), maxMB, maxDocs);
     IndexOutput out = dir.createOutput("bkd", IOContext.DEFAULT);
     IndexInput in = null;
 
@@ -770,7 +770,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);
+          w = new BKDWriter(numValues, dir, "_" + seg, new BKDConfig(numDataDims, numIndexDims, numBytesPerDim, maxPointsInLeafNode), maxMB, docValues.length);
           lastDocIDBase = docID;
         }
       }
@@ -793,7 +793,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);
+        w = new BKDWriter(numValues, dir, "_" + seg, new BKDConfig(numDataDims, numIndexDims, numBytesPerDim, maxPointsInLeafNode), maxMB, docValues.length);
         List<BKDReader> readers = new ArrayList<>();
         for(long fp : toMerge) {
           in.seek(fp);
@@ -1078,7 +1078,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);
+      BKDWriter w = new BKDWriter(numDocs+1, dir, "tmp", new BKDConfig(1, 1, Integer.BYTES, 2), 0.01f, numDocs);
       for(int i=0;i<numDocs;i++) {
         w.add(new byte[Integer.BYTES], i);
       }
@@ -1134,7 +1134,7 @@ public class TestBKD extends LuceneTestCase {
         System.arraycopy(pointValue1, i * numBytesPerDim, pointValue2, i * numBytesPerDim, numBytesPerDim);
     }
 
-    BKDWriter w = new BKDWriter(2 * numValues, dir, "_temp", numDataDims, numIndexDims, numBytesPerDim, maxPointsInLeafNode,
+    BKDWriter w = new BKDWriter(2 * numValues, dir, "_temp", new BKDConfig(numDataDims, numIndexDims, numBytesPerDim, maxPointsInLeafNode),
         maxMB, 2 * numValues);
     for (int i = 0; i < numValues; ++i) {
       w.add(pointValue1, i);
@@ -1194,7 +1194,7 @@ public class TestBKD extends LuceneTestCase {
   public void test2DLongOrdsOffline() throws Exception {
     try (Directory dir = newDirectory()) {
       int numDocs = 100000;
-      BKDWriter w = new BKDWriter(numDocs+1, dir, "tmp", 2, 2, Integer.BYTES, 2, 0.01f, numDocs);
+      BKDWriter w = new BKDWriter(numDocs+1, dir, "tmp", new BKDConfig(2, 2, Integer.BYTES, 2), 0.01f, numDocs);
       byte[] buffer = new byte[2*Integer.BYTES];
       for(int i=0;i<numDocs;i++) {
         random().nextBytes(buffer);
@@ -1248,7 +1248,7 @@ public class TestBKD extends LuceneTestCase {
 
     Directory dir = newFSDirectory(createTempDir());
     int numDocs = atLeast(10000);
-    BKDWriter w = new BKDWriter(numDocs+1, dir, "tmp", numDims, numIndexDims, bytesPerDim, 32, 1f, numDocs);
+    BKDWriter w = new BKDWriter(numDocs+1, dir, "tmp", new BKDConfig(numDims, numIndexDims, bytesPerDim, 32), 1f, numDocs);
     byte[] tmp = new byte[bytesUsed];
     byte[] buffer = new byte[numDims * bytesPerDim];
     for(int i=0;i<numDocs;i++) {
@@ -1307,7 +1307,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", new BKDConfig(1, 1, numBytesPerDim, maxPointsInLeafNode),
         BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP, numValues);
     for (int i = 0; i < numValues; ++i) {
       if (i == numValues / 2) {
@@ -1469,7 +1469,7 @@ public class TestBKD extends LuceneTestCase {
       }
     };
 
-    BKDWriter w = new BKDWriter(numValues, dir, "_temp", 1, 1, numBytesPerDim, BKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE,
+    BKDWriter w = new BKDWriter(numValues, dir, "_temp", new BKDConfig(1, 1, numBytesPerDim, BKDConfig.DEFAULT_MAX_POINTS_IN_LEAF_NODE),
         BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP, numValues);
     expectThrows(IllegalStateException.class, () -> {
       try (IndexOutput out = dir.createOutput("bkd", IOContext.DEFAULT)) {
@@ -1487,7 +1487,7 @@ public class TestBKD extends LuceneTestCase {
     final int numPointsAdded = 50; // exceeds totalPointCount
     final int numBytesPerDim = TestUtil.nextInt(random(), 1, 4);
     final byte[] pointValue = new byte[numBytesPerDim];
-    BKDWriter w = new BKDWriter(numValues, dir, "_temp", 1, 1, numBytesPerDim, 2,
+    BKDWriter w = new BKDWriter(numValues, dir, "_temp", new BKDConfig(1, 1, numBytesPerDim, 2),
         BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP, numValues);
     for(int i=0;i<numValues;i++) {
       random().nextBytes(pointValue);
@@ -1506,7 +1506,7 @@ public class TestBKD extends LuceneTestCase {
     final int numPointsAdded = 50; // exceeds totalPointCount
     final int numBytesPerDim = TestUtil.nextInt(random(), 1, 4);
     final byte[][] pointValue = new byte[11][numBytesPerDim];
-    BKDWriter w = new BKDWriter(numValues + 1, dir, "_temp", 1, 1, numBytesPerDim, 2,
+    BKDWriter w = new BKDWriter(numValues + 1, dir, "_temp", new BKDConfig(1, 1, numBytesPerDim, 2),
         BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP, numValues);
     for(int i=0;i<numValues + 1;i++) {
       random().nextBytes(pointValue[i]);
@@ -1590,5 +1590,4 @@ public class TestBKD extends LuceneTestCase {
     }
     dir.close();
   }
-
 }
diff --git a/lucene/core/src/test/org/apache/lucene/util/bkd/TestBKDRadixSelector.java b/lucene/core/src/test/org/apache/lucene/util/bkd/TestBKDRadixSelector.java
index 939991c..c3e2dda 100644
--- a/lucene/core/src/test/org/apache/lucene/util/bkd/TestBKDRadixSelector.java
+++ b/lucene/core/src/test/org/apache/lucene/util/bkd/TestBKDRadixSelector.java
@@ -34,11 +34,11 @@ public class TestBKDRadixSelector extends LuceneTestCase {
     int values = 4;
     Directory dir = getDirectory(values);
     int middle = 2;
-    int dimensions =1;
+    int dimensions = 1;
     int bytesPerDimensions = Integer.BYTES;
-    int packedLength = dimensions * bytesPerDimensions;
-    PointWriter points = getRandomPointWriter(dir, values, packedLength);
-    byte[] value = new byte[packedLength];
+    BKDConfig config = new BKDConfig(dimensions, dimensions, bytesPerDimensions, BKDConfig.DEFAULT_MAX_POINTS_IN_LEAF_NODE);
+    PointWriter points = getRandomPointWriter(config, dir, values);
+    byte[] value = new byte[config.packedBytesLength];
     NumericUtils.intToSortableBytes(1, value, 0);
     points.append(value, 0);
     NumericUtils.intToSortableBytes(2, value, 0);
@@ -48,8 +48,8 @@ public class TestBKDRadixSelector extends LuceneTestCase {
     NumericUtils.intToSortableBytes(4, value, 0);
     points.append(value, 3);
     points.close();
-    PointWriter copy = copyPoints(dir,points, packedLength);
-    verify(dir, copy, dimensions, dimensions, 0, values, middle, packedLength, bytesPerDimensions, 0);
+    PointWriter copy = copyPoints(config, dir,points);
+    verify(config, dir, copy, 0, values, middle, 0);
     dir.close();
   }
 
@@ -67,6 +67,7 @@ public class TestBKDRadixSelector extends LuceneTestCase {
   }
 
   private void doTestRandomBinary(int count) throws IOException {
+    BKDConfig config = getRandomConfig();
     int values = TestUtil.nextInt(random(), count, count*2);
     Directory dir = getDirectory(values);
     int start;
@@ -80,31 +81,27 @@ public class TestBKDRadixSelector extends LuceneTestCase {
     }
     int partitionPoint = TestUtil.nextInt(random(), start + 1, end - 1);
     int sortedOnHeap = random().nextInt(5000);
-    int indexDimensions =  TestUtil.nextInt(random(), 1, 8);
-    int dataDimensions =  TestUtil.nextInt(random(), indexDimensions, 8);
-    int bytesPerDimensions = TestUtil.nextInt(random(), 2, 30);
-    int packedLength = dataDimensions * bytesPerDimensions;
-    PointWriter points = getRandomPointWriter(dir, values, packedLength);
-    byte[] value = new byte[packedLength];
+    PointWriter points = getRandomPointWriter(config, dir, values);
+    byte[] value = new byte[config.packedBytesLength];
     for (int i =0; i < values; i++) {
       random().nextBytes(value);
       points.append(value, i);
     }
     points.close();
-    verify(dir, points, dataDimensions, indexDimensions, start, end, partitionPoint, packedLength, bytesPerDimensions, sortedOnHeap);
+    verify(config, dir, points,start, end, partitionPoint, sortedOnHeap);
     dir.close();
   }
 
   public void testRandomAllDimensionsEquals() throws IOException {
+    int dimensions =  TestUtil.nextInt(random(), 1, BKDConfig.MAX_INDEX_DIMS);
+    int bytesPerDimensions = TestUtil.nextInt(random(), 2, 30);
+    BKDConfig config = new BKDConfig(dimensions, dimensions, bytesPerDimensions, BKDConfig.DEFAULT_MAX_POINTS_IN_LEAF_NODE);
     int values =  TestUtil.nextInt(random(), 15000, 20000);
     Directory dir = getDirectory(values);
     int partitionPoint = random().nextInt(values);
     int sortedOnHeap = random().nextInt(5000);
-    int dimensions =  TestUtil.nextInt(random(), 1, 8);
-    int bytesPerDimensions = TestUtil.nextInt(random(), 2, 30);
-    int packedLength = dimensions * bytesPerDimensions;
-    PointWriter points = getRandomPointWriter(dir, values, packedLength);
-    byte[] value = new byte[packedLength];
+    PointWriter points = getRandomPointWriter(config, dir, values);
+    byte[] value = new byte[config.packedBytesLength];
     random().nextBytes(value);
     for (int i =0; i < values; i++) {
       if (random().nextBoolean()) {
@@ -114,7 +111,7 @@ public class TestBKDRadixSelector extends LuceneTestCase {
       }
     }
     points.close();
-    verify(dir, points, dimensions, dimensions, 0, values, partitionPoint, packedLength, bytesPerDimensions, sortedOnHeap);
+    verify(config, dir, points, 0, values, partitionPoint, sortedOnHeap);
     dir.close();
   }
 
@@ -123,12 +120,9 @@ public class TestBKDRadixSelector extends LuceneTestCase {
     Directory dir = getDirectory(values);
     int partitionPoint = random().nextInt(values);
     int sortedOnHeap = random().nextInt(5000);
-    int indexDimensions =  TestUtil.nextInt(random(), 1, 8);
-    int dataDimensions =  TestUtil.nextInt(random(), indexDimensions, 8);
-    int bytesPerDimensions = TestUtil.nextInt(random(), 2, 30);
-    int packedLength = dataDimensions * bytesPerDimensions;
-    PointWriter points = getRandomPointWriter(dir, values, packedLength);
-    byte[] value = new byte[packedLength];
+    BKDConfig config = getRandomConfig();
+    PointWriter points = getRandomPointWriter(config, dir, values);
+    byte[] value = new byte[config.packedBytesLength];
     random().nextBytes(value);
     for (int i =0; i < values; i++) {
       if (random().nextBoolean()) {
@@ -138,7 +132,7 @@ public class TestBKDRadixSelector extends LuceneTestCase {
       }
     }
     points.close();
-    verify(dir, points, dataDimensions, indexDimensions, 0, values, partitionPoint, packedLength, bytesPerDimensions, sortedOnHeap);
+    verify(config, dir, points, 0, values, partitionPoint, sortedOnHeap);
     dir.close();
   }
 
@@ -147,33 +141,27 @@ public class TestBKDRadixSelector extends LuceneTestCase {
     Directory dir = getDirectory(values);
     int partitionPoint = random().nextInt(values);
     int sortedOnHeap = random().nextInt(5000);
-    int indexDimensions =  TestUtil.nextInt(random(), 1, 8);
-    int dataDimensions =  TestUtil.nextInt(random(), indexDimensions, 8);
-    int bytesPerDimensions = TestUtil.nextInt(random(), 2, 30);
-    int packedLength = dataDimensions * bytesPerDimensions;
-    PointWriter points = getRandomPointWriter(dir, values, packedLength);
-    byte[] value = new byte[packedLength];
+    BKDConfig config = getRandomConfig();
+    PointWriter points = getRandomPointWriter(config, dir, values);
+    byte[] value = new byte[config.packedBytesLength];
     random().nextBytes(value);
     for (int i =0; i < values; i++) {
       points.append(value, 0);
     }
     points.close();
-    verify(dir, points, dataDimensions, indexDimensions, 0, values, partitionPoint, packedLength, bytesPerDimensions, sortedOnHeap);
+    verify(config, dir, points, 0, values, partitionPoint, sortedOnHeap);
     dir.close();
   }
 
   public void testRandomFewDifferentValues() throws IOException {
+    BKDConfig config = getRandomConfig();
     int values = atLeast(15000);
     Directory dir = getDirectory(values);
     int partitionPoint = random().nextInt(values);
     int sortedOnHeap = random().nextInt(5000);
-    int indexDimensions =  TestUtil.nextInt(random(), 1, 8);
-    int dataDimensions =  TestUtil.nextInt(random(), indexDimensions, 8);
-    int bytesPerDimensions = TestUtil.nextInt(random(), 2, 30);
-    int packedLength = dataDimensions * bytesPerDimensions;
-    PointWriter points = getRandomPointWriter(dir, values, packedLength);
+    PointWriter points = getRandomPointWriter(config, dir, values);
     int numberValues = random().nextInt(8) + 2;
-    byte[][] differentValues = new byte[numberValues][packedLength];
+    byte[][] differentValues = new byte[numberValues][config.packedBytesLength];
     for (int i =0; i < numberValues; i++) {
       random().nextBytes(differentValues[i]);
     }
@@ -181,57 +169,56 @@ public class TestBKDRadixSelector extends LuceneTestCase {
       points.append(differentValues[random().nextInt(numberValues)], i);
     }
     points.close();
-    verify(dir, points, dataDimensions, indexDimensions, 0, values, partitionPoint, packedLength, bytesPerDimensions, sortedOnHeap);
+    verify(config, dir, points, 0, values, partitionPoint, sortedOnHeap);
     dir.close();
   }
 
   public void testRandomDataDimDiffValues() throws IOException {
+    BKDConfig config = getRandomConfig();
     int values = atLeast(15000);
     Directory dir = getDirectory(values);
     int partitionPoint = random().nextInt(values);
     int sortedOnHeap = random().nextInt(5000);
-    int indexDimensions =  TestUtil.nextInt(random(), 1, 8);
-    int dataDimensions =  TestUtil.nextInt(random(), indexDimensions, 8);
-    int bytesPerDimensions = TestUtil.nextInt(random(), 2, 30);
-    int packedLength = dataDimensions * bytesPerDimensions;
-    PointWriter points = getRandomPointWriter(dir, values, packedLength);
-    byte[] value = new byte[packedLength];
-    byte[] dataValue = new byte[(dataDimensions - indexDimensions) * bytesPerDimensions];
+    PointWriter points = getRandomPointWriter(config, dir, values);
+    byte[] value = new byte[config.packedBytesLength];
+    int dataOnlyDims = config.numDims - config.numIndexDims;
+    byte[] dataValue = new byte[dataOnlyDims * config.bytesPerDim];
     random().nextBytes(value);
     for (int i =0; i < values; i++) {
       random().nextBytes(dataValue);
-      System.arraycopy(dataValue, 0, value, indexDimensions * bytesPerDimensions, (dataDimensions - indexDimensions) * bytesPerDimensions);
+      System.arraycopy(dataValue, 0, value, config.numIndexDims * config.bytesPerDim, dataOnlyDims * config.bytesPerDim);
       points.append(value, i);
     }
     points.close();
-    verify(dir, points, dataDimensions, indexDimensions, 0, values, partitionPoint, packedLength, bytesPerDimensions, sortedOnHeap);
+    verify(config, dir, points, 0, values, partitionPoint, sortedOnHeap);
     dir.close();
   }
 
-  private void verify(Directory dir, PointWriter points, int dataDimensions, int indexDimensions, long start, long end, long middle, int packedLength, int bytesPerDimensions, int sortedOnHeap) throws IOException{
-    BKDRadixSelector radixSelector = new BKDRadixSelector(dataDimensions, indexDimensions, bytesPerDimensions, sortedOnHeap, dir, "test");
-    //we only split by indexed dimension so we check for each only those dimension
-    for (int splitDim = 0; splitDim < indexDimensions; splitDim++) {
-      //We need to make a copy of the data as it is deleted in the process
-      BKDRadixSelector.PathSlice inputSlice = new BKDRadixSelector.PathSlice(copyPoints(dir, points, packedLength), 0, points.count());
-      int commonPrefixLengthInput = getRandomCommonPrefix(inputSlice, bytesPerDimensions, splitDim);
+  private void verify(BKDConfig config, Directory dir, PointWriter points, long start, long end, long middle, int sortedOnHeap) throws IOException{
+    BKDRadixSelector radixSelector = new BKDRadixSelector(config, sortedOnHeap, dir, "test");
+    int dataOnlyDims = config.numDims - config.numIndexDims;
+    // we only split by indexed dimension so we check for each only those dimension
+    for (int splitDim = 0; splitDim < config.numIndexDims; splitDim++) {
+      // We need to make a copy of the data as it is deleted in the process
+      BKDRadixSelector.PathSlice inputSlice = new BKDRadixSelector.PathSlice(copyPoints(config, dir, points), 0, points.count());
+      int commonPrefixLengthInput = getRandomCommonPrefix(config, inputSlice, splitDim);
       BKDRadixSelector.PathSlice[] slices = new BKDRadixSelector.PathSlice[2];
       byte[] partitionPoint = radixSelector.select(inputSlice, slices, start, end, middle, splitDim, commonPrefixLengthInput);
       assertEquals(middle - start, slices[0].count);
       assertEquals(end - middle, slices[1].count);
-      //check that left and right slices contain the correct points
-      byte[] max = getMax(slices[0], bytesPerDimensions, splitDim);
-      byte[] min = getMin(slices[1], bytesPerDimensions, splitDim);
-      int cmp = FutureArrays.compareUnsigned(max, 0, bytesPerDimensions, min, 0, bytesPerDimensions);
+      // check that left and right slices contain the correct points
+      byte[] max = getMax(config, slices[0], splitDim);
+      byte[] min = getMin(config, slices[1], splitDim);
+      int cmp = FutureArrays.compareUnsigned(max, 0, config.bytesPerDim, min, 0, config.bytesPerDim);
       assertTrue(cmp <= 0);
       if (cmp == 0) {
-        byte[] maxDataDim = getMaxDataDimension(slices[0], bytesPerDimensions, dataDimensions, indexDimensions, max, splitDim);
-        byte[] minDataDim = getMinDataDimension(slices[1], bytesPerDimensions, dataDimensions, indexDimensions, min, splitDim);
-        cmp = FutureArrays.compareUnsigned(maxDataDim, 0, (dataDimensions - indexDimensions) * bytesPerDimensions, minDataDim, 0, (dataDimensions - indexDimensions) * bytesPerDimensions);
+        byte[] maxDataDim = getMaxDataDimension(config, slices[0], max, splitDim);
+        byte[] minDataDim = getMinDataDimension(config, slices[1], min, splitDim);
+        cmp = FutureArrays.compareUnsigned(maxDataDim, 0, dataOnlyDims * config.bytesPerDim, minDataDim, 0, dataOnlyDims * config.bytesPerDim);
         assertTrue(cmp <= 0);
         if (cmp == 0) {
-          int maxDocID = getMaxDocId(slices[0], bytesPerDimensions, splitDim, partitionPoint, dataDimensions, indexDimensions,maxDataDim);
-          int minDocId = getMinDocId(slices[1], bytesPerDimensions, splitDim, partitionPoint, dataDimensions, indexDimensions,minDataDim);
+          int maxDocID = getMaxDocId(config, slices[0], splitDim, partitionPoint, maxDataDim);
+          int minDocId = getMinDocId(config, slices[1], splitDim, partitionPoint, minDataDim);
           assertTrue(minDocId >= maxDocID);
         }
       }
@@ -242,8 +229,8 @@ public class TestBKDRadixSelector extends LuceneTestCase {
     points.destroy();
   }
 
-  private PointWriter copyPoints(Directory dir, PointWriter points, int packedLength) throws IOException {
-    try (PointWriter copy  = getRandomPointWriter(dir, points.count(), packedLength);
+  private PointWriter copyPoints(BKDConfig config, Directory dir, PointWriter points) throws IOException {
+    try (PointWriter copy  = getRandomPointWriter(config, dir, points.count());
          PointReader reader = points.getReader(0, points.count())) {
       while (reader.next()) {
         copy.append(reader.pointValue());
@@ -253,21 +240,21 @@ public class TestBKDRadixSelector extends LuceneTestCase {
   }
 
   /** returns a common prefix length equal or lower than the current one */
-  private int getRandomCommonPrefix(BKDRadixSelector.PathSlice inputSlice, int bytesPerDimension, int splitDim) throws IOException {
-    byte[] pointsMax = getMax(inputSlice, bytesPerDimension, splitDim);
-    byte[] pointsMin = getMin(inputSlice, bytesPerDimension, splitDim);
-    int commonPrefixLength = FutureArrays.mismatch(pointsMin, 0, bytesPerDimension, pointsMax, 0, bytesPerDimension);
+  private int getRandomCommonPrefix(BKDConfig config, BKDRadixSelector.PathSlice inputSlice, int splitDim) throws IOException {
+    byte[] pointsMax = getMax(config, inputSlice, splitDim);
+    byte[] pointsMin = getMin(config, inputSlice, splitDim);
+    int commonPrefixLength = FutureArrays.mismatch(pointsMin, 0, config.bytesPerDim, pointsMax, 0, config.bytesPerDim);
     if (commonPrefixLength == -1) {
-      commonPrefixLength = bytesPerDimension;
+      commonPrefixLength = config.bytesPerDim;
     }
     return (random().nextBoolean()) ? commonPrefixLength : commonPrefixLength == 0 ? 0 : random().nextInt(commonPrefixLength);
   }
 
-  private PointWriter getRandomPointWriter(Directory dir, long numPoints, int packedBytesLength) throws IOException {
+  private PointWriter getRandomPointWriter(BKDConfig config, Directory dir, long numPoints) throws IOException {
     if (numPoints < 4096 && random().nextBoolean()) {
-      return new HeapPointWriter(Math.toIntExact(numPoints), packedBytesLength);
+      return new HeapPointWriter(config, Math.toIntExact(numPoints));
     } else {
-      return new OfflinePointWriter(dir, "test", packedBytesLength, "data", numPoints);
+      return new OfflinePointWriter(config, dir, "test", "data", numPoints);
     }
   }
 
@@ -281,35 +268,35 @@ public class TestBKDRadixSelector extends LuceneTestCase {
     return dir;
   }
 
-  private byte[] getMin(BKDRadixSelector.PathSlice p, int bytesPerDimension, int dimension) throws  IOException {
-    byte[] min = new byte[bytesPerDimension];
+  private byte[] getMin(BKDConfig config, BKDRadixSelector.PathSlice pathSlice, int dimension) throws  IOException {
+    byte[] min = new byte[config.bytesPerDim];
     Arrays.fill(min, (byte) 0xff);
-    try (PointReader reader = p.writer.getReader(p.start, p.count)) {
-      byte[] value = new byte[bytesPerDimension];
+    try (PointReader reader = pathSlice.writer.getReader(pathSlice.start, pathSlice.count)) {
+      byte[] value = new byte[config.bytesPerDim];
 
       while (reader.next()) {
         PointValue pointValue = reader.pointValue();
         BytesRef packedValue = pointValue.packedValue();
-        System.arraycopy(packedValue.bytes, packedValue.offset + dimension * bytesPerDimension, value, 0, bytesPerDimension);
-        if (FutureArrays.compareUnsigned(min, 0, bytesPerDimension, value, 0, bytesPerDimension) > 0) {
-          System.arraycopy(value, 0, min, 0, bytesPerDimension);
+        System.arraycopy(packedValue.bytes, packedValue.offset + dimension * config.bytesPerDim, value, 0, config.bytesPerDim);
+        if (FutureArrays.compareUnsigned(min, 0, config.bytesPerDim, value, 0, config.bytesPerDim) > 0) {
+          System.arraycopy(value, 0, min, 0, config.bytesPerDim);
         }
       }
     }
     return min;
   }
 
-  private int getMinDocId(BKDRadixSelector.PathSlice p, int bytesPerDimension, int dimension, byte[] partitionPoint, int dataDims, int indexDims, byte[] dataDim) throws  IOException {
-   int docID = Integer.MAX_VALUE;
+  private int getMinDocId(BKDConfig config, BKDRadixSelector.PathSlice p, int dimension, byte[] partitionPoint, byte[] dataDim) throws  IOException {
+    int docID = Integer.MAX_VALUE;
     try (PointReader reader = p.writer.getReader(p.start, p.count)) {
       while (reader.next()) {
         PointValue pointValue = reader.pointValue();
         BytesRef packedValue = pointValue.packedValue();
-        int offset = dimension * bytesPerDimension;
-        int dataOffset = indexDims * bytesPerDimension;
-        int dataLength = (dataDims - indexDims) * bytesPerDimension;
-        if (FutureArrays.compareUnsigned(packedValue.bytes, packedValue.offset + offset, packedValue.offset + offset + bytesPerDimension, partitionPoint, 0, bytesPerDimension) == 0
-          && FutureArrays.compareUnsigned(packedValue.bytes, packedValue.offset + dataOffset, packedValue.offset + dataOffset + dataLength, dataDim, 0, dataLength) == 0) {
+        int offset = dimension * config.bytesPerDim;
+        int dataOffset = config.packedIndexBytesLength;
+        int dataLength = (config.numDims - config.numIndexDims) * config.bytesPerDim;
+        if (FutureArrays.compareUnsigned(packedValue.bytes, packedValue.offset + offset, packedValue.offset + offset + config.bytesPerDim, partitionPoint, 0, config.bytesPerDim) == 0
+                && FutureArrays.compareUnsigned(packedValue.bytes, packedValue.offset + dataOffset, packedValue.offset + dataOffset + dataLength, dataDim, 0, dataLength) == 0) {
           int newDocID = pointValue.docID();
           if (newDocID < docID) {
             docID = newDocID;
@@ -320,19 +307,20 @@ public class TestBKDRadixSelector extends LuceneTestCase {
     return docID;
   }
 
-  private byte[] getMinDataDimension(BKDRadixSelector.PathSlice p, int bytesPerDimension, int dataDims, int indexDims, byte[] minDim, int splitDim) throws  IOException {
-    byte[] min = new byte[(dataDims - indexDims) * bytesPerDimension];
+  private byte[] getMinDataDimension(BKDConfig config, BKDRadixSelector.PathSlice p, byte[] minDim, int splitDim) throws  IOException {
+    final int numDataDims = config.numDims - config.numIndexDims;
+    byte[] min = new byte[numDataDims * config.bytesPerDim];
     Arrays.fill(min, (byte) 0xff);
-    int offset = splitDim * bytesPerDimension;
+    int offset = splitDim * config.bytesPerDim;
     try (PointReader reader = p.writer.getReader(p.start, p.count)) {
-      byte[] value = new byte[(dataDims - indexDims) * bytesPerDimension];
+      byte[] value = new byte[numDataDims * config.bytesPerDim];
       while (reader.next()) {
         PointValue pointValue = reader.pointValue();
         BytesRef packedValue = pointValue.packedValue();
-        if (FutureArrays.mismatch(minDim, 0, bytesPerDimension, packedValue.bytes, packedValue.offset + offset, packedValue.offset + offset + bytesPerDimension) == -1) {
-          System.arraycopy(packedValue.bytes, packedValue.offset + indexDims * bytesPerDimension, value, 0, (dataDims - indexDims) * bytesPerDimension);
-          if (FutureArrays.compareUnsigned(min, 0, (dataDims - indexDims) * bytesPerDimension, value, 0, (dataDims - indexDims) * bytesPerDimension) > 0) {
-            System.arraycopy(value, 0, min, 0, (dataDims - indexDims) * bytesPerDimension);
+        if (FutureArrays.mismatch(minDim, 0, config.bytesPerDim, packedValue.bytes, packedValue.offset + offset, packedValue.offset + offset + config.bytesPerDim) == -1) {
+          System.arraycopy(packedValue.bytes, packedValue.offset + config.numIndexDims * config.bytesPerDim, value, 0, numDataDims * config.bytesPerDim);
+          if (FutureArrays.compareUnsigned(min, 0, numDataDims * config.bytesPerDim, value, 0, numDataDims * config.bytesPerDim) > 0) {
+            System.arraycopy(value, 0, min, 0, numDataDims * config.bytesPerDim);
           }
         }
       }
@@ -340,36 +328,37 @@ public class TestBKDRadixSelector extends LuceneTestCase {
     return min;
   }
 
-  private byte[] getMax(BKDRadixSelector.PathSlice p, int bytesPerDimension, int dimension) throws  IOException {
-    byte[] max = new byte[bytesPerDimension];
+  private byte[] getMax(BKDConfig config, BKDRadixSelector.PathSlice p, int dimension) throws  IOException {
+    byte[] max = new byte[config.bytesPerDim];
     Arrays.fill(max, (byte) 0);
     try (PointReader reader = p.writer.getReader(p.start, p.count)) {
-      byte[] value = new byte[bytesPerDimension];
+      byte[] value = new byte[config.bytesPerDim];
       while (reader.next()) {
         PointValue pointValue = reader.pointValue();
         BytesRef packedValue = pointValue.packedValue();
-        System.arraycopy(packedValue.bytes, packedValue.offset + dimension * bytesPerDimension, value, 0, bytesPerDimension);
-        if (FutureArrays.compareUnsigned(max, 0, bytesPerDimension, value, 0, bytesPerDimension) < 0) {
-          System.arraycopy(value, 0, max, 0, bytesPerDimension);
+        System.arraycopy(packedValue.bytes, packedValue.offset + dimension * config.bytesPerDim, value, 0, config.bytesPerDim);
+        if (FutureArrays.compareUnsigned(max, 0, config.bytesPerDim, value, 0, config.bytesPerDim) < 0) {
+          System.arraycopy(value, 0, max, 0, config.bytesPerDim);
         }
       }
     }
     return max;
   }
 
-  private byte[] getMaxDataDimension(BKDRadixSelector.PathSlice p, int bytesPerDimension, int dataDims, int indexDims, byte[] maxDim, int splitDim) throws  IOException {
-    byte[] max = new byte[(dataDims - indexDims) * bytesPerDimension];
+  private byte[] getMaxDataDimension(BKDConfig config, BKDRadixSelector.PathSlice p, byte[] maxDim, int splitDim) throws  IOException {
+    final int numDataDims = config.numDims - config.numIndexDims;
+    byte[] max = new byte[numDataDims * config.bytesPerDim];
     Arrays.fill(max, (byte) 0);
-    int offset = splitDim * bytesPerDimension;
+    int offset = splitDim * config.bytesPerDim;
     try (PointReader reader = p.writer.getReader(p.start, p.count)) {
-      byte[] value = new byte[(dataDims - indexDims) * bytesPerDimension];
+      byte[] value = new byte[numDataDims * config.bytesPerDim];
       while (reader.next()) {
         PointValue pointValue = reader.pointValue();
         BytesRef packedValue = pointValue.packedValue();
-        if (FutureArrays.mismatch(maxDim, 0, bytesPerDimension, packedValue.bytes, packedValue.offset + offset, packedValue.offset + offset + bytesPerDimension) == -1) {
-          System.arraycopy(packedValue.bytes, packedValue.offset + indexDims * bytesPerDimension, value, 0, (dataDims - indexDims) * bytesPerDimension);
-          if (FutureArrays.compareUnsigned(max, 0, (dataDims - indexDims) * bytesPerDimension, value, 0, (dataDims - indexDims) * bytesPerDimension) < 0) {
-            System.arraycopy(value, 0, max, 0, (dataDims - indexDims) * bytesPerDimension);
+        if (FutureArrays.mismatch(maxDim, 0, config.bytesPerDim, packedValue.bytes, packedValue.offset + offset, packedValue.offset + offset + config.bytesPerDim) == -1) {
+          System.arraycopy(packedValue.bytes, packedValue.offset + config.packedIndexBytesLength, value, 0, numDataDims * config.bytesPerDim);
+          if (FutureArrays.compareUnsigned(max, 0, numDataDims * config.bytesPerDim, value, 0, numDataDims * config.bytesPerDim) < 0) {
+            System.arraycopy(value, 0, max, 0, numDataDims * config.bytesPerDim);
           }
         }
       }
@@ -377,17 +366,17 @@ public class TestBKDRadixSelector extends LuceneTestCase {
     return max;
   }
 
-  private int getMaxDocId(BKDRadixSelector.PathSlice p, int bytesPerDimension, int dimension, byte[] partitionPoint, int dataDims, int indexDims, byte[] dataDim) throws  IOException {
+  private int getMaxDocId(BKDConfig config, BKDRadixSelector.PathSlice p, int dimension, byte[] partitionPoint, byte[] dataDim) throws  IOException {
     int docID = Integer.MIN_VALUE;
     try (PointReader reader = p.writer.getReader(p.start, p.count)) {
       while (reader.next()) {
         PointValue pointValue = reader.pointValue();
         BytesRef packedValue = pointValue.packedValue();
-        int offset = dimension * bytesPerDimension;
-        int dataOffset = indexDims * bytesPerDimension;
-        int dataLength = (dataDims - indexDims) * bytesPerDimension;
-        if (FutureArrays.compareUnsigned(packedValue.bytes, packedValue.offset + offset, packedValue.offset + offset + bytesPerDimension, partitionPoint, 0, bytesPerDimension) == 0
-            && FutureArrays.compareUnsigned(packedValue.bytes, packedValue.offset + dataOffset, packedValue.offset + dataOffset + dataLength, dataDim, 0, dataLength) == 0) {
+        int offset = dimension * config.bytesPerDim;
+        int dataOffset = config.packedIndexBytesLength;
+        int dataLength = (config.numDims - config.numIndexDims) * config.bytesPerDim;
+        if (FutureArrays.compareUnsigned(packedValue.bytes, packedValue.offset + offset, packedValue.offset + offset + config.bytesPerDim, partitionPoint, 0, config.bytesPerDim) == 0
+                && FutureArrays.compareUnsigned(packedValue.bytes, packedValue.offset + dataOffset, packedValue.offset + dataOffset + dataLength, dataDim, 0, dataLength) == 0) {
           int newDocID = pointValue.docID();
           if (newDocID > docID) {
             docID = newDocID;
@@ -397,4 +386,12 @@ public class TestBKDRadixSelector extends LuceneTestCase {
     }
     return docID;
   }
+
+  private BKDConfig getRandomConfig() {
+    int numIndexDims = TestUtil.nextInt(random(), 1, BKDConfig.MAX_INDEX_DIMS);
+    int numDims = TestUtil.nextInt(random(), numIndexDims, BKDConfig.MAX_DIMS);
+    int bytesPerDim = TestUtil.nextInt(random(), 2, 30);
+    int maxPointsInLeafNode = TestUtil.nextInt(random(), 50, 2000);
+    return new BKDConfig(numDims, numIndexDims, bytesPerDim, maxPointsInLeafNode);
+  }
 }
diff --git a/lucene/core/src/test/org/apache/lucene/util/bkd/TestBKDRadixSort.java b/lucene/core/src/test/org/apache/lucene/util/bkd/TestBKDRadixSort.java
index 7fecca7..8379f66 100644
--- a/lucene/core/src/test/org/apache/lucene/util/bkd/TestBKDRadixSort.java
+++ b/lucene/core/src/test/org/apache/lucene/util/bkd/TestBKDRadixSort.java
@@ -29,43 +29,34 @@ import org.apache.lucene.util.TestUtil;
 public class TestBKDRadixSort extends LuceneTestCase {
 
   public void testRandom() throws IOException {
-    int numPoints = TestUtil.nextInt(random(), 1, BKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE);
-    int indexDimensions = TestUtil.nextInt(random(), 1, 8);
-    int dataDimensions = TestUtil.nextInt(random(), indexDimensions, 8);
-    int bytesPerDim = TestUtil.nextInt(random(), 2, 30);
-    int packedBytesLength = dataDimensions * bytesPerDim;
-    HeapPointWriter points = new HeapPointWriter(numPoints, packedBytesLength);
-    byte[] value = new byte[packedBytesLength];
+    BKDConfig config = getRandomConfig();
+    int numPoints = TestUtil.nextInt(random(), 1, BKDConfig.DEFAULT_MAX_POINTS_IN_LEAF_NODE);
+    HeapPointWriter points = new HeapPointWriter(config, numPoints);
+    byte[] value = new byte[config.packedBytesLength];
     for (int i = 0; i < numPoints; i++) {
       random().nextBytes(value);
       points.append(value, i);
     }
-    verifySort(points, dataDimensions, indexDimensions, 0, numPoints, bytesPerDim);
+    verifySort(config, points,  0, numPoints);
   }
 
   public void testRandomAllEquals() throws IOException {
-    int numPoints = TestUtil.nextInt(random(), 1, BKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE);
-    int indexDimensions = TestUtil.nextInt(random(), 1, 8);
-    int dataDimensions = TestUtil.nextInt(random(), indexDimensions, 8);
-    int bytesPerDim = TestUtil.nextInt(random(), 2, 30);
-    int packedBytesLength = dataDimensions * bytesPerDim;
-    HeapPointWriter points = new HeapPointWriter(numPoints, packedBytesLength);
-    byte[] value = new byte[packedBytesLength];
+    BKDConfig config = getRandomConfig();
+    int numPoints = TestUtil.nextInt(random(), 1, BKDConfig.DEFAULT_MAX_POINTS_IN_LEAF_NODE);
+    HeapPointWriter points = new HeapPointWriter(config, numPoints);
+    byte[] value = new byte[config.packedBytesLength];
     random().nextBytes(value);
     for (int i = 0; i < numPoints; i++) {
       points.append(value, random().nextInt(numPoints));
     }
-    verifySort(points, dataDimensions, indexDimensions, 0, numPoints, bytesPerDim);
+    verifySort(config, points, 0, numPoints);
   }
 
   public void testRandomLastByteTwoValues() throws IOException {
-    int numPoints = TestUtil.nextInt(random(), 1, BKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE);
-    int indexDimensions = TestUtil.nextInt(random(), 1, 8);
-    int dataDimensions = TestUtil.nextInt(random(), indexDimensions, 8);
-    int bytesPerDim = TestUtil.nextInt(random(), 2, 30);
-    int packedBytesLength = dataDimensions * bytesPerDim;
-    HeapPointWriter points = new HeapPointWriter(numPoints, packedBytesLength);
-    byte[] value = new byte[packedBytesLength];
+    BKDConfig config = getRandomConfig();
+    int numPoints = TestUtil.nextInt(random(), 1, BKDConfig.DEFAULT_MAX_POINTS_IN_LEAF_NODE);
+    HeapPointWriter points = new HeapPointWriter(config, numPoints);
+    byte[] value = new byte[config.packedBytesLength];
     random().nextBytes(value);
     for (int i = 0; i < numPoints; i++) {
       if (random().nextBoolean()) {
@@ -74,71 +65,64 @@ public class TestBKDRadixSort extends LuceneTestCase {
         points.append(value, 2);
       }
     }
-    verifySort(points, dataDimensions, indexDimensions, 0, numPoints, bytesPerDim);
+    verifySort(config, points, 0, numPoints);
   }
 
   public void testRandomFewDifferentValues() throws IOException {
-    int numPoints = TestUtil.nextInt(random(), 1, BKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE);
-    int indexDimensions = TestUtil.nextInt(random(), 1, 8);
-    int dataDimensions = TestUtil.nextInt(random(), indexDimensions, 8);
-    int bytesPerDim = TestUtil.nextInt(random(), 2, 30);
-    int packedBytesLength = dataDimensions * bytesPerDim;
-    HeapPointWriter points = new HeapPointWriter(numPoints, packedBytesLength);
+    BKDConfig config = getRandomConfig();
+    int numPoints = TestUtil.nextInt(random(), 1, BKDConfig.DEFAULT_MAX_POINTS_IN_LEAF_NODE);
+    HeapPointWriter points = new HeapPointWriter(config, numPoints);
     int numberValues = random().nextInt(8) + 2;
-    byte[][] differentValues = new byte[numberValues][packedBytesLength];
+    byte[][] differentValues = new byte[numberValues][config.packedBytesLength];
     for (int i = 0; i < numberValues; i++) {
       random().nextBytes(differentValues[i]);
     }
     for (int i = 0; i < numPoints; i++) {
       points.append(differentValues[random().nextInt(numberValues)], i);
     }
-    verifySort(points, dataDimensions, indexDimensions, 0, numPoints, bytesPerDim);
+    verifySort(config, points, 0, numPoints);
   }
 
   public void testRandomDataDimDifferent() throws IOException {
-    int numPoints = TestUtil.nextInt(random(), 1, BKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE);
-    int indexDimensions = TestUtil.nextInt(random(), 1, 8);
-    int dataDimensions = TestUtil.nextInt(random(), indexDimensions, 8);
-    int bytesPerDim = TestUtil.nextInt(random(), 2, 30);
-    int packedBytesLength = dataDimensions * bytesPerDim;
-    HeapPointWriter points = new HeapPointWriter(numPoints, packedBytesLength);
-    byte[] value = new byte[packedBytesLength];
-    int totalDataDimension = dataDimensions - indexDimensions;
-    byte[] dataDimensionValues = new byte[totalDataDimension * bytesPerDim];
+    BKDConfig config = getRandomConfig();
+    int numPoints = TestUtil.nextInt(random(), 1, BKDConfig.DEFAULT_MAX_POINTS_IN_LEAF_NODE);
+    HeapPointWriter points = new HeapPointWriter(config, numPoints);
+    byte[] value = new byte[config.packedBytesLength];
+    int totalDataDimension = config.numDims - config.numIndexDims;
+    byte[] dataDimensionValues = new byte[totalDataDimension * config.bytesPerDim];
     random().nextBytes(value);
     for (int i = 0; i < numPoints; i++) {
       random().nextBytes(dataDimensionValues);
-      System.arraycopy(dataDimensionValues, 0, value, indexDimensions * bytesPerDim, totalDataDimension * bytesPerDim);
+      System.arraycopy(dataDimensionValues, 0, value, config.packedIndexBytesLength, totalDataDimension * config.bytesPerDim);
       points.append(value, random().nextInt(numPoints));
     }
-    verifySort(points, dataDimensions, indexDimensions, 0, numPoints, bytesPerDim);
+    verifySort(config, points, 0, numPoints);
   }
 
-  private void verifySort(HeapPointWriter points, int dataDimensions, int indexDimensions, int start, int end, int bytesPerDim) throws IOException{
-    int packedBytesLength = dataDimensions * bytesPerDim;
+  private void verifySort(BKDConfig config, HeapPointWriter points,int start, int end) throws IOException{
     Directory dir = newDirectory();
-    BKDRadixSelector radixSelector = new BKDRadixSelector(dataDimensions, indexDimensions, bytesPerDim, 1000, dir, "test");
+    BKDRadixSelector radixSelector = new BKDRadixSelector(config, 1000, dir, "test");
     // we check for each dimension
-    for (int splitDim = 0; splitDim < dataDimensions; splitDim++) {
-      radixSelector.heapRadixSort(points, start, end, splitDim, getRandomCommonPrefix(points, start, end, bytesPerDim, splitDim));
-      byte[] previous = new byte[bytesPerDim * dataDimensions];
+    for (int splitDim = 0; splitDim < config.numDims; splitDim++) {
+      radixSelector.heapRadixSort(points, start, end, splitDim, getRandomCommonPrefix(config, points, start, end, splitDim));
+      byte[] previous = new byte[config.packedBytesLength];
       int previousDocId = -1;
       Arrays.fill(previous, (byte) 0);
-      int dimOffset = splitDim * bytesPerDim;
+      int dimOffset = splitDim * config.bytesPerDim;
       for (int j = start; j < end; j++) {
         PointValue pointValue = points.getPackedValueSlice(j);
         BytesRef value = pointValue.packedValue();
-        int cmp = FutureArrays.compareUnsigned(value.bytes, value.offset + dimOffset, value.offset + dimOffset + bytesPerDim, previous, dimOffset, dimOffset + bytesPerDim);
+        int cmp = FutureArrays.compareUnsigned(value.bytes, value.offset + dimOffset, value.offset + dimOffset + config.bytesPerDim, previous, dimOffset, dimOffset + config.bytesPerDim);
         assertTrue(cmp >= 0);
         if (cmp == 0) {
-          int dataOffset = indexDimensions * bytesPerDim;
-          cmp = FutureArrays.compareUnsigned(value.bytes, value.offset + dataOffset, value.offset + packedBytesLength, previous, dataOffset, packedBytesLength);
+          int dataOffset = config.numIndexDims * config.bytesPerDim;
+          cmp = FutureArrays.compareUnsigned(value.bytes, value.offset + dataOffset, value.offset + config.packedBytesLength, previous, dataOffset, config.packedBytesLength);
           assertTrue(cmp >= 0);
         }
         if (cmp == 0) {
           assertTrue(pointValue.docID() >= previousDocId);
         }
-        System.arraycopy(value.bytes, value.offset, previous, 0, packedBytesLength);
+        System.arraycopy(value.bytes, value.offset, previous, 0, config.packedBytesLength);
         previousDocId = pointValue.docID();
       }
     }
@@ -146,17 +130,17 @@ public class TestBKDRadixSort extends LuceneTestCase {
   }
 
   /** returns a common prefix length equal or lower than the current one */
-  private int getRandomCommonPrefix(HeapPointWriter points, int start, int end, int bytesPerDimension, int sortDim)  {
-    int commonPrefixLength = bytesPerDimension;
+  private int getRandomCommonPrefix(BKDConfig config, HeapPointWriter points, int start, int end, int sortDim)  {
+    int commonPrefixLength = config.bytesPerDim;
     PointValue value = points.getPackedValueSlice(start);
     BytesRef bytesRef = value.packedValue();
-    byte[] firstValue = new byte[bytesPerDimension];
-    int offset = sortDim * bytesPerDimension;
-    System.arraycopy(bytesRef.bytes, bytesRef.offset + offset, firstValue, 0, bytesPerDimension);
+    byte[] firstValue = new byte[config.bytesPerDim];
+    int offset = sortDim * config.bytesPerDim;
+    System.arraycopy(bytesRef.bytes, bytesRef.offset + offset, firstValue, 0, config.bytesPerDim);
     for (int i = start + 1; i < end; i++) {
       value = points.getPackedValueSlice(i);
       bytesRef = value.packedValue();
-      int diff = FutureArrays.mismatch(bytesRef.bytes, bytesRef.offset + offset, bytesRef.offset + offset + bytesPerDimension, firstValue, 0, bytesPerDimension);
+      int diff = FutureArrays.mismatch(bytesRef.bytes, bytesRef.offset + offset, bytesRef.offset + offset + config.bytesPerDim, firstValue, 0, config.bytesPerDim);
       if (diff != -1 && commonPrefixLength > diff) {
         if (diff == 0) {
           return diff;
@@ -166,4 +150,12 @@ public class TestBKDRadixSort extends LuceneTestCase {
     }
     return (random().nextBoolean()) ? commonPrefixLength : random().nextInt(commonPrefixLength);
   }
+
+  private BKDConfig getRandomConfig() {
+    int numIndexDims = TestUtil.nextInt(random(), 1, BKDConfig.MAX_INDEX_DIMS);
+    int numDims = TestUtil.nextInt(random(), numIndexDims, BKDConfig.MAX_DIMS);
+    int bytesPerDim = TestUtil.nextInt(random(), 2, 30);
+    int maxPointsInLeafNode = TestUtil.nextInt(random(), 50, 2000);
+    return new BKDConfig(numDims, numIndexDims, bytesPerDim, maxPointsInLeafNode);
+  }
 }
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 e6b467d..e75ab24 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
@@ -38,9 +38,10 @@ public class TestMutablePointsReaderUtils extends LuceneTestCase {
   private void doTestSort() {
     final int bytesPerDim = TestUtil.nextInt(random(), 1, 16);
     final int maxDoc = TestUtil.nextInt(random(), 1, 1 << random().nextInt(30));
-    Point[] points = createRandomPoints(1, 1, bytesPerDim, maxDoc, new int[1]);
+    BKDConfig config = new BKDConfig(1, 1, bytesPerDim, BKDConfig.DEFAULT_MAX_POINTS_IN_LEAF_NODE);
+    Point[] points = createRandomPoints(config, maxDoc, new int[1]);
     DummyPointsReader reader = new DummyPointsReader(points);
-    MutablePointsReaderUtils.sort(maxDoc, bytesPerDim, reader, 0, points.length);
+    MutablePointsReaderUtils.sort(config, maxDoc, reader, 0, points.length);
     Arrays.sort(points, new Comparator<Point>() {
       @Override
       public int compare(Point o1, Point o2) {
@@ -62,26 +63,24 @@ public class TestMutablePointsReaderUtils extends LuceneTestCase {
   }
 
   private void doTestSortByDim() {
-    final int numIndexDims = TestUtil.nextInt(random(), 1, 8);
-    final int numDataDims = TestUtil.nextInt(random(), numIndexDims, 8);
-    final int bytesPerDim = TestUtil.nextInt(random(), 1, 16);
+    BKDConfig config = createRandomConfig();
     final int maxDoc = TestUtil.nextInt(random(), 1, 1 << random().nextInt(30));
-    int[] commonPrefixLengths = new int[numDataDims];
-    Point[] points = createRandomPoints(numDataDims, numIndexDims, bytesPerDim, maxDoc, commonPrefixLengths);
+    int[] commonPrefixLengths = new int[config.numDims];
+    Point[] points = createRandomPoints(config, maxDoc, commonPrefixLengths);
     DummyPointsReader reader = new DummyPointsReader(points);
-    final int sortedDim = random().nextInt(numIndexDims);
-    MutablePointsReaderUtils.sortByDim(numDataDims, numIndexDims, sortedDim, bytesPerDim, commonPrefixLengths, reader, 0, points.length,
-        new BytesRef(), new BytesRef());
+    final int sortedDim = random().nextInt(config.numIndexDims);
+    MutablePointsReaderUtils.sortByDim(config, sortedDim, commonPrefixLengths, reader, 0, points.length,
+            new BytesRef(), new BytesRef());
     for (int i = 1; i < points.length; ++i) {
-      final int offset = sortedDim * bytesPerDim;
+      final int offset = sortedDim * config.bytesPerDim;
       BytesRef previousValue = reader.points[i-1].packedValue;
       BytesRef currentValue = reader.points[i].packedValue;
-      int cmp = FutureArrays.compareUnsigned(previousValue.bytes, previousValue.offset + offset, previousValue.offset + offset + bytesPerDim, currentValue.bytes, currentValue.offset + offset, currentValue.offset + offset + bytesPerDim);
+      int cmp = FutureArrays.compareUnsigned(previousValue.bytes, previousValue.offset + offset, previousValue.offset + offset + config.bytesPerDim, currentValue.bytes, currentValue.offset + offset, currentValue.offset + offset + config.bytesPerDim);
       if (cmp == 0) {
-        int dataDimOffset = numIndexDims * bytesPerDim;
-        int dataDimsLength = (numDataDims - numIndexDims) * bytesPerDim;
+        int dataDimOffset = config.packedIndexBytesLength;
+        int dataDimsLength = (config.numDims - config.numIndexDims) * config.bytesPerDim;
         cmp = FutureArrays.compareUnsigned(previousValue.bytes, previousValue.offset + dataDimOffset, previousValue.offset + dataDimOffset + dataDimsLength,
-            currentValue.bytes, currentValue.offset + dataDimOffset, currentValue.offset + dataDimOffset + dataDimsLength);
+                currentValue.bytes, currentValue.offset + dataDimOffset, currentValue.offset + dataDimOffset + dataDimsLength);
         if (cmp == 0) {
           cmp = reader.points[i - 1].doc - reader.points[i].doc;
         }
@@ -97,28 +96,26 @@ public class TestMutablePointsReaderUtils extends LuceneTestCase {
   }
 
   private void doTestPartition() {
-    final int numIndexDims = TestUtil.nextInt(random(), 1, 8);
-    final int numDataDims = TestUtil.nextInt(random(), numIndexDims, 8);
-    final int bytesPerDim = TestUtil.nextInt(random(), 1, 16);
-    int[] commonPrefixLengths  = new int[numDataDims];
+    BKDConfig config = createRandomConfig();
+    int[] commonPrefixLengths  = new int[config.numDims];
     final int maxDoc = TestUtil.nextInt(random(), 1, 1 << random().nextInt(30));
-    Point[] points = createRandomPoints(numDataDims, numIndexDims, bytesPerDim, maxDoc, commonPrefixLengths);
-    final int splitDim =  random().nextInt(numIndexDims);
+    Point[] points = createRandomPoints(config, maxDoc, commonPrefixLengths);
+    final int splitDim =  random().nextInt(config.numIndexDims);
     DummyPointsReader reader = new DummyPointsReader(points);
     final int pivot = TestUtil.nextInt(random(), 0, points.length - 1);
-    MutablePointsReaderUtils.partition(numDataDims, numIndexDims, maxDoc, splitDim, bytesPerDim, commonPrefixLengths[splitDim], reader, 0, points.length, pivot,
-        new BytesRef(), new BytesRef());
+    MutablePointsReaderUtils.partition(config, maxDoc, splitDim, commonPrefixLengths[splitDim], reader, 0, points.length, pivot,
+            new BytesRef(), new BytesRef());
     BytesRef pivotValue = reader.points[pivot].packedValue;
-    int offset = splitDim * bytesPerDim;
+    int offset = splitDim * config.bytesPerDim;
     for (int i = 0; i < points.length; ++i) {
       BytesRef value = reader.points[i].packedValue;
-      int cmp = FutureArrays.compareUnsigned(value.bytes, value.offset + offset, value.offset + offset + bytesPerDim,
-          pivotValue.bytes, pivotValue.offset + offset, pivotValue.offset + offset + bytesPerDim);
+      int cmp = FutureArrays.compareUnsigned(value.bytes, value.offset + offset, value.offset + offset + config.bytesPerDim,
+              pivotValue.bytes, pivotValue.offset + offset, pivotValue.offset + offset + config.bytesPerDim);
       if (cmp == 0) {
-        int dataDimOffset = numIndexDims * bytesPerDim;
-        int dataDimsLength = (numDataDims - numIndexDims) * bytesPerDim;
+        int dataDimOffset = config.packedIndexBytesLength;
+        int dataDimsLength = (config.numDims - config.numIndexDims) * config.bytesPerDim;
         cmp = FutureArrays.compareUnsigned(value.bytes, value.offset + dataDimOffset, value.offset + dataDimOffset + dataDimsLength,
-            pivotValue.bytes, pivotValue.offset + dataDimOffset, pivotValue.offset + dataDimOffset + dataDimsLength);
+                pivotValue.bytes, pivotValue.offset + dataDimOffset, pivotValue.offset + dataDimOffset + dataDimsLength);
         if (cmp == 0) {
           cmp = reader.points[i].doc - reader.points[pivot].doc;
         }
@@ -133,50 +130,58 @@ public class TestMutablePointsReaderUtils extends LuceneTestCase {
     }
   }
 
-  private static Point[] createRandomPoints(int numDataDims, int numIndexdims, int bytesPerDim, int maxDoc, int[] commonPrefixLengths) {
-    assertTrue(commonPrefixLengths.length == numDataDims);
-    final int packedBytesLength = numDataDims * bytesPerDim;
+  private static BKDConfig createRandomConfig() {
+    final int numIndexDims = TestUtil.nextInt(random(), 1, BKDConfig.MAX_INDEX_DIMS);
+    final int numDims = TestUtil.nextInt(random(), numIndexDims, BKDConfig.MAX_DIMS);
+    final int bytesPerDim = TestUtil.nextInt(random(), 1, 16);
+    final int maxPointsInLeafNode = TestUtil.nextInt(random(), 50, 2000);
+    return new BKDConfig(numDims, numIndexDims, bytesPerDim, maxPointsInLeafNode);
+  }
+
+  private static Point[] createRandomPoints(BKDConfig config, int maxDoc, int[] commonPrefixLengths) {
+    assertTrue(commonPrefixLengths.length == config.numDims);
     final int numPoints = TestUtil.nextInt(random(), 1, 100000);
     Point[] points = new Point[numPoints];
     if (random().nextInt(5) != 0) {
       for (int i = 0; i < numPoints; ++i) {
-        byte[] value = new byte[packedBytesLength];
+        byte[] value = new byte[config.packedBytesLength];
         random().nextBytes(value);
         points[i] = new Point(value, random().nextInt(maxDoc));
       }
-      for (int i = 0; i < numDataDims; ++i) {
-        commonPrefixLengths[i] = TestUtil.nextInt(random(), 0, bytesPerDim);
+      for (int i = 0; i < config.numDims; ++i) {
+        commonPrefixLengths[i] = TestUtil.nextInt(random(), 0, config.bytesPerDim);
       }
       BytesRef firstValue = points[0].packedValue;
       for (int i = 1; i < points.length; ++i) {
-        for (int dim = 0; dim < numDataDims; ++dim) {
-          int offset = dim * bytesPerDim;
+        for (int dim = 0; dim < config.numDims; ++dim) {
+          int offset = dim * config.bytesPerDim;
           BytesRef packedValue = points[i].packedValue;
           System.arraycopy(firstValue.bytes, firstValue.offset + offset, packedValue.bytes, packedValue.offset + offset, commonPrefixLengths[dim]);
         }
       }
     } else {
       //index dim are equal, data dims different
-      byte[] indexDims = new byte[numIndexdims * bytesPerDim];
+      int numDataDims = config.numDims - config.numIndexDims;
+      byte[] indexDims = new byte[config.packedIndexBytesLength];
       random().nextBytes(indexDims);
-      byte[] dataDims = new byte[(numDataDims - numIndexdims) * bytesPerDim];
+      byte[] dataDims = new byte[numDataDims * config.bytesPerDim];
       for (int i = 0; i < numPoints; ++i) {
-        byte[] value = new byte[packedBytesLength];
-        System.arraycopy(indexDims, 0, value, 0, numIndexdims * bytesPerDim);
+        byte[] value = new byte[config.packedBytesLength];
+        System.arraycopy(indexDims, 0, value, 0, config.packedIndexBytesLength);
         random().nextBytes(dataDims);
-        System.arraycopy(dataDims, 0, value, numIndexdims * bytesPerDim, (numDataDims - numIndexdims) * bytesPerDim);
+        System.arraycopy(dataDims, 0, value, config.packedIndexBytesLength, numDataDims * config.bytesPerDim);
         points[i] = new Point(value, random().nextInt(maxDoc));
       }
-      for (int i = 0; i < numIndexdims; ++i) {
-        commonPrefixLengths[i] = bytesPerDim;
+      for (int i = 0; i < config.numIndexDims; ++i) {
+        commonPrefixLengths[i] = config.bytesPerDim;
       }
-      for (int i = numDataDims; i < numDataDims; ++i) {
-        commonPrefixLengths[i] = TestUtil.nextInt(random(), 0, bytesPerDim);
+      for (int i = config.numIndexDims; i < config.numDims; ++i) {
+        commonPrefixLengths[i] = TestUtil.nextInt(random(), 0, config.bytesPerDim);
       }
       BytesRef firstValue = points[0].packedValue;
       for (int i = 1; i < points.length; ++i) {
-        for (int dim = numIndexdims; dim < numDataDims; ++dim) {
-          int offset = dim * bytesPerDim;
+        for (int dim = config.numIndexDims; dim < config.numDims; ++dim) {
+          int offset = dim * config.bytesPerDim;
           BytesRef packedValue = points[i].packedValue;
           System.arraycopy(firstValue.bytes, firstValue.offset + offset, packedValue.bytes, packedValue.offset + offset, commonPrefixLengths[dim]);
         }
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 5128e8d..8888d19 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
@@ -50,6 +50,7 @@ import org.apache.lucene.index.PointValues.IntersectVisitor;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
+import org.apache.lucene.util.bkd.BKDConfig;
 import org.apache.lucene.util.bkd.BKDWriter;
 
 /**
@@ -106,13 +107,16 @@ public class RandomCodec extends AssertingCodec {
 
             PointValues values = reader.getValues(fieldInfo.name);
 
+            BKDConfig config = new BKDConfig(fieldInfo.getPointDimensionCount(),
+                fieldInfo.getPointIndexDimensionCount(),
+                fieldInfo.getPointNumBytes(),
+                maxPointsInLeafNode);
+
+
             try (BKDWriter writer = new RandomlySplittingBKDWriter(writeState.segmentInfo.maxDoc(),
                                                                    writeState.directory,
                                                                    writeState.segmentInfo.name,
-                                                                   fieldInfo.getPointDimensionCount(),
-                                                                   fieldInfo.getPointIndexDimensionCount(),
-                                                                   fieldInfo.getPointNumBytes(),
-                                                                   maxPointsInLeafNode,
+                                                                   config,
                                                                    maxMBSortInHeap,
                                                                    values.size(),
                                                                    bkdSplitRandomSeed ^ fieldInfo.name.hashCode())) {
@@ -258,10 +262,9 @@ public class RandomCodec extends AssertingCodec {
 
     final Random random;
 
-    public RandomlySplittingBKDWriter(int maxDoc, Directory tempDir, String tempFileNamePrefix, int numDataDims, int numIndexDims,
-                                      int bytesPerDim, int maxPointsInLeafNode, double maxMBSortInHeap,
+    public RandomlySplittingBKDWriter(int maxDoc, Directory tempDir, String tempFileNamePrefix, BKDConfig config, double maxMBSortInHeap,
                                       long totalPointCount, int randomSeed) throws IOException {
-      super(maxDoc, tempDir, tempFileNamePrefix, numDataDims, numIndexDims, bytesPerDim, maxPointsInLeafNode, maxMBSortInHeap, totalPointCount);
+      super(maxDoc, tempDir, tempFileNamePrefix, config, maxMBSortInHeap, totalPointCount);
       this.random = new Random(randomSeed);
     }
 
@@ -286,7 +289,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(config.numIndexDims);
     }
   }
 }