You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by nk...@apache.org on 2018/10/18 16:41:16 UTC

[4/4] lucene-solr:branch_7x: LUCENE-8496: Selective indexing - modify BKDReader/BKDWriter to allow users to select a fewer number of dimensions to be used for creating the index than the total number of dimensions used for field encoding. i.e., dimension

LUCENE-8496: Selective indexing - modify BKDReader/BKDWriter to allow users to select a fewer number of dimensions to be used for creating the index than the total number of dimensions used for field encoding. i.e., dimensions 0 to N may be used to determine how to split the inner nodes, and dimensions N+1 to D are ignored and stored as data dimensions at the leaves.


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

Branch: refs/heads/branch_7x
Commit: 804afbfd47cc8d86ceda6ea66f0afe304af1ad1b
Parents: c9e4c3e
Author: Nicholas Knize <nk...@gmail.com>
Authored: Mon Oct 8 18:51:03 2018 -0500
Committer: Nicholas Knize <nk...@gmail.com>
Committed: Thu Oct 18 11:01:29 2018 -0500

----------------------------------------------------------------------
 .../codecs/simpletext/SimpleTextBKDReader.java  |  59 ++++---
 .../codecs/simpletext/SimpleTextBKDWriter.java  | 146 +++++++++-------
 .../simpletext/SimpleTextFieldInfosFormat.java  |  21 ++-
 .../simpletext/SimpleTextPointsReader.java      |  20 ++-
 .../simpletext/SimpleTextPointsWriter.java      |   6 +-
 .../org/apache/lucene/codecs/PointsWriter.java  |  13 +-
 .../lucene50/Lucene50FieldInfosFormat.java      |   2 +-
 .../lucene60/Lucene60FieldInfosFormat.java      |  21 ++-
 .../codecs/lucene60/Lucene60PointsReader.java   |   2 +-
 .../codecs/lucene60/Lucene60PointsWriter.java   |  14 +-
 .../codecs/perfield/PerFieldMergeState.java     |   2 +-
 .../org/apache/lucene/document/BinaryPoint.java |   4 +-
 .../org/apache/lucene/document/DoublePoint.java |  10 +-
 .../org/apache/lucene/document/DoubleRange.java |  10 +-
 .../org/apache/lucene/document/FieldType.java   |  66 +++++--
 .../org/apache/lucene/document/FloatPoint.java  |  10 +-
 .../org/apache/lucene/document/FloatRange.java  |  10 +-
 .../org/apache/lucene/document/IntPoint.java    |  10 +-
 .../org/apache/lucene/document/IntRange.java    |  10 +-
 .../org/apache/lucene/document/LatLonPoint.java |   6 +-
 .../org/apache/lucene/document/LongPoint.java   |  10 +-
 .../org/apache/lucene/document/LongRange.java   |  10 +-
 .../apache/lucene/document/RangeFieldQuery.java |   4 +-
 .../org/apache/lucene/index/CheckIndex.java     |  37 ++--
 .../org/apache/lucene/index/CodecReader.java    |   2 +-
 .../lucene/index/DefaultIndexingChain.java      |  15 +-
 .../java/org/apache/lucene/index/FieldInfo.java |  73 +++++---
 .../org/apache/lucene/index/FieldInfos.java     |  61 ++++---
 .../org/apache/lucene/index/IndexWriter.java    |   6 +-
 .../apache/lucene/index/IndexableFieldType.java |   9 +-
 .../org/apache/lucene/index/PointValues.java    |  11 +-
 .../apache/lucene/index/PointValuesWriter.java  |  20 ++-
 .../apache/lucene/index/SortingLeafReader.java  |   9 +-
 .../apache/lucene/search/PointInSetQuery.java   |   4 +-
 .../apache/lucene/search/PointRangeQuery.java   |   4 +-
 .../org/apache/lucene/util/bkd/BKDReader.java   | 151 +++++++++-------
 .../org/apache/lucene/util/bkd/BKDWriter.java   | 174 ++++++++++---------
 .../apache/lucene/document/TestFieldType.java   |  11 +-
 .../apache/lucene/index/TestIndexableField.java |   7 +-
 .../lucene/index/TestPendingSoftDeletes.java    |  10 +-
 .../apache/lucene/index/TestPointValues.java    |  14 +-
 .../apache/lucene/search/TestPointQueries.java  |   2 +-
 .../apache/lucene/util/TestDocIdSetBuilder.java |   7 +-
 .../apache/lucene/util/bkd/Test2BBKDPoints.java |   4 +-
 .../org/apache/lucene/util/bkd/TestBKD.java     | 142 ++++++++-------
 .../util/bkd/TestMutablePointsReaderUtils.java  |   7 +-
 .../search/highlight/TermVectorLeafReader.java  |   2 +-
 .../join/PointInSetIncludingScoreQuery.java     |   4 +-
 .../apache/lucene/index/memory/MemoryIndex.java |  24 ++-
 .../apache/lucene/document/BigIntegerPoint.java |  10 +-
 .../apache/lucene/document/HalfFloatPoint.java  |  10 +-
 .../lucene/spatial/bbox/BBoxStrategy.java       |   2 +-
 .../spatial/vector/PointVectorStrategy.java     |   2 +-
 .../codecs/asserting/AssertingPointsFormat.java |   4 +-
 .../codecs/cranky/CrankyPointsFormat.java       |  12 +-
 .../lucene/index/AssertingLeafReader.java       |  41 +++--
 .../index/BaseIndexFileFormatTestCase.java      |   2 +-
 .../lucene/index/BasePointsFormatTestCase.java  |  76 +++++---
 .../lucene/index/MismatchedLeafReader.java      |   3 +-
 .../org/apache/lucene/index/RandomCodec.java    |   9 +-
 .../lucene/index/RandomPostingsTester.java      |   4 +-
 .../org/apache/lucene/util/LuceneTestCase.java  |  16 +-
 .../java/org/apache/lucene/util/TestUtil.java   |   2 +-
 .../solr/handler/component/ExpandComponent.java |   3 +-
 .../org/apache/solr/legacy/BBoxStrategy.java    |   2 +-
 .../apache/solr/legacy/PointVectorStrategy.java |   2 +-
 .../org/apache/solr/schema/SchemaField.java     |   7 +-
 .../solr/search/CollapsingQParserPlugin.java    |   2 +-
 .../java/org/apache/solr/search/Insanity.java   |   2 +-
 .../apache/solr/uninverting/FieldCacheImpl.java |   6 +-
 .../solr/uninverting/UninvertingReader.java     |   6 +-
 .../solr/uninverting/TestUninvertingReader.java |   6 +-
 72 files changed, 891 insertions(+), 614 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/804afbfd/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextBKDReader.java
----------------------------------------------------------------------
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextBKDReader.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextBKDReader.java
index b7af45a..5227d6d 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextBKDReader.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextBKDReader.java
@@ -41,7 +41,8 @@ final class SimpleTextBKDReader extends PointValues implements Accountable {
   final private byte[] splitPackedValues; 
   final long[] leafBlockFPs;
   final private int leafNodeOffset;
-  final int numDims;
+  final int numDataDims;
+  final int numIndexDims;
   final int bytesPerDim;
   final int bytesPerIndexEntry;
   final IndexInput in;
@@ -52,16 +53,19 @@ final class SimpleTextBKDReader extends PointValues implements Accountable {
   final int docCount;
   final int version;
   protected final int packedBytesLength;
+  protected final int packedIndexBytesLength;
 
-  public SimpleTextBKDReader(IndexInput in, int numDims, int maxPointsInLeafNode, int bytesPerDim, long[] leafBlockFPs, byte[] splitPackedValues,
+  public SimpleTextBKDReader(IndexInput in, int numDataDims, int numIndexDims, 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.numDataDims = numDataDims;
+    this.numIndexDims = numIndexDims;
     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;
+    bytesPerIndexEntry = numIndexDims == 1 ? bytesPerDim : bytesPerDim + 1;
+    packedBytesLength = numDataDims * bytesPerDim;
+    packedIndexBytesLength = numIndexDims * bytesPerDim;
     this.leafNodeOffset = leafBlockFPs.length;
     this.leafBlockFPs = leafBlockFPs;
     this.splitPackedValues = splitPackedValues;
@@ -70,8 +74,8 @@ final class SimpleTextBKDReader extends PointValues implements Accountable {
     this.pointCount = pointCount;
     this.docCount = docCount;
     this.version = SimpleTextBKDWriter.VERSION_CURRENT;
-    assert minPackedValue.length == packedBytesLength;
-    assert maxPackedValue.length == packedBytesLength;
+    assert minPackedValue.length == packedIndexBytesLength;
+    assert maxPackedValue.length == packedIndexBytesLength;
   }
 
   /** Used to track all state for a single call to {@link #intersect}. */
@@ -115,7 +119,7 @@ final class SimpleTextBKDReader extends PointValues implements Accountable {
 
   /** Create a new {@link IntersectState} */
   public IntersectState getIntersectState(IntersectVisitor visitor) {
-    return new IntersectState(in.clone(), numDims,
+    return new IntersectState(in.clone(), numDataDims,
                               packedBytesLength,
                               maxPointsInLeafNode,
                               visitor);
@@ -181,7 +185,7 @@ final class SimpleTextBKDReader extends PointValues implements Accountable {
       scratchPackedValue[compressedByteOffset] = in.readByte();
       final int runLen = Byte.toUnsignedInt(in.readByte());
       for (int j = 0; j < runLen; ++j) {
-        for(int dim=0;dim<numDims;dim++) {
+        for(int dim=0;dim<numDataDims;dim++) {
           int prefix = commonPrefixLengths[dim];
           in.readBytes(scratchPackedValue, dim*bytesPerDim + prefix, bytesPerDim - prefix);
         }
@@ -196,14 +200,14 @@ final class SimpleTextBKDReader extends PointValues implements Accountable {
 
   private int readCompressedDim(IndexInput in) throws IOException {
     int compressedDim = in.readByte();
-    if (compressedDim < -1 || compressedDim >= numDims) {
+    if (compressedDim < -1 || compressedDim >= numIndexDims) {
       throw new CorruptIndexException("Got compressedDim="+compressedDim, in);
     }
     return compressedDim;
   }
 
   private void readCommonPrefixes(int[] commonPrefixLengths, byte[] scratchPackedValue, IndexInput in) throws IOException {
-    for(int dim=0;dim<numDims;dim++) {
+    for(int dim=0;dim<numDataDims;dim++) {
       int prefix = in.readVInt();
       commonPrefixLengths[dim] = prefix;
       if (prefix > 0) {
@@ -258,27 +262,27 @@ final class SimpleTextBKDReader extends PointValues implements Accountable {
 
       int address = nodeID * bytesPerIndexEntry;
       int splitDim;
-      if (numDims == 1) {
+      if (numIndexDims == 1) {
         splitDim = 0;
       } else {
         splitDim = splitPackedValues[address++] & 0xff;
       }
       
-      assert splitDim < numDims;
+      assert splitDim < numIndexDims;
 
       // TODO: can we alloc & reuse this up front?
 
-      byte[] splitPackedValue = new byte[packedBytesLength];
+      byte[] splitPackedValue = new byte[packedIndexBytesLength];
 
       // Recurse on left sub-tree:
-      System.arraycopy(cellMaxPacked, 0, splitPackedValue, 0, packedBytesLength);
+      System.arraycopy(cellMaxPacked, 0, splitPackedValue, 0, packedIndexBytesLength);
       System.arraycopy(splitPackedValues, address, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
       intersect(state,
                 2*nodeID,
                 cellMinPacked, splitPackedValue);
 
       // Recurse on right sub-tree:
-      System.arraycopy(cellMinPacked, 0, splitPackedValue, 0, packedBytesLength);
+      System.arraycopy(cellMinPacked, 0, splitPackedValue, 0, packedIndexBytesLength);
       System.arraycopy(splitPackedValues, address, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
       intersect(state,
                 2*nodeID+1,
@@ -307,27 +311,27 @@ final class SimpleTextBKDReader extends PointValues implements Accountable {
 
       int address = nodeID * bytesPerIndexEntry;
       int splitDim;
-      if (numDims == 1) {
+      if (numIndexDims == 1) {
         splitDim = 0;
       } else {
         splitDim = splitPackedValues[address++] & 0xff;
       }
       
-      assert splitDim < numDims;
+      assert splitDim < numIndexDims;
 
       // TODO: can we alloc & reuse this up front?
 
-      byte[] splitPackedValue = new byte[packedBytesLength];
+      byte[] splitPackedValue = new byte[packedIndexBytesLength];
 
       // Recurse on left sub-tree:
-      System.arraycopy(cellMaxPacked, 0, splitPackedValue, 0, packedBytesLength);
+      System.arraycopy(cellMaxPacked, 0, splitPackedValue, 0, packedIndexBytesLength);
       System.arraycopy(splitPackedValues, address, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
       final long leftCost = estimatePointCount(state,
                 2*nodeID,
                 cellMinPacked, splitPackedValue);
 
       // Recurse on right sub-tree:
-      System.arraycopy(cellMinPacked, 0, splitPackedValue, 0, packedBytesLength);
+      System.arraycopy(cellMinPacked, 0, splitPackedValue, 0, packedIndexBytesLength);
       System.arraycopy(splitPackedValues, address, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
       final long rightCost = estimatePointCount(state,
                 2*nodeID+1,
@@ -340,13 +344,13 @@ final class SimpleTextBKDReader extends PointValues implements Accountable {
   public void copySplitValue(int nodeID, byte[] splitPackedValue) {
     int address = nodeID * bytesPerIndexEntry;
     int splitDim;
-    if (numDims == 1) {
+    if (numIndexDims == 1) {
       splitDim = 0;
     } else {
       splitDim = splitPackedValues[address++] & 0xff;
     }
     
-    assert splitDim < numDims;
+    assert splitDim < numIndexDims;
     System.arraycopy(splitPackedValues, address, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
   }
 
@@ -367,8 +371,13 @@ final class SimpleTextBKDReader extends PointValues implements Accountable {
   }
 
   @Override
-  public int getNumDimensions() {
-    return numDims;
+  public int getNumDataDimensions() {
+    return numDataDims;
+  }
+
+  @Override
+  public int getNumIndexDimensions() {
+    return numIndexDims;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/804afbfd/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextBKDWriter.java
----------------------------------------------------------------------
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextBKDWriter.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextBKDWriter.java
index 3a161e8..9cacaa4 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextBKDWriter.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextBKDWriter.java
@@ -64,7 +64,8 @@ import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.INDEX_C
 import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.MAX_LEAF_POINTS;
 import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.MAX_VALUE;
 import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.MIN_VALUE;
-import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.NUM_DIMS;
+import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.NUM_DATA_DIMS;
+import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.NUM_INDEX_DIMS;
 import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.POINT_COUNT;
 import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.SPLIT_COUNT;
 import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.SPLIT_DIM;
@@ -104,15 +105,21 @@ final class SimpleTextBKDWriter implements Closeable {
   /** Maximum number of dimensions */
   public static final int MAX_DIMS = 8;
 
-  /** How many dimensions we are indexing */
-  protected final int numDims;
+  /** 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;
 
-  /** numDims * bytesPerDim */
+  /** numDataDims * bytesPerDim */
   protected final int packedBytesLength;
 
+  /** numIndexDims * bytesPerDim */
+  protected final int packedIndexBytesLength;
+
   final BytesRefBuilder scratch = new BytesRefBuilder();
 
   final TrackingDirectoryWrapper tempDir;
@@ -160,37 +167,39 @@ final class SimpleTextBKDWriter implements Closeable {
 
   private final int maxDoc;
 
-  public SimpleTextBKDWriter(int maxDoc, Directory tempDir, String tempFileNamePrefix, int numDims, int bytesPerDim,
+  public SimpleTextBKDWriter(int maxDoc, Directory tempDir, String tempFileNamePrefix, int numDataDims, int numIndexDims, int bytesPerDim,
                              int maxPointsInLeafNode, double maxMBSortInHeap, long totalPointCount, boolean singleValuePerDoc) throws IOException {
-    this(maxDoc, tempDir, tempFileNamePrefix, numDims, bytesPerDim, maxPointsInLeafNode, maxMBSortInHeap, totalPointCount, singleValuePerDoc,
+    this(maxDoc, tempDir, tempFileNamePrefix, numDataDims, numIndexDims, bytesPerDim, maxPointsInLeafNode, maxMBSortInHeap, totalPointCount, singleValuePerDoc,
          totalPointCount > Integer.MAX_VALUE, Math.max(1, (long) maxMBSortInHeap), OfflineSorter.MAX_TEMPFILES);
   }
 
-  private SimpleTextBKDWriter(int maxDoc, Directory tempDir, String tempFileNamePrefix, int numDims, int bytesPerDim,
+  private SimpleTextBKDWriter(int maxDoc, Directory tempDir, String tempFileNamePrefix, int numDataDims, int numIndexDims, int bytesPerDim,
                               int maxPointsInLeafNode, double maxMBSortInHeap, long totalPointCount,
                               boolean singleValuePerDoc, boolean longOrds, long offlineSorterBufferMB, int offlineSorterMaxTempFiles) throws IOException {
-    verifyParams(numDims, maxPointsInLeafNode, maxMBSortInHeap, totalPointCount);
+    verifyParams(numDataDims, numIndexDims, maxPointsInLeafNode, maxMBSortInHeap, totalPointCount);
     // We use tracking dir to deal with removing files on exception, so each place that
     // creates temp files doesn't need crazy try/finally/sucess logic:
     this.tempDir = new TrackingDirectoryWrapper(tempDir);
     this.tempFileNamePrefix = tempFileNamePrefix;
     this.maxPointsInLeafNode = maxPointsInLeafNode;
-    this.numDims = numDims;
+    this.numDataDims = numDataDims;
+    this.numIndexDims = numIndexDims;
     this.bytesPerDim = bytesPerDim;
     this.totalPointCount = totalPointCount;
     this.maxDoc = maxDoc;
     this.offlineSorterBufferMB = OfflineSorter.BufferSize.megabytes(offlineSorterBufferMB);
     this.offlineSorterMaxTempFiles = offlineSorterMaxTempFiles;
     docsSeen = new FixedBitSet(maxDoc);
-    packedBytesLength = numDims * bytesPerDim;
+    packedBytesLength = numDataDims * bytesPerDim;
+    packedIndexBytesLength = numIndexDims * bytesPerDim;
 
     scratchDiff = new byte[bytesPerDim];
     scratch1 = new byte[packedBytesLength];
     scratch2 = new byte[packedBytesLength];
-    commonPrefixLengths = new int[numDims];
+    commonPrefixLengths = new int[numDataDims];
 
-    minPackedValue = new byte[packedBytesLength];
-    maxPackedValue = new byte[packedBytesLength];
+    minPackedValue = new byte[packedIndexBytesLength];
+    maxPackedValue = new byte[packedIndexBytesLength];
 
     // If we may have more than 1+Integer.MAX_VALUE values, then we must encode ords with long (8 bytes), else we can use int (4 bytes).
     this.longOrds = longOrds;
@@ -217,7 +226,7 @@ final class SimpleTextBKDWriter implements Closeable {
     // bytes to points here.  Each dimension has its own sorted partition, so
     // we must divide by numDims as wel.
 
-    maxPointsSortInHeap = (int) (0.5 * (maxMBSortInHeap * 1024 * 1024) / (bytesPerDoc * numDims));
+    maxPointsSortInHeap = (int) (0.5 * (maxMBSortInHeap * 1024 * 1024) / (bytesPerDoc * numDataDims));
 
     // Finally, we must be able to hold at least the leaf node in heap during build:
     if (maxPointsSortInHeap < maxPointsInLeafNode) {
@@ -230,11 +239,14 @@ final class SimpleTextBKDWriter implements Closeable {
     this.maxMBSortInHeap = maxMBSortInHeap;
   }
 
-  public static void verifyParams(int numDims, int maxPointsInLeafNode, double maxMBSortInHeap, long totalPointCount) {
+  public static void verifyParams(int numDataDims, 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 (numDataDims < 1 || numDataDims > MAX_DIMS) {
+      throw new IllegalArgumentException("numDataDims must be 1 .. " + MAX_DIMS + " (got: " + numDataDims + ")");
+    }
+    if (numIndexDims < 1 || numIndexDims > numDataDims) {
+      throw new IllegalArgumentException("numIndexDims must be 1 .. " + numDataDims + " (got: " + numIndexDims + ")");
     }
     if (maxPointsInLeafNode <= 0) {
       throw new IllegalArgumentException("maxPointsInLeafNode must be > 0; got " + maxPointsInLeafNode);
@@ -283,10 +295,10 @@ final class SimpleTextBKDWriter implements Closeable {
 
     // TODO: we could specialize for the 1D case:
     if (pointCount == 0) {
-      System.arraycopy(packedValue, 0, minPackedValue, 0, packedBytesLength);
-      System.arraycopy(packedValue, 0, maxPackedValue, 0, packedBytesLength);
+      System.arraycopy(packedValue, 0, minPackedValue, 0, packedIndexBytesLength);
+      System.arraycopy(packedValue, 0, maxPackedValue, 0, packedIndexBytesLength);
     } else {
-      for(int dim=0;dim<numDims;dim++) {
+      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);
@@ -331,7 +343,7 @@ final class SimpleTextBKDWriter implements Closeable {
     public MergeReader(SimpleTextBKDReader bkd, MergeState.DocMap docMap) throws IOException {
       this.bkd = bkd;
       state = new SimpleTextBKDReader.IntersectState(bkd.in.clone(),
-                                                     bkd.numDims,
+                                                     bkd.numDataDims,
                                                      bkd.packedBytesLength,
                                                      bkd.maxPointsInLeafNode,
                                                      null);
@@ -433,7 +445,7 @@ final class SimpleTextBKDWriter implements Closeable {
    *  disk. This method does not use transient disk in order to reorder points.
    */
   public long writeField(IndexOutput out, String fieldName, MutablePointValues reader) throws IOException {
-    if (numDims == 1) {
+    if (numIndexDims == 1) {
       return writeField1Dim(out, fieldName, reader);
     } else {
       return writeFieldNDims(out, fieldName, reader);
@@ -476,7 +488,7 @@ final class SimpleTextBKDWriter implements Closeable {
     Arrays.fill(maxPackedValue, (byte) 0);
     for (int i = 0; i < Math.toIntExact(pointCount); ++i) {
       values.getValue(i, scratchBytesRef1);
-      for(int dim=0;dim<numDims;dim++) {
+      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);
@@ -502,7 +514,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, packedBytesLength, reader, 0, Math.toIntExact(reader.size()));
+    MutablePointsReaderUtils.sort(maxDoc, packedIndexBytesLength, reader, 0, Math.toIntExact(reader.size()));
 
     final OneDimensionBKDWriter oneDimWriter = new OneDimensionBKDWriter(out);
 
@@ -582,8 +594,8 @@ final class SimpleTextBKDWriter implements Closeable {
     int leafCount;
 
     OneDimensionBKDWriter(IndexOutput out) {
-      if (numDims != 1) {
-        throw new UnsupportedOperationException("numDims must be 1 but got " + numDims);
+      if (numIndexDims != 1) {
+        throw new UnsupportedOperationException("numIndexDims must be 1 but got " + numIndexDims);
       }
       if (pointCount != 0) {
         throw new IllegalStateException("cannot mix add and merge");
@@ -661,9 +673,9 @@ final class SimpleTextBKDWriter implements Closeable {
     private void writeLeafBlock() throws IOException {
       assert leafCount != 0;
       if (valueCount == 0) {
-        System.arraycopy(leafValues, 0, minPackedValue, 0, packedBytesLength);
+        System.arraycopy(leafValues, 0, minPackedValue, 0, packedIndexBytesLength);
       }
-      System.arraycopy(leafValues, (leafCount - 1) * packedBytesLength, maxPackedValue, 0, packedBytesLength);
+      System.arraycopy(leafValues, (leafCount - 1) * packedBytesLength, maxPackedValue, 0, packedIndexBytesLength);
 
       valueCount += leafCount;
 
@@ -676,7 +688,7 @@ final class SimpleTextBKDWriter implements Closeable {
 
       Arrays.fill(commonPrefixLengths, bytesPerDim);
       // Find per-dim common prefix:
-      for(int dim=0;dim<numDims;dim++) {
+      for(int dim=0;dim<numDataDims;dim++) {
         int offset1 = dim * bytesPerDim;
         int offset2 = (leafCount - 1) * packedBytesLength + offset1;
         for(int j=0;j<commonPrefixLengths[dim];j++) {
@@ -822,7 +834,7 @@ final class SimpleTextBKDWriter implements Closeable {
   }
 
   private PointWriter sort(int dim) throws IOException {
-    assert dim >= 0 && dim < numDims;
+    assert dim >= 0 && dim < numDataDims;
 
     if (heapPointWriter != null) {
 
@@ -855,7 +867,7 @@ final class SimpleTextBKDWriter implements Closeable {
       final int offset = bytesPerDim * dim;
 
       Comparator<BytesRef> cmp;
-      if (dim == numDims - 1) {
+      if (dim == numDataDims - 1) {
         // in that case the bytes for the dimension and for the doc id are contiguous,
         // so we don't need a branch
         cmp = new BytesRefComparator(bytesPerDim + Integer.BYTES) {
@@ -940,7 +952,7 @@ final class SimpleTextBKDWriter implements Closeable {
     }
 
     LongBitSet ordBitSet;
-    if (numDims > 1) {
+    if (numDataDims > 1) {
       if (singleValuePerDoc) {
         ordBitSet = new LongBitSet(maxDoc);
       } else {
@@ -975,7 +987,7 @@ final class SimpleTextBKDWriter implements Closeable {
     assert pointCount / numLeaves <= maxPointsInLeafNode: "pointCount=" + pointCount + " numLeaves=" + numLeaves + " maxPointsInLeafNode=" + maxPointsInLeafNode;
 
     // Sort all docs once by each dimension:
-    PathSlice[] sortedPointWriters = new PathSlice[numDims];
+    PathSlice[] sortedPointWriters = new PathSlice[numDataDims];
 
     // This is only used on exception; on normal code paths we close all files we opened:
     List<Closeable> toCloseHeroically = new ArrayList<>();
@@ -983,7 +995,7 @@ final class SimpleTextBKDWriter implements Closeable {
     boolean success = false;
     try {
       //long t0 = System.nanoTime();
-      for(int dim=0;dim<numDims;dim++) {
+      for(int dim=0;dim<numDataDims;dim++) {
         sortedPointWriters[dim] = new PathSlice(sort(dim), 0, pointCount);
       }
       //long t1 = System.nanoTime();
@@ -1031,8 +1043,12 @@ 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_DIMS);
-    writeInt(out, numDims);
+    write(out, NUM_DATA_DIMS);
+    writeInt(out, numDataDims);
+    newline(out);
+
+    write(out, NUM_INDEX_DIMS);
+    writeInt(out, numIndexDims);
     newline(out);
 
     write(out, BYTES_PER_DIM);
@@ -1116,7 +1132,7 @@ final class SimpleTextBKDWriter implements Closeable {
       BytesRef ref = packedValues.apply(i);
       assert ref.length == packedBytesLength;
 
-      for(int dim=0;dim<numDims;dim++) {
+      for(int dim=0;dim<numDataDims;dim++) {
         int prefix = commonPrefixLengths[dim];
         out.writeBytes(ref.bytes, ref.offset + dim*bytesPerDim + prefix, bytesPerDim-prefix);
       }
@@ -1200,7 +1216,7 @@ final class SimpleTextBKDWriter implements Closeable {
       boolean result = reader.next();
       assert result;
       System.arraycopy(reader.packedValue(), splitDim*bytesPerDim, scratch1, 0, bytesPerDim);
-      if (numDims > 1) {
+      if (numDataDims > 1) {
         assert ordBitSet.get(reader.ord()) == false;
         ordBitSet.set(reader.ord());
         // Subtract 1 from rightCount because we already did the first value above (so we could record the split value):
@@ -1215,7 +1231,7 @@ final class SimpleTextBKDWriter implements Closeable {
 
   /** Called only in assert */
   private boolean valueInBounds(BytesRef packedValue, byte[] minPackedValue, byte[] maxPackedValue) {
-    for(int dim=0;dim<numDims;dim++) {
+    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;
@@ -1231,7 +1247,7 @@ final class SimpleTextBKDWriter implements Closeable {
   protected int split(byte[] minPackedValue, byte[] maxPackedValue) {
     // Find which dim has the largest span so we can split on it:
     int splitDim = -1;
-    for(int dim=0;dim<numDims;dim++) {
+    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);
@@ -1279,7 +1295,7 @@ final class SimpleTextBKDWriter implements Closeable {
       reader.getValue(from, scratchBytesRef1);
       for (int i = from + 1; i < to; ++i) {
         reader.getValue(i, scratchBytesRef2);
-        for (int dim=0;dim<numDims;dim++) {
+        for (int dim=0;dim<numDataDims;dim++) {
           final int offset = dim * bytesPerDim;
           for(int j=0;j<commonPrefixLengths[dim];j++) {
             if (scratchBytesRef1.bytes[scratchBytesRef1.offset+offset+j] != scratchBytesRef2.bytes[scratchBytesRef2.offset+offset+j]) {
@@ -1291,14 +1307,14 @@ final class SimpleTextBKDWriter implements Closeable {
       }
 
       // Find the dimension that has the least number of unique bytes at commonPrefixLengths[dim]
-      FixedBitSet[] usedBytes = new FixedBitSet[numDims];
-      for (int dim = 0; dim < numDims; ++dim) {
+      FixedBitSet[] usedBytes = new FixedBitSet[numDataDims];
+      for (int dim = 0; dim < numDataDims; ++dim) {
         if (commonPrefixLengths[dim] < bytesPerDim) {
           usedBytes[dim] = new FixedBitSet(256);
         }
       }
       for (int i = from + 1; i < to; ++i) {
-        for (int dim=0;dim<numDims;dim++) {
+        for (int dim=0;dim<numDataDims;dim++) {
           if (usedBytes[dim] != null) {
             byte b = reader.getByteAt(i, dim * bytesPerDim + commonPrefixLengths[dim]);
             usedBytes[dim].set(Byte.toUnsignedInt(b));
@@ -1307,7 +1323,7 @@ final class SimpleTextBKDWriter implements Closeable {
       }
       int sortedDim = 0;
       int sortedDimCardinality = Integer.MAX_VALUE;
-      for (int dim = 0; dim < numDims; ++dim) {
+      for (int dim = 0; dim < numDataDims; ++dim) {
         if (usedBytes[dim] != null) {
           final int cardinality = usedBytes[dim].cardinality();
           if (cardinality < sortedDimCardinality) {
@@ -1370,8 +1386,8 @@ final class SimpleTextBKDWriter implements Closeable {
       reader.getValue(mid, scratchBytesRef1);
       System.arraycopy(scratchBytesRef1.bytes, scratchBytesRef1.offset + splitDim * bytesPerDim, splitPackedValues, address + 1, bytesPerDim);
 
-      byte[] minSplitPackedValue = ArrayUtil.copyOfSubArray(minPackedValue, 0, packedBytesLength);
-      byte[] maxSplitPackedValue = ArrayUtil.copyOfSubArray(maxPackedValue, 0, packedBytesLength);
+      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,
@@ -1399,7 +1415,7 @@ final class SimpleTextBKDWriter implements Closeable {
       assert slice.count == slices[0].count;
     }
 
-    if (numDims == 1 && slices[0].writer instanceof OfflinePointWriter && slices[0].count <= maxPointsSortInHeap) {
+    if (numDataDims == 1 && slices[0].writer instanceof OfflinePointWriter && slices[0].count <= maxPointsSortInHeap) {
       // Special case for 1D, to cutover to heap once we recurse deeply enough:
       slices[0] = switchToHeap(slices[0], toCloseHeroically);
     }
@@ -1412,7 +1428,7 @@ final class SimpleTextBKDWriter implements Closeable {
       int sortedDim = 0;
       int sortedDimCardinality = Integer.MAX_VALUE;
 
-      for (int dim=0;dim<numDims;dim++) {
+      for (int dim=0;dim<numDataDims;dim++) {
         if (slices[dim].writer instanceof HeapPointWriter == false) {
           // Adversarial cases can cause this, e.g. very lopsided data, all equal points, such that we started
           // offline, but then kept splitting only in one dimension, and so never had to rewrite into heap writer
@@ -1497,7 +1513,7 @@ final class SimpleTextBKDWriter implements Closeable {
       // Inner node: partition/recurse
 
       int splitDim;
-      if (numDims > 1) {
+      if (numIndexDims > 1) {
         splitDim = split(minPackedValue, maxPackedValue);
       } else {
         splitDim = 0;
@@ -1518,24 +1534,24 @@ final class SimpleTextBKDWriter implements Closeable {
 
       // Partition all PathSlice that are not the split dim into sorted left and right sets, so we can recurse:
 
-      PathSlice[] leftSlices = new PathSlice[numDims];
-      PathSlice[] rightSlices = new PathSlice[numDims];
+      PathSlice[] leftSlices = new PathSlice[numDataDims];
+      PathSlice[] rightSlices = new PathSlice[numDataDims];
 
-      byte[] minSplitPackedValue = new byte[packedBytesLength];
-      System.arraycopy(minPackedValue, 0, minSplitPackedValue, 0, packedBytesLength);
+      byte[] minSplitPackedValue = new byte[packedIndexBytesLength];
+      System.arraycopy(minPackedValue, 0, minSplitPackedValue, 0, packedIndexBytesLength);
 
-      byte[] maxSplitPackedValue = new byte[packedBytesLength];
-      System.arraycopy(maxPackedValue, 0, maxSplitPackedValue, 0, packedBytesLength);
+      byte[] maxSplitPackedValue = new byte[packedIndexBytesLength];
+      System.arraycopy(maxPackedValue, 0, maxSplitPackedValue, 0, packedIndexBytesLength);
 
       // When we are on this dim, below, we clear the ordBitSet:
       int dimToClear;
-      if (numDims - 1 == splitDim) {
-        dimToClear = numDims - 2;
+      if (numDataDims - 1 == splitDim) {
+        dimToClear = numDataDims - 2;
       } else {
-        dimToClear = numDims - 1;
+        dimToClear = numDataDims - 1;
       }
 
-      for(int dim=0;dim<numDims;dim++) {
+      for(int dim=0;dim<numDataDims;dim++) {
 
         if (dim == splitDim) {
           // No need to partition on this dim since it's a simple slice of the incoming already sorted slice, and we
@@ -1556,7 +1572,7 @@ final class SimpleTextBKDWriter implements Closeable {
 
           long nextRightCount = reader.split(source.count, ordBitSet, leftPointWriter, rightPointWriter, dim == dimToClear);
           if (rightCount != nextRightCount) {
-            throw new IllegalStateException("wrong number of points in split: expected=" + rightCount + " but actual=" + nextRightCount);
+            throw new IllegalStateException("wrong number of points in split: expected=" + rightCount + " but actual=" + nextRightCount + " in dim " + dim);
           }
 
           leftSlices[dim] = new PathSlice(leftPointWriter, 0, leftCount);
@@ -1571,7 +1587,7 @@ final class SimpleTextBKDWriter implements Closeable {
             ordBitSet, out,
             minPackedValue, maxSplitPackedValue,
             splitPackedValues, leafBlockFPs, toCloseHeroically);
-      for(int dim=0;dim<numDims;dim++) {
+      for(int dim=0;dim<numDataDims;dim++) {
         // Don't destroy the dim we split on because we just re-used what our caller above gave us for that dim:
         if (dim != splitDim) {
           leftSlices[dim].writer.destroy();
@@ -1584,7 +1600,7 @@ final class SimpleTextBKDWriter implements Closeable {
             ordBitSet, out,
             minSplitPackedValue, maxPackedValue,
             splitPackedValues, leafBlockFPs, toCloseHeroically);
-      for(int dim=0;dim<numDims;dim++) {
+      for(int dim=0;dim<numDataDims;dim++) {
         // Don't destroy the dim we split on because we just re-used what our caller above gave us for that dim:
         if (dim != splitDim) {
           rightSlices[dim].writer.destroy();
@@ -1618,10 +1634,10 @@ final class SimpleTextBKDWriter implements Closeable {
     if (ord > 0) {
       int cmp = FutureArrays.compareUnsigned(lastPackedValue, dimOffset, dimOffset + bytesPerDim, packedValue, packedValueOffset + dimOffset, packedValueOffset + dimOffset + bytesPerDim);
       if (cmp > 0) {
-        throw new AssertionError("values out of order: last value=" + new BytesRef(lastPackedValue) + " current value=" + new BytesRef(packedValue, packedValueOffset, packedBytesLength) + " ord=" + ord);
+        throw new AssertionError("values out of order: last value=" + new BytesRef(lastPackedValue) + " current value=" + new BytesRef(packedValue, packedValueOffset, packedBytesLength) + " ord=" + ord + " sortedDim=" + sortedDim);
       }
       if (cmp == 0 && doc < lastDoc) {
-        throw new AssertionError("docs out of order: last doc=" + lastDoc + " current doc=" + doc + " ord=" + ord);
+        throw new AssertionError("docs out of order: last doc=" + lastDoc + " current doc=" + doc + " ord=" + ord + " sortedDim=" + sortedDim);
       }
     }
     System.arraycopy(packedValue, packedValueOffset, lastPackedValue, 0, packedBytesLength);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/804afbfd/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosFormat.java
----------------------------------------------------------------------
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosFormat.java
index 1c40cbd..f52a84a 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosFormat.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosFormat.java
@@ -64,7 +64,8 @@ public class SimpleTextFieldInfosFormat extends FieldInfosFormat {
   static final BytesRef NUM_ATTS        =  new BytesRef("  attributes ");
   static final BytesRef ATT_KEY         =  new BytesRef("    key ");
   static final BytesRef ATT_VALUE       =  new BytesRef("    value ");
-  static final BytesRef DIM_COUNT       =  new BytesRef("  dimensional count ");
+  static final BytesRef DATA_DIM_COUNT  =  new BytesRef("  data dimensional count ");
+  static final BytesRef INDEX_DIM_COUNT =  new BytesRef("  index dimensional count ");
   static final BytesRef DIM_NUM_BYTES   =  new BytesRef("  dimensional num bytes ");
   static final BytesRef SOFT_DELETES    =  new BytesRef("  soft-deletes ");
   
@@ -134,8 +135,12 @@ public class SimpleTextFieldInfosFormat extends FieldInfosFormat {
         }
 
         SimpleTextUtil.readLine(input, scratch);
-        assert StringHelper.startsWith(scratch.get(), DIM_COUNT);
-        int dimensionalCount = Integer.parseInt(readString(DIM_COUNT.length, scratch));
+        assert StringHelper.startsWith(scratch.get(), DATA_DIM_COUNT);
+        int dataDimensionalCount = Integer.parseInt(readString(DATA_DIM_COUNT.length, scratch));
+
+        SimpleTextUtil.readLine(input, scratch);
+        assert StringHelper.startsWith(scratch.get(), INDEX_DIM_COUNT);
+        int indexDimensionalCount = Integer.parseInt(readString(INDEX_DIM_COUNT.length, scratch));
 
         SimpleTextUtil.readLine(input, scratch);
         assert StringHelper.startsWith(scratch.get(), DIM_NUM_BYTES);
@@ -147,7 +152,7 @@ public class SimpleTextFieldInfosFormat extends FieldInfosFormat {
 
         infos[i] = new FieldInfo(name, fieldNumber, storeTermVector, 
                                  omitNorms, storePayloads, indexOptions, docValuesType, dvGen, Collections.unmodifiableMap(atts),
-                                 dimensionalCount, dimensionalNumBytes, isSoftDeletesField);
+                                 dataDimensionalCount, indexDimensionalCount, dimensionalNumBytes, isSoftDeletesField);
       }
 
       SimpleTextUtil.checkFooter(input);
@@ -236,8 +241,12 @@ public class SimpleTextFieldInfosFormat extends FieldInfosFormat {
           }
         }
 
-        SimpleTextUtil.write(out, DIM_COUNT);
-        SimpleTextUtil.write(out, Integer.toString(fi.getPointDimensionCount()), scratch);
+        SimpleTextUtil.write(out, DATA_DIM_COUNT);
+        SimpleTextUtil.write(out, Integer.toString(fi.getPointDataDimensionCount()), scratch);
+        SimpleTextUtil.writeNewline(out);
+
+        SimpleTextUtil.write(out, INDEX_DIM_COUNT);
+        SimpleTextUtil.write(out, Integer.toString(fi.getPointIndexDimensionCount()), scratch);
         SimpleTextUtil.writeNewline(out);
         
         SimpleTextUtil.write(out, DIM_NUM_BYTES);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/804afbfd/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPointsReader.java
----------------------------------------------------------------------
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPointsReader.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPointsReader.java
index 453bd23..728aad1 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPointsReader.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPointsReader.java
@@ -47,7 +47,8 @@ import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.INDEX_C
 import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.MAX_LEAF_POINTS;
 import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.MAX_VALUE;
 import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.MIN_VALUE;
-import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.NUM_DIMS;
+import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.NUM_DATA_DIMS;
+import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.NUM_INDEX_DIMS;
 import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.POINT_COUNT;
 import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.SPLIT_COUNT;
 import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.SPLIT_DIM;
@@ -101,7 +102,10 @@ class SimpleTextPointsReader extends PointsReader {
     // NOTE: matches what writeIndex does in SimpleTextPointsWriter
     dataIn.seek(fp);
     readLine(dataIn);
-    int numDims = parseInt(NUM_DIMS);
+    int numDataDims = parseInt(NUM_DATA_DIMS);
+
+    readLine(dataIn);
+    int numIndexDims = parseInt(NUM_INDEX_DIMS);
 
     readLine(dataIn);
     int bytesPerDim = parseInt(BYTES_PER_DIM);
@@ -115,12 +119,12 @@ class SimpleTextPointsReader extends PointsReader {
     readLine(dataIn);
     assert startsWith(MIN_VALUE);
     BytesRef minValue = SimpleTextUtil.fromBytesRefString(stripPrefix(MIN_VALUE));
-    assert minValue.length == numDims*bytesPerDim;
+    assert minValue.length == numIndexDims*bytesPerDim;
 
     readLine(dataIn);
     assert startsWith(MAX_VALUE);
     BytesRef maxValue = SimpleTextUtil.fromBytesRefString(stripPrefix(MAX_VALUE));
-    assert maxValue.length == numDims*bytesPerDim;
+    assert maxValue.length == numIndexDims*bytesPerDim;
 
     readLine(dataIn);
     assert startsWith(POINT_COUNT);
@@ -140,7 +144,7 @@ class SimpleTextPointsReader extends PointsReader {
 
     byte[] splitPackedValues;
     int bytesPerIndexEntry;
-    if (numDims == 1) {
+    if (numIndexDims == 1) {
       bytesPerIndexEntry = bytesPerDim;
     } else {
       bytesPerIndexEntry = 1 + bytesPerDim;
@@ -150,7 +154,7 @@ class SimpleTextPointsReader extends PointsReader {
       readLine(dataIn);
       int address = bytesPerIndexEntry * i;
       int splitDim = parseInt(SPLIT_DIM);
-      if (numDims != 1) {
+      if (numIndexDims != 1) {
         splitPackedValues[address++] = (byte) splitDim;
       }
       readLine(dataIn);
@@ -160,7 +164,7 @@ class SimpleTextPointsReader extends PointsReader {
       System.arraycopy(br.bytes, br.offset, splitPackedValues, address, bytesPerDim);
     }
 
-    return new SimpleTextBKDReader(dataIn, numDims, maxPointsInLeafNode, bytesPerDim, leafBlockFPs, splitPackedValues, minValue.bytes, maxValue.bytes, pointCount, docCount);
+    return new SimpleTextBKDReader(dataIn, numDataDims, numIndexDims, maxPointsInLeafNode, bytesPerDim, leafBlockFPs, splitPackedValues, minValue.bytes, maxValue.bytes, pointCount, docCount);
   }
 
   private void readLine(IndexInput in) throws IOException {
@@ -191,7 +195,7 @@ class SimpleTextPointsReader extends PointsReader {
     if (fieldInfo == null) {
       throw new IllegalArgumentException("field=\"" + fieldName + "\" is unrecognized");
     }
-    if (fieldInfo.getPointDimensionCount() == 0) {
+    if (fieldInfo.getPointDataDimensionCount() == 0) {
       throw new IllegalArgumentException("field=\"" + fieldName + "\" did not index points");
     }
     return readers.get(fieldName);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/804afbfd/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 c3217f3..2da74d6 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPointsWriter.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPointsWriter.java
@@ -35,7 +35,8 @@ import org.apache.lucene.util.BytesRefBuilder;
 
 class SimpleTextPointsWriter extends PointsWriter {
 
-  public final static BytesRef NUM_DIMS      = new BytesRef("num dims ");
+  public final static BytesRef NUM_DATA_DIMS  = new BytesRef("num data dims ");
+  public final static BytesRef NUM_INDEX_DIMS = new BytesRef("num index dims ");
   public final static BytesRef 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 ");
@@ -76,7 +77,8 @@ class SimpleTextPointsWriter extends PointsWriter {
     try (SimpleTextBKDWriter writer = new SimpleTextBKDWriter(writeState.segmentInfo.maxDoc(),
                                                               writeState.directory,
                                                               writeState.segmentInfo.name,
-                                                              fieldInfo.getPointDimensionCount(),
+                                                              fieldInfo.getPointDataDimensionCount(),
+                                                              fieldInfo.getPointIndexDimensionCount(),
                                                               fieldInfo.getPointNumBytes(),
                                                               SimpleTextBKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE,
                                                               SimpleTextBKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/804afbfd/lucene/core/src/java/org/apache/lucene/codecs/PointsWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/PointsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/PointsWriter.java
index d9a0b30..dd1eaeb 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/PointsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/PointsWriter.java
@@ -48,7 +48,7 @@ public abstract class PointsWriter implements Closeable {
       PointsReader pointsReader = mergeState.pointsReaders[i];
       if (pointsReader != null) {
         FieldInfo readerFieldInfo = mergeState.fieldInfos[i].fieldInfo(fieldInfo.name);
-        if (readerFieldInfo != null && readerFieldInfo.getPointDimensionCount() > 0) {
+        if (readerFieldInfo != null && readerFieldInfo.getPointDataDimensionCount() > 0) {
           PointValues values = pointsReader.getValues(fieldInfo.name);
           if (values != null) {
             maxPointCount += values.size();
@@ -92,7 +92,7 @@ public abstract class PointsWriter implements Closeable {
                     continue;
                   }
 
-                  if (readerFieldInfo.getPointDimensionCount() == 0) {
+                  if (readerFieldInfo.getPointDataDimensionCount() == 0) {
                     // This segment saw this field, but the field did not index points in it:
                     continue;
                   }
@@ -143,7 +143,12 @@ public abstract class PointsWriter implements Closeable {
               }
 
               @Override
-              public int getNumDimensions() {
+              public int getNumDataDimensions() {
+                throw new UnsupportedOperationException();
+              }
+
+              @Override
+              public int getNumIndexDimensions() {
                 throw new UnsupportedOperationException();
               }
 
@@ -182,7 +187,7 @@ public abstract class PointsWriter implements Closeable {
     }
     // merge field at a time
     for (FieldInfo fieldInfo : mergeState.mergeFieldInfos) {
-      if (fieldInfo.getPointDimensionCount() != 0) {
+      if (fieldInfo.getPointDataDimensionCount() != 0) {
         mergeOneField(mergeState, fieldInfo);
       }
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/804afbfd/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java
index 30dca70..0ad0cad 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java
@@ -148,7 +148,7 @@ public final class Lucene50FieldInfosFormat extends FieldInfosFormat {
           lastAttributes = attributes;
           try {
             infos[i] = new FieldInfo(name, fieldNumber, storeTermVector, omitNorms, storePayloads, 
-                                     indexOptions, docValuesType, dvGen, attributes, 0, 0, false);
+                                     indexOptions, docValuesType, dvGen, attributes, 0, 0, 0, false);
             infos[i].checkConsistency();
           } catch (IllegalStateException e) {
             throw new CorruptIndexException("invalid fieldinfo for field: " + name + ", fieldNumber=" + fieldNumber, input, e);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/804afbfd/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60FieldInfosFormat.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60FieldInfosFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60FieldInfosFormat.java
index 522a73f..e2ca9ee 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60FieldInfosFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60FieldInfosFormat.java
@@ -114,7 +114,7 @@ public final class Lucene60FieldInfosFormat extends FieldInfosFormat {
       Throwable priorE = null;
       FieldInfo infos[] = null;
       try {
-        CodecUtil.checkIndexHeader(input,
+        int version = CodecUtil.checkIndexHeader(input,
                                    Lucene60FieldInfosFormat.CODEC_NAME, 
                                    Lucene60FieldInfosFormat.FORMAT_START, 
                                    Lucene60FieldInfosFormat.FORMAT_CURRENT,
@@ -149,9 +149,13 @@ public final class Lucene60FieldInfosFormat extends FieldInfosFormat {
             attributes = lastAttributes;
           }
           lastAttributes = attributes;
-          int pointDimensionCount = input.readVInt();
+          int pointDataDimensionCount = input.readVInt();
           int pointNumBytes;
-          if (pointDimensionCount != 0) {
+          int pointIndexDimensionCount = pointDataDimensionCount;
+          if (pointDataDimensionCount != 0) {
+            if (version >= Lucene60FieldInfosFormat.FORMAT_SELECTIVE_INDEXING) {
+              pointIndexDimensionCount = input.readVInt();
+            }
             pointNumBytes = input.readVInt();
           } else {
             pointNumBytes = 0;
@@ -160,7 +164,7 @@ public final class Lucene60FieldInfosFormat extends FieldInfosFormat {
           try {
             infos[i] = new FieldInfo(name, fieldNumber, storeTermVector, omitNorms, storePayloads, 
                                      indexOptions, docValuesType, dvGen, attributes,
-                                     pointDimensionCount, pointNumBytes, isSoftDeletesField);
+                                     pointDataDimensionCount, pointIndexDimensionCount, pointNumBytes, isSoftDeletesField);
             infos[i].checkConsistency();
           } catch (IllegalStateException e) {
             throw new CorruptIndexException("invalid fieldinfo for field: " + name + ", fieldNumber=" + fieldNumber, input, e);
@@ -287,9 +291,9 @@ public final class Lucene60FieldInfosFormat extends FieldInfosFormat {
         output.writeByte(docValuesByte(fi.getDocValuesType()));
         output.writeLong(fi.getDocValuesGen());
         output.writeMapOfStrings(fi.attributes());
-        int pointDimensionCount = fi.getPointDimensionCount();
-        output.writeVInt(pointDimensionCount);
-        if (pointDimensionCount != 0) {
+        output.writeVInt(fi.getPointDataDimensionCount());
+        if (fi.getPointDataDimensionCount() != 0) {
+          output.writeVInt(fi.getPointIndexDimensionCount());
           output.writeVInt(fi.getPointNumBytes());
         }
       }
@@ -304,7 +308,8 @@ public final class Lucene60FieldInfosFormat extends FieldInfosFormat {
   static final String CODEC_NAME = "Lucene60FieldInfos";
   static final int FORMAT_START = 0;
   static final int FORMAT_SOFT_DELETES = 1;
-  static final int FORMAT_CURRENT = FORMAT_SOFT_DELETES;
+  static final int FORMAT_SELECTIVE_INDEXING = 2;
+  static final int FORMAT_CURRENT = FORMAT_SELECTIVE_INDEXING;
   
   // Field flags
   static final byte STORE_TERMVECTOR = 0x1;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/804afbfd/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsReader.java
index 0eecdbb..b5ff680 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsReader.java
@@ -124,7 +124,7 @@ public class Lucene60PointsReader extends PointsReader implements Closeable {
     if (fieldInfo == null) {
       throw new IllegalArgumentException("field=\"" + fieldName + "\" is unrecognized");
     }
-    if (fieldInfo.getPointDimensionCount() == 0) {
+    if (fieldInfo.getPointDataDimensionCount() == 0) {
       throw new IllegalArgumentException("field=\"" + fieldName + "\" did not index point values");
     }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/804afbfd/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java
index 4f51e26..fddf08c 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java
@@ -94,7 +94,8 @@ public class Lucene60PointsWriter extends PointsWriter implements Closeable {
     try (BKDWriter writer = new BKDWriter(writeState.segmentInfo.maxDoc(),
                                           writeState.directory,
                                           writeState.segmentInfo.name,
-                                          fieldInfo.getPointDimensionCount(),
+                                          fieldInfo.getPointDataDimensionCount(),
+                                          fieldInfo.getPointIndexDimensionCount(),
                                           fieldInfo.getPointNumBytes(),
                                           maxPointsInLeafNode,
                                           maxMBSortInHeap,
@@ -152,8 +153,8 @@ public class Lucene60PointsWriter extends PointsWriter implements Closeable {
     }
 
     for (FieldInfo fieldInfo : mergeState.mergeFieldInfos) {
-      if (fieldInfo.getPointDimensionCount() != 0) {
-        if (fieldInfo.getPointDimensionCount() == 1) {
+      if (fieldInfo.getPointDataDimensionCount() != 0) {
+        if (fieldInfo.getPointDataDimensionCount() == 1) {
 
           boolean singleValuePerDoc = true;
 
@@ -164,7 +165,7 @@ public class Lucene60PointsWriter extends PointsWriter implements Closeable {
             if (reader != null) {
               FieldInfos readerFieldInfos = mergeState.fieldInfos[i];
               FieldInfo readerFieldInfo = readerFieldInfos.fieldInfo(fieldInfo.name);
-              if (readerFieldInfo != null && readerFieldInfo.getPointDimensionCount() > 0) {
+              if (readerFieldInfo != null && readerFieldInfo.getPointDataDimensionCount() > 0) {
                 PointValues values = reader.getValues(fieldInfo.name);
                 if (values != null) {
                   totMaxSize += values.size();
@@ -181,7 +182,8 @@ public class Lucene60PointsWriter extends PointsWriter implements Closeable {
           try (BKDWriter writer = new BKDWriter(writeState.segmentInfo.maxDoc(),
                                                 writeState.directory,
                                                 writeState.segmentInfo.name,
-                                                fieldInfo.getPointDimensionCount(),
+                                                fieldInfo.getPointDataDimensionCount(),
+                                                fieldInfo.getPointIndexDimensionCount(),
                                                 fieldInfo.getPointNumBytes(),
                                                 maxPointsInLeafNode,
                                                 maxMBSortInHeap,
@@ -204,7 +206,7 @@ public class Lucene60PointsWriter extends PointsWriter implements Closeable {
 
                 FieldInfos readerFieldInfos = mergeState.fieldInfos[i];
                 FieldInfo readerFieldInfo = readerFieldInfos.fieldInfo(fieldInfo.name);
-                if (readerFieldInfo != null && readerFieldInfo.getPointDimensionCount() > 0) {
+                if (readerFieldInfo != null && readerFieldInfo.getPointDataDimensionCount() > 0) {
                   BKDReader bkdReader = reader60.readers.get(readerFieldInfo.number);
                   if (bkdReader != null) {
                     bkdReaders.add(bkdReader);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/804afbfd/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldMergeState.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldMergeState.java b/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldMergeState.java
index 991eedf..291a384 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldMergeState.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldMergeState.java
@@ -134,7 +134,7 @@ final class PerFieldMergeState {
           hasNorms |= fi.hasNorms();
           hasDocValues |= fi.getDocValuesType() != DocValuesType.NONE;
           hasPayloads |= fi.hasPayloads();
-          hasPointValues |= (fi.getPointDimensionCount() != 0);
+          hasPointValues |= (fi.getPointDataDimensionCount() != 0);
         }
       }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/804afbfd/lucene/core/src/java/org/apache/lucene/document/BinaryPoint.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/BinaryPoint.java b/lucene/core/src/java/org/apache/lucene/document/BinaryPoint.java
index 693a00f..f4b625e 100644
--- a/lucene/core/src/java/org/apache/lucene/document/BinaryPoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/BinaryPoint.java
@@ -124,8 +124,8 @@ public final class BinaryPoint extends Field {
   /** Expert API */
   public BinaryPoint(String name, byte[] packedPoint, IndexableFieldType type) {
     super(name, packedPoint, type);
-    if (packedPoint.length != type.pointDimensionCount() * type.pointNumBytes()) {
-      throw new IllegalArgumentException("packedPoint is length=" + packedPoint.length + " but type.pointDimensionCount()=" + type.pointDimensionCount() + " and type.pointNumBytes()=" + type.pointNumBytes());
+    if (packedPoint.length != type.pointDataDimensionCount() * type.pointNumBytes()) {
+      throw new IllegalArgumentException("packedPoint is length=" + packedPoint.length + " but type.pointDimensionCount()=" + type.pointDataDimensionCount() + " and type.pointNumBytes()=" + type.pointNumBytes());
     }
   }
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/804afbfd/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java b/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java
index 6547402..e442bec 100644
--- a/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java
@@ -85,8 +85,8 @@ public final class DoublePoint extends Field {
 
   /** Change the values of this field */
   public void setDoubleValues(double... point) {
-    if (type.pointDimensionCount() != point.length) {
-      throw new IllegalArgumentException("this field (name=" + name + ") uses " + type.pointDimensionCount() + " dimensions; cannot change to (incoming) " + point.length + " dimensions");
+    if (type.pointDataDimensionCount() != point.length) {
+      throw new IllegalArgumentException("this field (name=" + name + ") uses " + type.pointDataDimensionCount() + " dimensions; cannot change to (incoming) " + point.length + " dimensions");
     }
     fieldsData = pack(point);
   }
@@ -98,8 +98,8 @@ public final class DoublePoint extends Field {
 
   @Override
   public Number numericValue() {
-    if (type.pointDimensionCount() != 1) {
-      throw new IllegalStateException("this field (name=" + name + ") uses " + type.pointDimensionCount() + " dimensions; cannot convert to a single numeric value");
+    if (type.pointDataDimensionCount() != 1) {
+      throw new IllegalStateException("this field (name=" + name + ") uses " + type.pointDataDimensionCount() + " dimensions; cannot convert to a single numeric value");
     }
     BytesRef bytes = (BytesRef) fieldsData;
     assert bytes.length == Double.BYTES;
@@ -142,7 +142,7 @@ public final class DoublePoint extends Field {
     result.append(':');
 
     BytesRef bytes = (BytesRef) fieldsData;
-    for (int dim = 0; dim < type.pointDimensionCount(); dim++) {
+    for (int dim = 0; dim < type.pointDataDimensionCount(); dim++) {
       if (dim > 0) {
         result.append(',');
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/804afbfd/lucene/core/src/java/org/apache/lucene/document/DoubleRange.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/DoubleRange.java b/lucene/core/src/java/org/apache/lucene/document/DoubleRange.java
index cf308c3..726d0a1 100644
--- a/lucene/core/src/java/org/apache/lucene/document/DoubleRange.java
+++ b/lucene/core/src/java/org/apache/lucene/document/DoubleRange.java
@@ -77,8 +77,8 @@ public class DoubleRange extends Field {
    */
   public void setRangeValues(double[] min, double[] max) {
     checkArgs(min, max);
-    if (min.length*2 != type.pointDimensionCount() || max.length*2 != type.pointDimensionCount()) {
-      throw new IllegalArgumentException("field (name=" + name + ") uses " + type.pointDimensionCount()/2
+    if (min.length*2 != type.pointDataDimensionCount() || max.length*2 != type.pointDataDimensionCount()) {
+      throw new IllegalArgumentException("field (name=" + name + ") uses " + type.pointDataDimensionCount()/2
           + " dimensions; cannot change to (incoming) " + min.length + " dimensions");
     }
 
@@ -148,7 +148,7 @@ public class DoubleRange extends Field {
    * @return the decoded min value
    */
   public double getMin(int dimension) {
-    FutureObjects.checkIndex(dimension, type.pointDimensionCount()/2);
+    FutureObjects.checkIndex(dimension, type.pointDataDimensionCount()/2);
     return decodeMin(((BytesRef)fieldsData).bytes, dimension);
   }
 
@@ -158,7 +158,7 @@ public class DoubleRange extends Field {
    * @return the decoded max value
    */
   public double getMax(int dimension) {
-    FutureObjects.checkIndex(dimension, type.pointDimensionCount()/2);
+    FutureObjects.checkIndex(dimension, type.pointDataDimensionCount()/2);
     return decodeMax(((BytesRef)fieldsData).bytes, dimension);
   }
 
@@ -244,7 +244,7 @@ public class DoubleRange extends Field {
     sb.append(':');
     byte[] b = ((BytesRef)fieldsData).bytes;
     toString(b, 0);
-    for (int d = 0; d < type.pointDimensionCount() / 2; ++d) {
+    for (int d = 0; d < type.pointDataDimensionCount() / 2; ++d) {
       sb.append(' ');
       sb.append(toString(b, d));
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/804afbfd/lucene/core/src/java/org/apache/lucene/document/FieldType.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/FieldType.java b/lucene/core/src/java/org/apache/lucene/document/FieldType.java
index 3c7d276..a21572e 100644
--- a/lucene/core/src/java/org/apache/lucene/document/FieldType.java
+++ b/lucene/core/src/java/org/apache/lucene/document/FieldType.java
@@ -38,7 +38,8 @@ public class FieldType implements IndexableFieldType  {
   private IndexOptions indexOptions = IndexOptions.NONE;
   private boolean frozen;
   private DocValuesType docValuesType = DocValuesType.NONE;
-  private int dimensionCount;
+  private int dataDimensionCount;
+  private int indexDimensionCount;
   private int dimensionNumBytes;
 
   /**
@@ -54,7 +55,8 @@ public class FieldType implements IndexableFieldType  {
     this.omitNorms = ref.omitNorms();
     this.indexOptions = ref.indexOptions();
     this.docValuesType = ref.docValuesType();
-    this.dimensionCount = ref.pointDimensionCount();
+    this.dataDimensionCount = ref.pointDataDimensionCount();
+    this.indexDimensionCount = ref.pointIndexDimensionCount();
     this.dimensionNumBytes = ref.pointNumBytes();
     // Do not copy frozen!
   }
@@ -279,11 +281,24 @@ public class FieldType implements IndexableFieldType  {
    * Enables points indexing.
    */
   public void setDimensions(int dimensionCount, int dimensionNumBytes) {
-    if (dimensionCount < 0) {
-      throw new IllegalArgumentException("dimensionCount must be >= 0; got " + dimensionCount);
+    this.setDimensions(dimensionCount, dimensionCount, dimensionNumBytes);
+  }
+
+  /**
+   * Enables points indexing with selectable dimension indexing.
+   */
+  public void setDimensions(int dataDimensionCount, int indexDimensionCount, int dimensionNumBytes) {
+    if (dataDimensionCount < 0) {
+      throw new IllegalArgumentException("dataDimensionCount must be >= 0; got " + dataDimensionCount);
+    }
+    if (dataDimensionCount > PointValues.MAX_DIMENSIONS) {
+      throw new IllegalArgumentException("dataDimensionCount must be <= " + PointValues.MAX_DIMENSIONS + "; got " + dataDimensionCount);
     }
-    if (dimensionCount > PointValues.MAX_DIMENSIONS) {
-      throw new IllegalArgumentException("dimensionCount must be <= " + PointValues.MAX_DIMENSIONS + "; got " + dimensionCount);
+    if (indexDimensionCount < 0) {
+      throw new IllegalArgumentException("indexDimensionCount must be >= 0; got " + indexDimensionCount);
+    }
+    if (indexDimensionCount > dataDimensionCount) {
+      throw new IllegalArgumentException("indexDimensionCount must be <= dataDimensionCount: " + dataDimensionCount + "; got " + indexDimensionCount);
     }
     if (dimensionNumBytes < 0) {
       throw new IllegalArgumentException("dimensionNumBytes must be >= 0; got " + dimensionNumBytes);
@@ -291,23 +306,34 @@ public class FieldType implements IndexableFieldType  {
     if (dimensionNumBytes > PointValues.MAX_NUM_BYTES) {
       throw new IllegalArgumentException("dimensionNumBytes must be <= " + PointValues.MAX_NUM_BYTES + "; got " + dimensionNumBytes);
     }
-    if (dimensionCount == 0) {
+    if (dataDimensionCount == 0) {
+      if (indexDimensionCount != 0) {
+        throw new IllegalArgumentException("when dataDimensionCount is 0, indexDimensionCount must be 0; got " + indexDimensionCount);
+      }
       if (dimensionNumBytes != 0) {
-        throw new IllegalArgumentException("when dimensionCount is 0, dimensionNumBytes must 0; got " + dimensionNumBytes);
+        throw new IllegalArgumentException("when dataDimensionCount is 0, dimensionNumBytes must be 0; got " + dimensionNumBytes);
       }
+    } else if (indexDimensionCount == 0) {
+      throw new IllegalArgumentException("when dataDimensionCount is > 0, indexDimensionCount must be > 0; got " + indexDimensionCount);
     } else if (dimensionNumBytes == 0) {
-      if (dimensionCount != 0) {
-        throw new IllegalArgumentException("when dimensionNumBytes is 0, dimensionCount must 0; got " + dimensionCount);
+      if (dataDimensionCount != 0) {
+        throw new IllegalArgumentException("when dimensionNumBytes is 0, dataDimensionCount must be 0; got " + dataDimensionCount);
       }
     }
 
-    this.dimensionCount = dimensionCount;
+    this.dataDimensionCount = dataDimensionCount;
+    this.indexDimensionCount = indexDimensionCount;
     this.dimensionNumBytes = dimensionNumBytes;
   }
 
   @Override
-  public int pointDimensionCount() {
-    return dimensionCount;
+  public int pointDataDimensionCount() {
+    return dataDimensionCount;
+  }
+
+  @Override
+  public int pointIndexDimensionCount() {
+    return indexDimensionCount;
   }
 
   @Override
@@ -349,12 +375,14 @@ public class FieldType implements IndexableFieldType  {
         result.append(indexOptions);
       }
     }
-    if (dimensionCount != 0) {
+    if (dataDimensionCount != 0) {
       if (result.length() > 0) {
         result.append(",");
       }
-      result.append("pointDimensionCount=");
-      result.append(dimensionCount);
+      result.append("pointDataDimensionCount=");
+      result.append(dataDimensionCount);
+      result.append(",pointIndexDimensionCount=");
+      result.append(indexDimensionCount);
       result.append(",pointNumBytes=");
       result.append(dimensionNumBytes);
     }
@@ -399,7 +427,8 @@ public class FieldType implements IndexableFieldType  {
   public int hashCode() {
     final int prime = 31;
     int result = 1;
-    result = prime * result + dimensionCount;
+    result = prime * result + dataDimensionCount;
+    result = prime * result + indexDimensionCount;
     result = prime * result + dimensionNumBytes;
     result = prime * result + ((docValuesType == null) ? 0 : docValuesType.hashCode());
     result = prime * result + indexOptions.hashCode();
@@ -419,7 +448,8 @@ public class FieldType implements IndexableFieldType  {
     if (obj == null) return false;
     if (getClass() != obj.getClass()) return false;
     FieldType other = (FieldType) obj;
-    if (dimensionCount != other.dimensionCount) return false;
+    if (dataDimensionCount != other.dataDimensionCount) return false;
+    if (indexDimensionCount != other.indexDimensionCount) return false;
     if (dimensionNumBytes != other.dimensionNumBytes) return false;
     if (docValuesType != other.docValuesType) return false;
     if (indexOptions != other.indexOptions) return false;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/804afbfd/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java b/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java
index 0ec67fd..b6d508f 100644
--- a/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java
@@ -85,8 +85,8 @@ public final class FloatPoint extends Field {
 
   /** Change the values of this field */
   public void setFloatValues(float... point) {
-    if (type.pointDimensionCount() != point.length) {
-      throw new IllegalArgumentException("this field (name=" + name + ") uses " + type.pointDimensionCount() + " dimensions; cannot change to (incoming) " + point.length + " dimensions");
+    if (type.pointDataDimensionCount() != point.length) {
+      throw new IllegalArgumentException("this field (name=" + name + ") uses " + type.pointDataDimensionCount() + " dimensions; cannot change to (incoming) " + point.length + " dimensions");
     }
     fieldsData = pack(point);
   }
@@ -98,8 +98,8 @@ public final class FloatPoint extends Field {
 
   @Override
   public Number numericValue() {
-    if (type.pointDimensionCount() != 1) {
-      throw new IllegalStateException("this field (name=" + name + ") uses " + type.pointDimensionCount() + " dimensions; cannot convert to a single numeric value");
+    if (type.pointDataDimensionCount() != 1) {
+      throw new IllegalStateException("this field (name=" + name + ") uses " + type.pointDataDimensionCount() + " dimensions; cannot convert to a single numeric value");
     }
     BytesRef bytes = (BytesRef) fieldsData;
     assert bytes.length == Float.BYTES;
@@ -142,7 +142,7 @@ public final class FloatPoint extends Field {
     result.append(':');
 
     BytesRef bytes = (BytesRef) fieldsData;
-    for (int dim = 0; dim < type.pointDimensionCount(); dim++) {
+    for (int dim = 0; dim < type.pointDataDimensionCount(); dim++) {
       if (dim > 0) {
         result.append(',');
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/804afbfd/lucene/core/src/java/org/apache/lucene/document/FloatRange.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/FloatRange.java b/lucene/core/src/java/org/apache/lucene/document/FloatRange.java
index 9b555d6..191631b 100644
--- a/lucene/core/src/java/org/apache/lucene/document/FloatRange.java
+++ b/lucene/core/src/java/org/apache/lucene/document/FloatRange.java
@@ -77,8 +77,8 @@ public class FloatRange extends Field {
    */
   public void setRangeValues(float[] min, float[] max) {
     checkArgs(min, max);
-    if (min.length*2 != type.pointDimensionCount() || max.length*2 != type.pointDimensionCount()) {
-      throw new IllegalArgumentException("field (name=" + name + ") uses " + type.pointDimensionCount()/2
+    if (min.length*2 != type.pointDataDimensionCount() || max.length*2 != type.pointDataDimensionCount()) {
+      throw new IllegalArgumentException("field (name=" + name + ") uses " + type.pointDataDimensionCount()/2
           + " dimensions; cannot change to (incoming) " + min.length + " dimensions");
     }
 
@@ -148,7 +148,7 @@ public class FloatRange extends Field {
    * @return the decoded min value
    */
   public float getMin(int dimension) {
-    FutureObjects.checkIndex(dimension, type.pointDimensionCount()/2);
+    FutureObjects.checkIndex(dimension, type.pointDataDimensionCount()/2);
     return decodeMin(((BytesRef)fieldsData).bytes, dimension);
   }
 
@@ -158,7 +158,7 @@ public class FloatRange extends Field {
    * @return the decoded max value
    */
   public float getMax(int dimension) {
-    FutureObjects.checkIndex(dimension, type.pointDimensionCount()/2);
+    FutureObjects.checkIndex(dimension, type.pointDataDimensionCount()/2);
     return decodeMax(((BytesRef)fieldsData).bytes, dimension);
   }
 
@@ -244,7 +244,7 @@ public class FloatRange extends Field {
     sb.append(':');
     byte[] b = ((BytesRef)fieldsData).bytes;
     toString(b, 0);
-    for (int d = 0; d < type.pointDimensionCount() / 2; ++d) {
+    for (int d = 0; d < type.pointDataDimensionCount() / 2; ++d) {
       sb.append(' ');
       sb.append(toString(b, d));
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/804afbfd/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/IntPoint.java b/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
index 8b3484b..da4f391 100644
--- a/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
@@ -59,8 +59,8 @@ public final class IntPoint extends Field {
 
   /** Change the values of this field */
   public void setIntValues(int... point) {
-    if (type.pointDimensionCount() != point.length) {
-      throw new IllegalArgumentException("this field (name=" + name + ") uses " + type.pointDimensionCount() + " dimensions; cannot change to (incoming) " + point.length + " dimensions");
+    if (type.pointDataDimensionCount() != point.length) {
+      throw new IllegalArgumentException("this field (name=" + name + ") uses " + type.pointDataDimensionCount() + " dimensions; cannot change to (incoming) " + point.length + " dimensions");
     }
     fieldsData = pack(point);
   }
@@ -72,8 +72,8 @@ public final class IntPoint extends Field {
 
   @Override
   public Number numericValue() {
-    if (type.pointDimensionCount() != 1) {
-      throw new IllegalStateException("this field (name=" + name + ") uses " + type.pointDimensionCount() + " dimensions; cannot convert to a single numeric value");
+    if (type.pointDataDimensionCount() != 1) {
+      throw new IllegalStateException("this field (name=" + name + ") uses " + type.pointDataDimensionCount() + " dimensions; cannot convert to a single numeric value");
     }
     BytesRef bytes = (BytesRef) fieldsData;
     assert bytes.length == Integer.BYTES;
@@ -116,7 +116,7 @@ public final class IntPoint extends Field {
     result.append(':');
 
     BytesRef bytes = (BytesRef) fieldsData;
-    for (int dim = 0; dim < type.pointDimensionCount(); dim++) {
+    for (int dim = 0; dim < type.pointDataDimensionCount(); dim++) {
       if (dim > 0) {
         result.append(',');
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/804afbfd/lucene/core/src/java/org/apache/lucene/document/IntRange.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/IntRange.java b/lucene/core/src/java/org/apache/lucene/document/IntRange.java
index e67b94f..b24e80a 100644
--- a/lucene/core/src/java/org/apache/lucene/document/IntRange.java
+++ b/lucene/core/src/java/org/apache/lucene/document/IntRange.java
@@ -77,8 +77,8 @@ public class IntRange extends Field {
    */
   public void setRangeValues(int[] min, int[] max) {
     checkArgs(min, max);
-    if (min.length*2 != type.pointDimensionCount() || max.length*2 != type.pointDimensionCount()) {
-      throw new IllegalArgumentException("field (name=" + name + ") uses " + type.pointDimensionCount()/2
+    if (min.length*2 != type.pointDataDimensionCount() || max.length*2 != type.pointDataDimensionCount()) {
+      throw new IllegalArgumentException("field (name=" + name + ") uses " + type.pointDataDimensionCount()/2
           + " dimensions; cannot change to (incoming) " + min.length + " dimensions");
     }
 
@@ -148,7 +148,7 @@ public class IntRange extends Field {
    * @return the decoded min value
    */
   public int getMin(int dimension) {
-    FutureObjects.checkIndex(dimension, type.pointDimensionCount()/2);
+    FutureObjects.checkIndex(dimension, type.pointDataDimensionCount()/2);
     return decodeMin(((BytesRef)fieldsData).bytes, dimension);
   }
 
@@ -158,7 +158,7 @@ public class IntRange extends Field {
    * @return the decoded max value
    */
   public int getMax(int dimension) {
-    FutureObjects.checkIndex(dimension, type.pointDimensionCount()/2);
+    FutureObjects.checkIndex(dimension, type.pointDataDimensionCount()/2);
     return decodeMax(((BytesRef)fieldsData).bytes, dimension);
   }
 
@@ -244,7 +244,7 @@ public class IntRange extends Field {
     sb.append(':');
     byte[] b = ((BytesRef)fieldsData).bytes;
     toString(b, 0);
-    for (int d = 0; d < type.pointDimensionCount() / 2; ++d) {
+    for (int d = 0; d < type.pointDataDimensionCount() / 2; ++d) {
       sb.append(' ');
       sb.append(toString(b, d));
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/804afbfd/lucene/core/src/java/org/apache/lucene/document/LatLonPoint.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/LatLonPoint.java b/lucene/core/src/java/org/apache/lucene/document/LatLonPoint.java
index a44795d..9cb94ae 100644
--- a/lucene/core/src/java/org/apache/lucene/document/LatLonPoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/LatLonPoint.java
@@ -145,9 +145,9 @@ public class LatLonPoint extends Field {
   /** helper: checks a fieldinfo and throws exception if its definitely not a LatLonPoint */
   static void checkCompatible(FieldInfo fieldInfo) {
     // point/dv properties could be "unset", if you e.g. used only StoredField with this same name in the segment.
-    if (fieldInfo.getPointDimensionCount() != 0 && fieldInfo.getPointDimensionCount() != TYPE.pointDimensionCount()) {
-      throw new IllegalArgumentException("field=\"" + fieldInfo.name + "\" was indexed with numDims=" + fieldInfo.getPointDimensionCount() + 
-                                         " but this point type has numDims=" + TYPE.pointDimensionCount() + 
+    if (fieldInfo.getPointDataDimensionCount() != 0 && fieldInfo.getPointDataDimensionCount() != TYPE.pointDataDimensionCount()) {
+      throw new IllegalArgumentException("field=\"" + fieldInfo.name + "\" was indexed with numDims=" + fieldInfo.getPointDataDimensionCount() +
+          " but this point type has numDims=" + TYPE.pointDataDimensionCount() +
                                          ", is the field really a LatLonPoint?");
     }
     if (fieldInfo.getPointNumBytes() != 0 && fieldInfo.getPointNumBytes() != TYPE.pointNumBytes()) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/804afbfd/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/LongPoint.java b/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
index 686086c..b3189ca 100644
--- a/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
@@ -59,8 +59,8 @@ public final class LongPoint extends Field {
 
   /** Change the values of this field */
   public void setLongValues(long... point) {
-    if (type.pointDimensionCount() != point.length) {
-      throw new IllegalArgumentException("this field (name=" + name + ") uses " + type.pointDimensionCount() + " dimensions; cannot change to (incoming) " + point.length + " dimensions");
+    if (type.pointDataDimensionCount() != point.length) {
+      throw new IllegalArgumentException("this field (name=" + name + ") uses " + type.pointDataDimensionCount() + " dimensions; cannot change to (incoming) " + point.length + " dimensions");
     }
     fieldsData = pack(point);
   }
@@ -72,8 +72,8 @@ public final class LongPoint extends Field {
 
   @Override
   public Number numericValue() {
-    if (type.pointDimensionCount() != 1) {
-      throw new IllegalStateException("this field (name=" + name + ") uses " + type.pointDimensionCount() + " dimensions; cannot convert to a single numeric value");
+    if (type.pointDataDimensionCount() != 1) {
+      throw new IllegalStateException("this field (name=" + name + ") uses " + type.pointDataDimensionCount() + " dimensions; cannot convert to a single numeric value");
     }
     BytesRef bytes = (BytesRef) fieldsData;
     assert bytes.length == Long.BYTES;
@@ -116,7 +116,7 @@ public final class LongPoint extends Field {
     result.append(':');
 
     BytesRef bytes = (BytesRef) fieldsData;
-    for (int dim = 0; dim < type.pointDimensionCount(); dim++) {
+    for (int dim = 0; dim < type.pointDataDimensionCount(); dim++) {
       if (dim > 0) {
         result.append(',');
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/804afbfd/lucene/core/src/java/org/apache/lucene/document/LongRange.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/LongRange.java b/lucene/core/src/java/org/apache/lucene/document/LongRange.java
index 1a1b19a..9056d2d 100644
--- a/lucene/core/src/java/org/apache/lucene/document/LongRange.java
+++ b/lucene/core/src/java/org/apache/lucene/document/LongRange.java
@@ -77,8 +77,8 @@ public class LongRange extends Field {
    */
   public void setRangeValues(long[] min, long[] max) {
     checkArgs(min, max);
-    if (min.length*2 != type.pointDimensionCount() || max.length*2 != type.pointDimensionCount()) {
-      throw new IllegalArgumentException("field (name=" + name + ") uses " + type.pointDimensionCount()/2
+    if (min.length*2 != type.pointDataDimensionCount() || max.length*2 != type.pointDataDimensionCount()) {
+      throw new IllegalArgumentException("field (name=" + name + ") uses " + type.pointDataDimensionCount()/2
           + " dimensions; cannot change to (incoming) " + min.length + " dimensions");
     }
 
@@ -146,7 +146,7 @@ public class LongRange extends Field {
    * @return the decoded min value
    */
   public long getMin(int dimension) {
-    FutureObjects.checkIndex(dimension, type.pointDimensionCount()/2);
+    FutureObjects.checkIndex(dimension, type.pointDataDimensionCount()/2);
     return decodeMin(((BytesRef)fieldsData).bytes, dimension);
   }
 
@@ -156,7 +156,7 @@ public class LongRange extends Field {
    * @return the decoded max value
    */
   public long getMax(int dimension) {
-    FutureObjects.checkIndex(dimension, type.pointDimensionCount()/2);
+    FutureObjects.checkIndex(dimension, type.pointDataDimensionCount()/2);
     return decodeMax(((BytesRef)fieldsData).bytes, dimension);
   }
 
@@ -242,7 +242,7 @@ public class LongRange extends Field {
     sb.append(':');
     byte[] b = ((BytesRef)fieldsData).bytes;
     toString(b, 0);
-    for (int d = 0; d < type.pointDimensionCount() / 2; ++d) {
+    for (int d = 0; d < type.pointDataDimensionCount() / 2; ++d) {
       sb.append(' ');
       sb.append(toString(b, d));
     }