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/11/01 11:19:51 UTC

svn commit: r1711732 [1/2] - in /lucene/dev/trunk: lucene/ lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/ lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ lucene/core/src/java/org/apache/lucene/codecs/ lucene/core...

Author: mikemccand
Date: Sun Nov  1 10:19:50 2015
New Revision: 1711732

URL: http://svn.apache.org/viewvc?rev=1711732&view=rev
Log:
LUCENE-6861: switch trunk to Luceen60 codec, implementing dimensional format

Added:
    lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/
    lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/Lucene54Codec.java
      - copied unchanged from r1711727, lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54Codec.java
    lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/package.html   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene60/
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60Codec.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60DimensionalFormat.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60DimensionalReader.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60DimensionalWriter.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60FieldInfosFormat.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene60/package-info.java   (with props)
Removed:
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54Codec.java
Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/Codec.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DimensionalValues.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/MultiDimensionalValues.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SlowCodecReaderWrapper.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
    lucene/dev/trunk/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50StoredFieldsFormatHighCompression.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene53/TestLucene53NormsFormat.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/Test2BTerms.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDimensionalValues.java
    lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/index/SortingLeafReader.java
    lucene/dev/trunk/lucene/misc/src/test/org/apache/lucene/index/SorterTestBase.java
    lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/bkdtree/TestBKDTree.java
    lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/rangetree/TestRangeTree.java
    lucene/dev/trunk/lucene/spatial3d/src/test/org/apache/lucene/bkdtree3d/TestGeo3DPointField.java
    lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingDimensionalFormat.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/RandomIndexWriter.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1711732&r1=1711731&r2=1711732&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Sun Nov  1 10:19:50 2015
@@ -43,6 +43,9 @@ New Features
   document and codec APIs, including a simple text implementation.
   (Mike McCandless)
 
+* LUCENE-6861: Create Lucene60Codec, supporting dimensional values.
+  (Mike McCandless)
+
 API Changes
 
 * LUCENE-3312: The API of oal.document was restructured to

Added: lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/package.html?rev=1711732&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/package.html (added)
+++ lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/package.html Sun Nov  1 10:19:50 2015
@@ -0,0 +1,25 @@
+<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
+<!--
+ 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.
+-->
+<html>
+<head>
+   <meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
+</head>
+<body>
+Lucene 5.4 file format.
+</body>
+</html>

Modified: lucene/dev/trunk/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java?rev=1711732&r1=1711731&r2=1711732&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java (original)
+++ lucene/dev/trunk/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java Sun Nov  1 10:19:50 2015
@@ -29,13 +29,13 @@ import org.apache.lucene.benchmark.byTas
 import org.apache.lucene.benchmark.byTask.utils.Config;
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.lucene54.Lucene54Codec;
+import org.apache.lucene.codecs.lucene60.Lucene60Codec;
 import org.apache.lucene.index.ConcurrentMergeScheduler;
 import org.apache.lucene.index.IndexCommit;
 import org.apache.lucene.index.IndexDeletionPolicy;
 import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
+import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.LogMergePolicy;
 import org.apache.lucene.index.MergePolicy;
 import org.apache.lucene.index.MergeScheduler;
@@ -139,7 +139,7 @@ public class CreateIndexTask extends Per
     if (defaultCodec == null && postingsFormat != null) {
       try {
         final PostingsFormat postingsFormatChosen = PostingsFormat.forName(postingsFormat);
-        iwConf.setCodec(new Lucene54Codec() {
+        iwConf.setCodec(new Lucene60Codec() {
           @Override
           public PostingsFormat getPostingsFormatForField(String field) {
             return postingsFormatChosen;

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=1711732&r1=1711731&r2=1711732&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 Sun Nov  1 10:19:50 2015
@@ -57,7 +57,7 @@ public abstract class Codec implements N
     }
     
     // TODO: should we use this, or maybe a system property is better?
-    static Codec defaultCodec = LOADER.lookup("Lucene54");
+    static Codec defaultCodec = LOADER.lookup("Lucene60");
   }
 
   private final String name;

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60Codec.java?rev=1711732&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60Codec.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60Codec.java Sun Nov  1 10:19:50 2015
@@ -0,0 +1,176 @@
+package org.apache.lucene.codecs.lucene60;
+
+/*
+ * 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.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;
+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.StoredFieldsFormat;
+import org.apache.lucene.codecs.TermVectorsFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50CompoundFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50SegmentInfoFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
+import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat;
+import org.apache.lucene.codecs.lucene53.Lucene53NormsFormat;
+import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
+import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
+
+/**
+ * Implements the Lucene 6.0 index format, with configurable per-field postings
+ * and docvalues formats.
+ * <p>
+ * If you want to reuse functionality of this codec in another codec, extend
+ * {@link FilterCodec}.
+ *
+ * @see org.apache.lucene.codecs.lucene60 package documentation for file format details.
+ *
+ * @lucene.experimental
+ */
+public class Lucene60Codec extends Codec {
+  private final TermVectorsFormat vectorsFormat = new Lucene50TermVectorsFormat();
+  private final FieldInfosFormat fieldInfosFormat = new Lucene60FieldInfosFormat();
+  private final SegmentInfoFormat segmentInfosFormat = new Lucene50SegmentInfoFormat();
+  private final LiveDocsFormat liveDocsFormat = new Lucene50LiveDocsFormat();
+  private final CompoundFormat compoundFormat = new Lucene50CompoundFormat();
+  
+  private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
+    @Override
+    public PostingsFormat getPostingsFormatForField(String field) {
+      return Lucene60Codec.this.getPostingsFormatForField(field);
+    }
+  };
+  
+  private final DocValuesFormat docValuesFormat = new PerFieldDocValuesFormat() {
+    @Override
+    public DocValuesFormat getDocValuesFormatForField(String field) {
+      return Lucene60Codec.this.getDocValuesFormatForField(field);
+    }
+  };
+  
+  private final StoredFieldsFormat storedFieldsFormat;
+
+  /** 
+   * Instantiates a new codec.
+   */
+  public Lucene60Codec() {
+    this(Mode.BEST_SPEED);
+  }
+  
+  /** 
+   * Instantiates a new codec, specifying the stored fields compression
+   * mode to use.
+   * @param mode stored fields compression mode to use for newly 
+   *             flushed/merged segments.
+   */
+  public Lucene60Codec(Mode mode) {
+    super("Lucene60");
+    this.storedFieldsFormat = new Lucene50StoredFieldsFormat(Objects.requireNonNull(mode));
+  }
+  
+  @Override
+  public final StoredFieldsFormat storedFieldsFormat() {
+    return storedFieldsFormat;
+  }
+  
+  @Override
+  public final TermVectorsFormat termVectorsFormat() {
+    return vectorsFormat;
+  }
+
+  @Override
+  public final PostingsFormat postingsFormat() {
+    return postingsFormat;
+  }
+  
+  @Override
+  public final FieldInfosFormat fieldInfosFormat() {
+    return fieldInfosFormat;
+  }
+  
+  @Override
+  public final SegmentInfoFormat segmentInfoFormat() {
+    return segmentInfosFormat;
+  }
+  
+  @Override
+  public final LiveDocsFormat liveDocsFormat() {
+    return liveDocsFormat;
+  }
+
+  @Override
+  public final CompoundFormat compoundFormat() {
+    return compoundFormat;
+  }
+
+  @Override
+  public final DimensionalFormat dimensionalFormat() {
+    return new Lucene60DimensionalFormat();
+  }
+
+  /** Returns the postings format that should be used for writing 
+   *  new segments of <code>field</code>.
+   *  
+   *  The default implementation always returns "Lucene50".
+   *  <p>
+   *  <b>WARNING:</b> if you subclass, you are responsible for index 
+   *  backwards compatibility: future version of Lucene are only 
+   *  guaranteed to be able to read the default implementation. 
+   */
+  public PostingsFormat getPostingsFormatForField(String field) {
+    return defaultFormat;
+  }
+  
+  /** Returns the docvalues format that should be used for writing 
+   *  new segments of <code>field</code>.
+   *  
+   *  The default implementation always returns "Lucene50".
+   *  <p>
+   *  <b>WARNING:</b> if you subclass, you are responsible for index 
+   *  backwards compatibility: future version of Lucene are only 
+   *  guaranteed to be able to read the default implementation. 
+   */
+  public DocValuesFormat getDocValuesFormatForField(String field) {
+    return defaultDVFormat;
+  }
+  
+  @Override
+  public final DocValuesFormat docValuesFormat() {
+    return docValuesFormat;
+  }
+
+  private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene50");
+  private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene54");
+
+  private final NormsFormat normsFormat = new Lucene53NormsFormat();
+
+  @Override
+  public final NormsFormat normsFormat() {
+    return normsFormat;
+  }
+}

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60DimensionalFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60DimensionalFormat.java?rev=1711732&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60DimensionalFormat.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60DimensionalFormat.java Sun Nov  1 10:19:50 2015
@@ -0,0 +1,107 @@
+package org.apache.lucene.codecs.lucene60;
+
+/*
+ * 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.CodecUtil;
+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;
+
+/**
+ * Lucene 6.0 dimensional format, which encodes dimensional values in a block KD-tree structure
+ * for fast shape intersection filtering. See <a href="https://www.cs.duke.edu/~pankaj/publications/papers/bkd-sstd.pdf">this paper</a> for details.
+ *
+ * <p>This data structure is written as a series of blocks on disk, with an in-memory perfectly balanced
+ * binary tree of split values referencing those blocks at the leaves.
+ *
+ * <p>The <code>.dim</code> file has both blocks and the index split
+ * values, for each field.  The file starts with {@link CodecUtil#writeIndexHeader}.
+ *
+ * <p>The blocks are written like this:
+ *
+ * <ul>
+ *  <li> count (vInt)
+ *  <li> delta-docID (vInt) <sup>count</sup> (delta coded docIDs, in sorted order)
+ *  <li> packedValue<sup>count</sup> (the <code>byte[]</code> value of each dimension packed into a single <code>byte[]</code>)
+ * </ul>
+ *
+ * <p>After all blocks for a field are written, then the index is written:
+ * <ul>
+ *  <li> numDims (vInt)
+ *  <li> maxPointsInLeafNode (vInt)
+ *  <li> bytesPerDim (vInt)
+ *  <li> count (vInt)
+ *  <li> byte[bytesPerDim]<sup>count</sup> (packed <code>byte[]</code> all split values)
+ *  <li> delta-blockFP (vLong)<sup>count</sup> (delta-coded file pointers to the on-disk leaf blocks))
+ * </ul>
+ *
+ * <p>After all fields blocks + index data are written, {@link CodecUtil#writeFooter} writes the checksum.
+ *
+ * <p>The <code>.dii</code> file records the file pointer in the <code>.dim</code> file where each field's
+ * index data was written.  It starts with {@link CodecUtil#writeIndexHeader}, then has:
+ *
+ * <ul>
+ *   <li> fieldCount (vInt)
+ *   <li> (fieldNumber (vInt), fieldFilePointer (vLong))<sup>fieldCount</sup>
+ * </ul>
+ *
+ * <p> After that, {@link CodecUtil#writeFooter} writes the checksum.
+ *
+ * <p>After all fields blocks + index data are written, {@link CodecUtil#writeFooter} writes the checksum.
+
+ * @lucene.experimental
+ */
+
+public final class Lucene60DimensionalFormat extends DimensionalFormat {
+
+  static final String CODEC_NAME = "Lucene60DimensionalFormat";
+
+  /**
+   * Filename extension for the leaf blocks
+   */
+  public static final String DATA_EXTENSION = "dim";
+
+  /**
+   * Filename extension for the index per field
+   */
+  public static final String INDEX_EXTENSION = "dii";
+
+  static final int DATA_VERSION_START = 0;
+  static final int DATA_VERSION_CURRENT = DATA_VERSION_START;
+
+  static final int INDEX_VERSION_START = 0;
+  static final int INDEX_VERSION_CURRENT = INDEX_VERSION_START;
+
+  /** Sole constructor */
+  public Lucene60DimensionalFormat() {
+  }
+
+  @Override
+  public DimensionalWriter fieldsWriter(SegmentWriteState state) throws IOException {
+    return new Lucene60DimensionalWriter(state);
+  }
+
+  @Override
+  public DimensionalReader fieldsReader(SegmentReadState state) throws IOException {
+    return new Lucene60DimensionalReader(state);
+  }
+}

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60DimensionalReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60DimensionalReader.java?rev=1711732&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60DimensionalReader.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60DimensionalReader.java Sun Nov  1 10:19:50 2015
@@ -0,0 +1,132 @@
+package org.apache.lucene.codecs.lucene60;
+
+
+/*
+ * 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 java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.DimensionalReader;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.Accountables;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.bkd.BKDReader;
+
+/** Reads dimensional values previously written with {@link Lucene60DimensionalWriter} */
+public class Lucene60DimensionalReader extends DimensionalReader implements Closeable {
+  final IndexInput dataIn;
+  final SegmentReadState readState;
+  final Map<Integer,BKDReader> readers = new HashMap<>();
+
+  /** Sole constructor */
+  public Lucene60DimensionalReader(SegmentReadState readState) throws IOException {
+    this.readState = readState;
+    String dataFileName = IndexFileNames.segmentFileName(readState.segmentInfo.name,
+                                                         readState.segmentSuffix,
+                                                         Lucene60DimensionalFormat.DATA_EXTENSION);
+    dataIn = readState.directory.openInput(dataFileName, readState.context);
+    String indexFileName = IndexFileNames.segmentFileName(readState.segmentInfo.name,
+                                                          readState.segmentSuffix,
+                                                          Lucene60DimensionalFormat.INDEX_EXTENSION);
+
+    boolean success = false;
+
+    // Read index file
+    try (ChecksumIndexInput indexIn = readState.directory.openChecksumInput(indexFileName, readState.context)) {
+      CodecUtil.checkIndexHeader(indexIn,
+                                 Lucene60DimensionalFormat.CODEC_NAME,
+                                 Lucene60DimensionalFormat.INDEX_VERSION_START,
+                                 Lucene60DimensionalFormat.INDEX_VERSION_START,
+                                 readState.segmentInfo.getId(),
+                                 readState.segmentSuffix);
+      int count = indexIn.readVInt();
+      for(int i=0;i<count;i++) {
+        int fieldNumber = indexIn.readVInt();
+        long fp = indexIn.readVLong();
+        dataIn.seek(fp);
+        readers.put(fieldNumber, new BKDReader(dataIn));
+      }
+      CodecUtil.checkFooter(indexIn);
+      success = true;
+    } finally {
+      if (success == false) {
+        IOUtils.closeWhileHandlingException(dataIn);
+      }
+    }
+  }
+
+  @Override
+  public void intersect(String field, IntersectVisitor visitor) throws IOException {
+    FieldInfo fieldInfo = readState.fieldInfos.fieldInfo(field);
+    if (fieldInfo == null) {
+      throw new IllegalArgumentException("field=\"" + field + "\" is unrecognized");
+    }
+    if (fieldInfo.getDimensionCount() == 0) {
+      throw new IllegalArgumentException("field=\"" + field + "\" did not index dimensional values");
+    }
+
+    BKDReader reader = readers.get(fieldInfo.number);
+    assert reader != null;
+
+    reader.intersect(visitor);
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    long sizeInBytes = 0;
+    for(BKDReader reader : readers.values()) {
+      sizeInBytes += reader.ramBytesUsed();
+    }
+    return sizeInBytes;
+  }
+
+  @Override
+  public Collection<Accountable> getChildResources() {
+    List<Accountable> resources = new ArrayList<>();
+    for(Map.Entry<Integer,BKDReader> ent : readers.entrySet()) {
+      resources.add(Accountables.namedAccountable(readState.fieldInfos.fieldInfo(ent.getKey()).name,
+                                                  ent.getValue()));
+    }
+    return Collections.unmodifiableList(resources);
+  }
+
+  @Override
+  public void checkIntegrity() throws IOException {
+    CodecUtil.checksumEntireFile(dataIn);
+  }
+
+  @Override
+  public void close() throws IOException {
+    dataIn.close();
+    // Free up heap:
+    readers.clear();
+  }
+}
+  

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60DimensionalWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60DimensionalWriter.java?rev=1711732&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60DimensionalWriter.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60DimensionalWriter.java Sun Nov  1 10:19:50 2015
@@ -0,0 +1,140 @@
+package org.apache.lucene.codecs.lucene60;
+
+
+/*
+ * 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 java.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.codecs.CodecUtil;
+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.IOUtils;
+import org.apache.lucene.util.bkd.BKDWriter;
+
+/** Writes dimensional values */
+public class Lucene60DimensionalWriter extends DimensionalWriter implements Closeable {
+  
+  final IndexOutput dataOut;
+  final Map<String,Long> indexFPs = new HashMap<>();
+  final SegmentWriteState writeState;
+  final int maxPointsInLeafNode;
+  final double maxMBSortInHeap;
+  private boolean closed;
+
+  /** Full constructor */
+  public Lucene60DimensionalWriter(SegmentWriteState writeState, int maxPointsInLeafNode, double maxMBSortInHeap) throws IOException {
+    assert writeState.fieldInfos.hasDimensionalValues();
+    this.writeState = writeState;
+    this.maxPointsInLeafNode = maxPointsInLeafNode;
+    this.maxMBSortInHeap = maxMBSortInHeap;
+    String dataFileName = IndexFileNames.segmentFileName(writeState.segmentInfo.name,
+                                                         writeState.segmentSuffix,
+                                                         Lucene60DimensionalFormat.DATA_EXTENSION);
+    dataOut = writeState.directory.createOutput(dataFileName, writeState.context);
+    boolean success = false;
+    try {
+      CodecUtil.writeIndexHeader(dataOut,
+                                 Lucene60DimensionalFormat.CODEC_NAME,
+                                 Lucene60DimensionalFormat.DATA_VERSION_CURRENT,
+                                 writeState.segmentInfo.getId(),
+                                 writeState.segmentSuffix);
+      success = true;
+    } finally {
+      if (success == false) {
+        IOUtils.closeWhileHandlingException(dataOut);
+      }
+    }
+  }
+
+  /** Uses the defaults values for {@code maxPointsInLeafNode} (1024) and {@code maxMBSortInHeap} (16.0) */
+  public Lucene60DimensionalWriter(SegmentWriteState writeState) throws IOException {
+    this(writeState, BKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE, BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP);
+  }
+
+  @Override
+  public void writeField(FieldInfo fieldInfo, DimensionalReader values) throws IOException {
+
+    try (BKDWriter writer = new BKDWriter(writeState.directory,
+                                          writeState.segmentInfo.name,
+                                          fieldInfo.getDimensionCount(),
+                                          fieldInfo.getDimensionNumBytes(),
+                                          maxPointsInLeafNode,
+                                          maxMBSortInHeap)) {
+
+      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));
+    }
+  }
+
+
+  @Override
+  public void close() throws IOException {
+    if (closed == false) {
+      CodecUtil.writeFooter(dataOut);
+      dataOut.close();
+      closed = true;
+
+      String indexFileName = IndexFileNames.segmentFileName(writeState.segmentInfo.name,
+                                                            writeState.segmentSuffix,
+                                                            Lucene60DimensionalFormat.INDEX_EXTENSION);
+      // Write index file
+      try (IndexOutput indexOut = writeState.directory.createOutput(indexFileName, writeState.context)) {
+        CodecUtil.writeIndexHeader(indexOut,
+                                   Lucene60DimensionalFormat.CODEC_NAME,
+                                   Lucene60DimensionalFormat.INDEX_VERSION_CURRENT,
+                                   writeState.segmentInfo.getId(),
+                                   writeState.segmentSuffix);
+        int count = indexFPs.size();
+        indexOut.writeVInt(count);
+        for(Map.Entry<String,Long> ent : indexFPs.entrySet()) {
+          FieldInfo fieldInfo = writeState.fieldInfos.fieldInfo(ent.getKey());
+          if (fieldInfo == null) {
+            throw new IllegalStateException("wrote field=\"" + ent.getKey() + "\" but that field doesn't exist in FieldInfos");
+          }
+          indexOut.writeVInt(fieldInfo.number);
+          indexOut.writeVLong(ent.getValue());
+        }
+        CodecUtil.writeFooter(indexOut);
+      }
+    }
+  }
+}

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60FieldInfosFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60FieldInfosFormat.java?rev=1711732&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60FieldInfosFormat.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60FieldInfosFormat.java Sun Nov  1 10:19:50 2015
@@ -0,0 +1,311 @@
+package org.apache.lucene.codecs.lucene60;
+
+/*
+ * 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.Collections;
+import java.util.Map;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.FieldInfosFormat;
+import org.apache.lucene.document.DimensionalField;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.DocValuesType;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.IndexOptions;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+
+/**
+ * Lucene 6.0 Field Infos format.
+ * <p>Field names are stored in the field info file, with suffix <tt>.fnm</tt>.
+ * <p>FieldInfos (.fnm) --&gt; Header,FieldsCount, &lt;FieldName,FieldNumber,
+ * FieldBits,DocValuesBits,DocValuesGen,Attributes,DimensionCount,DimensionNumBytes&gt; <sup>FieldsCount</sup>,Footer
+ * <p>Data types:
+ * <ul>
+ *   <li>Header --&gt; {@link CodecUtil#checkIndexHeader IndexHeader}</li>
+ *   <li>FieldsCount --&gt; {@link DataOutput#writeVInt VInt}</li>
+ *   <li>FieldName --&gt; {@link DataOutput#writeString String}</li>
+ *   <li>FieldBits, IndexOptions, DocValuesBits --&gt; {@link DataOutput#writeByte Byte}</li>
+ *   <li>FieldNumber, DimensionCount, DimensionNumBytes --&gt; {@link DataOutput#writeInt VInt}</li>
+ *   <li>Attributes --&gt; {@link DataOutput#writeMapOfStrings Map&lt;String,String&gt;}</li>
+ *   <li>DocValuesGen --&gt; {@link DataOutput#writeLong(long) Int64}</li>
+ *   <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
+ * </ul>
+ * Field Descriptions:
+ * <ul>
+ *   <li>FieldsCount: the number of fields in this file.</li>
+ *   <li>FieldName: name of the field as a UTF-8 String.</li>
+ *   <li>FieldNumber: the field's number. Note that unlike previous versions of
+ *       Lucene, the fields are not numbered implicitly by their order in the
+ *       file, instead explicitly.</li>
+ *   <li>FieldBits: a byte containing field options.
+ *     <ul>
+ *       <li>The low order bit (0x1) is one for fields that have term vectors
+ *           stored, and zero for fields without term vectors.</li>
+ *       <li>If the second lowest order-bit is set (0x2), norms are omitted for the
+ *           indexed field.</li>
+ *       <li>If the third lowest-order bit is set (0x4), payloads are stored for the
+ *           indexed field.</li>
+ *     </ul>
+ *   </li>
+ *   <li>IndexOptions: a byte containing index options.
+ *     <ul>
+ *       <li>0: not indexed</li>
+ *       <li>1: indexed as DOCS_ONLY</li>
+ *       <li>2: indexed as DOCS_AND_FREQS</li>
+ *       <li>3: indexed as DOCS_AND_FREQS_AND_POSITIONS</li>
+ *       <li>4: indexed as DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS</li>
+ *     </ul>
+ *   </li>
+ *   <li>DocValuesBits: a byte containing per-document value types. The type
+ *       recorded as two four-bit integers, with the high-order bits representing
+ *       <code>norms</code> options, and the low-order bits representing 
+ *       {@code DocValues} options. Each four-bit integer can be decoded as such:
+ *     <ul>
+ *       <li>0: no DocValues for this field.</li>
+ *       <li>1: NumericDocValues. ({@link DocValuesType#NUMERIC})</li>
+ *       <li>2: BinaryDocValues. ({@code DocValuesType#BINARY})</li>
+ *       <li>3: SortedDocValues. ({@code DocValuesType#SORTED})</li>
+ *      </ul>
+ *   </li>
+ *   <li>DocValuesGen is the generation count of the field's DocValues. If this is -1,
+ *       there are no DocValues updates to that field. Anything above zero means there 
+ *       are updates stored by {@link DocValuesFormat}.</li>
+ *   <li>Attributes: a key-value map of codec-private attributes.</li>
+ *   <li>DimensionCount, DimensionNumBytes: these are non-zero only if the field is
+ *       indexed dimensionally using {@link DimensionalField}</li>
+ * </ul>
+ *
+ * @lucene.experimental
+ */
+public final class Lucene60FieldInfosFormat extends FieldInfosFormat {
+
+  /** Sole constructor. */
+  public Lucene60FieldInfosFormat() {
+  }
+  
+  @Override
+  public FieldInfos read(Directory directory, SegmentInfo segmentInfo, String segmentSuffix, IOContext context) throws IOException {
+    final String fileName = IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, EXTENSION);
+    try (ChecksumIndexInput input = directory.openChecksumInput(fileName, context)) {
+      Throwable priorE = null;
+      FieldInfo infos[] = null;
+      try {
+        CodecUtil.checkIndexHeader(input,
+                                   Lucene60FieldInfosFormat.CODEC_NAME, 
+                                   Lucene60FieldInfosFormat.FORMAT_START, 
+                                   Lucene60FieldInfosFormat.FORMAT_CURRENT,
+                                   segmentInfo.getId(), segmentSuffix);
+        
+        final int size = input.readVInt(); //read in the size
+        infos = new FieldInfo[size];
+        
+        // previous field's attribute map, we share when possible:
+        Map<String,String> lastAttributes = Collections.emptyMap();
+        
+        for (int i = 0; i < size; i++) {
+          String name = input.readString();
+          final int fieldNumber = input.readVInt();
+          if (fieldNumber < 0) {
+            throw new CorruptIndexException("invalid field number for field: " + name + ", fieldNumber=" + fieldNumber, input);
+          }
+          byte bits = input.readByte();
+          boolean storeTermVector = (bits & STORE_TERMVECTOR) != 0;
+          boolean omitNorms = (bits & OMIT_NORMS) != 0;
+          boolean storePayloads = (bits & STORE_PAYLOADS) != 0;
+
+          final IndexOptions indexOptions = getIndexOptions(input, input.readByte());
+          
+          // DV Types are packed in one byte
+          final DocValuesType docValuesType = getDocValuesType(input, input.readByte());
+          final long dvGen = input.readLong();
+          Map<String,String> attributes = input.readMapOfStrings();
+          // just use the last field's map if its the same
+          if (attributes.equals(lastAttributes)) {
+            attributes = lastAttributes;
+          }
+          lastAttributes = attributes;
+          int dimensionCount = input.readVInt();
+          int dimensionNumBytes;
+          if (dimensionCount != 0) {
+            dimensionNumBytes = input.readVInt();
+          } else {
+            dimensionNumBytes = 0;
+          }
+
+          try {
+            infos[i] = new FieldInfo(name, fieldNumber, storeTermVector, omitNorms, storePayloads, 
+                                     indexOptions, docValuesType, dvGen, attributes,
+                                     dimensionCount, dimensionNumBytes);
+            infos[i].checkConsistency();
+          } catch (IllegalStateException e) {
+            throw new CorruptIndexException("invalid fieldinfo for field: " + name + ", fieldNumber=" + fieldNumber, input, e);
+          }
+        }
+      } catch (Throwable exception) {
+        priorE = exception;
+      } finally {
+        CodecUtil.checkFooter(input, priorE);
+      }
+      return new FieldInfos(infos);
+    }
+  }
+  
+  static {
+    // We "mirror" DocValues enum values with the constants below; let's try to ensure if we add a new DocValuesType while this format is
+    // still used for writing, we remember to fix this encoding:
+    assert DocValuesType.values().length == 6;
+  }
+
+  private static byte docValuesByte(DocValuesType type) {
+    switch(type) {
+    case NONE:
+      return 0;
+    case NUMERIC:
+      return 1;
+    case BINARY:
+      return 2;
+    case SORTED:
+      return 3;
+    case SORTED_SET:
+      return 4;
+    case SORTED_NUMERIC:
+      return 5;
+    default:
+      // BUG
+      throw new AssertionError("unhandled DocValuesType: " + type);
+    }
+  }
+
+  private static DocValuesType getDocValuesType(IndexInput input, byte b) throws IOException {
+    switch(b) {
+    case 0:
+      return DocValuesType.NONE;
+    case 1:
+      return DocValuesType.NUMERIC;
+    case 2:
+      return DocValuesType.BINARY;
+    case 3:
+      return DocValuesType.SORTED;
+    case 4:
+      return DocValuesType.SORTED_SET;
+    case 5:
+      return DocValuesType.SORTED_NUMERIC;
+    default:
+      throw new CorruptIndexException("invalid docvalues byte: " + b, input);
+    }
+  }
+
+  static {
+    // We "mirror" IndexOptions enum values with the constants below; let's try to ensure if we add a new IndexOption while this format is
+    // still used for writing, we remember to fix this encoding:
+    assert IndexOptions.values().length == 5;
+  }
+
+  private static byte indexOptionsByte(IndexOptions indexOptions) {
+    switch (indexOptions) {
+    case NONE:
+      return 0;
+    case DOCS:
+      return 1;
+    case DOCS_AND_FREQS:
+      return 2;
+    case DOCS_AND_FREQS_AND_POSITIONS:
+      return 3;
+    case DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS:
+      return 4;
+    default:
+      // BUG:
+      throw new AssertionError("unhandled IndexOptions: " + indexOptions);
+    }
+  }
+  
+  private static IndexOptions getIndexOptions(IndexInput input, byte b) throws IOException {
+    switch (b) {
+    case 0:
+      return IndexOptions.NONE;
+    case 1:
+      return IndexOptions.DOCS;
+    case 2:
+      return IndexOptions.DOCS_AND_FREQS;
+    case 3:
+      return IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
+    case 4:
+      return IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS;
+    default:
+      // BUG
+      throw new CorruptIndexException("invalid IndexOptions byte: " + b, input);
+    }
+  }
+
+  @Override
+  public void write(Directory directory, SegmentInfo segmentInfo, String segmentSuffix, FieldInfos infos, IOContext context) throws IOException {
+    final String fileName = IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, EXTENSION);
+    try (IndexOutput output = directory.createOutput(fileName, context)) {
+      CodecUtil.writeIndexHeader(output, Lucene60FieldInfosFormat.CODEC_NAME, Lucene60FieldInfosFormat.FORMAT_CURRENT, segmentInfo.getId(), segmentSuffix);
+      output.writeVInt(infos.size());
+      for (FieldInfo fi : infos) {
+        fi.checkConsistency();
+
+        output.writeString(fi.name);
+        output.writeVInt(fi.number);
+
+        byte bits = 0x0;
+        if (fi.hasVectors()) bits |= STORE_TERMVECTOR;
+        if (fi.omitsNorms()) bits |= OMIT_NORMS;
+        if (fi.hasPayloads()) bits |= STORE_PAYLOADS;
+        output.writeByte(bits);
+
+        output.writeByte(indexOptionsByte(fi.getIndexOptions()));
+
+        // pack the DV type and hasNorms in one byte
+        output.writeByte(docValuesByte(fi.getDocValuesType()));
+        output.writeLong(fi.getDocValuesGen());
+        output.writeMapOfStrings(fi.attributes());
+        int dimensionCount = fi.getDimensionCount();
+        output.writeVInt(dimensionCount);
+        if (dimensionCount != 0) {
+          output.writeVInt(fi.getDimensionNumBytes());
+        }
+      }
+      CodecUtil.writeFooter(output);
+    }
+  }
+  
+  /** Extension of field infos */
+  static final String EXTENSION = "fnm";
+  
+  // Codec header
+  static final String CODEC_NAME = "Lucene60FieldInfos";
+  static final int FORMAT_START = 0;
+  static final int FORMAT_CURRENT = FORMAT_START;
+  
+  // Field flags
+  static final byte STORE_TERMVECTOR = 0x1;
+  static final byte OMIT_NORMS = 0x2;
+  static final byte STORE_PAYLOADS = 0x4;
+}

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene60/package-info.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene60/package-info.java?rev=1711732&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene60/package-info.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene60/package-info.java Sun Nov  1 10:19:50 2015
@@ -0,0 +1,412 @@
+/*
+ * 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.
+ */
+
+/**
+ * Lucene 6.0 file format.
+ * 
+ * <h1>Apache Lucene - Index File Formats</h1>
+ * <div>
+ * <ul>
+ * <li><a href="#Introduction">Introduction</a></li>
+ * <li><a href="#Definitions">Definitions</a>
+ *   <ul>
+ *   <li><a href="#Inverted_Indexing">Inverted Indexing</a></li>
+ *   <li><a href="#Types_of_Fields">Types of Fields</a></li>
+ *   <li><a href="#Segments">Segments</a></li>
+ *   <li><a href="#Document_Numbers">Document Numbers</a></li>
+ *   </ul>
+ * </li>
+ * <li><a href="#Overview">Index Structure Overview</a></li>
+ * <li><a href="#File_Naming">File Naming</a></li>
+ * <li><a href="#file-names">Summary of File Extensions</a>
+ *   <ul>
+ *   <li><a href="#Lock_File">Lock File</a></li>
+ *   <li><a href="#History">History</a></li>
+ *   <li><a href="#Limitations">Limitations</a></li>
+ *   </ul>
+ * </li>
+ * </ul>
+ * </div>
+ * <a name="Introduction"></a>
+ * <h2>Introduction</h2>
+ * <div>
+ * <p>This document defines the index file formats used in this version of Lucene.
+ * If you are using a different version of Lucene, please consult the copy of
+ * <code>docs/</code> that was distributed with
+ * the version you are using.</p>
+ * <p>Apache Lucene is written in Java, but several efforts are underway to write
+ * <a href="http://wiki.apache.org/lucene-java/LuceneImplementations">versions of
+ * Lucene in other programming languages</a>. If these versions are to remain
+ * compatible with Apache Lucene, then a language-independent definition of the
+ * Lucene index format is required. This document thus attempts to provide a
+ * complete and independent definition of the Apache Lucene file formats.</p>
+ * <p>As Lucene evolves, this document should evolve. Versions of Lucene in
+ * different programming languages should endeavor to agree on file formats, and
+ * generate new versions of this document.</p>
+ * </div>
+ * <a name="Definitions"></a>
+ * <h2>Definitions</h2>
+ * <div>
+ * <p>The fundamental concepts in Lucene are index, document, field and term.</p>
+ * <p>An index contains a sequence of documents.</p>
+ * <ul>
+ * <li>A document is a sequence of fields.</li>
+ * <li>A field is a named sequence of terms.</li>
+ * <li>A term is a sequence of bytes.</li>
+ * </ul>
+ * <p>The same sequence of bytes in two different fields is considered a different 
+ * term. Thus terms are represented as a pair: the string naming the field, and the
+ * bytes within the field.</p>
+ * <a name="Inverted_Indexing"></a>
+ * <h3>Inverted Indexing</h3>
+ * <p>The index stores statistics about terms in order to make term-based search
+ * more efficient. Lucene's index falls into the family of indexes known as an
+ * <i>inverted index.</i> This is because it can list, for a term, the documents
+ * that contain it. This is the inverse of the natural relationship, in which
+ * documents list terms.</p>
+ * <a name="Types_of_Fields"></a>
+ * <h3>Types of Fields</h3>
+ * <p>In Lucene, fields may be <i>stored</i>, in which case their text is stored
+ * in the index literally, in a non-inverted manner. Fields that are inverted are
+ * called <i>indexed</i>. A field may be both stored and indexed.</p>
+ * <p>The text of a field may be <i>tokenized</i> into terms to be indexed, or the
+ * text of a field may be used literally as a term to be indexed. Most fields are
+ * tokenized, but sometimes it is useful for certain identifier fields to be
+ * indexed literally.</p>
+ * <p>See the {@link org.apache.lucene.document.Field Field}
+ * java docs for more information on Fields.</p>
+ * <a name="Segments"></a>
+ * <h3>Segments</h3>
+ * <p>Lucene indexes may be composed of multiple sub-indexes, or <i>segments</i>.
+ * Each segment is a fully independent index, which could be searched separately.
+ * Indexes evolve by:</p>
+ * <ol>
+ * <li>Creating new segments for newly added documents.</li>
+ * <li>Merging existing segments.</li>
+ * </ol>
+ * <p>Searches may involve multiple segments and/or multiple indexes, each index
+ * potentially composed of a set of segments.</p>
+ * <a name="Document_Numbers"></a>
+ * <h3>Document Numbers</h3>
+ * <p>Internally, Lucene refers to documents by an integer <i>document number</i>.
+ * The first document added to an index is numbered zero, and each subsequent
+ * document added gets a number one greater than the previous.</p>
+ * <p>Note that a document's number may change, so caution should be taken when
+ * storing these numbers outside of Lucene. In particular, numbers may change in
+ * the following situations:</p>
+ * <ul>
+ * <li>
+ * <p>The numbers stored in each segment are unique only within the segment, and
+ * must be converted before they can be used in a larger context. The standard
+ * technique is to allocate each segment a range of values, based on the range of
+ * numbers used in that segment. To convert a document number from a segment to an
+ * external value, the segment's <i>base</i> document number is added. To convert
+ * an external value back to a segment-specific value, the segment is identified
+ * by the range that the external value is in, and the segment's base value is
+ * subtracted. For example two five document segments might be combined, so that
+ * the first segment has a base value of zero, and the second of five. Document
+ * three from the second segment would have an external value of eight.</p>
+ * </li>
+ * <li>
+ * <p>When documents are deleted, gaps are created in the numbering. These are
+ * eventually removed as the index evolves through merging. Deleted documents are
+ * dropped when segments are merged. A freshly-merged segment thus has no gaps in
+ * its numbering.</p>
+ * </li>
+ * </ul>
+ * </div>
+ * <a name="Overview"></a>
+ * <h2>Index Structure Overview</h2>
+ * <div>
+ * <p>Each segment index maintains the following:</p>
+ * <ul>
+ * <li>
+ * {@link org.apache.lucene.codecs.lucene50.Lucene50SegmentInfoFormat Segment info}.
+ *    This contains metadata about a segment, such as the number of documents,
+ *    what files it uses, 
+ * </li>
+ * <li>
+ * {@link org.apache.lucene.codecs.lucene50.Lucene50FieldInfosFormat Field names}. 
+ *    This contains the set of field names used in the index.
+ * </li>
+ * <li>
+ * {@link org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat Stored Field values}. 
+ * This contains, for each document, a list of attribute-value pairs, where the attributes 
+ * are field names. These are used to store auxiliary information about the document, such as 
+ * its title, url, or an identifier to access a database. The set of stored fields are what is 
+ * returned for each hit when searching. This is keyed by document number.
+ * </li>
+ * <li>
+ * {@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term dictionary}. 
+ * A dictionary containing all of the terms used in all of the
+ * indexed fields of all of the documents. The dictionary also contains the number
+ * of documents which contain the term, and pointers to the term's frequency and
+ * proximity data.
+ * </li>
+ * <li>
+ * {@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Frequency data}. 
+ * For each term in the dictionary, the numbers of all the
+ * documents that contain that term, and the frequency of the term in that
+ * document, unless frequencies are omitted (IndexOptions.DOCS_ONLY)
+ * </li>
+ * <li>
+ * {@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Proximity data}. 
+ * For each term in the dictionary, the positions that the
+ * term occurs in each document. Note that this will not exist if all fields in
+ * all documents omit position data.
+ * </li>
+ * <li>
+ * {@link org.apache.lucene.codecs.lucene53.Lucene53NormsFormat Normalization factors}. 
+ * For each field in each document, a value is stored
+ * that is multiplied into the score for hits on that field.
+ * </li>
+ * <li>
+ * {@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vectors}. 
+ * For each field in each document, the term vector (sometimes
+ * called document vector) may be stored. A term vector consists of term text and
+ * term frequency. To add Term Vectors to your index see the 
+ * {@link org.apache.lucene.document.Field Field} constructors
+ * </li>
+ * <li>
+ * {@link org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat Per-document values}. 
+ * Like stored values, these are also keyed by document
+ * number, but are generally intended to be loaded into main memory for fast
+ * access. Whereas stored values are generally intended for summary results from
+ * searches, per-document values are useful for things like scoring factors.
+ * </li>
+ * <li>
+ * {@link org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat Live documents}. 
+ * An optional file indicating which documents are live.
+ * </li>
+ * <li>
+ * {@link org.apache.lucene.codecs.lucene60.Lucene60DimensionalFormat Dimensional values}. 
+ * Optional pair of files, recording dimesionally indexed fields, to enable fast
+ * numeric range filtering and large numeric values like BigInteger and BigDecimal (1D)
+ * and geo shape intersection (2D, 3D).
+ * </li>
+ * </ul>
+ * <p>Details on each of these are provided in their linked pages.</p>
+ * </div>
+ * <a name="File_Naming"></a>
+ * <h2>File Naming</h2>
+ * <div>
+ * <p>All files belonging to a segment have the same name with varying extensions.
+ * The extensions correspond to the different file formats described below. When
+ * using the Compound File format (default in 1.4 and greater) these files (except
+ * for the Segment info file, the Lock file, and Deleted documents file) are collapsed 
+ * into a single .cfs file (see below for details)</p>
+ * <p>Typically, all segments in an index are stored in a single directory,
+ * although this is not required.</p>
+ * <p>As of version 2.1 (lock-less commits), file names are never re-used.
+ * That is, when any file is saved
+ * to the Directory it is given a never before used filename. This is achieved
+ * using a simple generations approach. For example, the first segments file is
+ * segments_1, then segments_2, etc. The generation is a sequential long integer
+ * represented in alpha-numeric (base 36) form.</p>
+ * </div>
+ * <a name="file-names"></a>
+ * <h2>Summary of File Extensions</h2>
+ * <div>
+ * <p>The following table summarizes the names and extensions of the files in
+ * Lucene:</p>
+ * <table cellspacing="1" cellpadding="4" summary="lucene filenames by extension">
+ * <tr>
+ * <th>Name</th>
+ * <th>Extension</th>
+ * <th>Brief Description</th>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.index.SegmentInfos Segments File}</td>
+ * <td>segments_N</td>
+ * <td>Stores information about a commit point</td>
+ * </tr>
+ * <tr>
+ * <td><a href="#Lock_File">Lock File</a></td>
+ * <td>write.lock</td>
+ * <td>The Write lock prevents multiple IndexWriters from writing to the same
+ * file.</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50SegmentInfoFormat Segment Info}</td>
+ * <td>.si</td>
+ * <td>Stores metadata about a segment</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50CompoundFormat Compound File}</td>
+ * <td>.cfs, .cfe</td>
+ * <td>An optional "virtual" file consisting of all the other index files for
+ * systems that frequently run out of file handles.</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50FieldInfosFormat Fields}</td>
+ * <td>.fnm</td>
+ * <td>Stores information about the fields</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat Field Index}</td>
+ * <td>.fdx</td>
+ * <td>Contains pointers to field data</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat Field Data}</td>
+ * <td>.fdt</td>
+ * <td>The stored fields for documents</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Dictionary}</td>
+ * <td>.tim</td>
+ * <td>The term dictionary, stores term info</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Index}</td>
+ * <td>.tip</td>
+ * <td>The index into the Term Dictionary</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Frequencies}</td>
+ * <td>.doc</td>
+ * <td>Contains the list of docs which contain each term along with frequency</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Positions}</td>
+ * <td>.pos</td>
+ * <td>Stores position information about where a term occurs in the index</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Payloads}</td>
+ * <td>.pay</td>
+ * <td>Stores additional per-position metadata information such as character offsets and user payloads</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene53.Lucene53NormsFormat Norms}</td>
+ * <td>.nvd, .nvm</td>
+ * <td>Encodes length and boost factors for docs and fields</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat Per-Document Values}</td>
+ * <td>.dvd, .dvm</td>
+ * <td>Encodes additional scoring factors or other per-document information.</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vector Index}</td>
+ * <td>.tvx</td>
+ * <td>Stores offset into the document data file</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vector Documents}</td>
+ * <td>.tvd</td>
+ * <td>Contains information about each document that has term vectors</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vector Fields}</td>
+ * <td>.tvf</td>
+ * <td>The field level info about term vectors</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat Live Documents}</td>
+ * <td>.liv</td>
+ * <td>Info about what files are live</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene60.Lucene60DimensionalFormat Dimensional values}</td>
+ * <td>.dii, .dim</td>
+ * <td>Holds dimensionally indexed fields, if any</td>
+ * </tr>
+ * </table>
+ * </div>
+ * <a name="Lock_File"></a>
+ * <h2>Lock File</h2>
+ * The write lock, which is stored in the index directory by default, is named
+ * "write.lock". If the lock directory is different from the index directory then
+ * the write lock will be named "XXXX-write.lock" where XXXX is a unique prefix
+ * derived from the full path to the index directory. When this file is present, a
+ * writer is currently modifying the index (adding or removing documents). This
+ * lock file ensures that only one writer is modifying the index at a time.
+ * <a name="History"></a>
+ * <h2>History</h2>
+ * <p>Compatibility notes are provided in this document, describing how file
+ * formats have changed from prior versions:</p>
+ * <ul>
+ * <li>In version 2.1, the file format was changed to allow lock-less commits (ie,
+ * no more commit lock). The change is fully backwards compatible: you can open a
+ * pre-2.1 index for searching or adding/deleting of docs. When the new segments
+ * file is saved (committed), it will be written in the new file format (meaning
+ * no specific "upgrade" process is needed). But note that once a commit has
+ * occurred, pre-2.1 Lucene will not be able to read the index.</li>
+ * <li>In version 2.3, the file format was changed to allow segments to share a
+ * single set of doc store (vectors &amp; stored fields) files. This allows for
+ * faster indexing in certain cases. The change is fully backwards compatible (in
+ * the same way as the lock-less commits change in 2.1).</li>
+ * <li>In version 2.4, Strings are now written as true UTF-8 byte sequence, not
+ * Java's modified UTF-8. See <a href="http://issues.apache.org/jira/browse/LUCENE-510">
+ * LUCENE-510</a> for details.</li>
+ * <li>In version 2.9, an optional opaque Map&lt;String,String&gt; CommitUserData
+ * may be passed to IndexWriter's commit methods (and later retrieved), which is
+ * recorded in the segments_N file. See <a href="http://issues.apache.org/jira/browse/LUCENE-1382">
+ * LUCENE-1382</a> for details. Also,
+ * diagnostics were added to each segment written recording details about why it
+ * was written (due to flush, merge; which OS/JRE was used; etc.). See issue
+ * <a href="http://issues.apache.org/jira/browse/LUCENE-1654">LUCENE-1654</a> for details.</li>
+ * <li>In version 3.0, compressed fields are no longer written to the index (they
+ * can still be read, but on merge the new segment will write them, uncompressed).
+ * See issue <a href="http://issues.apache.org/jira/browse/LUCENE-1960">LUCENE-1960</a> 
+ * for details.</li>
+ * <li>In version 3.1, segments records the code version that created them. See
+ * <a href="http://issues.apache.org/jira/browse/LUCENE-2720">LUCENE-2720</a> for details. 
+ * Additionally segments track explicitly whether or not they have term vectors. 
+ * See <a href="http://issues.apache.org/jira/browse/LUCENE-2811">LUCENE-2811</a> 
+ * for details.</li>
+ * <li>In version 3.2, numeric fields are written as natively to stored fields
+ * file, previously they were stored in text format only.</li>
+ * <li>In version 3.4, fields can omit position data while still indexing term
+ * frequencies.</li>
+ * <li>In version 4.0, the format of the inverted index became extensible via
+ * the {@link org.apache.lucene.codecs.Codec Codec} api. Fast per-document storage
+ * ({@code DocValues}) was introduced. Normalization factors need no longer be a 
+ * single byte, they can be any {@link org.apache.lucene.index.NumericDocValues NumericDocValues}. 
+ * Terms need not be unicode strings, they can be any byte sequence. Term offsets 
+ * can optionally be indexed into the postings lists. Payloads can be stored in the 
+ * term vectors.</li>
+ * <li>In version 4.1, the format of the postings list changed to use either
+ * of FOR compression or variable-byte encoding, depending upon the frequency
+ * of the term. Terms appearing only once were changed to inline directly into
+ * the term dictionary. Stored fields are compressed by default. </li>
+ * <li>In version 4.2, term vectors are compressed by default. DocValues has 
+ * a new multi-valued type (SortedSet), that can be used for faceting/grouping/joining
+ * on multi-valued fields.</li>
+ * <li>In version 4.5, DocValues were extended to explicitly represent missing values.</li>
+ * <li>In version 4.6, FieldInfos were extended to support per-field DocValues generation, to 
+ * allow updating NumericDocValues fields.</li>
+ * <li>In version 4.8, checksum footers were added to the end of each index file 
+ * for improved data integrity. Specifically, the last 8 bytes of every index file
+ * contain the zlib-crc32 checksum of the file.</li>
+ * <li>In version 4.9, DocValues has a new multi-valued numeric type (SortedNumeric)
+ * that is suitable for faceting/sorting/analytics.
+ * </li>
+ * </ul>
+ * <a name="Limitations"></a>
+ * <h2>Limitations</h2>
+ * <div>
+ * <p>Lucene uses a Java <code>int</code> to refer to
+ * document numbers, and the index file format uses an <code>Int32</code>
+ * on-disk to store document numbers. This is a limitation
+ * of both the index file format and the current implementation. Eventually these
+ * should be replaced with either <code>UInt64</code> values, or
+ * better yet, {@link org.apache.lucene.store.DataOutput#writeVInt VInt} values which have no limit.</p>
+ * </div>
+ */
+package org.apache.lucene.codecs.lucene60;

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java?rev=1711732&r1=1711731&r2=1711732&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java Sun Nov  1 10:19:50 2015
@@ -33,6 +33,7 @@ import java.util.Locale;
 import java.util.Map;
 
 import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.DimensionalReader;
 import org.apache.lucene.codecs.DocValuesProducer;
 import org.apache.lucene.codecs.NormsProducer;
 import org.apache.lucene.codecs.PostingsFormat;
@@ -212,6 +213,9 @@ public class CheckIndex implements Close
       
       /** Status for testing of DocValues (null if DocValues could not be tested). */
       public DocValuesStatus docValuesStatus;
+
+      /** Status for testing of DimensionalValues (null if DimensionalValues could not be tested). */
+      public DimensionalValuesStatus dimensionalValuesStatus;
     }
     
     /**
@@ -351,6 +355,24 @@ public class CheckIndex implements Close
       /** Exception thrown during doc values test (null on success) */
       public Throwable error = null;
     }
+
+    /**
+     * Status from testing DimensionalValues
+     */
+    public static final class DimensionalValuesStatus {
+
+      DimensionalValuesStatus() {
+      }
+
+      /** Total number of dimensional values points tested. */
+      public long totalValuePoints;
+
+      /** Total number of fields with dimensional values. */
+      public int totalValueFields;
+      
+      /** Exception thrown during doc values test (null on success) */
+      public Throwable error = null;
+    }
   }
 
   /** Create a new CheckIndex on the directory. */
@@ -704,6 +726,9 @@ public class CheckIndex implements Close
           // Test Docvalues
           segInfoStat.docValuesStatus = testDocValues(reader, infoStream, failFast);
 
+          // Test DimensionalValues
+          segInfoStat.dimensionalValuesStatus = testDimensionalValues(reader, infoStream, failFast);
+
           // Rethrow the first exception we encountered
           //  This will cause stats for failed segments to be incremented properly
           if (segInfoStat.liveDocStatus.error != null) {
@@ -1659,7 +1684,102 @@ public class CheckIndex implements Close
 
     return status;
   }
-  
+
+  /**
+   * Test the dimensional values index.
+   * @lucene.experimental
+   */
+  public static Status.DimensionalValuesStatus testDimensionalValues(CodecReader reader, PrintStream infoStream, boolean failFast) throws IOException {
+    FieldInfos fieldInfos = reader.getFieldInfos();
+    Status.DimensionalValuesStatus status = new Status.DimensionalValuesStatus();
+    try {
+      if (fieldInfos.hasDimensionalValues()) {
+        DimensionalReader values = reader.getDimensionalReader();
+        if (values == null) {
+          throw new RuntimeException("there are fields with dimensional values, but reader.getDimensionalRader() is null");
+        }
+        for (FieldInfo fieldInfo : fieldInfos) {
+          if (fieldInfo.getDimensionCount() > 0) {
+            status.totalValueFields++;
+            int dimCount = fieldInfo.getDimensionCount();
+            int bytesPerDim = fieldInfo.getDimensionNumBytes();
+            byte[] lastMinPackedValue = new byte[dimCount*bytesPerDim];
+            BytesRef lastMinPacked = new BytesRef(lastMinPackedValue);
+            byte[] lastMaxPackedValue = new byte[dimCount*bytesPerDim];
+            BytesRef lastMaxPacked = new BytesRef(lastMaxPackedValue);
+            BytesRef scratch = new BytesRef();
+            lastMaxPacked.length = bytesPerDim;
+            lastMinPacked.length = bytesPerDim;
+            scratch.length = bytesPerDim;
+            values.intersect(fieldInfo.name,
+                             new DimensionalValues.IntersectVisitor() {
+                               @Override
+                               public void visit(int docID) {
+                                 throw new RuntimeException("codec called IntersectVisitor.visit without a packed value for docID=" + docID);
+                               }
+
+                               @Override
+                               public void visit(int docID, byte[] packedValue) {
+                                 checkPackedValue("packed value", packedValue, docID);
+                                 scratch.bytes = packedValue;
+
+                                 for(int dim=0;dim<dimCount;dim++) {
+                                   lastMaxPacked.offset = bytesPerDim * dim;
+                                   lastMinPacked.offset = bytesPerDim * dim;
+                                   scratch.offset = bytesPerDim * dim;
+
+                                   if (scratch.compareTo(lastMinPacked) < 0) {
+                                     // This doc's point, in this dimension, is lower than the minimum value of the last cell checked:
+                                     throw new RuntimeException("packed value " + Arrays.toString(packedValue) + " for docID=" + docID + " is out-of-bounds of the last cell min=" + Arrays.toString(lastMinPackedValue) + " max=" + Arrays.toString(lastMaxPackedValue) + " dim=" + dim);
+                                   }
+                                   if (scratch.compareTo(lastMaxPacked) > 0) {
+                                     // This doc's point, in this dimension, is greater than the maximum value of the last cell checked:
+                                     throw new RuntimeException("packed value " + Arrays.toString(packedValue) + " for docID=" + docID + " is out-of-bounds of the last cell min=" + Arrays.toString(lastMinPackedValue) + " max=" + Arrays.toString(lastMaxPackedValue) + " dim=" + dim);
+                                   }
+                                 }
+
+                                 status.totalValuePoints++;
+                               }
+
+                               @Override
+                               public DimensionalValues.Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
+                                 checkPackedValue("min packed value", minPackedValue, -1);
+                                 System.arraycopy(minPackedValue, 0, lastMinPackedValue, 0, minPackedValue.length);
+                                 checkPackedValue("max packed value", maxPackedValue, -1);
+                                 System.arraycopy(maxPackedValue, 0, lastMaxPackedValue, 0, maxPackedValue.length);
+
+                                 // We always pretend the query shape is so complex that it crosses every cell, so
+                                 // that packedValue is passed for every document
+                                 return DimensionalValues.Relation.QUERY_CROSSES_CELL;
+                               }
+
+                               private void checkPackedValue(String desc, byte[] packedValue, int docID) {
+                                 if (packedValue == null) {
+                                   throw new RuntimeException(desc + " is null for docID=" + docID);
+                                 }
+
+                                 if (packedValue.length != dimCount * bytesPerDim) {
+                                   throw new RuntimeException(desc + " has incorrect length=" + packedValue.length + " vs expected=" + (dimCount * bytesPerDim) + " for docID=" + docID);
+                                 }
+                               }
+                             });
+          }
+        }
+      }
+    } catch (Throwable e) {
+      if (failFast) {
+        IOUtils.reThrow(e);
+      }
+      msg(infoStream, "ERROR: " + e);
+      status.error = e;
+      if (infoStream != null) {
+        e.printStackTrace(infoStream);
+      }
+    }
+
+    return status;
+  }
+
   /**
    * Test stored fields.
    * @lucene.experimental

Modified: 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=1711732&r1=1711731&r2=1711732&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DimensionalValues.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DimensionalValues.java Sun Nov  1 10:19:50 2015
@@ -35,7 +35,8 @@ public abstract class DimensionalValues
     /** 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};
+    QUERY_CROSSES_CELL
+  };
 
   /** We recurse the BKD tree, using a provided instance of this to guide the recursion.
    *
@@ -54,6 +55,8 @@ public abstract class DimensionalValues
     Relation compare(byte[] minPackedValue, byte[] maxPackedValue);
   }
 
-  /** Finds all documents and points matching the provided visitor */
+  /** Finds all documents and points matching the provided visitor.
+   *  This method does not enforce live docs, so it's up to the caller
+   *  to test whether each document is deleted, if necessary. */
   public abstract void intersect(String fieldName, IntersectVisitor visitor) throws IOException;
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java?rev=1711732&r1=1711731&r2=1711732&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java Sun Nov  1 10:19:50 2015
@@ -82,7 +82,7 @@ public class FieldInfos implements Itera
       hasNorms |= info.hasNorms();
       hasDocValues |= info.getDocValuesType() != DocValuesType.NONE;
       hasPayloads |= info.hasPayloads();
-      hasDimensionalValues |= info.getDimensionCount() != 0;
+      hasDimensionalValues |= (info.getDimensionCount() != 0);
     }
     
     this.hasVectors = hasVectors;

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiDimensionalValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiDimensionalValues.java?rev=1711732&r1=1711731&r2=1711732&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiDimensionalValues.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiDimensionalValues.java Sun Nov  1 10:19:50 2015
@@ -51,6 +51,10 @@ class MultiDimensionalValues extends Dim
       }
     }
 
+    if (values.isEmpty()) {
+      return null;
+    }
+
     return new MultiDimensionalValues(values, docBases);
   }
 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SlowCodecReaderWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SlowCodecReaderWrapper.java?rev=1711732&r1=1711731&r2=1711732&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SlowCodecReaderWrapper.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SlowCodecReaderWrapper.java Sun Nov  1 10:19:50 2015
@@ -46,7 +46,7 @@ public final class SlowCodecReaderWrappe
    */
   public static CodecReader wrap(final LeafReader reader) throws IOException {
     if (reader instanceof CodecReader) {
-      return (CodecReader)reader;
+      return (CodecReader) reader;
     } else {
       // simulate it slowly, over the leafReader api:
       reader.checkIntegrity();
@@ -130,6 +130,9 @@ public final class SlowCodecReaderWrappe
   }
 
   private static DimensionalReader dimensionalValuesToReader(DimensionalValues values) {
+    if (values == null) {
+      return null;
+    }
     return new DimensionalReader() {
       @Override
       public void intersect(String fieldName, IntersectVisitor visitor) throws IOException {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java?rev=1711732&r1=1711731&r2=1711732&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java Sun Nov  1 10:19:50 2015
@@ -79,8 +79,10 @@ public class BKDWriter implements Closea
   /** How many bytes each docs takes in the fixed-width offline format */
   private final int bytesPerDoc;
 
+  /** Default maximum number of point in each leaf block */
   public static final int DEFAULT_MAX_POINTS_IN_LEAF_NODE = 1024;
 
+  /** Default maximum heap to use, before spilling to (slower) disk */
   public static final float DEFAULT_MAX_MB_SORT_IN_HEAP = 16.0f;
 
   /** Maximum number of dimensions */
@@ -370,6 +372,11 @@ public class BKDWriter implements Closea
 
     // TODO: specialize the 1D case?  it's much faster at indexing time (no partitioning on recruse...)
 
+    // Catch user silliness:
+    if (heapPointWriter == null && tempInput == null) {
+      throw new IllegalStateException("already finished");
+    }
+
     if (offlinePointWriter != null) {
       offlinePointWriter.close();
     }

Modified: lucene/dev/trunk/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec?rev=1711732&r1=1711731&r2=1711732&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec (original)
+++ lucene/dev/trunk/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec Sun Nov  1 10:19:50 2015
@@ -13,4 +13,4 @@
 #  See the License for the specific language governing permissions and
 #  limitations under the License.
 
-org.apache.lucene.codecs.lucene54.Lucene54Codec
+org.apache.lucene.codecs.lucene60.Lucene60Codec

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50StoredFieldsFormatHighCompression.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50StoredFieldsFormatHighCompression.java?rev=1711732&r1=1711731&r2=1711732&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50StoredFieldsFormatHighCompression.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50StoredFieldsFormatHighCompression.java Sun Nov  1 10:19:50 2015
@@ -19,7 +19,7 @@ package org.apache.lucene.codecs.lucene5
 
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
-import org.apache.lucene.codecs.lucene54.Lucene54Codec;
+import org.apache.lucene.codecs.lucene60.Lucene60Codec;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.StoredField;
 import org.apache.lucene.index.BaseStoredFieldsFormatTestCase;
@@ -34,7 +34,7 @@ import com.carrotsearch.randomizedtestin
 public class TestLucene50StoredFieldsFormatHighCompression extends BaseStoredFieldsFormatTestCase {
   @Override
   protected Codec getCodec() {
-    return new Lucene54Codec(Mode.BEST_COMPRESSION);
+    return new Lucene60Codec(Mode.BEST_COMPRESSION);
   }
   
   /**
@@ -45,7 +45,7 @@ public class TestLucene50StoredFieldsFor
     Directory dir = newDirectory();
     for (int i = 0; i < 10; i++) {
       IndexWriterConfig iwc = newIndexWriterConfig();
-      iwc.setCodec(new Lucene54Codec(RandomPicks.randomFrom(random(), Mode.values())));
+      iwc.setCodec(new Lucene60Codec(RandomPicks.randomFrom(random(), Mode.values())));
       IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig());
       Document doc = new Document();
       doc.add(new StoredField("field1", "value1"));
@@ -72,7 +72,7 @@ public class TestLucene50StoredFieldsFor
   
   public void testInvalidOptions() throws Exception {
     try {
-      new Lucene54Codec(null);
+      new Lucene60Codec(null);
       fail("didn't hit exception");
     } catch (NullPointerException expected) {
       // expected

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene53/TestLucene53NormsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene53/TestLucene53NormsFormat.java?rev=1711732&r1=1711731&r2=1711732&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene53/TestLucene53NormsFormat.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene53/TestLucene53NormsFormat.java Sun Nov  1 10:19:50 2015
@@ -18,18 +18,17 @@ package org.apache.lucene.codecs.lucene5
  */
 
 import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.lucene54.Lucene54Codec;
+import org.apache.lucene.codecs.lucene60.Lucene60Codec;
 import org.apache.lucene.index.BaseNormsFormatTestCase;
 
 /**
  * Tests Lucene53NormsFormat
  */
 public class TestLucene53NormsFormat extends BaseNormsFormatTestCase {
-  private final Codec codec = new Lucene54Codec();
+  private final Codec codec = new Lucene60Codec();
   
   @Override
   protected Codec getCodec() {
     return codec;
   }
-
 }

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/Test2BTerms.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/Test2BTerms.java?rev=1711732&r1=1711731&r2=1711732&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/Test2BTerms.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/Test2BTerms.java Sun Nov  1 10:19:50 2015
@@ -53,7 +53,7 @@ import com.carrotsearch.randomizedtestin
 // disk (but, should run successfully).  Best to run w/
 // -Dtests.codec=<current codec>, and w/ plenty of RAM, eg:
 //
-//   ant test -Dtests.monster=true -Dtests.heapsize=8g -Dtests.codec=Lucene53 -Dtestcase=Test2BTerms
+//   ant test -Dtests.monster=true -Dtests.heapsize=8g -Dtests.codec=Lucene60 -Dtestcase=Test2BTerms
 //
 @SuppressCodecs({ "SimpleText", "Memory", "Direct" })
 @Monster("very slow, use 5g minimum heap")