You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2015/10/27 10:31:36 UTC

svn commit: r1710752 [1/3] - in /lucene/dev/trunk: lucene/ lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/ lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/ lucene/core/src/java/org/apache/lucene/codecs/ lucene/core/src/jav...

Author: mikemccand
Date: Tue Oct 27 09:31:35 2015
New Revision: 1710752

URL: http://svn.apache.org/viewvc?rev=1710752&view=rev
Log:
LUCENE-6825: add dimensionally indexed values

Added:
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextBKDReader.java   (with props)
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDimensionalFormat.java   (with props)
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDimensionalReader.java   (with props)
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDimensionalWriter.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DimensionalFormat.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DimensionalReader.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DimensionalWriter.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/DimensionalField.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DimensionalValues.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DimensionalValuesWriter.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiDimensionalValues.java   (with props)
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDimensionalValues.java   (with props)
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingDimensionalFormat.java   (with props)
Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50Codec.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCodec.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCompoundFormat.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosFormat.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextUtil.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/Codec.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/FilterCodec.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene53/Lucene53Codec.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/Document.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/FieldType.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CodecReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FilterCodecReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FilterLeafReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/IndexableFieldType.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/LeafReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/LogMergePolicy.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MergeState.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/ParallelLeafReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SlowCodecReaderWrapper.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/bkd/BKDReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/bkd/BKDUtil.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/document/TestFieldType.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexableField.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/bkd/TestBKD.java
    lucene/dev/trunk/lucene/highlighter/src/java/org/apache/lucene/search/highlight/TermVectorLeafReader.java
    lucene/dev/trunk/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
    lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/index/MergeReaderWrapper.java
    lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/index/SortingLeafReader.java
    lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/uninverting/UninvertingReader.java
    lucene/dev/trunk/lucene/misc/src/test/org/apache/lucene/index/SorterTestBase.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingCodec.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/MismatchedLeafReader.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/RandomPostingsTester.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/Insanity.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestDocSet.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1710752&r1=1710751&r2=1710752&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Tue Oct 27 09:31:35 2015
@@ -39,6 +39,10 @@ New Features
 
 * LUCENE-6825: Add low-level support for block-KD trees (Mike McCandless)
 
+* LUCENE-6852: Add support for dimensionally indexed values to index,
+  document and codec APIs, including a simple text implementation.
+  (Mike McCandless)
+
 API Changes
 
 * LUCENE-3312: The API of oal.document was restructured to

Modified: lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50Codec.java?rev=1710752&r1=1710751&r2=1710752&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50Codec.java (original)
+++ lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50Codec.java Tue Oct 27 09:31:35 2015
@@ -21,6 +21,7 @@ import java.util.Objects;
 
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.CompoundFormat;
+import org.apache.lucene.codecs.DimensionalFormat;
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.FieldInfosFormat;
 import org.apache.lucene.codecs.FilterCodec;
@@ -152,6 +153,11 @@ public class Lucene50Codec extends Codec
     return docValuesFormat;
   }
 
+  @Override
+  public final DimensionalFormat dimensionalFormat() {
+    return DimensionalFormat.EMPTY;
+  }
+
   private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene50");
   private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene50");
 

Added: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextBKDReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextBKDReader.java?rev=1710752&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextBKDReader.java (added)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextBKDReader.java Tue Oct 27 09:31:35 2015
@@ -0,0 +1,95 @@
+package org.apache.lucene.codecs.simpletext;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+
+import org.apache.lucene.index.DimensionalValues.IntersectVisitor;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.StringHelper;
+import org.apache.lucene.util.bkd.BKDReader;
+
+import static org.apache.lucene.codecs.simpletext.SimpleTextDimensionalWriter.BLOCK_COUNT;
+import static org.apache.lucene.codecs.simpletext.SimpleTextDimensionalWriter.BLOCK_DOC_ID;
+import static org.apache.lucene.codecs.simpletext.SimpleTextDimensionalWriter.BLOCK_VALUE;
+
+class SimpleTextBKDReader extends BKDReader {
+
+  public SimpleTextBKDReader(IndexInput datIn, int numDims, int maxPointsInLeafNode, int bytesPerDim, long[] leafBlockFPs, byte[] splitPackedValues) throws IOException {
+    super(datIn, numDims, maxPointsInLeafNode, bytesPerDim, leafBlockFPs, splitPackedValues);
+  }
+
+  @Override
+  protected void visitDocIDs(IndexInput in, long blockFP, IntersectVisitor visitor) throws IOException {
+    BytesRefBuilder scratch = new BytesRefBuilder();
+    in.seek(blockFP);
+    readLine(in, scratch);
+    int count = parseInt(scratch, BLOCK_COUNT);
+    for(int i=0;i<count;i++) {
+      readLine(in, scratch);
+      visitor.visit(parseInt(scratch, BLOCK_DOC_ID));
+    }
+  }
+
+  @Override
+  protected int readDocIDs(IndexInput in, long blockFP, int[] docIDs) throws IOException {
+    BytesRefBuilder scratch = new BytesRefBuilder();
+    in.seek(blockFP);
+    readLine(in, scratch);
+    int count = parseInt(scratch, BLOCK_COUNT);
+    for(int i=0;i<count;i++) {
+      readLine(in, scratch);
+      docIDs[i] = parseInt(scratch, BLOCK_DOC_ID);
+    }
+    return count;
+  }
+
+  @Override
+  protected void visitDocValues(byte[] scratchPackedValue, IndexInput in, int[] docIDs, int count, IntersectVisitor visitor) throws IOException {
+    assert scratchPackedValue.length == packedBytesLength;
+    BytesRefBuilder scratch = new BytesRefBuilder();
+    for(int i=0;i<count;i++) {
+      readLine(in, scratch);
+      assert startsWith(scratch, BLOCK_VALUE);
+      BytesRef br = SimpleTextUtil.fromBytesRefString(stripPrefix(scratch, BLOCK_VALUE));
+      assert br.length == packedBytesLength;
+      System.arraycopy(br.bytes, br.offset, scratchPackedValue, 0, packedBytesLength);
+      visitor.visit(docIDs[i], scratchPackedValue);
+    }
+  }
+
+  private int parseInt(BytesRefBuilder scratch, BytesRef prefix) {
+    assert startsWith(scratch, prefix);
+    return Integer.parseInt(stripPrefix(scratch, prefix));
+  }
+
+  private String stripPrefix(BytesRefBuilder scratch, BytesRef prefix) {
+    return new String(scratch.bytes(), prefix.length, scratch.length() - prefix.length, StandardCharsets.UTF_8);
+  }
+
+  private boolean startsWith(BytesRefBuilder scratch, BytesRef prefix) {
+    return StringHelper.startsWith(scratch.get(), prefix);
+  }
+
+  private void readLine(IndexInput in, BytesRefBuilder scratch) throws IOException {
+    SimpleTextUtil.readLine(in, scratch);
+  }
+}

Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCodec.java?rev=1710752&r1=1710751&r2=1710752&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCodec.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCodec.java Tue Oct 27 09:31:35 2015
@@ -19,12 +19,13 @@ package org.apache.lucene.codecs.simplet
 
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.CompoundFormat;
+import org.apache.lucene.codecs.DimensionalFormat;
+import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.FieldInfosFormat;
 import org.apache.lucene.codecs.LiveDocsFormat;
+import org.apache.lucene.codecs.NormsFormat;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.SegmentInfoFormat;
-import org.apache.lucene.codecs.DocValuesFormat;
-import org.apache.lucene.codecs.NormsFormat;
 import org.apache.lucene.codecs.StoredFieldsFormat;
 import org.apache.lucene.codecs.TermVectorsFormat;
 
@@ -44,11 +45,12 @@ public final class SimpleTextCodec exten
   private final LiveDocsFormat liveDocs = new SimpleTextLiveDocsFormat();
   private final DocValuesFormat dvFormat = new SimpleTextDocValuesFormat();
   private final CompoundFormat compoundFormat = new SimpleTextCompoundFormat();
+  private final DimensionalFormat dimensionalFormat = new SimpleTextDimensionalFormat();
   
   public SimpleTextCodec() {
     super("SimpleText");
   }
-  
+
   @Override
   public PostingsFormat postingsFormat() {
     return postings;
@@ -93,4 +95,9 @@ public final class SimpleTextCodec exten
   public CompoundFormat compoundFormat() {
     return compoundFormat;
   }
+
+  @Override
+  public DimensionalFormat dimensionalFormat() {
+    return dimensionalFormat;
+  }
 }

Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCompoundFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCompoundFormat.java?rev=1710752&r1=1710751&r2=1710752&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCompoundFormat.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCompoundFormat.java Tue Oct 27 09:31:35 2015
@@ -214,7 +214,7 @@ public class SimpleTextCompoundFormat ex
   }
   
   // helper method to strip strip away 'prefix' from 'scratch' and return as String
-  private String stripPrefix(BytesRefBuilder scratch, BytesRef prefix) throws IOException {
+  private String stripPrefix(BytesRefBuilder scratch, BytesRef prefix) {
     return new String(scratch.bytes(), prefix.length, scratch.length() - prefix.length, StandardCharsets.UTF_8);
   }
   

Added: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDimensionalFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDimensionalFormat.java?rev=1710752&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDimensionalFormat.java (added)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDimensionalFormat.java Tue Oct 27 09:31:35 2015
@@ -0,0 +1,53 @@
+package org.apache.lucene.codecs.simpletext;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.DimensionalFormat;
+import org.apache.lucene.codecs.DimensionalReader;
+import org.apache.lucene.codecs.DimensionalWriter;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+
+/** For debugging, curiosity, transparency only!!  Do not
+ *  use this codec in production.
+ *
+ *  <p>This codec stores all dimensional data in a single
+ *  human-readable text file (_N.dim).  You can view this in
+ *  any text editor, and even edit it to alter your index.
+ *
+ *  @lucene.experimental */
+public final class SimpleTextDimensionalFormat extends DimensionalFormat {
+  
+  @Override
+  public DimensionalWriter fieldsWriter(SegmentWriteState state) throws IOException {
+    return new SimpleTextDimensionalWriter(state);
+  }
+
+  @Override
+  public DimensionalReader fieldsReader(SegmentReadState state) throws IOException {
+    return new SimpleTextDimensionalReader(state);
+  }
+
+  /** Extension of dimensional data file */
+  static final String DIMENSIONAL_EXTENSION = "dim";
+
+  /** Extension of dimensional index file */
+  static final String DIMENSIONAL_INDEX_EXTENSION = "dii";
+}

Added: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDimensionalReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDimensionalReader.java?rev=1710752&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDimensionalReader.java (added)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDimensionalReader.java Tue Oct 27 09:31:35 2015
@@ -0,0 +1,176 @@
+package org.apache.lucene.codecs.simpletext;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.codecs.DimensionalReader;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.store.BufferedChecksumIndexInput;
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.StringHelper;
+import org.apache.lucene.util.bkd.BKDReader;
+
+import static org.apache.lucene.codecs.simpletext.SimpleTextDimensionalWriter.BLOCK_FP;
+import static org.apache.lucene.codecs.simpletext.SimpleTextDimensionalWriter.BYTES_PER_DIM;
+import static org.apache.lucene.codecs.simpletext.SimpleTextDimensionalWriter.FIELD_COUNT;
+import static org.apache.lucene.codecs.simpletext.SimpleTextDimensionalWriter.FIELD_FP;
+import static org.apache.lucene.codecs.simpletext.SimpleTextDimensionalWriter.FIELD_FP_NAME;
+import static org.apache.lucene.codecs.simpletext.SimpleTextDimensionalWriter.INDEX_COUNT;
+import static org.apache.lucene.codecs.simpletext.SimpleTextDimensionalWriter.MAX_LEAF_POINTS;
+import static org.apache.lucene.codecs.simpletext.SimpleTextDimensionalWriter.NUM_DIMS;
+import static org.apache.lucene.codecs.simpletext.SimpleTextDimensionalWriter.SPLIT_COUNT;
+import static org.apache.lucene.codecs.simpletext.SimpleTextDimensionalWriter.SPLIT_DIM;
+import static org.apache.lucene.codecs.simpletext.SimpleTextDimensionalWriter.SPLIT_VALUE;
+
+class SimpleTextDimensionalReader extends DimensionalReader {
+
+  private final IndexInput dataIn;
+  final SegmentReadState readState;
+  final Map<String,BKDReader> readers = new HashMap<>();
+  final BytesRefBuilder scratch = new BytesRefBuilder();
+
+  public SimpleTextDimensionalReader(SegmentReadState readState) throws IOException {
+    // Initialize readers now:
+    String fileName = IndexFileNames.segmentFileName(readState.segmentInfo.name, readState.segmentSuffix, SimpleTextDimensionalFormat.DIMENSIONAL_EXTENSION);
+    dataIn = readState.directory.openInput(fileName, IOContext.DEFAULT);
+    String indexFileName = IndexFileNames.segmentFileName(readState.segmentInfo.name, readState.segmentSuffix, SimpleTextDimensionalFormat.DIMENSIONAL_INDEX_EXTENSION);
+    try (ChecksumIndexInput in = readState.directory.openChecksumInput(indexFileName, IOContext.DEFAULT)) {
+      readLine(in);
+      int count = parseInt(FIELD_COUNT);
+      for(int i=0;i<count;i++) {
+        readLine(in);
+        String fieldName = stripPrefix(FIELD_FP_NAME);
+        readLine(in);
+        long fp = parseLong(FIELD_FP);
+        readers.put(fieldName, initReader(fp));
+      }
+      SimpleTextUtil.checkFooter(in);
+    }
+    this.readState = readState;
+  }
+
+  private BKDReader initReader(long fp) throws IOException {
+    // NOTE: matches what writeIndex does in SimpleTextDimensionalWriter
+    dataIn.seek(fp);
+    readLine(dataIn);
+    int numDims = parseInt(NUM_DIMS);
+
+    readLine(dataIn);
+    int bytesPerDim = parseInt(BYTES_PER_DIM);
+
+    readLine(dataIn);
+    int maxPointsInLeafNode = parseInt(MAX_LEAF_POINTS);
+
+    readLine(dataIn);
+    int count = parseInt(INDEX_COUNT);
+    long[] leafBlockFPs = new long[count];
+    for(int i=0;i<count;i++) {
+      readLine(dataIn);
+      leafBlockFPs[i] = parseLong(BLOCK_FP);
+    }
+    readLine(dataIn);
+    count = parseInt(SPLIT_COUNT);
+
+    byte[] splitPackedValues = new byte[count * (1 + bytesPerDim)];
+    for(int i=0;i<count;i++) {
+      readLine(dataIn);
+      splitPackedValues[(1 + bytesPerDim) * i] = (byte) parseInt(SPLIT_DIM);
+      readLine(dataIn);
+      assert startsWith(SPLIT_VALUE);
+      BytesRef br = SimpleTextUtil.fromBytesRefString(stripPrefix(SPLIT_VALUE));
+      assert br.length == bytesPerDim;
+      System.arraycopy(br.bytes, br.offset, splitPackedValues, (1 + bytesPerDim) * i + 1, bytesPerDim);
+    }
+
+    return new SimpleTextBKDReader(dataIn, numDims, maxPointsInLeafNode, bytesPerDim, leafBlockFPs, splitPackedValues);
+  }
+
+  private void readLine(IndexInput in) throws IOException {
+    SimpleTextUtil.readLine(in, scratch);
+  }
+
+  private boolean startsWith(BytesRef prefix) {
+    return StringHelper.startsWith(scratch.get(), prefix);
+  }
+
+  private int parseInt(BytesRef prefix) {
+    assert startsWith(prefix);
+    return Integer.parseInt(stripPrefix(prefix));
+  }
+
+  private long parseLong(BytesRef prefix) {
+    assert startsWith(prefix);
+    return Long.parseLong(stripPrefix(prefix));
+  }
+
+  private String stripPrefix(BytesRef prefix) {
+    return new String(scratch.bytes(), prefix.length, scratch.length() - prefix.length, StandardCharsets.UTF_8);
+  }
+
+  /** Finds all documents and points matching the provided visitor */
+  @Override
+  public void intersect(String field, IntersectVisitor visitor) throws IOException {
+    BKDReader bkdReader = readers.get(field);
+    if (bkdReader == null) {
+      throw new IllegalArgumentException("field=\"" + field + "\" was not indexed with dimensional values");
+    }
+    bkdReader.intersect(visitor);
+  }
+
+  @Override
+  public void checkIntegrity() throws IOException {
+    BytesRefBuilder scratch = new BytesRefBuilder();
+    IndexInput clone = dataIn.clone();
+    clone.seek(0);
+
+    // checksum is fixed-width encoded with 20 bytes, plus 1 byte for newline (the space is included in SimpleTextUtil.CHECKSUM):
+    long footerStartPos = dataIn.length() - (SimpleTextUtil.CHECKSUM.length + 21);
+    ChecksumIndexInput input = new BufferedChecksumIndexInput(clone);
+    while (true) {
+      SimpleTextUtil.readLine(input, scratch);
+      if (input.getFilePointer() >= footerStartPos) {
+        // Make sure we landed at precisely the right location:
+        if (input.getFilePointer() != footerStartPos) {
+          throw new CorruptIndexException("SimpleText failure: footer does not start at expected position current=" + input.getFilePointer() + " vs expected=" + footerStartPos, input);
+        }
+        SimpleTextUtil.checkFooter(input);
+        break;
+      }
+    }
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return 0L;
+  }
+
+  @Override
+  public void close() throws IOException {
+    dataIn.close();
+  }
+}

Added: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDimensionalWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDimensionalWriter.java?rev=1710752&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDimensionalWriter.java (added)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDimensionalWriter.java Tue Oct 27 09:31:35 2015
@@ -0,0 +1,203 @@
+package org.apache.lucene.codecs.simpletext;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.codecs.DimensionalReader;
+import org.apache.lucene.codecs.DimensionalWriter;
+import org.apache.lucene.index.DimensionalValues.IntersectVisitor;
+import org.apache.lucene.index.DimensionalValues.Relation;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.IndexFileNames;
+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 SimpleTextDimensionalWriter extends DimensionalWriter {
+
+  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 ");
+
+  private IndexOutput dataOut;
+  final BytesRefBuilder scratch = new BytesRefBuilder();
+  final SegmentWriteState writeState;
+  final Map<String,Long> indexFPs = new HashMap<>();
+
+  public SimpleTextDimensionalWriter(SegmentWriteState writeState) throws IOException {
+    String fileName = IndexFileNames.segmentFileName(writeState.segmentInfo.name, writeState.segmentSuffix, SimpleTextDimensionalFormat.DIMENSIONAL_EXTENSION);
+    dataOut = writeState.directory.createOutput(fileName, writeState.context);
+    this.writeState = writeState;
+  }
+
+  @Override
+  public void writeField(FieldInfo fieldInfo, DimensionalReader values) throws IOException {
+
+    // We use the normal BKDWriter, but subclass to customize how it writes the index and blocks to disk:
+    BKDWriter writer = new BKDWriter(writeState.directory,
+                                     writeState.segmentInfo.name,
+                                     fieldInfo.getDimensionCount(),
+                                     fieldInfo.getDimensionNumBytes(),
+                                     BKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE,
+                                     BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP) {
+
+        @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);
+
+          for(int i=0;i<leafBlockFPs.length;i++) {
+            write(out, BLOCK_FP);
+            writeLong(out, leafBlockFPs[i]);
+            newline(out);
+          }
+
+          assert (splitPackedValues.length % (1 + fieldInfo.getDimensionNumBytes())) == 0;
+          int count = splitPackedValues.length / (1 + fieldInfo.getDimensionNumBytes());
+          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.getDimensionNumBytes())] & 0xff);
+            newline(out);
+            write(out, SPLIT_VALUE);
+            BytesRef br = new BytesRef(splitPackedValues, 1+(i * (1+fieldInfo.getDimensionNumBytes())), fieldInfo.getDimensionNumBytes());
+            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 writeLeafBlockPackedValue(IndexOutput out, byte[] bytes, int offset, int length) throws IOException {
+          assert length == packedBytesLength;
+          write(out, BLOCK_VALUE);
+          write(out, new BytesRef(bytes, offset, length).toString());
+          newline(out);
+        }          
+      };
+
+    values.intersect(fieldInfo.name, new IntersectVisitor() {
+        @Override
+        public void visit(int docID) {
+          throw new IllegalStateException();
+        }
+
+        public void visit(int docID, byte[] packedValue) throws IOException {
+          writer.add(packedValue, docID);
+        }
+
+        @Override
+        public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
+          return Relation.QUERY_CROSSES_CELL;
+        }
+      });
+    indexFPs.put(fieldInfo.name, writer.finish(dataOut));
+  }
+
+  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 close() throws IOException {
+    if (dataOut != null) {
+      SimpleTextUtil.writeChecksum(dataOut, scratch);
+      dataOut.close();
+      dataOut = null;
+
+      // Write index file
+      String fileName = IndexFileNames.segmentFileName(writeState.segmentInfo.name, writeState.segmentSuffix, SimpleTextDimensionalFormat.DIMENSIONAL_INDEX_EXTENSION);
+      try (IndexOutput indexOut = writeState.directory.createOutput(fileName, writeState.context)) {
+        int count = indexFPs.size();
+        write(indexOut, FIELD_COUNT);
+        write(indexOut, Integer.toString(count));
+        newline(indexOut);
+        for(Map.Entry<String,Long> ent : indexFPs.entrySet()) {
+          write(indexOut, FIELD_FP_NAME);
+          write(indexOut, ent.getKey());
+          newline(indexOut);
+          write(indexOut, FIELD_FP);
+          write(indexOut, Long.toString(ent.getValue()));
+          newline(indexOut);
+        }
+        SimpleTextUtil.writeChecksum(indexOut, scratch);
+      }
+    }
+  }
+}

Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java?rev=1710752&r1=1710751&r2=1710752&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java Tue Oct 27 09:31:35 2015
@@ -24,7 +24,6 @@ import java.nio.charset.StandardCharsets
 import java.text.DecimalFormat;
 import java.text.DecimalFormatSymbols;
 import java.text.ParseException;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.Locale;
 import java.util.Map;
@@ -499,7 +498,7 @@ class SimpleTextDocValuesReader extends
   }
 
   /** Used only in ctor: */
-  private String stripPrefix(BytesRef prefix) throws IOException {
+  private String stripPrefix(BytesRef prefix) {
     return new String(scratch.bytes(), prefix.length, scratch.length() - prefix.length, StandardCharsets.UTF_8);
   }
 

Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosFormat.java?rev=1710752&r1=1710751&r2=1710752&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosFormat.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosFormat.java Tue Oct 27 09:31:35 2015
@@ -64,6 +64,8 @@ public class SimpleTextFieldInfosFormat
   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 DIM_NUM_BYTES   =  new BytesRef("  dimensional num bytes ");
   
   @Override
   public FieldInfos read(Directory directory, SegmentInfo segmentInfo, String segmentSuffix, IOContext iocontext) throws IOException {
@@ -130,8 +132,17 @@ public class SimpleTextFieldInfosFormat
           atts.put(key, value);
         }
 
+        SimpleTextUtil.readLine(input, scratch);
+        assert StringHelper.startsWith(scratch.get(), DIM_COUNT);
+        int dimensionalCount = Integer.parseInt(readString(DIM_COUNT.length, scratch));
+
+        SimpleTextUtil.readLine(input, scratch);
+        assert StringHelper.startsWith(scratch.get(), DIM_NUM_BYTES);
+        int dimensionalNumBytes = Integer.parseInt(readString(DIM_NUM_BYTES.length, scratch));
+
         infos[i] = new FieldInfo(name, fieldNumber, storeTermVector, 
-          omitNorms, storePayloads, indexOptions, docValuesType, dvGen, Collections.unmodifiableMap(atts));
+                                 omitNorms, storePayloads, indexOptions, docValuesType, dvGen, Collections.unmodifiableMap(atts),
+                                 dimensionalCount, dimensionalNumBytes);
       }
 
       SimpleTextUtil.checkFooter(input);
@@ -219,6 +230,14 @@ public class SimpleTextFieldInfosFormat
             SimpleTextUtil.writeNewline(out);
           }
         }
+
+        SimpleTextUtil.write(out, DIM_COUNT);
+        SimpleTextUtil.write(out, Integer.toString(fi.getDimensionCount()), scratch);
+        SimpleTextUtil.writeNewline(out);
+        
+        SimpleTextUtil.write(out, DIM_NUM_BYTES);
+        SimpleTextUtil.write(out, Integer.toString(fi.getDimensionNumBytes()), scratch);
+        SimpleTextUtil.writeNewline(out);
       }
       SimpleTextUtil.writeChecksum(out, scratch);
       success = true;

Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextUtil.java?rev=1710752&r1=1710751&r2=1710752&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextUtil.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextUtil.java Tue Oct 27 09:31:35 2015
@@ -28,7 +28,6 @@ import org.apache.lucene.store.IndexOutp
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.StringHelper;
-import org.apache.lucene.util.UnicodeUtil;
 
 class SimpleTextUtil {
   public final static byte NEWLINE = 10;
@@ -77,15 +76,15 @@ class SimpleTextUtil {
     // same number of bytes
     // (BaseIndexFileFormatTestCase.testMergeStability cares):
     String checksum = String.format(Locale.ROOT, "%020d", out.getChecksum());
-    SimpleTextUtil.write(out, CHECKSUM);
-    SimpleTextUtil.write(out, checksum, scratch);
-    SimpleTextUtil.writeNewline(out);
+    write(out, CHECKSUM);
+    write(out, checksum, scratch);
+    writeNewline(out);
   }
   
   public static void checkFooter(ChecksumIndexInput input) throws IOException {
     BytesRefBuilder scratch = new BytesRefBuilder();
     String expectedChecksum = String.format(Locale.ROOT, "%020d", input.getChecksum());
-    SimpleTextUtil.readLine(input, scratch);
+    readLine(input, scratch);
     if (StringHelper.startsWith(scratch.get(), CHECKSUM) == false) {
       throw new CorruptIndexException("SimpleText failure: expected checksum line but got " + scratch.get().utf8ToString(), input);
     }
@@ -97,4 +96,21 @@ class SimpleTextUtil {
       throw new CorruptIndexException("Unexpected stuff at the end of file, please be careful with your text editor!", input);
     }
   }
+
+  /** Inverse of {@link BytesRef#toString}. */
+  public static BytesRef fromBytesRefString(String s) {
+    if (s.length() < 2) {
+      throw new IllegalArgumentException("string " + s + " was not created from BytesRef.toString?");
+    }
+    if (s.charAt(0) != '[' || s.charAt(s.length()-1) != ']') {
+      throw new IllegalArgumentException("string " + s + " was not created from BytesRef.toString?");
+    }
+    String[] parts = s.substring(1, s.length()-1).split(" ");
+    byte[] bytes = new byte[parts.length];
+    for(int i=0;i<parts.length;i++) {
+      bytes[i] = (byte) Integer.parseInt(parts[i], 16);
+    }
+
+    return new BytesRef(bytes);
+  }
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/Codec.java?rev=1710752&r1=1710751&r2=1710752&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/Codec.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/Codec.java Tue Oct 27 09:31:35 2015
@@ -18,8 +18,8 @@ package org.apache.lucene.codecs;
  */
 
 import java.util.Objects;
-import java.util.Set;
 import java.util.ServiceLoader; // javadocs
+import java.util.Set;
 
 import org.apache.lucene.index.IndexWriterConfig; // javadocs
 import org.apache.lucene.util.NamedSPILoader;
@@ -107,6 +107,9 @@ public abstract class Codec implements N
   
   /** Encodes/decodes compound files */
   public abstract CompoundFormat compoundFormat();
+
+  /** Encodes/decodes dimensional index */
+  public abstract DimensionalFormat dimensionalFormat();
   
   /** looks up a codec by name */
   public static Codec forName(String name) {

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DimensionalFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DimensionalFormat.java?rev=1710752&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DimensionalFormat.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DimensionalFormat.java Tue Oct 27 09:31:35 2015
@@ -0,0 +1,81 @@
+package org.apache.lucene.codecs;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+
+/** 
+ * Encodes/decodes indexed dimensional data.
+ *
+ * @lucene.experimental */
+public abstract class DimensionalFormat {
+
+  /**
+   * Creates a new dimensional format.
+   */
+  protected DimensionalFormat() {
+  }
+
+  /** Writes a new segment */
+  public abstract DimensionalWriter fieldsWriter(SegmentWriteState state) throws IOException;
+
+  /** Reads a segment.  NOTE: by the time this call
+   *  returns, it must hold open any files it will need to
+   *  use; else, those files may be deleted. 
+   *  Additionally, required files may be deleted during the execution of 
+   *  this call before there is a chance to open them. Under these 
+   *  circumstances an IOException should be thrown by the implementation. 
+   *  IOExceptions are expected and will automatically cause a retry of the 
+   *  segment opening logic with the newly revised segments.
+   *  */
+  public abstract DimensionalReader fieldsReader(SegmentReadState state) throws IOException;
+
+  /** A {@code DimensionalFormat} that has nothing indexed */
+  public static final DimensionalFormat EMPTY = new DimensionalFormat() {
+      @Override
+      public DimensionalWriter fieldsWriter(SegmentWriteState state) {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public DimensionalReader fieldsReader(SegmentReadState state) {
+        return new DimensionalReader() {
+          @Override
+          public void close() {
+          }
+
+          @Override
+          public long ramBytesUsed() {
+            return 0L;
+          }
+
+          @Override
+          public void checkIntegrity() {
+          }
+
+          @Override
+          public void intersect(String fieldName, IntersectVisitor visitor) {
+            throw new IllegalArgumentException("field=\"" + fieldName + "\" was not indexed with dimensional values");
+          }
+        };
+      }
+    };
+}

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DimensionalReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DimensionalReader.java?rev=1710752&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DimensionalReader.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DimensionalReader.java Tue Oct 27 09:31:35 2015
@@ -0,0 +1,51 @@
+package org.apache.lucene.codecs;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.lucene.index.DimensionalValues;
+import org.apache.lucene.util.Accountable;
+
+/** Abstract API to visit dimensional values.
+ *
+ * @lucene.experimental
+ */
+public abstract class DimensionalReader extends DimensionalValues implements Closeable, Accountable {
+
+  /** Sole constructor. (For invocation by subclass constructors, typically implicit.) */
+  protected DimensionalReader() {}
+
+  /** 
+   * Checks consistency of this reader.
+   * <p>
+   * Note that this may be costly in terms of I/O, e.g. 
+   * may involve computing a checksum value against large data files.
+   * @lucene.internal
+   */
+  public abstract void checkIntegrity() throws IOException;
+
+  /** 
+   * Returns an instance optimized for merging.
+   * <p>
+   * The default implementation returns {@code this} */
+  public DimensionalReader getMergeInstance() throws IOException {
+    return this;
+  }
+}

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DimensionalWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DimensionalWriter.java?rev=1710752&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DimensionalWriter.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DimensionalWriter.java Tue Oct 27 09:31:35 2015
@@ -0,0 +1,99 @@
+package org.apache.lucene.codecs;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.MergeState;
+
+/** Abstract API to write dimensional values
+ *
+ * @lucene.experimental
+ */
+
+public abstract class DimensionalWriter implements Closeable {
+  /** Sole constructor. (For invocation by subclass 
+   *  constructors, typically implicit.) */
+  protected DimensionalWriter() {
+  }
+
+  /** Write all values contained in the provided reader */
+  public abstract void writeField(FieldInfo fieldInfo, DimensionalReader values) throws IOException;
+
+  /** Default merge implementation to merge incoming dimensional readers by visiting all their points and
+   *  adding to this writer */
+  public void merge(MergeState mergeState) throws IOException {
+    for (FieldInfo fieldInfo : mergeState.mergeFieldInfos) {
+      if (fieldInfo.getDimensionCount() != 0) {
+        writeField(fieldInfo,
+                   new DimensionalReader() {
+                     @Override
+                     public void intersect(String fieldName, IntersectVisitor mergedVisitor) throws IOException {
+                       if (fieldName.equals(fieldInfo.name) == false) {
+                         throw new IllegalArgumentException("field name must match the field being merged");
+                       }
+                       for (int i=0;i<mergeState.dimensionalReaders.length;i++) {
+                         DimensionalReader dimensionalReader = mergeState.dimensionalReaders[i];
+                         MergeState.DocMap docMap = mergeState.docMaps[i];
+                         int docBase = mergeState.docBase[i];
+                         dimensionalReader.intersect(fieldInfo.name,
+                                                     new IntersectVisitor() {
+                                                       @Override
+                                                       public void visit(int docID) {
+                                                         // Should never be called because our compare method never returns Relation.CELL_INSIDE_QUERY
+                                                         throw new IllegalStateException();
+                                                       }
+
+                                                       @Override
+                                                       public void visit(int docID, byte[] packedValue) throws IOException {
+                                                         int newDocID = docMap.get(docID);
+                                                         if (newDocID != -1) {
+                                                           // Not deleted:
+                                                           mergedVisitor.visit(docBase + newDocID, packedValue);
+                                                         }
+                                                       }
+
+                                                       @Override
+                                                       public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
+                                                         // Forces this segment's DimensionalReader to always visit all docs + values:
+                                                         return Relation.QUERY_CROSSES_CELL;
+                                                       }
+                                                     });
+                       }
+                     }
+
+                     @Override
+                     public void checkIntegrity() {
+                       throw new UnsupportedOperationException();
+                     }
+
+                     @Override
+                     public long ramBytesUsed() {
+                       return 0L;
+                     }
+
+                     @Override
+                     public void close() {
+                     }
+                   });
+      }
+    }
+  }
+}

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/FilterCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/FilterCodec.java?rev=1710752&r1=1710751&r2=1710752&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/FilterCodec.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/FilterCodec.java Tue Oct 27 09:31:35 2015
@@ -103,4 +103,9 @@ public abstract class FilterCodec extend
   public CompoundFormat compoundFormat() {
     return delegate.compoundFormat();
   }
+
+  @Override
+  public DimensionalFormat dimensionalFormat() {
+    return delegate.dimensionalFormat();
+  }
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java?rev=1710752&r1=1710751&r2=1710752&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java Tue Oct 27 09:31:35 2015
@@ -152,7 +152,7 @@ public final class Lucene50FieldInfosFor
           lastAttributes = attributes;
           try {
             infos[i] = new FieldInfo(name, fieldNumber, storeTermVector, omitNorms, storePayloads, 
-                                     indexOptions, docValuesType, dvGen, attributes);
+                                     indexOptions, docValuesType, dvGen, attributes, 0, 0);
             infos[i].checkConsistency();
           } catch (IllegalStateException e) {
             throw new CorruptIndexException("invalid fieldinfo for field: " + name + ", fieldNumber=" + fieldNumber, input, e);

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene53/Lucene53Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene53/Lucene53Codec.java?rev=1710752&r1=1710751&r2=1710752&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene53/Lucene53Codec.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene53/Lucene53Codec.java Tue Oct 27 09:31:35 2015
@@ -21,6 +21,9 @@ import java.util.Objects;
 
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.CompoundFormat;
+import org.apache.lucene.codecs.DimensionalFormat;
+import org.apache.lucene.codecs.DimensionalReader;
+import org.apache.lucene.codecs.DimensionalWriter;
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.FieldInfosFormat;
 import org.apache.lucene.codecs.FilterCodec;
@@ -39,6 +42,8 @@ import org.apache.lucene.codecs.lucene50
 import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat;
 import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
 import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
 
 /**
  * Implements the Lucene 5.3 index format, with configurable per-field postings
@@ -157,6 +162,11 @@ public class Lucene53Codec extends Codec
     return docValuesFormat;
   }
 
+  @Override
+  public final DimensionalFormat dimensionalFormat() {
+    return DimensionalFormat.EMPTY;
+  }
+
   private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene50");
   private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene50");
 

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/DimensionalField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/DimensionalField.java?rev=1710752&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/DimensionalField.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/DimensionalField.java Tue Oct 27 09:31:35 2015
@@ -0,0 +1,122 @@
+package org.apache.lucene.document;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.util.BytesRef;
+
+/** A field that is indexed dimensionally such that finding
+ *  all documents within an N-dimensional at search time is
+ *  efficient.  Muliple values for the same field in one documents
+ *  is allowed. */
+
+public final class DimensionalField extends Field {
+
+  private static FieldType getType(byte[][] point) {
+    if (point == null) {
+      throw new IllegalArgumentException("point cannot be null");
+    }
+    if (point.length == 0) {
+      throw new IllegalArgumentException("point cannot be 0 dimensions");
+    }
+    int bytesPerDim = -1;
+    for(int i=0;i<point.length;i++) {
+      byte[] oneDim = point[i];
+      if (oneDim == null) {
+        throw new IllegalArgumentException("point cannot have null values");
+      }
+      if (oneDim.length == 0) {
+        throw new IllegalArgumentException("point cannot have 0-length values");
+      }
+      if (bytesPerDim == -1) {
+        bytesPerDim = oneDim.length;
+      } else if (bytesPerDim != oneDim.length) {
+        throw new IllegalArgumentException("all dimensions must have same bytes length; got " + bytesPerDim + " and " + oneDim.length);
+      }
+    }
+    return getType(point.length, bytesPerDim);
+  }
+
+  private static FieldType getType(int numDims, int bytesPerDim) {
+    FieldType type = new FieldType();
+    type.setDimensions(numDims, bytesPerDim);
+    type.freeze();
+    return type;
+  }
+
+  private static BytesRef pack(byte[]... point) {
+    if (point == null) {
+      throw new IllegalArgumentException("point cannot be null");
+    }
+    if (point.length == 0) {
+      throw new IllegalArgumentException("point cannot be 0 dimensions");
+    }
+    int bytesPerDim = -1;
+    for(byte[] dim : point) {
+      if (dim == null) {
+        throw new IllegalArgumentException("point cannot have null values");
+      }
+      if (bytesPerDim == -1) {
+        if (dim.length == 0) {
+          throw new IllegalArgumentException("point cannot have 0-length values");
+        }
+        bytesPerDim = dim.length;
+      } else if (dim.length != bytesPerDim) {
+        throw new IllegalArgumentException("all dimensions must have same bytes length; got " + bytesPerDim + " and " + dim.length);
+      }
+    }
+    byte[] packed = new byte[bytesPerDim*point.length];
+    for(int i=0;i<point.length;i++) {
+      System.arraycopy(point[i], 0, packed, i*bytesPerDim, bytesPerDim);
+    }
+    return new BytesRef(packed);
+  }
+
+  /** Sugar API: indexes a one-dimensional point */
+  public DimensionalField(String name, byte[] dim1) {
+    super(name, dim1, getType(1, dim1.length));
+  }
+
+  /** Sugar API: indexes a two-dimensional point */
+  public DimensionalField(String name, byte[] dim1, byte[] dim2) {
+    super(name, pack(dim1, dim2), getType(2, dim1.length));
+  }
+
+  /** Sugar API: indexes a three-dimensional point */
+  public DimensionalField(String name, byte[] dim1, byte[] dim2, byte[] dim3) {
+    super(name, pack(dim1, dim2, dim3), getType(3, dim1.length));
+  }
+
+  /** General purpose API: creates a new DimensionalField, indexing the
+   *  provided N-dimensional binary point.
+   *
+   *  @param name field name
+   *  @param point byte[][] value
+   *  @throws IllegalArgumentException if the field name or value is null.
+   */
+  public DimensionalField(String name, byte[][] point) {
+    super(name, pack(point), getType(point));
+  }
+
+  /** Expert API */
+  public DimensionalField(String name, byte[] packedPoint, FieldType type) {
+    super(name, packedPoint, type);
+    if (packedPoint.length != type.dimensionCount() * type.dimensionNumBytes()) {
+      throw new IllegalArgumentException("packedPoint is length=" + packedPoint.length + " but type.dimensionCount()=" + type.dimensionCount() + " and type.dimensionNumBytes()=" + type.dimensionNumBytes());
+    }
+  }
+}

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/Document.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/Document.java?rev=1710752&r1=1710751&r2=1710752&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/Document.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/Document.java Tue Oct 27 09:31:35 2015
@@ -312,7 +312,7 @@ public final class Document implements I
     return new FilterIterator<StorableField, Field>(fields.iterator()) {
       @Override
       protected boolean predicateFunction(Field field) {
-        return field.type.stored() || field.type.docValuesType() != DocValuesType.NONE;
+        return field.type.stored() || field.type.docValuesType() != DocValuesType.NONE || field.type.dimensionCount() != 0;
       }
     };
   }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/FieldType.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/FieldType.java?rev=1710752&r1=1710751&r2=1710752&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/FieldType.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/FieldType.java Tue Oct 27 09:31:35 2015
@@ -55,6 +55,8 @@ public class FieldType implements Indexa
   private boolean frozen;
   private int numericPrecisionStep = NumericUtils.PRECISION_STEP_DEFAULT;
   private DocValuesType docValuesType = DocValuesType.NONE;
+  private int dimensionCount;
+  private int dimensionNumBytes;
 
   /**
    * Create a new mutable FieldType with all of the properties from <code>ref</code>
@@ -71,6 +73,8 @@ public class FieldType implements Indexa
     this.numericType = ref.numericType();
     this.numericPrecisionStep = ref.numericPrecisionStep();
     this.docValuesType = ref.docValuesType();
+    this.dimensionCount = dimensionCount;
+    this.dimensionNumBytes = dimensionNumBytes;
     // Do not copy frozen!
   }
   
@@ -342,6 +346,40 @@ public class FieldType implements Indexa
     return numericPrecisionStep;
   }
 
+  /**
+   * Enables dimensional indexing.
+   */
+  public void setDimensions(int dimensionCount, int dimensionNumBytes) {
+    if (dimensionCount < 0) {
+      throw new IllegalArgumentException("dimensionCount must be >= 0; got " + dimensionCount);
+    }
+    if (dimensionNumBytes < 0) {
+      throw new IllegalArgumentException("dimensionNumBytes must be >= 0; got " + dimensionNumBytes);
+    }
+    if (dimensionCount == 0) {
+      if (dimensionNumBytes != 0) {
+        throw new IllegalArgumentException("when dimensionCount is 0 dimensionNumBytes must 0; got " + dimensionNumBytes);
+      }
+    } else if (dimensionNumBytes == 0) {
+      if (dimensionCount != 0) {
+        throw new IllegalArgumentException("when dimensionNumBytes is 0 dimensionCount must 0; got " + dimensionCount);
+      }
+    }
+
+    this.dimensionCount = dimensionCount;
+    this.dimensionNumBytes = dimensionNumBytes;
+  }
+
+  @Override
+  public int dimensionCount() {
+    return dimensionCount;
+  }
+
+  @Override
+  public int dimensionNumBytes() {
+    return dimensionNumBytes;
+  }
+
   /** Prints a Field for human consumption. */
   @Override
   public final String toString() {
@@ -381,6 +419,12 @@ public class FieldType implements Indexa
         result.append(",numericPrecisionStep=");
         result.append(numericPrecisionStep);
       }
+      if (dimensionCount != 0) {
+        result.append(",dimensionCount=");
+        result.append(dimensionCount);
+        result.append(",dimensionNumBytes=");
+        result.append(dimensionNumBytes);
+      }
     }
     if (docValuesType != DocValuesType.NONE) {
       if (result.length() > 0) {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CodecReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CodecReader.java?rev=1710752&r1=1710751&r2=1710752&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CodecReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CodecReader.java Tue Oct 27 09:31:35 2015
@@ -25,6 +25,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.lucene.codecs.DimensionalReader;
 import org.apache.lucene.codecs.DocValuesProducer;
 import org.apache.lucene.codecs.FieldsProducer;
 import org.apache.lucene.codecs.NormsProducer;
@@ -74,6 +75,12 @@ public abstract class CodecReader extend
    * @lucene.internal
    */
   public abstract FieldsProducer getPostingsReader();
+
+  /**
+   * Expert: retrieve underlying DimensionalReader
+   * @lucene.internal
+   */
+  public abstract DimensionalReader getDimensionalReader();
   
   @Override
   public final void document(int docID, StoredFieldVisitor visitor) throws IOException {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java?rev=1710752&r1=1710751&r2=1710752&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java Tue Oct 27 09:31:35 2015
@@ -23,6 +23,8 @@ import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.codecs.DimensionalFormat;
+import org.apache.lucene.codecs.DimensionalWriter;
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.NormsConsumer;
@@ -91,6 +93,7 @@ final class DefaultIndexingChain extends
     int maxDoc = state.segmentInfo.maxDoc();
     writeNorms(state);
     writeDocValues(state);
+    writeDimensionalValues(state);
     
     // it's possible all docs hit non-aborting exceptions...
     initStoredFieldsWriter();
@@ -118,6 +121,44 @@ final class DefaultIndexingChain extends
     docWriter.codec.fieldInfosFormat().write(state.directory, state.segmentInfo, "", state.fieldInfos, IOContext.DEFAULT);
   }
 
+  /** Writes all buffered dimensional values. */
+  private void writeDimensionalValues(SegmentWriteState state) throws IOException {
+    DimensionalWriter dimensionalWriter = null;
+    boolean success = false;
+    try {
+      for (int i=0;i<fieldHash.length;i++) {
+        PerField perField = fieldHash[i];
+        while (perField != null) {
+          if (perField.dimensionalValuesWriter != null) {
+            if (perField.fieldInfo.getDimensionCount() == 0) {
+              // BUG
+              throw new AssertionError("segment=" + state.segmentInfo + ": field=\"" + perField.fieldInfo.name + "\" has no dimensional values but wrote them");
+            }
+            if (dimensionalWriter == null) {
+              // lazy init
+              DimensionalFormat fmt = state.segmentInfo.getCodec().dimensionalFormat();
+              dimensionalWriter = fmt.fieldsWriter(state);
+            }
+
+            perField.dimensionalValuesWriter.flush(state, dimensionalWriter);
+            perField.dimensionalValuesWriter = null;
+          } else if (perField.fieldInfo.getDimensionCount() != 0) {
+            // BUG
+            throw new AssertionError("segment=" + state.segmentInfo + ": field=\"" + perField.fieldInfo.name + "\" has dimensional values but did not write them");
+          }
+          perField = perField.next;
+        }
+      }
+      success = true;
+    } finally {
+      if (success) {
+        IOUtils.close(dimensionalWriter);
+      } else {
+        IOUtils.closeWhileHandlingException(dimensionalWriter);
+      }
+    }
+  }
+
   /** Writes all buffered doc values (called from {@link #flush}). */
   private void writeDocValues(SegmentWriteState state) throws IOException {
     int maxDoc = state.segmentInfo.maxDoc();
@@ -355,6 +396,9 @@ final class DefaultIndexingChain extends
         if (dvType != DocValuesType.NONE) {
           indexDocValue(fp, dvType, field);
         }
+        if (fieldType.dimensionCount() != 0) {
+          indexDimensionalValue(fp, field);
+        }
       }
     } finally {
       if (abort == false) {
@@ -387,8 +431,27 @@ final class DefaultIndexingChain extends
     }
   }
 
-  /** Called from processDocument to index one field's doc
-   *  value */
+  /** Called from processDocument to index one field's dimensional value */
+  private void indexDimensionalValue(PerField fp, StorableField field) throws IOException {
+    int dimensionCount = field.fieldType().dimensionCount();
+
+    int dimensionNumBytes = field.fieldType().dimensionNumBytes();
+
+    // Record dimensions for this field; this setter will throw IllegalArgExc if
+    // the dimensions were already set to something different:
+    if (fp.fieldInfo.getDimensionCount() == 0) {
+      fieldInfos.globalFieldNumbers.setDimensions(fp.fieldInfo.number, fp.fieldInfo.name, dimensionCount, dimensionNumBytes);
+    }
+
+    fp.fieldInfo.setDimensions(dimensionCount, dimensionNumBytes);
+
+    if (fp.dimensionalValuesWriter == null) {
+      fp.dimensionalValuesWriter = new DimensionalValuesWriter(docWriter, fp.fieldInfo);
+    }
+    fp.dimensionalValuesWriter.addPackedValue(docState.docID, field.binaryValue());
+  }
+
+  /** Called from processDocument to index one field's doc value */
   private void indexDocValue(PerField fp, DocValuesType dvType, StorableField field) throws IOException {
 
     if (fp.fieldInfo.getDocValuesType() == DocValuesType.NONE) {
@@ -516,6 +579,9 @@ final class DefaultIndexingChain extends
     // segment:
     DocValuesWriter docValuesWriter;
 
+    // Non-null if this field ever had dimensional values in this segment:
+    DimensionalValuesWriter dimensionalValuesWriter;
+
     /** We use this to know when a PerField is seen for the
      *  first time in the current document. */
     long fieldGen = -1;

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DimensionalValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DimensionalValues.java?rev=1710752&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DimensionalValues.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DimensionalValues.java Tue Oct 27 09:31:35 2015
@@ -0,0 +1,59 @@
+package org.apache.lucene.index;
+
+import java.io.IOException;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/** Allows recursively visiting indexed dimensional values
+ *
+ *  @lucene.experimental */
+public abstract class DimensionalValues {
+
+  /** Defautl constructor */
+  protected DimensionalValues() {
+  }
+
+  /** Used by {@link #intersect} to check how each recursive cell corresponds to the query. */
+  public enum Relation {
+    /** Return this if the cell is fully contained by the query */
+    CELL_INSIDE_QUERY,
+    /** Return this if the cell and query do not overlap */
+    QUERY_OUTSIDE_CELL,
+    /** Return this if the cell partially overlapps the query */
+    QUERY_CROSSES_CELL};
+
+  /** We recurse the BKD tree, using a provided instance of this to guide the recursion.
+   *
+   * @lucene.experimental */
+  public interface IntersectVisitor {
+    /** Called for all docs in a leaf cell that's fully contained by the query.  The
+     *  consumer should blindly accept the docID. */
+    void visit(int docID) throws IOException;
+
+    /** Called for all docs in a leaf cell that crosses the query.  The consumer
+     *  should scrutinize the packedValue to decide whether to accept it. */
+    void visit(int docID, byte[] packedValue) throws IOException;
+
+    /** Called for non-leaf cells to test how the cell relates to the query, to
+     *  determine how to further recurse down the treer. */
+    Relation compare(byte[] minPackedValue, byte[] maxPackedValue);
+  }
+
+  /** Finds all documents and points matching the provided visitor */
+  public abstract void intersect(String fieldName, IntersectVisitor visitor) throws IOException;
+}

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DimensionalValuesWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DimensionalValuesWriter.java?rev=1710752&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DimensionalValuesWriter.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DimensionalValuesWriter.java Tue Oct 27 09:31:35 2015
@@ -0,0 +1,96 @@
+package org.apache.lucene.index;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.DimensionalReader;
+import org.apache.lucene.codecs.DimensionalWriter;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.ByteBlockPool;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.Counter;
+import org.apache.lucene.util.RamUsageEstimator;
+
+/** Buffers up pending byte[][] value(s) per doc, then flushes when segment flushes. */
+class DimensionalValuesWriter {
+  private final FieldInfo fieldInfo;
+  private final ByteBlockPool bytes;
+  private final Counter iwBytesUsed;
+  private int[] docIDs;
+  private int numDocs;
+  private final byte[] packedValue;
+
+  public DimensionalValuesWriter(DocumentsWriterPerThread docWriter, FieldInfo fieldInfo) {
+    this.fieldInfo = fieldInfo;
+    this.iwBytesUsed = docWriter.bytesUsed;
+    this.bytes = new ByteBlockPool(docWriter.byteBlockAllocator);
+    docIDs = new int[16];
+    iwBytesUsed.addAndGet(16 * RamUsageEstimator.NUM_BYTES_INT);
+    packedValue = new byte[fieldInfo.getDimensionCount() * fieldInfo.getDimensionNumBytes()];
+  }
+
+  public void addPackedValue(int docID, BytesRef value) {
+    if (value == null) {
+      throw new IllegalArgumentException("field=" + fieldInfo.name + ": dimensional value cannot be null");
+    }
+    if (value.length != fieldInfo.getDimensionCount() * fieldInfo.getDimensionNumBytes()) {
+      throw new IllegalArgumentException("field=" + fieldInfo.name + ": this field's value has length=" + value.length + " but should be " + (fieldInfo.getDimensionCount() * fieldInfo.getDimensionNumBytes()));
+    }
+    if (docIDs.length == numDocs) {
+      docIDs = ArrayUtil.grow(docIDs, numDocs+1);
+      iwBytesUsed.addAndGet((docIDs.length - numDocs) * RamUsageEstimator.NUM_BYTES_INT);
+    }
+    bytes.append(value);
+    docIDs[numDocs] = docID;
+    numDocs++;
+  }
+
+  public void flush(SegmentWriteState state, DimensionalWriter writer) throws IOException {
+
+    final int maxDoc = state.segmentInfo.maxDoc();
+
+    writer.writeField(fieldInfo,
+                      new DimensionalReader() {
+                        @Override
+                        public void intersect(String fieldName, IntersectVisitor visitor) throws IOException {
+                          if (fieldName.equals(fieldInfo.name) == false) {
+                            throw new IllegalArgumentException("fieldName must be the same");
+                          }
+                          for(int i=0;i<numDocs;i++) {
+                            bytes.readBytes(packedValue.length * i, packedValue, 0, packedValue.length);
+                            visitor.visit(docIDs[i], packedValue);
+                          }
+                        }
+
+                        @Override
+                        public void checkIntegrity() {
+                          throw new UnsupportedOperationException();
+                        }
+
+                        @Override
+                        public long ramBytesUsed() {
+                          return 0L;
+                        }
+
+                        @Override
+                        public void close() {
+                        }
+                      });
+  }
+}

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java?rev=1710752&r1=1710751&r2=1710752&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java Tue Oct 27 09:31:35 2015
@@ -20,6 +20,8 @@ package org.apache.lucene.index;
 import java.util.Map;
 import java.util.Objects;
 
+import org.apache.lucene.codecs.DimensionalFormat;
+
 /**
  *  Access to the Field Info file that describes document fields and whether or
  *  not they are indexed. Each segment has a separate Field Info file. Objects
@@ -47,14 +49,20 @@ public final class FieldInfo {
   private final Map<String,String> attributes;
 
   private long dvGen;
+
+  /** If both of these are positive it means this is a dimensionally indexed
+   *  field (see {@link DimensionalFormat}). */
+  private int dimensionCount;
+  private int dimensionNumBytes;
+
   /**
    * Sole constructor.
    *
    * @lucene.experimental
    */
   public FieldInfo(String name, int number, boolean storeTermVector, boolean omitNorms, 
-      boolean storePayloads, IndexOptions indexOptions, DocValuesType docValues,
-      long dvGen, Map<String,String> attributes) {
+                   boolean storePayloads, IndexOptions indexOptions, DocValuesType docValues,
+                   long dvGen, Map<String,String> attributes, int dimensionCount, int dimensionNumBytes) {
     this.name = Objects.requireNonNull(name);
     this.number = number;
     this.docValuesType = Objects.requireNonNull(docValues, "DocValuesType cannot be null (field: \"" + name + "\")");
@@ -70,6 +78,8 @@ public final class FieldInfo {
     }
     this.dvGen = dvGen;
     this.attributes = Objects.requireNonNull(attributes);
+    this.dimensionCount = dimensionCount;
+    this.dimensionNumBytes = dimensionNumBytes;
     assert checkConsistency();
   }
 
@@ -94,6 +104,22 @@ public final class FieldInfo {
         throw new IllegalStateException("non-indexed field '" + name + "' cannot omit norms");
       }
     }
+
+    if (dimensionCount < 0) {
+      throw new IllegalStateException("dimensionCount must be >= 0; got " + dimensionCount);
+    }
+
+    if (dimensionNumBytes < 0) {
+      throw new IllegalStateException("dimensionNumBytes must be >= 0; got " + dimensionNumBytes);
+    }
+
+    if (dimensionCount != 0 && dimensionNumBytes == 0) {
+      throw new IllegalStateException("dimensionNumBytes must be > 0 when dimensionCount=" + dimensionCount);
+    }
+
+    if (dimensionNumBytes != 0 && dimensionCount == 0) {
+      throw new IllegalStateException("dimensionCount must be > 0 when dimensionNumBytes=" + dimensionNumBytes);
+    }
     
     if (dvGen != -1 && docValuesType == DocValuesType.NONE) {
       throw new IllegalStateException("field '" + name + "' cannot have a docvalues update generation without having docvalues");
@@ -103,7 +129,8 @@ public final class FieldInfo {
   }
 
   // should only be called by FieldInfos#addOrUpdate
-  void update(boolean storeTermVector, boolean omitNorms, boolean storePayloads, IndexOptions indexOptions) {
+  void update(boolean storeTermVector, boolean omitNorms, boolean storePayloads, IndexOptions indexOptions,
+              int dimensionCount, int dimensionNumBytes) {
     if (indexOptions == null) {
       throw new NullPointerException("IndexOptions cannot be null (field: \"" + name + "\")");
     }
@@ -117,6 +144,11 @@ public final class FieldInfo {
       }
     }
 
+    if (this.dimensionCount == 0 && dimensionCount != 0) {
+      this.dimensionCount = dimensionCount;
+      this.dimensionNumBytes = dimensionNumBytes;
+    }
+
     if (this.indexOptions != IndexOptions.NONE) { // if updated field data is not for indexing, leave the updates out
       this.storeTermVector |= storeTermVector;                // once vector, always vector
       this.storePayloads |= storePayloads;
@@ -133,6 +165,36 @@ public final class FieldInfo {
     assert checkConsistency();
   }
 
+  /** Record that this field is indexed dimensionally, with the
+   *  specified number of dimensions and bytes per dimension. */
+  public void setDimensions(int count, int numBytes) {
+    if (count <= 0) {
+      throw new IllegalArgumentException("dimension count must be >= 0; got " + count + " for field=\"" + name + "\"");
+    }
+    if (numBytes <= 0) {
+      throw new IllegalArgumentException("dimension numBytes must be >= 0; got " + numBytes + " for field=\"" + name + "\"");
+    }
+    if (dimensionCount != 0 && dimensionCount != count) {
+      throw new IllegalArgumentException("cannot change dimension count from " + dimensionCount + " to " + count + " for field=\"" + name + "\"");
+    }
+    if (dimensionNumBytes != 0 && dimensionNumBytes != numBytes) {
+      throw new IllegalArgumentException("cannot change dimension numBytes from " + dimensionNumBytes + " to " + numBytes + " for field=\"" + name + "\"");
+    }
+
+    dimensionCount = count;
+    dimensionNumBytes = numBytes;
+  }
+
+  /** Return dimension count */
+  public int getDimensionCount() {
+    return dimensionCount;
+  }
+
+  /** Return number of bytes per dimension */
+  public int getDimensionNumBytes() {
+    return dimensionNumBytes;
+  }
+
   void setDocValuesType(DocValuesType type) {
     if (type == null) {
       throw new NullPointerException("DocValuesType cannot be null (field: \"" + name + "\")");