You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by kr...@apache.org on 2016/12/15 21:35:23 UTC

[08/50] [abbrv] lucene-solr:jira/solr-8593: LUCENE-7563: use a compressed format for the in-heap BKD index

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5e8db2e0/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPointsWriter.java
----------------------------------------------------------------------
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPointsWriter.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPointsWriter.java
index c06c128..9d2db89 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
@@ -20,7 +20,6 @@ package org.apache.lucene.codecs.simpletext;
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
-import java.util.function.IntFunction;
 
 import org.apache.lucene.codecs.PointsReader;
 import org.apache.lucene.codecs.PointsWriter;
@@ -33,29 +32,28 @@ 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.BKDWriter;
 
 class SimpleTextPointsWriter extends PointsWriter {
 
-  final static BytesRef NUM_DIMS      = new BytesRef("num dims ");
-  final static BytesRef BYTES_PER_DIM = new BytesRef("bytes per dim ");
-  final static BytesRef MAX_LEAF_POINTS = new BytesRef("max leaf points ");
-  final static BytesRef INDEX_COUNT = new BytesRef("index count ");
-  final static BytesRef BLOCK_COUNT   = new BytesRef("block count ");
-  final static BytesRef BLOCK_DOC_ID  = new BytesRef("  doc ");
-  final static BytesRef BLOCK_FP      = new BytesRef("  block fp ");
-  final static BytesRef BLOCK_VALUE   = new BytesRef("  block value ");
-  final static BytesRef SPLIT_COUNT   = new BytesRef("split count ");
-  final static BytesRef SPLIT_DIM     = new BytesRef("  split dim ");
-  final static BytesRef SPLIT_VALUE   = new BytesRef("  split value ");
-  final static BytesRef FIELD_COUNT   = new BytesRef("field count ");
-  final static BytesRef FIELD_FP_NAME = new BytesRef("  field fp name ");
-  final static BytesRef FIELD_FP      = new BytesRef("  field fp ");
-  final static BytesRef MIN_VALUE     = new BytesRef("min value ");
-  final static BytesRef MAX_VALUE     = new BytesRef("max value ");
-  final static BytesRef POINT_COUNT   = new BytesRef("point count ");
-  final static BytesRef DOC_COUNT     = new BytesRef("doc count ");
-  final static BytesRef END           = new BytesRef("END");
+  public final static BytesRef NUM_DIMS      = new BytesRef("num dims ");
+  public final static BytesRef BYTES_PER_DIM = new BytesRef("bytes per dim ");
+  public final static BytesRef MAX_LEAF_POINTS = new BytesRef("max leaf points ");
+  public final static BytesRef INDEX_COUNT = new BytesRef("index count ");
+  public final static BytesRef BLOCK_COUNT   = new BytesRef("block count ");
+  public final static BytesRef BLOCK_DOC_ID  = new BytesRef("  doc ");
+  public final static BytesRef BLOCK_FP      = new BytesRef("  block fp ");
+  public final static BytesRef BLOCK_VALUE   = new BytesRef("  block value ");
+  public final static BytesRef SPLIT_COUNT   = new BytesRef("split count ");
+  public final static BytesRef SPLIT_DIM     = new BytesRef("  split dim ");
+  public final static BytesRef SPLIT_VALUE   = new BytesRef("  split value ");
+  public final static BytesRef FIELD_COUNT   = new BytesRef("field count ");
+  public final static BytesRef FIELD_FP_NAME = new BytesRef("  field fp name ");
+  public final static BytesRef FIELD_FP      = new BytesRef("  field fp ");
+  public final static BytesRef MIN_VALUE     = new BytesRef("min value ");
+  public final static BytesRef MAX_VALUE     = new BytesRef("max value ");
+  public final static BytesRef POINT_COUNT   = new BytesRef("point count ");
+  public final static BytesRef DOC_COUNT     = new BytesRef("doc count ");
+  public final static BytesRef END           = new BytesRef("END");
 
   private IndexOutput dataOut;
   final BytesRefBuilder scratch = new BytesRefBuilder();
@@ -75,105 +73,15 @@ class SimpleTextPointsWriter extends PointsWriter {
     boolean singleValuePerDoc = values.size() == values.getDocCount();
 
     // We use the normal BKDWriter, but subclass to customize how it writes the index and blocks to disk:
-    try (BKDWriter writer = new BKDWriter(writeState.segmentInfo.maxDoc(),
-                                          writeState.directory,
-                                          writeState.segmentInfo.name,
-                                          fieldInfo.getPointDimensionCount(),
-                                          fieldInfo.getPointNumBytes(),
-                                          BKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE,
-                                          BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP,
-                                          values.size(),
-                                          singleValuePerDoc) {
-
-        @Override
-        protected void writeIndex(IndexOutput out, long[] leafBlockFPs, byte[] splitPackedValues) throws IOException {
-          write(out, NUM_DIMS);
-          writeInt(out, numDims);
-          newline(out);
-
-          write(out, BYTES_PER_DIM);
-          writeInt(out, bytesPerDim);
-          newline(out);
-
-          write(out, MAX_LEAF_POINTS);
-          writeInt(out, maxPointsInLeafNode);
-          newline(out);
-
-          write(out, INDEX_COUNT);
-          writeInt(out, leafBlockFPs.length);
-          newline(out);
-
-          write(out, MIN_VALUE);
-          BytesRef br = new BytesRef(minPackedValue, 0, minPackedValue.length);
-          write(out, br.toString());
-          newline(out);
-
-          write(out, MAX_VALUE);
-          br = new BytesRef(maxPackedValue, 0, maxPackedValue.length);
-          write(out, br.toString());
-          newline(out);
-
-          write(out, POINT_COUNT);
-          writeLong(out, pointCount);
-          newline(out);
-
-          write(out, DOC_COUNT);
-          writeInt(out, docsSeen.cardinality());
-          newline(out);
-
-          for(int i=0;i<leafBlockFPs.length;i++) {
-            write(out, BLOCK_FP);
-            writeLong(out, leafBlockFPs[i]);
-            newline(out);
-          }
-
-          assert (splitPackedValues.length % (1 + fieldInfo.getPointNumBytes())) == 0;
-          int count = splitPackedValues.length / (1 + fieldInfo.getPointNumBytes());
-          assert count == leafBlockFPs.length;
-
-          write(out, SPLIT_COUNT);
-          writeInt(out, count);
-          newline(out);
-
-          for(int i=0;i<count;i++) {
-            write(out, SPLIT_DIM);
-            writeInt(out, splitPackedValues[i * (1 + fieldInfo.getPointNumBytes())] & 0xff);
-            newline(out);
-            write(out, SPLIT_VALUE);
-            br = new BytesRef(splitPackedValues, 1+(i * (1+fieldInfo.getPointNumBytes())), fieldInfo.getPointNumBytes());
-            write(out, br.toString());
-            newline(out);
-          }
-        }
-
-        @Override
-        protected void writeLeafBlockDocs(IndexOutput out, int[] docIDs, int start, int count) throws IOException {
-          write(out, BLOCK_COUNT);
-          writeInt(out, count);
-          newline(out);
-          for(int i=0;i<count;i++) {
-            write(out, BLOCK_DOC_ID);
-            writeInt(out, docIDs[start+i]);
-            newline(out);
-          }
-        }
-
-        @Override
-        protected void writeCommonPrefixes(IndexOutput out, int[] commonPrefixLengths, byte[] packedValue) {
-          // NOTE: we don't do prefix coding, so we ignore commonPrefixLengths
-        }
-
-        @Override
-        protected void writeLeafBlockPackedValues(IndexOutput out, int[] commonPrefixLengths, int count, int sortedDim, IntFunction<BytesRef> packedValues) throws IOException {
-          for (int i = 0; i < count; ++i) {
-            BytesRef packedValue = packedValues.apply(i);
-            // NOTE: we don't do prefix coding, so we ignore commonPrefixLengths
-            write(out, BLOCK_VALUE);
-            write(out, packedValue.toString());
-            newline(out);
-          }
-        }
-      }) {
+    try (SimpleTextBKDWriter writer = new SimpleTextBKDWriter(writeState.segmentInfo.maxDoc(),
+                                                              writeState.directory,
+                                                              writeState.segmentInfo.name,
+                                                              fieldInfo.getPointDimensionCount(),
+                                                              fieldInfo.getPointNumBytes(),
+                                                              SimpleTextBKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE,
+                                                              SimpleTextBKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP,
+                                                              values.size(),
+                                                              singleValuePerDoc)) {
 
       values.intersect(new IntersectVisitor() {
           @Override
@@ -198,26 +106,6 @@ class SimpleTextPointsWriter extends PointsWriter {
     }
   }
 
-  private void write(IndexOutput out, String s) throws IOException {
-    SimpleTextUtil.write(out, s, scratch);
-  }
-
-  private void writeInt(IndexOutput out, int x) throws IOException {
-    SimpleTextUtil.write(out, Integer.toString(x), scratch);
-  }
-
-  private void writeLong(IndexOutput out, long x) throws IOException {
-    SimpleTextUtil.write(out, Long.toString(x), scratch);
-  }
-
-  private void write(IndexOutput out, BytesRef b) throws IOException {
-    SimpleTextUtil.write(out, b);
-  }
-
-  private void newline(IndexOutput out) throws IOException {
-    SimpleTextUtil.writeNewline(out);
-  }
-
   @Override
   public void finish() throws IOException {
     SimpleTextUtil.write(dataOut, END);
@@ -250,4 +138,24 @@ class SimpleTextPointsWriter extends PointsWriter {
       }
     }
   }
+
+  private void write(IndexOutput out, String s) throws IOException {
+    SimpleTextUtil.write(out, s, scratch);
+  }
+
+  private void writeInt(IndexOutput out, int x) throws IOException {
+    SimpleTextUtil.write(out, Integer.toString(x), scratch);
+  }
+
+  private void writeLong(IndexOutput out, long x) throws IOException {
+    SimpleTextUtil.write(out, Long.toString(x), scratch);
+  }
+
+  private void write(IndexOutput out, BytesRef b) throws IOException {
+    SimpleTextUtil.write(out, b);
+  }
+
+  private void newline(IndexOutput out) throws IOException {
+    SimpleTextUtil.writeNewline(out);
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5e8db2e0/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsFormat.java
index e558d0d..1d2285c 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsFormat.java
@@ -28,7 +28,8 @@ import org.apache.lucene.index.SegmentWriteState;
 
 /**
  * Lucene 6.0 point format, which encodes dimensional values in a block KD-tree structure
- * for fast shape intersection filtering. See <a href="https://www.cs.duke.edu/~pankaj/publications/papers/bkd-sstd.pdf">this paper</a> for details.
+ * for fast 1D range and N dimesional shape intersection filtering.
+ * See <a href="https://www.cs.duke.edu/~pankaj/publications/papers/bkd-sstd.pdf">this paper</a> for details.
  *
  * <p>This data structure is written as a series of blocks on disk, with an in-memory perfectly balanced
  * binary tree of split values referencing those blocks at the leaves.
@@ -50,10 +51,13 @@ import org.apache.lucene.index.SegmentWriteState;
  *  <li> maxPointsInLeafNode (vInt)
  *  <li> bytesPerDim (vInt)
  *  <li> count (vInt)
- *  <li> byte[bytesPerDim]<sup>count</sup> (packed <code>byte[]</code> all split values)
- *  <li> delta-blockFP (vLong)<sup>count</sup> (delta-coded file pointers to the on-disk leaf blocks))
+ *  <li> packed index (byte[])
  * </ul>
  *
+ * <p>The packed index uses hierarchical delta and prefix coding to compactly encode the file pointer for
+ * all leaf blocks, once the tree is traversed, as well as the split dimension and split value for each
+ * inner node of the tree.
+ *
  * <p>After all fields blocks + index data are written, {@link CodecUtil#writeFooter} writes the checksum.
  *
  * <p>The <code>.dii</code> file records the file pointer in the <code>.dim</code> file where each field's

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5e8db2e0/lucene/core/src/java/org/apache/lucene/codecs/lucene60/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/package-info.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/package-info.java
index 8968a6d..a914001 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/package-info.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/package-info.java
@@ -16,7 +16,7 @@
  */
 
 /**
- * Components from the Lucene 6.0 index format.  See {@link org.apache.lucene.codecs.lucene62}
- * for an overview of the index format.
+ * Components from the Lucene 6.0 index format.  See {@link org.apache.lucene.codecs.lucene70}
+ * for an overview of the current index format.
  */
 package org.apache.lucene.codecs.lucene60;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5e8db2e0/lucene/core/src/java/org/apache/lucene/codecs/lucene62/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene62/package-info.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene62/package-info.java
index 2fe2dc7..fb55673 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene62/package-info.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene62/package-info.java
@@ -17,8 +17,8 @@
 
 /**
  * Components from the Lucene 6.2 index format
- * See {@link org.apache.lucene.codecs.lucene62} for an overview
- * of the index format.
+ * See {@link org.apache.lucene.codecs.lucene70} for an overview
+ * of the current index format.
  */
 
 package org.apache.lucene.codecs.lucene62;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5e8db2e0/lucene/core/src/java/org/apache/lucene/codecs/lucene70/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/package-info.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/package-info.java
index 9b432f7..cab2859 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/package-info.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/package-info.java
@@ -185,6 +185,12 @@
  * {@link org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat Live documents}. 
  * An optional file indicating which documents are live.
  * </li>
+ * <li>
+ * {@link org.apache.lucene.codecs.lucene60.Lucene60PointsFormat Point values}.
+ * Optional pair of files, recording dimensionally indexed fields, to enable fast
+ * numeric range filtering and large numeric values like BigInteger and BigDecimal (1D)
+ * and geographic shape intersection (2D, 3D).
+ * </li>
  * </ul>
  * <p>Details on each of these are provided in their linked pages.</p>
  * </div>
@@ -300,7 +306,12 @@
  * <tr>
  * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat Live Documents}</td>
  * <td>.liv</td>
- * <td>Info about what files are live</td>
+ * <td>Info about what documents are live</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene60.Lucene60PointsFormat Point values}</td>
+ * <td>.dii, .dim</td>
+ * <td>Holds indexed points, if any</td>
  * </tr>
  * </table>
  * </div>
@@ -374,6 +385,8 @@
  * that is suitable for faceting/sorting/analytics.
  * <li>In version 5.4, DocValues have been improved to store more information on disk:
  * addresses for binary fields and ord indexes for multi-valued fields.
+ * <li>In version 6.0, Points were added, for multi-dimensional range/distance search.
+ * <li>In version 6.2, new Segment info format that reads/writes the index sort, to support index sorting.
  * <li>In version 7.0, DocValues have been improved to better support sparse doc values
  * thanks to an iterator API.
  * </li>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5e8db2e0/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
index 7bc08f3..fd8011d 100644
--- a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
+++ b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
@@ -1801,161 +1801,32 @@ public final class CheckIndex implements Closeable {
         }
         for (FieldInfo fieldInfo : fieldInfos) {
           if (fieldInfo.getPointDimensionCount() > 0) {
-            FixedBitSet docsSeen = new FixedBitSet(reader.maxDoc());
-            status.totalValueFields++;
-            int dimCount = fieldInfo.getPointDimensionCount();
-            int bytesPerDim = fieldInfo.getPointNumBytes();
-            int packedBytesCount = dimCount * bytesPerDim;
-            byte[] lastMinPackedValue = new byte[packedBytesCount];
-            byte[] lastMaxPackedValue = new byte[packedBytesCount];
-            BytesRef scratch = new BytesRef();
-            scratch.length = bytesPerDim;
-            byte[] lastPackedValue = new byte[packedBytesCount];
-
-            long[] pointCountSeen = new long[1];
-
             PointValues values = pointsReader.getValues(fieldInfo.name);
             if (values == null) {
               continue;
             }
-            byte[] globalMinPackedValue = values.getMinPackedValue();
+
+            status.totalValueFields++;
+
             long size = values.size();
             int docCount = values.getDocCount();
 
-            if (docCount > size) {
-              throw new RuntimeException("point values for field \"" + fieldInfo.name + "\" claims to have size=" + size + " points and inconsistent docCount=" + docCount);
-            }
+            VerifyPointsVisitor visitor = new VerifyPointsVisitor(fieldInfo.name, reader.maxDoc(), values);
+            values.intersect(visitor);
 
-            if (docCount > reader.maxDoc()) {
-              throw new RuntimeException("point values for field \"" + fieldInfo.name + "\" claims to have docCount=" + docCount + " but that's greater than maxDoc=" + reader.maxDoc());
+            if (visitor.getPointCountSeen() != size) {
+              throw new RuntimeException("point values for field \"" + fieldInfo.name + "\" claims to have size=" + size + " points, but in fact has " + visitor.getPointCountSeen());
             }
 
-            if (globalMinPackedValue == null) {
-              if (size != 0) {
-                throw new RuntimeException("getMinPackedValue is null points for field \"" + fieldInfo.name + "\" yet size=" + size);
-              }
-            } else if (globalMinPackedValue.length != packedBytesCount) {
-              throw new RuntimeException("getMinPackedValue for field \"" + fieldInfo.name + "\" return length=" + globalMinPackedValue.length + " array, but should be " + packedBytesCount);
-            }
-            byte[] globalMaxPackedValue = values.getMaxPackedValue();
-            if (globalMaxPackedValue == null) {
-              if (size != 0) {
-                throw new RuntimeException("getMaxPackedValue is null points for field \"" + fieldInfo.name + "\" yet size=" + size);
-              }
-            } else if (globalMaxPackedValue.length != packedBytesCount) {
-              throw new RuntimeException("getMaxPackedValue for field \"" + fieldInfo.name + "\" return length=" + globalMaxPackedValue.length + " array, but should be " + packedBytesCount);
-            }
-
-            values.intersect(new PointValues.IntersectVisitor() {
-
-                               private int lastDocID = -1;
-
-                               @Override
-                               public void visit(int docID) {
-                                 throw new RuntimeException("codec called IntersectVisitor.visit without a packed value for docID=" + docID);
-                               }
-
-                               @Override
-                               public void visit(int docID, byte[] packedValue) {
-                                 checkPackedValue("packed value", packedValue, docID);
-                                 pointCountSeen[0]++;
-                                 docsSeen.set(docID);
-
-                                 for(int dim=0;dim<dimCount;dim++) {
-                                   int offset = bytesPerDim * dim;
-
-                                   // Compare to last cell:
-                                   if (StringHelper.compare(bytesPerDim, packedValue, offset, lastMinPackedValue, offset) < 0) {
-                                     // This doc's point, in this dimension, is lower than the minimum value of the last cell checked:
-                                     throw new RuntimeException("packed points value " + Arrays.toString(packedValue) + " for field=\"" + fieldInfo.name + "\", docID=" + docID + " is out-of-bounds of the last cell min=" + Arrays.toString(lastMinPackedValue) + " max=" + Arrays.toString(lastMaxPackedValue) + " dim=" + dim);
-                                   }
-
-                                   if (StringHelper.compare(bytesPerDim, packedValue, offset, lastMaxPackedValue, offset) > 0) {
-                                     // This doc's point, in this dimension, is greater than the maximum value of the last cell checked:
-                                     throw new RuntimeException("packed points value " + Arrays.toString(packedValue) + " for field=\"" + fieldInfo.name + "\", docID=" + docID + " is out-of-bounds of the last cell min=" + Arrays.toString(lastMinPackedValue) + " max=" + Arrays.toString(lastMaxPackedValue) + " dim=" + dim);
-                                   }
-                                 }
-
-                                 // In the 1D case, PointValues must make a single in-order sweep through all values, and tie-break by
-                                 // increasing docID:
-                                 if (dimCount == 1) {
-                                   int cmp = StringHelper.compare(bytesPerDim, lastPackedValue, 0, packedValue, 0);
-                                   if (cmp > 0) {
-                                     throw new RuntimeException("packed points value " + Arrays.toString(packedValue) + " for field=\"" + fieldInfo.name + "\", for docID=" + docID + " is out-of-order vs the previous document's value " + Arrays.toString(lastPackedValue));
-                                   } else if (cmp == 0) {
-                                     if (docID < lastDocID) {
-                                       throw new RuntimeException("packed points value is the same, but docID=" + docID + " is out of order vs previous docID=" + lastDocID + ", field=\"" + fieldInfo.name + "\"");
-                                     }
-                                   }
-                                   System.arraycopy(packedValue, 0, lastPackedValue, 0, bytesPerDim);
-                                   lastDocID = docID;
-                                 }
-
-                                 status.totalValuePoints++;
-                               }
-
-                               @Override
-                               public PointValues.Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
-                                 checkPackedValue("min packed value", minPackedValue, -1);
-                                 System.arraycopy(minPackedValue, 0, lastMinPackedValue, 0, packedBytesCount);
-                                 checkPackedValue("max packed value", maxPackedValue, -1);
-                                 System.arraycopy(maxPackedValue, 0, lastMaxPackedValue, 0, packedBytesCount);
-
-                                 for(int dim=0;dim<dimCount;dim++) {
-                                   int offset = bytesPerDim * dim;
-
-                                   if (StringHelper.compare(bytesPerDim, minPackedValue, offset, maxPackedValue, offset) > 0) {
-                                     throw new RuntimeException("packed points cell minPackedValue " + Arrays.toString(minPackedValue) +
-                                                                " is out-of-bounds of the cell's maxPackedValue " + Arrays.toString(maxPackedValue) + " dim=" + dim + " field=\"" + fieldInfo.name + "\"");
-                                   }
-
-                                   // Make sure this cell is not outside of the global min/max:
-                                   if (StringHelper.compare(bytesPerDim, minPackedValue, offset, globalMinPackedValue, offset) < 0) {
-                                     throw new RuntimeException("packed points cell minPackedValue " + Arrays.toString(minPackedValue) +
-                                                                " is out-of-bounds of the global minimum " + Arrays.toString(globalMinPackedValue) + " dim=" + dim + " field=\"" + fieldInfo.name + "\"");
-                                   }
-
-                                   if (StringHelper.compare(bytesPerDim, maxPackedValue, offset, globalMinPackedValue, offset) < 0) {
-                                     throw new RuntimeException("packed points cell maxPackedValue " + Arrays.toString(maxPackedValue) +
-                                                                " is out-of-bounds of the global minimum " + Arrays.toString(globalMinPackedValue) + " dim=" + dim + " field=\"" + fieldInfo.name + "\"");
-                                   }
-
-                                   if (StringHelper.compare(bytesPerDim, minPackedValue, offset, globalMaxPackedValue, offset) > 0) {
-                                     throw new RuntimeException("packed points cell minPackedValue " + Arrays.toString(minPackedValue) +
-                                                                " is out-of-bounds of the global maximum " + Arrays.toString(globalMaxPackedValue) + " dim=" + dim + " field=\"" + fieldInfo.name + "\"");
-                                   }
-                                   if (StringHelper.compare(bytesPerDim, maxPackedValue, offset, globalMaxPackedValue, offset) > 0) {
-                                     throw new RuntimeException("packed points cell maxPackedValue " + Arrays.toString(maxPackedValue) +
-                                                                " is out-of-bounds of the global maximum " + Arrays.toString(globalMaxPackedValue) + " dim=" + dim + " field=\"" + fieldInfo.name + "\"");
-                                   }
-                                 }                                   
-
-                                 // We always pretend the query shape is so complex that it crosses every cell, so
-                                 // that packedValue is passed for every document
-                                 return PointValues.Relation.CELL_CROSSES_QUERY;
-                               }
-
-                               private void checkPackedValue(String desc, byte[] packedValue, int docID) {
-                                 if (packedValue == null) {
-                                   throw new RuntimeException(desc + " is null for docID=" + docID + " field=\"" + fieldInfo.name + "\"");
-                                 }
-
-                                 if (packedValue.length != packedBytesCount) {
-                                   throw new RuntimeException(desc + " has incorrect length=" + packedValue.length + " vs expected=" + packedBytesCount + " for docID=" + docID + " field=\"" + fieldInfo.name + "\"");
-                                 }
-                               }
-                             });
-
-            if (pointCountSeen[0] != size) {
-              throw new RuntimeException("point values for field \"" + fieldInfo.name + "\" claims to have size=" + size + " points, but in fact has " + pointCountSeen[0]);
+            if (visitor.getDocCountSeen() != docCount) {
+              throw new RuntimeException("point values for field \"" + fieldInfo.name + "\" claims to have docCount=" + docCount + " but in fact has " + visitor.getDocCountSeen());
             }
 
-            if (docsSeen.cardinality() != docCount) {
-              throw new RuntimeException("point values for field \"" + fieldInfo.name + "\" claims to have docCount=" + docCount + " but in fact has " + docsSeen.cardinality());
-            }
+            status.totalValuePoints += visitor.getPointCountSeen();
           }
         }
       }
+
       msg(infoStream, String.format(Locale.ROOT, "OK [%d fields, %d points] [took %.3f sec]", status.totalValueFields, status.totalValuePoints, nsToSec(System.nanoTime()-startNS)));
 
     } catch (Throwable e) {
@@ -1972,6 +1843,167 @@ public final class CheckIndex implements Closeable {
     return status;
   }
 
+  /** Walks the entire N-dimensional points space, verifying that all points fall within the last cell's boundaries.
+   *
+   * @lucene.internal */
+  public static class VerifyPointsVisitor implements PointValues.IntersectVisitor {
+    private long pointCountSeen;
+    private int lastDocID = -1;
+    private final int maxDoc;
+    private final FixedBitSet docsSeen;
+    private final byte[] lastMinPackedValue;
+    private final byte[] lastMaxPackedValue;
+    private final byte[] lastPackedValue;
+    private final byte[] globalMinPackedValue;
+    private final byte[] globalMaxPackedValue;
+    private final int packedBytesCount;
+    private final int numDims;
+    private final int bytesPerDim;
+    private final String fieldName;
+
+    /** Sole constructor */
+    public VerifyPointsVisitor(String fieldName, int maxDoc, PointValues values) throws IOException {
+      this.maxDoc = maxDoc;
+      this.fieldName = fieldName;
+      numDims = values.getNumDimensions();
+      bytesPerDim = values.getBytesPerDimension();
+      packedBytesCount = numDims * bytesPerDim;
+      globalMinPackedValue = values.getMinPackedValue();
+      globalMaxPackedValue = values.getMaxPackedValue();
+      docsSeen = new FixedBitSet(maxDoc);
+      lastMinPackedValue = new byte[packedBytesCount];
+      lastMaxPackedValue = new byte[packedBytesCount];
+      lastPackedValue = new byte[packedBytesCount];
+
+      if (values.getDocCount() > values.size()) {
+        throw new RuntimeException("point values for field \"" + fieldName + "\" claims to have size=" + values.size() + " points and inconsistent docCount=" + values.getDocCount());
+      }
+
+      if (values.getDocCount() > maxDoc) {
+        throw new RuntimeException("point values for field \"" + fieldName + "\" claims to have docCount=" + values.getDocCount() + " but that's greater than maxDoc=" + maxDoc);
+      }
+
+      if (globalMinPackedValue == null) {
+        if (values.size() != 0) {
+          throw new RuntimeException("getMinPackedValue is null points for field \"" + fieldName + "\" yet size=" + values.size());
+        }
+      } else if (globalMinPackedValue.length != packedBytesCount) {
+        throw new RuntimeException("getMinPackedValue for field \"" + fieldName + "\" return length=" + globalMinPackedValue.length + " array, but should be " + packedBytesCount);
+      }
+      if (globalMaxPackedValue == null) {
+        if (values.size() != 0) {
+          throw new RuntimeException("getMaxPackedValue is null points for field \"" + fieldName + "\" yet size=" + values.size());
+        }
+      } else if (globalMaxPackedValue.length != packedBytesCount) {
+        throw new RuntimeException("getMaxPackedValue for field \"" + fieldName + "\" return length=" + globalMaxPackedValue.length + " array, but should be " + packedBytesCount);
+      }
+    }
+
+    /** Returns total number of points in this BKD tree */
+    public long getPointCountSeen() {
+      return pointCountSeen;
+    }
+
+    /** Returns total number of unique docIDs in this BKD tree */
+    public long getDocCountSeen() {
+      return docsSeen.cardinality();
+    }
+
+    @Override
+    public void visit(int docID) {
+      throw new RuntimeException("codec called IntersectVisitor.visit without a packed value for docID=" + docID);
+    }
+
+    @Override
+    public void visit(int docID, byte[] packedValue) {
+      checkPackedValue("packed value", packedValue, docID);
+      pointCountSeen++;
+      docsSeen.set(docID);
+
+      for(int dim=0;dim<numDims;dim++) {
+        int offset = bytesPerDim * dim;
+
+        // Compare to last cell:
+        if (StringHelper.compare(bytesPerDim, packedValue, offset, lastMinPackedValue, offset) < 0) {
+          // This doc's point, in this dimension, is lower than the minimum value of the last cell checked:
+          throw new RuntimeException("packed points value " + Arrays.toString(packedValue) + " for field=\"" + fieldName + "\", docID=" + docID + " is out-of-bounds of the last cell min=" + Arrays.toString(lastMinPackedValue) + " max=" + Arrays.toString(lastMaxPackedValue) + " dim=" + dim);
+        }
+
+        if (StringHelper.compare(bytesPerDim, packedValue, offset, lastMaxPackedValue, offset) > 0) {
+          // This doc's point, in this dimension, is greater than the maximum value of the last cell checked:
+          throw new RuntimeException("packed points value " + Arrays.toString(packedValue) + " for field=\"" + fieldName + "\", docID=" + docID + " is out-of-bounds of the last cell min=" + Arrays.toString(lastMinPackedValue) + " max=" + Arrays.toString(lastMaxPackedValue) + " dim=" + dim);
+        }
+      }
+
+      // In the 1D case, PointValues must make a single in-order sweep through all values, and tie-break by
+      // increasing docID:
+      if (numDims == 1) {
+        int cmp = StringHelper.compare(bytesPerDim, lastPackedValue, 0, packedValue, 0);
+        if (cmp > 0) {
+          throw new RuntimeException("packed points value " + Arrays.toString(packedValue) + " for field=\"" + fieldName + "\", for docID=" + docID + " is out-of-order vs the previous document's value " + Arrays.toString(lastPackedValue));
+        } else if (cmp == 0) {
+          if (docID < lastDocID) {
+            throw new RuntimeException("packed points value is the same, but docID=" + docID + " is out of order vs previous docID=" + lastDocID + ", field=\"" + fieldName + "\"");
+          }
+        }
+        System.arraycopy(packedValue, 0, lastPackedValue, 0, bytesPerDim);
+        lastDocID = docID;
+      }
+    }
+
+    @Override
+    public PointValues.Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
+      checkPackedValue("min packed value", minPackedValue, -1);
+      System.arraycopy(minPackedValue, 0, lastMinPackedValue, 0, packedBytesCount);
+      checkPackedValue("max packed value", maxPackedValue, -1);
+      System.arraycopy(maxPackedValue, 0, lastMaxPackedValue, 0, packedBytesCount);
+
+      for(int dim=0;dim<numDims;dim++) {
+        int offset = bytesPerDim * dim;
+
+        if (StringHelper.compare(bytesPerDim, minPackedValue, offset, maxPackedValue, offset) > 0) {
+          throw new RuntimeException("packed points cell minPackedValue " + Arrays.toString(minPackedValue) +
+                                     " is out-of-bounds of the cell's maxPackedValue " + Arrays.toString(maxPackedValue) + " dim=" + dim + " field=\"" + fieldName + "\"");
+        }
+
+        // Make sure this cell is not outside of the global min/max:
+        if (StringHelper.compare(bytesPerDim, minPackedValue, offset, globalMinPackedValue, offset) < 0) {
+          throw new RuntimeException("packed points cell minPackedValue " + Arrays.toString(minPackedValue) +
+                                     " is out-of-bounds of the global minimum " + Arrays.toString(globalMinPackedValue) + " dim=" + dim + " field=\"" + fieldName + "\"");
+        }
+
+        if (StringHelper.compare(bytesPerDim, maxPackedValue, offset, globalMinPackedValue, offset) < 0) {
+          throw new RuntimeException("packed points cell maxPackedValue " + Arrays.toString(maxPackedValue) +
+                                     " is out-of-bounds of the global minimum " + Arrays.toString(globalMinPackedValue) + " dim=" + dim + " field=\"" + fieldName + "\"");
+        }
+
+        if (StringHelper.compare(bytesPerDim, minPackedValue, offset, globalMaxPackedValue, offset) > 0) {
+          throw new RuntimeException("packed points cell minPackedValue " + Arrays.toString(minPackedValue) +
+                                     " is out-of-bounds of the global maximum " + Arrays.toString(globalMaxPackedValue) + " dim=" + dim + " field=\"" + fieldName + "\"");
+        }
+        if (StringHelper.compare(bytesPerDim, maxPackedValue, offset, globalMaxPackedValue, offset) > 0) {
+          throw new RuntimeException("packed points cell maxPackedValue " + Arrays.toString(maxPackedValue) +
+                                     " is out-of-bounds of the global maximum " + Arrays.toString(globalMaxPackedValue) + " dim=" + dim + " field=\"" + fieldName + "\"");
+        }
+      }                                   
+
+      // We always pretend the query shape is so complex that it crosses every cell, so
+      // that packedValue is passed for every document
+      return PointValues.Relation.CELL_CROSSES_QUERY;
+    }
+
+    private void checkPackedValue(String desc, byte[] packedValue, int docID) {
+      if (packedValue == null) {
+        throw new RuntimeException(desc + " is null for docID=" + docID + " field=\"" + fieldName + "\"");
+      }
+
+      if (packedValue.length != packedBytesCount) {
+        throw new RuntimeException(desc + " has incorrect length=" + packedValue.length + " vs expected=" + packedBytesCount + " for docID=" + docID + " field=\"" + fieldName + "\"");
+      }
+    }
+  }
+
+  
   /**
    * Test stored fields.
    * @lucene.experimental

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5e8db2e0/lucene/core/src/java/org/apache/lucene/util/bkd/BKDReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/bkd/BKDReader.java b/lucene/core/src/java/org/apache/lucene/util/bkd/BKDReader.java
index 6bf7dfc..6cccf4c 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
@@ -17,14 +17,15 @@
 package org.apache.lucene.util.bkd;
 
 import java.io.IOException;
-import java.util.Arrays;
 
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.PointValues;
+import org.apache.lucene.store.ByteArrayDataInput;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.MathUtil;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.StringHelper;
 
@@ -32,14 +33,12 @@ import org.apache.lucene.util.StringHelper;
  *
  * @lucene.experimental */
 
-public class BKDReader extends PointValues implements Accountable {
+public final class BKDReader extends PointValues implements Accountable {
   // Packed array of byte[] holding all split values in the full binary tree:
-  final private byte[] splitPackedValues; 
-  final long[] leafBlockFPs;
-  final private int leafNodeOffset;
+  final int leafNodeOffset;
   final int numDims;
   final int bytesPerDim;
-  final int bytesPerIndexEntry;
+  final int numLeaves;
   final IndexInput in;
   final int maxPointsInLeafNode;
   final byte[] minPackedValue;
@@ -49,6 +48,14 @@ public class BKDReader extends PointValues implements Accountable {
   final int version;
   protected final int packedBytesLength;
 
+  // Used for 6.4.0+ index format:
+  final byte[] packedIndex;
+
+  // Used for Legacy (pre-6.4.0) index format, to hold a compact form of the index:
+  final private byte[] splitPackedValues; 
+  final int bytesPerIndexEntry;
+  final long[] leafBlockFPs;
+
   /** Caller must pre-seek the provided {@link IndexInput} to the index location that {@link BKDWriter#finish} returned */
   public BKDReader(IndexInput in) throws IOException {
     version = CodecUtil.checkHeader(in, BKDWriter.CODEC_NAME, BKDWriter.VERSION_START, BKDWriter.VERSION_CURRENT);
@@ -59,7 +66,7 @@ public class BKDReader extends PointValues implements Accountable {
     packedBytesLength = numDims * bytesPerDim;
 
     // Read index:
-    int numLeaves = in.readVInt();
+    numLeaves = in.readVInt();
     assert numLeaves > 0;
     leafNodeOffset = numLeaves;
 
@@ -78,205 +85,380 @@ public class BKDReader extends PointValues implements Accountable {
     pointCount = in.readVLong();
     docCount = in.readVInt();
 
-    splitPackedValues = new byte[bytesPerIndexEntry*numLeaves];
-
-    // TODO: don't write split packed values[0]!
-    in.readBytes(splitPackedValues, 0, splitPackedValues.length);
-
-    // Read the file pointers to the start of each leaf block:
-    long[] leafBlockFPs = new long[numLeaves];
-    long lastFP = 0;
-    for(int i=0;i<numLeaves;i++) {
-      long delta = in.readVLong();
-      leafBlockFPs[i] = lastFP + delta;
-      lastFP += delta;
-    }
-
-    // Possibly rotate the leaf block FPs, if the index not fully balanced binary tree (only happens
-    // if it was created by BKDWriter.merge).  In this case the leaf nodes may straddle the two bottom
-    // levels of the binary tree:
-    if (numDims == 1 && numLeaves > 1) {
-      //System.out.println("BKDR: numLeaves=" + numLeaves);
-      int levelCount = 2;
-      while (true) {
-        //System.out.println("  cycle levelCount=" + levelCount);
-        if (numLeaves >= levelCount && numLeaves <= 2*levelCount) {
-          int lastLevel = 2*(numLeaves - levelCount);
-          assert lastLevel >= 0;
-          /*
-          System.out.println("BKDR: lastLevel=" + lastLevel + " vs " + levelCount);
-          System.out.println("FPs before:");
-          for(int i=0;i<leafBlockFPs.length;i++) {
-            System.out.println("  " + i + " " + leafBlockFPs[i]);
-          }
-          */
-          if (lastLevel != 0) {
-            // Last level is partially filled, so we must rotate the leaf FPs to match.  We do this here, after loading
-            // at read-time, so that we can still delta code them on disk at write:
-            //System.out.println("BKDR: now rotate index");
-            long[] newLeafBlockFPs = new long[numLeaves];
-            System.arraycopy(leafBlockFPs, lastLevel, newLeafBlockFPs, 0, leafBlockFPs.length - lastLevel);
-            System.arraycopy(leafBlockFPs, 0, newLeafBlockFPs, leafBlockFPs.length - lastLevel, lastLevel);
-            leafBlockFPs = newLeafBlockFPs;
-          }
-          /*
-          System.out.println("FPs:");
-          for(int i=0;i<leafBlockFPs.length;i++) {
-            System.out.println("  " + i + " " + leafBlockFPs[i]);
+    if (version >= BKDWriter.VERSION_PACKED_INDEX) {
+      int numBytes = in.readVInt();
+      packedIndex = new byte[numBytes];
+      in.readBytes(packedIndex, 0, numBytes);
+      leafBlockFPs = null;
+      splitPackedValues = null;
+    } else {
+      // legacy un-packed index
+
+      splitPackedValues = new byte[bytesPerIndexEntry*numLeaves];
+
+      in.readBytes(splitPackedValues, 0, splitPackedValues.length);
+
+      // Read the file pointers to the start of each leaf block:
+      long[] leafBlockFPs = new long[numLeaves];
+      long lastFP = 0;
+      for(int i=0;i<numLeaves;i++) {
+        long delta = in.readVLong();
+        leafBlockFPs[i] = lastFP + delta;
+        lastFP += delta;
+      }
+
+      // Possibly rotate the leaf block FPs, if the index not fully balanced binary tree (only happens
+      // if it was created by BKDWriter.merge or OneDimWriter).  In this case the leaf nodes may straddle the two bottom
+      // levels of the binary tree:
+      if (numDims == 1 && numLeaves > 1) {
+        int levelCount = 2;
+        while (true) {
+          if (numLeaves >= levelCount && numLeaves <= 2*levelCount) {
+            int lastLevel = 2*(numLeaves - levelCount);
+            assert lastLevel >= 0;
+            if (lastLevel != 0) {
+              // Last level is partially filled, so we must rotate the leaf FPs to match.  We do this here, after loading
+              // at read-time, so that we can still delta code them on disk at write:
+              long[] newLeafBlockFPs = new long[numLeaves];
+              System.arraycopy(leafBlockFPs, lastLevel, newLeafBlockFPs, 0, leafBlockFPs.length - lastLevel);
+              System.arraycopy(leafBlockFPs, 0, newLeafBlockFPs, leafBlockFPs.length - lastLevel, lastLevel);
+              leafBlockFPs = newLeafBlockFPs;
+            }
+            break;
           }
-          */
-          break;
-        }
 
-        levelCount *= 2;
+          levelCount *= 2;
+        }
       }
+      
+      this.leafBlockFPs = leafBlockFPs;
+      packedIndex = null;
     }
 
-    this.leafBlockFPs = leafBlockFPs;
     this.in = in;
   }
 
-  /** Called by consumers that have their own on-disk format for the index (e.g. SimpleText) */
-  protected BKDReader(IndexInput in, int numDims, int maxPointsInLeafNode, int bytesPerDim, long[] leafBlockFPs, byte[] splitPackedValues,
-                      byte[] minPackedValue, byte[] maxPackedValue, long pointCount, int docCount) throws IOException {
-    this.in = in;
-    this.numDims = numDims;
-    this.maxPointsInLeafNode = maxPointsInLeafNode;
-    this.bytesPerDim = bytesPerDim;
-    // no version check here because callers of this API (SimpleText) have no back compat:
-    bytesPerIndexEntry = numDims == 1 ? bytesPerDim : bytesPerDim + 1;
-    packedBytesLength = numDims * bytesPerDim;
-    this.leafNodeOffset = leafBlockFPs.length;
-    this.leafBlockFPs = leafBlockFPs;
-    this.splitPackedValues = splitPackedValues;
-    this.minPackedValue = minPackedValue;
-    this.maxPackedValue = maxPackedValue;
-    this.pointCount = pointCount;
-    this.docCount = docCount;
-    this.version = BKDWriter.VERSION_CURRENT;
-    assert minPackedValue.length == packedBytesLength;
-    assert maxPackedValue.length == packedBytesLength;
+  long getMinLeafBlockFP() {
+    if (packedIndex != null) {
+      return new ByteArrayDataInput(packedIndex).readVLong();
+    } else {
+      long minFP = Long.MAX_VALUE;
+      for(long fp : leafBlockFPs) {
+        minFP = Math.min(minFP, fp);
+      }
+      return minFP;
+    }
   }
 
-  private static class VerifyVisitor implements IntersectVisitor {
-    byte[] cellMinPacked;
-    byte[] cellMaxPacked;
-    byte[] lastPackedValue;
-    final int numDims;
-    final int bytesPerDim;
-    final int maxDoc;
+  /** Used to walk the in-heap index
+   *
+   * @lucene.internal */
+  public abstract class IndexTree implements Cloneable {
+    protected int nodeID;
+    // level is 1-based so that we can do level-1 w/o checking each time:
+    protected int level;
+    protected int splitDim;
+    protected final byte[][] splitPackedValueStack;
+
+    protected IndexTree() {
+      int treeDepth = getTreeDepth();
+      splitPackedValueStack = new byte[treeDepth+1][];
+      nodeID = 1;
+      level = 1;
+      splitPackedValueStack[level] = new byte[packedBytesLength];
+    }      
+
+    public void pushLeft() {
+      nodeID *= 2;
+      level++;
+      if (splitPackedValueStack[level] == null) {
+        splitPackedValueStack[level] = new byte[packedBytesLength];
+      }
+    }
+
+    /** Clone, but you are not allowed to pop up past the point where the clone happened. */
+    public abstract IndexTree clone();
+    
+    public void pushRight() {
+      nodeID = nodeID * 2 + 1;
+      level++;
+      if (splitPackedValueStack[level] == null) {
+        splitPackedValueStack[level] = new byte[packedBytesLength];
+      }
+    }
+
+    public void pop() {
+      nodeID /= 2;
+      level--;
+      splitDim = -1;
+      //System.out.println("  pop nodeID=" + nodeID);
+    }
 
-    public VerifyVisitor(int numDims, int bytesPerDim, int maxDoc) {
-      this.numDims = numDims;
-      this.bytesPerDim = bytesPerDim;
-      this.maxDoc = maxDoc;
+    public boolean isLeafNode() {
+      return nodeID >= leafNodeOffset;
     }
 
-    @Override
-    public void visit(int docID) {
-      throw new UnsupportedOperationException();
+    public boolean nodeExists() {
+      return nodeID - leafNodeOffset < leafNodeOffset;
+    }
+
+    public int getNodeID() {
+      return nodeID;
+    }
+
+    public byte[] getSplitPackedValue() {
+      assert isLeafNode() == false;
+      assert splitPackedValueStack[level] != null: "level=" + level;
+      return splitPackedValueStack[level];
+    }
+                                                       
+    /** Only valid after pushLeft or pushRight, not pop! */
+    public int getSplitDim() {
+      assert isLeafNode() == false;
+      return splitDim;
+    }
+
+    /** Only valid after pushLeft or pushRight, not pop! */
+    public abstract BytesRef getSplitDimValue();
+    
+    /** Only valid after pushLeft or pushRight, not pop! */
+    public abstract long getLeafBlockFP();
+  }
+
+  /** Reads the original simple yet heap-heavy index format */
+  private final class LegacyIndexTree extends IndexTree {
+
+    private long leafBlockFP;
+    private final byte[] splitDimValue = new byte[bytesPerDim];
+    private final BytesRef scratch = new BytesRef();
+
+    public LegacyIndexTree() {
+      setNodeData();
+      scratch.bytes = splitDimValue;
+      scratch.length = bytesPerDim;
     }
 
     @Override
-    public void visit(int docID, byte[] packedValue) {
-      if (docID < 0 || docID >= maxDoc) {
-        throw new RuntimeException("docID=" + docID + " is out of bounds of 0.." + maxDoc);
-      }
-      for(int dim=0;dim<numDims;dim++) {
-        if (StringHelper.compare(bytesPerDim, cellMinPacked, dim*bytesPerDim, packedValue, dim*bytesPerDim) > 0) {
-          throw new RuntimeException("value=" + new BytesRef(packedValue, dim*bytesPerDim, bytesPerDim) + " for docID=" + docID + " dim=" + dim + " is less than this leaf block's minimum=" + new BytesRef(cellMinPacked, dim*bytesPerDim, bytesPerDim));
-        }
-        if (StringHelper.compare(bytesPerDim, cellMaxPacked, dim*bytesPerDim, packedValue, dim*bytesPerDim) < 0) {
-          throw new RuntimeException("value=" + new BytesRef(packedValue, dim*bytesPerDim, bytesPerDim) + " for docID=" + docID + " dim=" + dim + " is greater than this leaf block's maximum=" + new BytesRef(cellMaxPacked, dim*bytesPerDim, bytesPerDim));
-        }
-      }
+    public LegacyIndexTree clone() {
+      LegacyIndexTree index = new LegacyIndexTree();
+      index.nodeID = nodeID;
+      index.level = level;
+      index.splitDim = splitDim;
+      index.leafBlockFP = leafBlockFP;
+      index.splitPackedValueStack[index.level] = splitPackedValueStack[index.level].clone();
+
+      return index;
+    }
+    
+    @Override
+    public void pushLeft() {
+      super.pushLeft();
+      setNodeData();
+    }
+    
+    @Override
+    public void pushRight() {
+      super.pushRight();
+      setNodeData();
+    }
 
-      if (numDims == 1) {
-        // With only 1D, all values should always be in sorted order
-        if (lastPackedValue == null) {
-          lastPackedValue = Arrays.copyOf(packedValue, packedValue.length);
-        } else if (StringHelper.compare(bytesPerDim, lastPackedValue, 0, packedValue, 0) > 0) {
-          throw new RuntimeException("value=" + new BytesRef(packedValue) + " for docID=" + docID + " dim=0" + " sorts before last value=" + new BytesRef(lastPackedValue));
+    private void setNodeData() {
+      if (isLeafNode()) {
+        leafBlockFP = leafBlockFPs[nodeID - leafNodeOffset];
+        splitDim = -1;
+      } else {
+        leafBlockFP = -1;
+        int address = nodeID * bytesPerIndexEntry;
+        if (numDims == 1) {
+          splitDim = 0;
+          if (version < BKDWriter.VERSION_IMPLICIT_SPLIT_DIM_1D) {
+            // skip over wastefully encoded 0 splitDim:
+            assert splitPackedValues[address] == 0;
+            address++;
+          }
         } else {
-          System.arraycopy(packedValue, 0, lastPackedValue, 0, bytesPerDim);
+          splitDim = splitPackedValues[address++] & 0xff;
         }
+        System.arraycopy(splitPackedValues, address, splitDimValue, 0, bytesPerDim);
       }
     }
 
     @Override
-    public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
-      throw new UnsupportedOperationException();
+    public long getLeafBlockFP() {
+      assert isLeafNode();
+      return leafBlockFP;
+    }
+
+    @Override
+    public BytesRef getSplitDimValue() {
+      assert isLeafNode() == false;
+      return scratch;
     }
-  }
 
-  /** Only used for debugging, to make sure all values in each leaf block fall within the range expected by the index */
-  // TODO: maybe we can get this into CheckIndex?
-  public void verify(int maxDoc) throws IOException {
-    //System.out.println("BKDR.verify this=" + this);
-    // Visits every doc in every leaf block and confirms that
-    // their values agree with the index:
-    byte[] rootMinPacked = new byte[packedBytesLength];
-    byte[] rootMaxPacked = new byte[packedBytesLength];
-    Arrays.fill(rootMaxPacked, (byte) 0xff);
-    verify(getIntersectState(new VerifyVisitor(numDims, bytesPerDim, maxDoc)), 1, rootMinPacked, rootMaxPacked);
+    @Override
+    public void pop() {
+      super.pop();
+      leafBlockFP = -1;
+    }
   }
 
-  private void verify(IntersectState state, int nodeID, byte[] cellMinPacked, byte[] cellMaxPacked) throws IOException {
+  /** Reads the new packed byte[] index format which can be up to ~63% smaller than the legacy index format on 20M NYC taxis tests.  This
+   *  format takes advantage of the limited access pattern to the BKD tree at search time, i.e. starting at the root node and recursing
+   *  downwards one child at a time. */
+  private final class PackedIndexTree extends IndexTree {
+    // used to read the packed byte[]
+    private final ByteArrayDataInput in;
+    // holds the minimum (left most) leaf block file pointer for each level we've recursed to:
+    private final long[] leafBlockFPStack;
+    // holds the address, in the packed byte[] index, of the left-node of each level:
+    private final int[] leftNodePositions;
+    // holds the address, in the packed byte[] index, of the right-node of each level:
+    private final int[] rightNodePositions;
+    // holds the splitDim for each level:
+    private final int[] splitDims;
+    // true if the per-dim delta we read for the node at this level is a negative offset vs. the last split on this dim; this is a packed
+    // 2D array, i.e. to access array[level][dim] you read from negativeDeltas[level*numDims+dim].  this will be true if the last time we
+    // split on this dimension, we next pushed to the left sub-tree:
+    private final boolean[] negativeDeltas;
+    // holds the packed per-level split values; the intersect method uses this to save the cell min/max as it recurses:
+    private final byte[][] splitValuesStack;
+    // scratch value to return from getPackedValue:
+    private final BytesRef scratch;
+
+    public PackedIndexTree() {
+      int treeDepth = getTreeDepth();
+      leafBlockFPStack = new long[treeDepth+1];
+      leftNodePositions = new int[treeDepth+1];
+      rightNodePositions = new int[treeDepth+1];
+      splitValuesStack = new byte[treeDepth+1][];
+      splitDims = new int[treeDepth+1];
+      negativeDeltas = new boolean[numDims*(treeDepth+1)];
+
+      in = new ByteArrayDataInput(packedIndex);
+      splitValuesStack[0] = new byte[packedBytesLength];
+      readNodeData(false);
+      scratch = new BytesRef();
+      scratch.length = bytesPerDim;
+    }
 
-    if (nodeID >= leafNodeOffset) {
-      int leafID = nodeID - leafNodeOffset;
+    @Override
+    public PackedIndexTree clone() {
+      PackedIndexTree index = new PackedIndexTree();
+      index.nodeID = nodeID;
+      index.level = level;
+      index.splitDim = splitDim;
+      System.arraycopy(negativeDeltas, level*numDims, index.negativeDeltas, level*numDims, numDims);
+      index.leafBlockFPStack[level] = leafBlockFPStack[level];
+      index.leftNodePositions[level] = leftNodePositions[level];
+      index.rightNodePositions[level] = rightNodePositions[level];
+      index.splitValuesStack[index.level] = splitValuesStack[index.level].clone();
+      System.arraycopy(negativeDeltas, level*numDims, index.negativeDeltas, level*numDims, numDims);
+      index.splitDims[level] = splitDims[level];
+      return index;
+    }
 
-      // In the unbalanced case it's possible the left most node only has one child:
-      if (leafID < leafBlockFPs.length) {
-        //System.out.println("CHECK nodeID=" + nodeID + " leaf=" + (nodeID-leafNodeOffset) + " offset=" + leafNodeOffset + " fp=" + leafBlockFPs[leafID]);
-        //System.out.println("BKDR.verify leafID=" + leafID + " nodeID=" + nodeID + " fp=" + leafBlockFPs[leafID] + " min=" + new BytesRef(cellMinPacked) + " max=" + new BytesRef(cellMaxPacked));
+    @Override
+    public void pushLeft() {
+      int nodePosition = leftNodePositions[level];
+      super.pushLeft();
+      System.arraycopy(negativeDeltas, (level-1)*numDims, negativeDeltas, level*numDims, numDims);
+      assert splitDim != -1;
+      negativeDeltas[level*numDims+splitDim] = true;
+      in.setPosition(nodePosition);
+      readNodeData(true);
+    }
+    
+    @Override
+    public void pushRight() {
+      int nodePosition = rightNodePositions[level];
+      super.pushRight();
+      System.arraycopy(negativeDeltas, (level-1)*numDims, negativeDeltas, level*numDims, numDims);
+      assert splitDim != -1;
+      negativeDeltas[level*numDims+splitDim] = false;
+      in.setPosition(nodePosition);
+      readNodeData(false);
+    }
 
-        // Leaf node: check that all values are in fact in bounds:
-        VerifyVisitor visitor = (VerifyVisitor) state.visitor;
-        visitor.cellMinPacked = cellMinPacked;
-        visitor.cellMaxPacked = cellMaxPacked;
+    @Override
+    public void pop() {
+      super.pop();
+      splitDim = splitDims[level];
+    }
 
-        int count = readDocIDs(state.in, leafBlockFPs[leafID], state.scratchDocIDs);
-        visitDocValues(state.commonPrefixLengths, state.scratchPackedValue, state.in, state.scratchDocIDs, count, state.visitor);
-      } else {
-        //System.out.println("BKDR.verify skip leafID=" + leafID);
+    @Override
+    public long getLeafBlockFP() {
+      assert isLeafNode(): "nodeID=" + nodeID + " is not a leaf";
+      return leafBlockFPStack[level];
+    }
+
+    @Override
+    public BytesRef getSplitDimValue() {
+      assert isLeafNode() == false;
+      scratch.bytes = splitValuesStack[level];
+      scratch.offset = splitDim * bytesPerDim;
+      return scratch;
+    }
+
+    private void readNodeData(boolean isLeft) {
+
+      leafBlockFPStack[level] = leafBlockFPStack[level-1];
+
+      // read leaf block FP delta
+      if (isLeft == false) {
+        leafBlockFPStack[level] += in.readVLong();
       }
-    } else {
-      // Non-leaf node:
-
-      int address = nodeID * bytesPerIndexEntry;
-      int splitDim;
-      if (numDims == 1) {
-        splitDim = 0;
-        if (version < BKDWriter.VERSION_IMPLICIT_SPLIT_DIM_1D) {
-          // skip over wastefully encoded 0 splitDim:
-          assert splitPackedValues[address] == 0;
-          address++;
-        }
+
+      if (isLeafNode()) {
+        splitDim = -1;
       } else {
-        splitDim = splitPackedValues[address++] & 0xff;
-      }
-      
-      assert splitDim < numDims;
 
-      byte[] splitPackedValue = new byte[packedBytesLength];
+        // read split dim, prefix, firstDiffByteDelta encoded as int:
+        int code = in.readVInt();
+        splitDim = code % numDims;
+        splitDims[level] = splitDim;
+        code /= numDims;
+        int prefix = code % (1+bytesPerDim);
+        int suffix = bytesPerDim - prefix;
 
-      // Recurse on left sub-tree:
-      System.arraycopy(cellMaxPacked, 0, splitPackedValue, 0, packedBytesLength);
-      System.arraycopy(splitPackedValues, address, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
-      verify(state,
-             2*nodeID,
-             cellMinPacked, splitPackedValue);
+        if (splitValuesStack[level] == null) {
+          splitValuesStack[level] = new byte[packedBytesLength];
+        }
+        System.arraycopy(splitValuesStack[level-1], 0, splitValuesStack[level], 0, packedBytesLength);
+        if (suffix > 0) {
+          int firstDiffByteDelta = code / (1+bytesPerDim);
+          if (negativeDeltas[level*numDims + 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);
+        } else {
+          // our split value is == last split value in this dim, which can happen when there are many duplicate values
+        }
 
-      // Recurse on right sub-tree:
-      System.arraycopy(cellMinPacked, 0, splitPackedValue, 0, packedBytesLength);
-      System.arraycopy(splitPackedValues, address, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
-      verify(state,
-             2*nodeID+1,
-             splitPackedValue, cellMaxPacked);
+        int leftNumBytes;
+        if (nodeID * 2 < leafNodeOffset) {
+          leftNumBytes = in.readVInt();
+        } else {
+          leftNumBytes = 0;
+        }
+
+        leftNodePositions[level] = in.getPosition();
+        rightNodePositions[level] = leftNodePositions[level] + leftNumBytes;
+      }
     }
   }
 
+  private int getTreeDepth() {
+    // First +1 because all the non-leave nodes makes another power
+    // of 2; e.g. to have a fully balanced tree with 4 leaves you
+    // need a depth=3 tree:
+
+    // Second +1 because MathUtil.log computes floor of the logarithm; e.g.
+    // with 5 leaves you need a depth=4 tree:
+    return MathUtil.log(numLeaves, 2) + 2;
+  }
+
   /** Used to track all state for a single call to {@link #intersect}. */
   public static final class IntersectState {
     final IndexInput in;
@@ -285,57 +467,73 @@ public class BKDReader extends PointValues implements Accountable {
     final int[] commonPrefixLengths;
 
     final IntersectVisitor visitor;
+    public final IndexTree index;
 
     public IntersectState(IndexInput in, int numDims,
                           int packedBytesLength,
                           int maxPointsInLeafNode,
-                          IntersectVisitor visitor) {
+                          IntersectVisitor visitor,
+                          IndexTree indexVisitor) {
       this.in = in;
       this.visitor = visitor;
       this.commonPrefixLengths = new int[numDims];
       this.scratchDocIDs = new int[maxPointsInLeafNode];
       this.scratchPackedValue = new byte[packedBytesLength];
+      this.index = indexVisitor;
     }
   }
 
   public void intersect(IntersectVisitor visitor) throws IOException {
-    intersect(getIntersectState(visitor), 1, minPackedValue, maxPackedValue);
+    intersect(getIntersectState(visitor), minPackedValue, maxPackedValue);
   }
 
   /** Fast path: this is called when the query box fully encompasses all cells under this node. */
-  private void addAll(IntersectState state, int nodeID) throws IOException {
+  private void addAll(IntersectState state) throws IOException {
     //System.out.println("R: addAll nodeID=" + nodeID);
 
-    if (nodeID >= leafNodeOffset) {
+    if (state.index.isLeafNode()) {
       //System.out.println("ADDALL");
-      visitDocIDs(state.in, leafBlockFPs[nodeID-leafNodeOffset], state.visitor);
+      if (state.index.nodeExists()) {
+        visitDocIDs(state.in, state.index.getLeafBlockFP(), state.visitor);
+      }
       // TODO: we can assert that the first value here in fact matches what the index claimed?
     } else {
-      addAll(state, 2*nodeID);
-      addAll(state, 2*nodeID+1);
+      state.index.pushLeft();
+      addAll(state);
+      state.index.pop();
+
+      state.index.pushRight();
+      addAll(state);
+      state.index.pop();
     }
   }
 
   /** Create a new {@link IntersectState} */
   public IntersectState getIntersectState(IntersectVisitor visitor) {
+    IndexTree index;
+    if (packedIndex != null) {
+      index = new PackedIndexTree();
+    } else {
+      index = new LegacyIndexTree();
+    }
     return new IntersectState(in.clone(), numDims,
                               packedBytesLength,
                               maxPointsInLeafNode,
-                              visitor);
+                              visitor,
+                              index);
   }
 
   /** Visits all docIDs and packed values in a single leaf block */
-  public void visitLeafBlockValues(int nodeID, IntersectState state) throws IOException {
-    int leafID = nodeID - leafNodeOffset;
+  public void visitLeafBlockValues(IndexTree index, IntersectState state) throws IOException {
 
     // Leaf node; scan and filter all points in this block:
-    int count = readDocIDs(state.in, leafBlockFPs[leafID], state.scratchDocIDs);
+    int count = readDocIDs(state.in, index.getLeafBlockFP(), state.scratchDocIDs);
 
     // Again, this time reading values and checking with the visitor
     visitDocValues(state.commonPrefixLengths, state.scratchPackedValue, state.in, state.scratchDocIDs, count, state.visitor);
   }
 
-  protected void visitDocIDs(IndexInput in, long blockFP, IntersectVisitor visitor) throws IOException {
+  private void visitDocIDs(IndexInput in, long blockFP, IntersectVisitor visitor) throws IOException {
     // Leaf node
     in.seek(blockFP);
 
@@ -350,7 +548,7 @@ public class BKDReader extends PointValues implements Accountable {
     }
   }
 
-  protected int readDocIDs(IndexInput in, long blockFP, int[] docIDs) throws IOException {
+  int readDocIDs(IndexInput in, long blockFP, int[] docIDs) throws IOException {
     in.seek(blockFP);
 
     // How many points are stored in this leaf cell:
@@ -365,7 +563,7 @@ public class BKDReader extends PointValues implements Accountable {
     return count;
   }
 
-  protected void visitDocValues(int[] commonPrefixLengths, byte[] scratchPackedValue, IndexInput in, int[] docIDs, int count, IntersectVisitor visitor) throws IOException {
+  void visitDocValues(int[] commonPrefixLengths, byte[] scratchPackedValue, IndexInput in, int[] docIDs, int count, IntersectVisitor visitor) throws IOException {
     visitor.grow(count);
 
     readCommonPrefixes(commonPrefixLengths, scratchPackedValue, in);
@@ -434,13 +632,10 @@ public class BKDReader extends PointValues implements Accountable {
     }
   }
 
-  private void intersect(IntersectState state,
-                         int nodeID,
-                         byte[] cellMinPacked, byte[] cellMaxPacked)
-    throws IOException {
+  private void intersect(IntersectState state, byte[] cellMinPacked, byte[] cellMaxPacked) throws IOException {
 
     /*
-    System.out.println("\nR: intersect nodeID=" + nodeID);
+    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));
     }
@@ -450,24 +645,18 @@ public class BKDReader extends PointValues implements Accountable {
 
     if (r == Relation.CELL_OUTSIDE_QUERY) {
       // This cell is fully outside of the query shape: stop recursing
-      return;
     } else if (r == Relation.CELL_INSIDE_QUERY) {
       // This cell is fully inside of the query shape: recursively add all points in this cell without filtering
-      addAll(state, nodeID);
-      return;
-    } else {
-      // The cell crosses the shape boundary, or the cell fully contains the query, so we fall through and do full filtering
-    }
-
-    if (nodeID >= leafNodeOffset) {
+      addAll(state);
+      // 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?
-
-      int leafID = nodeID - leafNodeOffset;
       
       // In the unbalanced case it's possible the left most node only has one child:
-      if (leafID < leafBlockFPs.length) {
+      if (state.index.nodeExists()) {
         // Leaf node; scan and filter all points in this block:
-        int count = readDocIDs(state.in, leafBlockFPs[leafID], state.scratchDocIDs);
+        int count = readDocIDs(state.in, state.index.getLeafBlockFP(), state.scratchDocIDs);
 
         // Again, this time reading values and checking with the visitor
         visitDocValues(state.commonPrefixLengths, state.scratchPackedValue, state.in, state.scratchDocIDs, count, state.visitor);
@@ -476,65 +665,45 @@ public class BKDReader extends PointValues implements Accountable {
     } else {
       
       // Non-leaf node: recurse on the split left and right nodes
-
-      int address = nodeID * bytesPerIndexEntry;
-      int splitDim;
-      if (numDims == 1) {
-        splitDim = 0;
-        if (version < BKDWriter.VERSION_IMPLICIT_SPLIT_DIM_1D) {
-          // skip over wastefully encoded 0 splitDim:
-          assert splitPackedValues[address] == 0;
-          address++;
-        }
-      } else {
-        splitDim = splitPackedValues[address++] & 0xff;
-      }
-      
+      int splitDim = state.index.getSplitDim();
+      assert splitDim >= 0: "splitDim=" + splitDim;
       assert splitDim < numDims;
 
-      // TODO: can we alloc & reuse this up front?
+      byte[] splitPackedValue = state.index.getSplitPackedValue();
+      BytesRef splitDimValue = state.index.getSplitDimValue();
+      assert splitDimValue.length == bytesPerDim;
+      //System.out.println("  splitDimValue=" + splitDimValue + " splitDim=" + splitDim);
 
-      byte[] splitPackedValue = new byte[packedBytesLength];
+      // make sure cellMin <= splitValue <= cellMax:
+      assert StringHelper.compare(bytesPerDim, cellMinPacked, splitDim*bytesPerDim, splitDimValue.bytes, splitDimValue.offset) <= 0: "bytesPerDim=" + bytesPerDim + " splitDim=" + splitDim + " numDims=" + numDims;
+      assert StringHelper.compare(bytesPerDim, cellMaxPacked, splitDim*bytesPerDim, splitDimValue.bytes, splitDimValue.offset) >= 0: "bytesPerDim=" + bytesPerDim + " splitDim=" + splitDim + " numDims=" + numDims;
 
       // Recurse on left sub-tree:
       System.arraycopy(cellMaxPacked, 0, splitPackedValue, 0, packedBytesLength);
-      System.arraycopy(splitPackedValues, address, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
-      intersect(state,
-                2*nodeID,
-                cellMinPacked, splitPackedValue);
+      System.arraycopy(splitDimValue.bytes, splitDimValue.offset, splitPackedValue, splitDim*bytesPerDim, 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);
 
       // Recurse on right sub-tree:
       System.arraycopy(cellMinPacked, 0, splitPackedValue, 0, packedBytesLength);
-      System.arraycopy(splitPackedValues, address, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
-      intersect(state,
-                2*nodeID+1,
-                splitPackedValue, cellMaxPacked);
+      System.arraycopy(splitDimValue.bytes, splitDimValue.offset, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
+      state.index.pushRight();
+      intersect(state, splitPackedValue, cellMaxPacked);
+      state.index.pop();
     }
   }
 
-  /** Copies the split value for this node into the provided byte array */
-  public void copySplitValue(int nodeID, byte[] splitPackedValue) {
-    int address = nodeID * bytesPerIndexEntry;
-    int splitDim;
-    if (numDims == 1) {
-      splitDim = 0;
-      if (version < BKDWriter.VERSION_IMPLICIT_SPLIT_DIM_1D) {
-        // skip over wastefully encoded 0 splitDim:
-        assert splitPackedValues[address] == 0;
-        address++;
-      }
-    } else {
-      splitDim = splitPackedValues[address++] & 0xff;
-    }
-    
-    assert splitDim < numDims;
-    System.arraycopy(splitPackedValues, address, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
-  }
-
   @Override
   public long ramBytesUsed() {
-    return RamUsageEstimator.sizeOf(splitPackedValues) +
-        RamUsageEstimator.sizeOf(leafBlockFPs);
+    if (packedIndex != null) {
+      return packedIndex.length;
+    } else {
+      return RamUsageEstimator.sizeOf(splitPackedValues) + RamUsageEstimator.sizeOf(leafBlockFPs);
+    }
   }
 
   @Override