You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2014/08/14 17:30:45 UTC

svn commit: r1617975 [1/3] - in /lucene/dev/trunk: lucene/ lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/ lucene/core/src/java/org/apache/lucene/codecs/ lucene/core/src/ja...

Author: rmuir
Date: Thu Aug 14 15:30:44 2014
New Revision: 1617975

URL: http://svn.apache.org/r1617975
Log:
LUCENE-5882: Add 4.10 docvaluesformat

Added:
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene410/
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene410/Lucene410Codec.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesConsumer.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesFormat.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesProducer.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene410/package.html   (with props)
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene410/
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene410/TestLucene410DocValuesFormat.java   (with props)
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene49/
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene49/Lucene49RWCodec.java   (with props)
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene49/Lucene49RWDocValuesFormat.java   (with props)
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene49/package.html   (with props)
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/build.xml
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/FixedGapTermsIndexWriter.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/lucene49/Lucene49Codec.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesConsumer.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesFormat.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49NormsFormat.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/package.html
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/StringHelper.java
    lucene/dev/trunk/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
    lucene/dev/trunk/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/TestExternalCodecs.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene49/TestLucene49DocValuesFormat.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene49/TestLucene49NormsFormat.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldDocValuesFormat.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat2.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveChecksumFooter.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReaderReopen.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDuelingCodecs.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterThreadsToSegments.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestOrdinalMap.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestTieredMergePolicy.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestSortedSetSelector.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestNamedSPILoader.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestStringHelper.java
    lucene/dev/trunk/lucene/misc/src/test/org/apache/lucene/uninverting/TestFieldCacheVsDocValues.java
    lucene/dev/trunk/lucene/spatial/src/test/org/apache/lucene/spatial/SpatialTestCase.java
    lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.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/codecs/asserting/AssertingDocValuesFormat.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardCodec.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/CompressingCodec.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BaseStoredFieldsFormatTestCase.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.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/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java
    lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/schema_codec.xml
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestCodecSupport.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1617975&r1=1617974&r2=1617975&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Thu Aug 14 15:30:44 2014
@@ -217,6 +217,9 @@ Optimizations
 * LUCENE-5884: Optimize FST.ramBytesUsed.  (Adrien Grand, Robert Muir,
   Mike McCandless)
 
+* LUCENE-5882: Add Lucene410DocValuesFormat, with faster term lookups
+  for SORTED/SORTED_SET fields.  (Robert Muir)
+
 Bug Fixes
 
 * LUCENE-5796: Fixes the Scorer.getChildren() method for two combinations 

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=1617975&r1=1617974&r2=1617975&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 Thu Aug 14 15:30:44 2014
@@ -21,7 +21,7 @@ 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.lucene49.Lucene49Codec;
+import org.apache.lucene.codecs.lucene410.Lucene410Codec;
 import org.apache.lucene.index.ConcurrentMergeScheduler;
 import org.apache.lucene.index.IndexCommit;
 import org.apache.lucene.index.IndexDeletionPolicy;
@@ -140,7 +140,7 @@ public class CreateIndexTask extends Per
     if (defaultCodec == null && postingsFormat != null) {
       try {
         final PostingsFormat postingsFormatChosen = PostingsFormat.forName(postingsFormat);
-        iwConf.setCodec(new Lucene49Codec(){
+        iwConf.setCodec(new Lucene410Codec(){
           @Override
           public PostingsFormat getPostingsFormatForField(String field) {
             return postingsFormatChosen;

Modified: lucene/dev/trunk/lucene/build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/build.xml?rev=1617975&r1=1617974&r2=1617975&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/build.xml (original)
+++ lucene/dev/trunk/lucene/build.xml Thu Aug 14 15:30:44 2014
@@ -288,7 +288,7 @@
             <classpath>
               <path refid="groovy.classpath"/>
             </classpath><![CDATA[
-            //                   private static Codec defaultCodec   =   Codec    .   forName    (   "Lucene49"   )   ;
+            //                   private static Codec defaultCodec   =   Codec    .   forName    (   "Lucene410"   )   ;
             defaultCodecMatcher = self.getToken() =~ /defaultCodec\s*=\s*Codec\s*\.\s*forName\s*\(\s*"([^"]+)"\s*\)\s*;/
             self.setToken(defaultCodecMatcher[0][1].toLowerCase());
             ]]>

Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/FixedGapTermsIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/FixedGapTermsIndexWriter.java?rev=1617975&r1=1617974&r2=1617975&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/FixedGapTermsIndexWriter.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/FixedGapTermsIndexWriter.java Thu Aug 14 15:30:44 2014
@@ -27,6 +27,7 @@ import org.apache.lucene.index.SegmentWr
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
 import org.apache.lucene.util.packed.PackedInts;
 
@@ -105,15 +106,7 @@ public class FixedGapTermsIndexWriter ex
     // As long as codec sorts terms in unicode codepoint
     // order, we can safely strip off the non-distinguishing
     // suffix to save RAM in the loaded terms index.
-    final int idxTermOffset = indexedTerm.offset;
-    final int priorTermOffset = priorTerm.offset;
-    final int limit = Math.min(priorTerm.length, indexedTerm.length);
-    for(int byteIdx=0;byteIdx<limit;byteIdx++) {
-      if (priorTerm.bytes[priorTermOffset+byteIdx] != indexedTerm.bytes[idxTermOffset+byteIdx]) {
-        return byteIdx+1;
-      }
-    }
-    return Math.min(1+priorTerm.length, indexedTerm.length);
+    return StringHelper.sortKeyLength(priorTerm, indexedTerm);
   }
 
   private class SimpleFieldWriter extends FieldWriter {

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=1617975&r1=1617974&r2=1617975&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 Thu Aug 14 15:30:44 2014
@@ -119,7 +119,7 @@ public abstract class Codec implements N
     loader.reload(classloader);
   }
   
-  private static Codec defaultCodec = Codec.forName("Lucene49");
+  private static Codec defaultCodec = Codec.forName("Lucene410");
   
   /** expert: returns the default codec used for newly created
    *  {@link IndexWriterConfig}s.

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=1617975&r1=1617974&r2=1617975&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 Thu Aug 14 15:30:44 2014
@@ -21,13 +21,13 @@ package org.apache.lucene.codecs;
  * A codec that forwards all its method calls to another codec.
  * <p>
  * Extend this class when you need to reuse the functionality of an existing
- * codec. For example, if you want to build a codec that redefines Lucene49's
+ * codec. For example, if you want to build a codec that redefines Lucene410's
  * {@link LiveDocsFormat}:
  * <pre class="prettyprint">
  *   public final class CustomCodec extends FilterCodec {
  *
  *     public CustomCodec() {
- *       super("CustomCodec", new Lucene49Codec());
+ *       super("CustomCodec", new Lucene410Codec());
  *     }
  *
  *     public LiveDocsFormat liveDocsFormat() {

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene410/Lucene410Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene410/Lucene410Codec.java?rev=1617975&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene410/Lucene410Codec.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene410/Lucene410Codec.java Thu Aug 14 15:30:44 2014
@@ -0,0 +1,140 @@
+package org.apache.lucene.codecs.lucene410;
+
+/*
+ * 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.codecs.Codec;
+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.lucene40.Lucene40LiveDocsFormat;
+import org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat;
+import org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat;
+import org.apache.lucene.codecs.lucene46.Lucene46FieldInfosFormat;
+import org.apache.lucene.codecs.lucene46.Lucene46SegmentInfoFormat;
+import org.apache.lucene.codecs.lucene49.Lucene49NormsFormat;
+import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
+import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
+
+/**
+ * Implements the Lucene 4.10 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.lucene410 package documentation for file format details.
+ * @lucene.experimental
+ */
+// NOTE: if we make largish changes in a minor release, easier to just make Lucene411Codec or whatever
+// if they are backwards compatible or smallish we can probably do the backwards in the postingsreader
+// (it writes a minor version, etc).
+public class Lucene410Codec extends Codec {
+  private final StoredFieldsFormat fieldsFormat = new Lucene41StoredFieldsFormat();
+  private final TermVectorsFormat vectorsFormat = new Lucene42TermVectorsFormat();
+  private final FieldInfosFormat fieldInfosFormat = new Lucene46FieldInfosFormat();
+  private final SegmentInfoFormat segmentInfosFormat = new Lucene46SegmentInfoFormat();
+  private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat();
+  
+  private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
+    @Override
+    public PostingsFormat getPostingsFormatForField(String field) {
+      return Lucene410Codec.this.getPostingsFormatForField(field);
+    }
+  };
+  
+  private final DocValuesFormat docValuesFormat = new PerFieldDocValuesFormat() {
+    @Override
+    public DocValuesFormat getDocValuesFormatForField(String field) {
+      return Lucene410Codec.this.getDocValuesFormatForField(field);
+    }
+  };
+
+  /** Sole constructor. */
+  public Lucene410Codec() {
+    super("Lucene410");
+  }
+  
+  @Override
+  public final StoredFieldsFormat storedFieldsFormat() {
+    return fieldsFormat;
+  }
+  
+  @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;
+  }
+
+  /** Returns the postings format that should be used for writing 
+   *  new segments of <code>field</code>.
+   *  
+   *  The default implementation always returns "Lucene41"
+   */
+  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 "Lucene410"
+   */
+  public DocValuesFormat getDocValuesFormatForField(String field) {
+    return defaultDVFormat;
+  }
+  
+  @Override
+  public final DocValuesFormat docValuesFormat() {
+    return docValuesFormat;
+  }
+
+  private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
+  private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene410");
+
+  private final NormsFormat normsFormat = new Lucene49NormsFormat();
+
+  @Override
+  public final NormsFormat normsFormat() {
+    return normsFormat;
+  }
+}

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesConsumer.java?rev=1617975&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesConsumer.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesConsumer.java Thu Aug 14 15:30:44 2014
@@ -0,0 +1,559 @@
+package org.apache.lucene.codecs.lucene410;
+
+/*
+ * 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; // javadocs
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.DocValuesConsumer;
+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.store.RAMOutputStream;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.MathUtil;
+import org.apache.lucene.util.PagedBytes;
+import org.apache.lucene.util.PagedBytes.PagedBytesDataInput;
+import org.apache.lucene.util.StringHelper;
+import org.apache.lucene.util.packed.DirectWriter;
+import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
+import org.apache.lucene.util.packed.PackedInts;
+
+/** writer for {@link Lucene410DocValuesFormat} */
+class Lucene410DocValuesConsumer extends DocValuesConsumer implements Closeable {
+
+  static final int BLOCK_SIZE = 16384;
+  
+  // address terms in blocks of 16 terms
+  static final int INTERVAL_SHIFT = 4;
+  static final int INTERVAL_COUNT = 1 << INTERVAL_SHIFT;
+  static final int INTERVAL_MASK = INTERVAL_COUNT - 1;
+  
+  // build reverse index from every 1024th term
+  static final int REVERSE_INTERVAL_SHIFT = 10;
+  static final int REVERSE_INTERVAL_COUNT = 1 << REVERSE_INTERVAL_SHIFT;
+  static final int REVERSE_INTERVAL_MASK = REVERSE_INTERVAL_COUNT - 1;
+  
+  // for conversion from reverse index to block
+  static final int BLOCK_INTERVAL_SHIFT = REVERSE_INTERVAL_SHIFT - INTERVAL_SHIFT;
+  static final int BLOCK_INTERVAL_COUNT = 1 << BLOCK_INTERVAL_SHIFT;
+  static final int BLOCK_INTERVAL_MASK = BLOCK_INTERVAL_COUNT - 1;
+
+  /** Compressed using packed blocks of ints. */
+  public static final int DELTA_COMPRESSED = 0;
+  /** Compressed by computing the GCD. */
+  public static final int GCD_COMPRESSED = 1;
+  /** Compressed by giving IDs to unique values. */
+  public static final int TABLE_COMPRESSED = 2;
+  /** Compressed with monotonically increasing values */
+  public static final int MONOTONIC_COMPRESSED = 3;
+  
+  /** Uncompressed binary, written directly (fixed length). */
+  public static final int BINARY_FIXED_UNCOMPRESSED = 0;
+  /** Uncompressed binary, written directly (variable length). */
+  public static final int BINARY_VARIABLE_UNCOMPRESSED = 1;
+  /** Compressed binary with shared prefixes */
+  public static final int BINARY_PREFIX_COMPRESSED = 2;
+
+  /** Standard storage for sorted set values with 1 level of indirection:
+   *  docId -> address -> ord. */
+  public static final int SORTED_WITH_ADDRESSES = 0;
+  /** Single-valued sorted set values, encoded as sorted values, so no level
+   *  of indirection: docId -> ord. */
+  public static final int SORTED_SINGLE_VALUED = 1;
+
+  IndexOutput data, meta;
+  final int maxDoc;
+  
+  /** expert: Creates a new writer */
+  public Lucene410DocValuesConsumer(SegmentWriteState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
+    boolean success = false;
+    try {
+      String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
+      data = state.directory.createOutput(dataName, state.context);
+      CodecUtil.writeHeader(data, dataCodec, Lucene410DocValuesFormat.VERSION_CURRENT);
+      String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
+      meta = state.directory.createOutput(metaName, state.context);
+      CodecUtil.writeHeader(meta, metaCodec, Lucene410DocValuesFormat.VERSION_CURRENT);
+      maxDoc = state.segmentInfo.getDocCount();
+      success = true;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(this);
+      }
+    }
+  }
+  
+  @Override
+  public void addNumericField(FieldInfo field, Iterable<Number> values) throws IOException {
+    addNumericField(field, values, true);
+  }
+
+  void addNumericField(FieldInfo field, Iterable<Number> values, boolean optimizeStorage) throws IOException {
+    long count = 0;
+    long minValue = Long.MAX_VALUE;
+    long maxValue = Long.MIN_VALUE;
+    long gcd = 0;
+    boolean missing = false;
+    // TODO: more efficient?
+    HashSet<Long> uniqueValues = null;
+    if (optimizeStorage) {
+      uniqueValues = new HashSet<>();
+
+      for (Number nv : values) {
+        final long v;
+        if (nv == null) {
+          v = 0;
+          missing = true;
+        } else {
+          v = nv.longValue();
+        }
+
+        if (gcd != 1) {
+          if (v < Long.MIN_VALUE / 2 || v > Long.MAX_VALUE / 2) {
+            // in that case v - minValue might overflow and make the GCD computation return
+            // wrong results. Since these extreme values are unlikely, we just discard
+            // GCD computation for them
+            gcd = 1;
+          } else if (count != 0) { // minValue needs to be set first
+            gcd = MathUtil.gcd(gcd, v - minValue);
+          }
+        }
+
+        minValue = Math.min(minValue, v);
+        maxValue = Math.max(maxValue, v);
+
+        if (uniqueValues != null) {
+          if (uniqueValues.add(v)) {
+            if (uniqueValues.size() > 256) {
+              uniqueValues = null;
+            }
+          }
+        }
+
+        ++count;
+      }
+    } else {
+      for (Number nv : values) {
+        long v = nv.longValue();
+        minValue = Math.min(minValue, v);
+        maxValue = Math.max(maxValue, v);
+        ++count;
+      }
+    }
+    
+    final long delta = maxValue - minValue;
+    final int deltaBitsRequired = DirectWriter.unsignedBitsRequired(delta);
+    final int tableBitsRequired = uniqueValues == null
+        ? Integer.MAX_VALUE
+        : DirectWriter.bitsRequired(uniqueValues.size() - 1);
+
+    final int format;
+    if (uniqueValues != null && tableBitsRequired < deltaBitsRequired) {
+      format = TABLE_COMPRESSED;
+    } else if (gcd != 0 && gcd != 1) {
+      final long gcdDelta = (maxValue - minValue) / gcd;
+      final long gcdBitsRequired = DirectWriter.unsignedBitsRequired(gcdDelta);
+      format = gcdBitsRequired < deltaBitsRequired ? GCD_COMPRESSED : DELTA_COMPRESSED;
+    } else {
+      format = DELTA_COMPRESSED;
+    }
+    meta.writeVInt(field.number);
+    meta.writeByte(Lucene410DocValuesFormat.NUMERIC);
+    meta.writeVInt(format);
+    if (missing) {
+      meta.writeLong(data.getFilePointer());
+      writeMissingBitset(values);
+    } else {
+      meta.writeLong(-1L);
+    }
+    meta.writeLong(data.getFilePointer());
+    meta.writeVLong(count);
+
+    switch (format) {
+      case GCD_COMPRESSED:
+        meta.writeLong(minValue);
+        meta.writeLong(gcd);
+        final long maxDelta = (maxValue - minValue) / gcd;
+        final int bits = DirectWriter.unsignedBitsRequired(maxDelta);
+        meta.writeVInt(bits);
+        final DirectWriter quotientWriter = DirectWriter.getInstance(data, count, bits);
+        for (Number nv : values) {
+          long value = nv == null ? 0 : nv.longValue();
+          quotientWriter.add((value - minValue) / gcd);
+        }
+        quotientWriter.finish();
+        break;
+      case DELTA_COMPRESSED:
+        final long minDelta = delta < 0 ? 0 : minValue;
+        meta.writeLong(minDelta);
+        meta.writeVInt(deltaBitsRequired);
+        final DirectWriter writer = DirectWriter.getInstance(data, count, deltaBitsRequired);
+        for (Number nv : values) {
+          long v = nv == null ? 0 : nv.longValue();
+          writer.add(v - minDelta);
+        }
+        writer.finish();
+        break;
+      case TABLE_COMPRESSED:
+        final Long[] decode = uniqueValues.toArray(new Long[uniqueValues.size()]);
+        Arrays.sort(decode);
+        final HashMap<Long,Integer> encode = new HashMap<>();
+        meta.writeVInt(decode.length);
+        for (int i = 0; i < decode.length; i++) {
+          meta.writeLong(decode[i]);
+          encode.put(decode[i], i);
+        }
+        meta.writeVInt(tableBitsRequired);
+        final DirectWriter ordsWriter = DirectWriter.getInstance(data, count, tableBitsRequired);
+        for (Number nv : values) {
+          ordsWriter.add(encode.get(nv == null ? 0 : nv.longValue()));
+        }
+        ordsWriter.finish();
+        break;
+      default:
+        throw new AssertionError();
+    }
+    meta.writeLong(data.getFilePointer());
+  }
+  
+  // TODO: in some cases representing missing with minValue-1 wouldn't take up additional space and so on,
+  // but this is very simple, and algorithms only check this for values of 0 anyway (doesnt slow down normal decode)
+  void writeMissingBitset(Iterable<?> values) throws IOException {
+    byte bits = 0;
+    int count = 0;
+    for (Object v : values) {
+      if (count == 8) {
+        data.writeByte(bits);
+        count = 0;
+        bits = 0;
+      }
+      if (v != null) {
+        bits |= 1 << (count & 7);
+      }
+      count++;
+    }
+    if (count > 0) {
+      data.writeByte(bits);
+    }
+  }
+
+  @Override
+  public void addBinaryField(FieldInfo field, Iterable<BytesRef> values) throws IOException {
+    // write the byte[] data
+    meta.writeVInt(field.number);
+    meta.writeByte(Lucene410DocValuesFormat.BINARY);
+    int minLength = Integer.MAX_VALUE;
+    int maxLength = Integer.MIN_VALUE;
+    final long startFP = data.getFilePointer();
+    long count = 0;
+    boolean missing = false;
+    for(BytesRef v : values) {
+      final int length;
+      if (v == null) {
+        length = 0;
+        missing = true;
+      } else {
+        length = v.length;
+      }
+      minLength = Math.min(minLength, length);
+      maxLength = Math.max(maxLength, length);
+      if (v != null) {
+        data.writeBytes(v.bytes, v.offset, v.length);
+      }
+      count++;
+    }
+    meta.writeVInt(minLength == maxLength ? BINARY_FIXED_UNCOMPRESSED : BINARY_VARIABLE_UNCOMPRESSED);
+    if (missing) {
+      meta.writeLong(data.getFilePointer());
+      writeMissingBitset(values);
+    } else {
+      meta.writeLong(-1L);
+    }
+    meta.writeVInt(minLength);
+    meta.writeVInt(maxLength);
+    meta.writeVLong(count);
+    meta.writeLong(startFP);
+    
+    // if minLength == maxLength, its a fixed-length byte[], we are done (the addresses are implicit)
+    // otherwise, we need to record the length fields...
+    if (minLength != maxLength) {
+      meta.writeLong(data.getFilePointer());
+      meta.writeVInt(PackedInts.VERSION_CURRENT);
+      meta.writeVInt(BLOCK_SIZE);
+
+      final MonotonicBlockPackedWriter writer = new MonotonicBlockPackedWriter(data, BLOCK_SIZE);
+      long addr = 0;
+      writer.add(addr);
+      for (BytesRef v : values) {
+        if (v != null) {
+          addr += v.length;
+        }
+        writer.add(addr);
+      }
+      writer.finish();
+    }
+  }
+  
+  /** expert: writes a value dictionary for a sorted/sortedset field */
+  private void addTermsDict(FieldInfo field, final Iterable<BytesRef> values) throws IOException {
+    // first check if its a "fixed-length" terms dict
+    int minLength = Integer.MAX_VALUE;
+    int maxLength = Integer.MIN_VALUE;
+    long numValues = 0;
+    for (BytesRef v : values) {
+      minLength = Math.min(minLength, v.length);
+      maxLength = Math.max(maxLength, v.length);
+      numValues++;
+    }
+    if (minLength == maxLength) {
+      // no index needed: direct addressing by mult
+      addBinaryField(field, values);
+    } else if (numValues < REVERSE_INTERVAL_COUNT) {
+      // low cardinality: waste a few KB of ram, but can't really use fancy index etc
+      addBinaryField(field, values);
+    } else {
+      assert numValues > 0; // we don't have to handle the empty case
+      // header
+      meta.writeVInt(field.number);
+      meta.writeByte(Lucene410DocValuesFormat.BINARY);
+      meta.writeVInt(BINARY_PREFIX_COMPRESSED);
+      meta.writeLong(-1L);
+      // now write the bytes: sharing prefixes within a block
+      final long startFP = data.getFilePointer();
+      // currently, we have to store the delta from expected for every 1/nth term
+      // we could avoid this, but its not much and less overall RAM than the previous approach!
+      RAMOutputStream addressBuffer = new RAMOutputStream();
+      MonotonicBlockPackedWriter termAddresses = new MonotonicBlockPackedWriter(addressBuffer, BLOCK_SIZE);
+      // buffers up 16 terms
+      RAMOutputStream bytesBuffer = new RAMOutputStream();
+      // buffers up block header
+      RAMOutputStream headerBuffer = new RAMOutputStream();
+      BytesRefBuilder lastTerm = new BytesRefBuilder();
+      lastTerm.grow(maxLength);
+      long count = 0;
+      int suffixDeltas[] = new int[INTERVAL_COUNT];
+      for (BytesRef v : values) {
+        int termPosition = (int) (count & INTERVAL_MASK);
+        if (termPosition == 0) {
+          termAddresses.add(data.getFilePointer() - startFP);
+          // abs-encode first term
+          headerBuffer.writeVInt(v.length);
+          headerBuffer.writeBytes(v.bytes, v.offset, v.length);
+          lastTerm.copyBytes(v);
+        } else {
+          // prefix-code: we only share at most 255 characters, to encode the length as a single
+          // byte and have random access. Larger terms just get less compression.
+          int sharedPrefix = Math.min(255, StringHelper.bytesDifference(lastTerm.get(), v));
+          bytesBuffer.writeByte((byte) sharedPrefix);
+          bytesBuffer.writeBytes(v.bytes, v.offset + sharedPrefix, v.length - sharedPrefix);
+          // we can encode one smaller, because terms are unique.
+          suffixDeltas[termPosition] = v.length - sharedPrefix - 1;
+        }
+        
+        count++;
+        // flush block
+        if ((count & INTERVAL_MASK) == 0) {
+          flushTermsDictBlock(headerBuffer, bytesBuffer, suffixDeltas);
+        }
+      }
+      // flush trailing crap
+      int leftover = (int) (count & INTERVAL_MASK);
+      if (leftover > 0) {
+        Arrays.fill(suffixDeltas, leftover, suffixDeltas.length, 0);
+        flushTermsDictBlock(headerBuffer, bytesBuffer, suffixDeltas);
+      }
+      final long indexStartFP = data.getFilePointer();
+      // write addresses of indexed terms
+      termAddresses.finish();
+      addressBuffer.writeTo(data);
+      addressBuffer = null;
+      termAddresses = null;
+      meta.writeVInt(minLength);
+      meta.writeVInt(maxLength);
+      meta.writeVLong(count);
+      meta.writeLong(startFP);
+      meta.writeLong(indexStartFP);
+      meta.writeVInt(PackedInts.VERSION_CURRENT);
+      meta.writeVInt(BLOCK_SIZE);
+      addReverseTermIndex(field, values, maxLength);
+    }
+  }
+  
+  // writes term dictionary "block"
+  // first term is absolute encoded as vint length + bytes.
+  // lengths of subsequent N terms are encoded as either N bytes or N shorts.
+  // in the double-byte case, the first byte is indicated with -1.
+  // subsequent terms are encoded as byte suffixLength + bytes.
+  private void flushTermsDictBlock(RAMOutputStream headerBuffer, RAMOutputStream bytesBuffer, int suffixDeltas[]) throws IOException {
+    boolean twoByte = false;
+    for (int i = 1; i < suffixDeltas.length; i++) {
+      if (suffixDeltas[i] > 254) {
+        twoByte = true;
+      }
+    }
+    if (twoByte) {
+      headerBuffer.writeByte((byte)255);
+      for (int i = 1; i < suffixDeltas.length; i++) {
+        headerBuffer.writeShort((short) suffixDeltas[i]);
+      }
+    } else {
+      for (int i = 1; i < suffixDeltas.length; i++) {
+        headerBuffer.writeByte((byte) suffixDeltas[i]);
+      }
+    }
+    headerBuffer.writeTo(data);
+    headerBuffer.reset();
+    bytesBuffer.writeTo(data);
+    bytesBuffer.reset();
+  }
+  
+  // writes reverse term index: used for binary searching a term into a range of 64 blocks
+  // for every 64 blocks (1024 terms) we store a term, trimming any suffix unnecessary for comparison
+  // terms are written as a contiguous byte[], but never spanning 2^15 byte boundaries.
+  private void addReverseTermIndex(FieldInfo field, final Iterable<BytesRef> values, int maxLength) throws IOException {
+    long count = 0;
+    BytesRefBuilder priorTerm = new BytesRefBuilder();
+    priorTerm.grow(maxLength);
+    BytesRef indexTerm = new BytesRef();
+    long startFP = data.getFilePointer();
+    PagedBytes pagedBytes = new PagedBytes(15);
+    MonotonicBlockPackedWriter addresses = new MonotonicBlockPackedWriter(data, BLOCK_SIZE);
+    
+    for (BytesRef b : values) {
+      int termPosition = (int) (count & REVERSE_INTERVAL_MASK);
+      if (termPosition == 0) {
+        int len = StringHelper.sortKeyLength(priorTerm.get(), b);
+        indexTerm.bytes = b.bytes;
+        indexTerm.offset = b.offset;
+        indexTerm.length = len;
+        addresses.add(pagedBytes.copyUsingLengthPrefix(indexTerm));
+      } else if (termPosition == REVERSE_INTERVAL_MASK) {
+        priorTerm.copyBytes(b);
+      }
+      count++;
+    }
+    addresses.finish();
+    long numBytes = pagedBytes.getPointer();
+    pagedBytes.freeze(true);
+    PagedBytesDataInput in = pagedBytes.getDataInput();
+    meta.writeLong(startFP);
+    data.writeVLong(numBytes);
+    data.copyBytes(in, numBytes);
+  }
+
+  @Override
+  public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
+    meta.writeVInt(field.number);
+    meta.writeByte(Lucene410DocValuesFormat.SORTED);
+    addTermsDict(field, values);
+    addNumericField(field, docToOrd, false);
+  }
+
+  @Override
+  public void addSortedNumericField(FieldInfo field, final Iterable<Number> docToValueCount, final Iterable<Number> values) throws IOException {
+    meta.writeVInt(field.number);
+    meta.writeByte(Lucene410DocValuesFormat.SORTED_NUMERIC);
+    if (isSingleValued(docToValueCount)) {
+      meta.writeVInt(SORTED_SINGLE_VALUED);
+      // The field is single-valued, we can encode it as NUMERIC
+      addNumericField(field, singletonView(docToValueCount, values, null));
+    } else {
+      meta.writeVInt(SORTED_WITH_ADDRESSES);
+      // write the stream of values as a numeric field
+      addNumericField(field, values, true);
+      // write the doc -> ord count as a absolute index to the stream
+      addAddresses(field, docToValueCount);
+    }
+  }
+
+  @Override
+  public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, final Iterable<Number> docToOrdCount, final Iterable<Number> ords) throws IOException {
+    meta.writeVInt(field.number);
+    meta.writeByte(Lucene410DocValuesFormat.SORTED_SET);
+
+    if (isSingleValued(docToOrdCount)) {
+      meta.writeVInt(SORTED_SINGLE_VALUED);
+      // The field is single-valued, we can encode it as SORTED
+      addSortedField(field, values, singletonView(docToOrdCount, ords, -1L));
+    } else {
+      meta.writeVInt(SORTED_WITH_ADDRESSES);
+
+      // write the ord -> byte[] as a binary field
+      addTermsDict(field, values);
+
+      // write the stream of ords as a numeric field
+      // NOTE: we could return an iterator that delta-encodes these within a doc
+      addNumericField(field, ords, false);
+
+      // write the doc -> ord count as a absolute index to the stream
+      addAddresses(field, docToOrdCount);
+    }
+  }
+  
+  // writes addressing information as MONOTONIC_COMPRESSED integer
+  private void addAddresses(FieldInfo field, Iterable<Number> values) throws IOException {
+    meta.writeVInt(field.number);
+    meta.writeByte(Lucene410DocValuesFormat.NUMERIC);
+    meta.writeVInt(MONOTONIC_COMPRESSED);
+    meta.writeLong(-1L);
+    meta.writeLong(data.getFilePointer());
+    meta.writeVLong(maxDoc);
+    meta.writeVInt(PackedInts.VERSION_CURRENT);
+    meta.writeVInt(BLOCK_SIZE);
+
+    final MonotonicBlockPackedWriter writer = new MonotonicBlockPackedWriter(data, BLOCK_SIZE);
+    long addr = 0;
+    writer.add(addr);
+    for (Number v : values) {
+      addr += v.longValue();
+      writer.add(addr);
+    }
+    writer.finish();
+    meta.writeLong(data.getFilePointer());
+  }
+
+  @Override
+  public void close() throws IOException {
+    boolean success = false;
+    try {
+      if (meta != null) {
+        meta.writeVInt(-1); // write EOF marker
+        CodecUtil.writeFooter(meta); // write checksum
+      }
+      if (data != null) {
+        CodecUtil.writeFooter(data); // write checksum
+      }
+      success = true;
+    } finally {
+      if (success) {
+        IOUtils.close(data, meta);
+      } else {
+        IOUtils.closeWhileHandlingException(data, meta);
+      }
+      meta = data = null;
+    }
+  }
+}

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesFormat.java?rev=1617975&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesFormat.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene410/Lucene410DocValuesFormat.java Thu Aug 14 15:30:44 2014
@@ -0,0 +1,195 @@
+package org.apache.lucene.codecs.lucene410;
+
+/*
+ * 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.DocValuesConsumer;
+import org.apache.lucene.codecs.DocValuesProducer;
+import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.index.FieldInfo.DocValuesType;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.SmallFloat;
+import org.apache.lucene.util.fst.FST;
+import org.apache.lucene.util.packed.DirectWriter;
+import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
+
+/**
+ * Lucene 4.10 DocValues format.
+ * <p>
+ * Encodes the five per-document value types (Numeric,Binary,Sorted,SortedSet,SortedNumeric) with these strategies:
+ * <p>
+ * {@link DocValuesType#NUMERIC NUMERIC}:
+ * <ul>
+ *    <li>Delta-compressed: per-document integers written as deltas from the minimum value,
+ *        compressed with bitpacking. For more information, see {@link DirectWriter}.
+ *    <li>Table-compressed: when the number of unique values is very small (&lt; 256), and
+ *        when there are unused "gaps" in the range of values used (such as {@link SmallFloat}), 
+ *        a lookup table is written instead. Each per-document entry is instead the ordinal 
+ *        to this table, and those ordinals are compressed with bitpacking ({@link DirectWriter}). 
+ *    <li>GCD-compressed: when all numbers share a common divisor, such as dates, the greatest
+ *        common denominator (GCD) is computed, and quotients are stored using Delta-compressed Numerics.
+ *    <li>Monotonic-compressed: when all numbers are monotonically increasing offsets, they are written
+ *        as blocks of bitpacked integers, encoding the deviation from the expected delta.
+ * </ul>
+ * <p>
+ * {@link DocValuesType#BINARY BINARY}:
+ * <ul>
+ *    <li>Fixed-width Binary: one large concatenated byte[] is written, along with the fixed length.
+ *        Each document's value can be addressed directly with multiplication ({@code docID * length}). 
+ *    <li>Variable-width Binary: one large concatenated byte[] is written, along with end addresses 
+ *        for each document. The addresses are written as Monotonic-compressed numerics.
+ *    <li>Prefix-compressed Binary: values are written in chunks of 16, with the first value written
+ *        completely and other values sharing prefixes. chunk addresses are written as Monotonic-compressed
+ *        numerics. A reverse lookup index is written from a portion of every 1024th term.
+ * </ul>
+ * <p>
+ * {@link DocValuesType#SORTED SORTED}:
+ * <ul>
+ *    <li>Sorted: a mapping of ordinals to deduplicated terms is written as Binary, 
+ *        along with the per-document ordinals written using one of the numeric strategies above.
+ * </ul>
+ * <p>
+ * {@link DocValuesType#SORTED_SET SORTED_SET}:
+ * <ul>
+ *    <li>SortedSet: a mapping of ordinals to deduplicated terms is written as Binary, 
+ *        an ordinal list and per-document index into this list are written using the numeric strategies 
+ *        above. 
+ * </ul>
+ * <p>
+ * {@link DocValuesType#SORTED_NUMERIC SORTED_NUMERIC}:
+ * <ul>
+ *    <li>SortedNumeric: a value list and per-document index into this list are written using the numeric
+ *        strategies above.
+ * </ul>
+ * <p>
+ * Files:
+ * <ol>
+ *   <li><tt>.dvd</tt>: DocValues data</li>
+ *   <li><tt>.dvm</tt>: DocValues metadata</li>
+ * </ol>
+ * <ol>
+ *   <li><a name="dvm" id="dvm"></a>
+ *   <p>The DocValues metadata or .dvm file.</p>
+ *   <p>For DocValues field, this stores metadata, such as the offset into the 
+ *      DocValues data (.dvd)</p>
+ *   <p>DocValues metadata (.dvm) --&gt; Header,&lt;Entry&gt;<sup>NumFields</sup>,Footer</p>
+ *   <ul>
+ *     <li>Entry --&gt; NumericEntry | BinaryEntry | SortedEntry | SortedSetEntry | SortedNumericEntry</li>
+ *     <li>NumericEntry --&gt; GCDNumericEntry | TableNumericEntry | DeltaNumericEntry</li>
+ *     <li>GCDNumericEntry --&gt; NumericHeader,MinValue,GCD,BitsPerValue</li>
+ *     <li>TableNumericEntry --&gt; NumericHeader,TableSize,{@link DataOutput#writeLong Int64}<sup>TableSize</sup>,BitsPerValue</li>
+ *     <li>DeltaNumericEntry --&gt; NumericHeader,MinValue,BitsPerValue</li>
+ *     <li>MonotonicNumericEntry --&gt; NumericHeader,PackedVersion,BlockSize</li>
+ *     <li>NumericHeader --&gt; FieldNumber,EntryType,NumericType,MissingOffset,DataOffset,Count,EndOffset</li>
+ *     <li>BinaryEntry --&gt; FixedBinaryEntry | VariableBinaryEntry | PrefixBinaryEntry</li>
+ *     <li>FixedBinaryEntry --&gt; BinaryHeader</li>
+ *     <li>VariableBinaryEntry --&gt; BinaryHeader,AddressOffset,PackedVersion,BlockSize</li>
+ *     <li>PrefixBinaryEntry --&gt; BinaryHeader,AddressInterval,AddressOffset,PackedVersion,BlockSize</li>
+ *     <li>BinaryHeader --&gt; FieldNumber,EntryType,BinaryType,MissingOffset,MinLength,MaxLength,DataOffset</li>
+ *     <li>SortedEntry --&gt; FieldNumber,EntryType,BinaryEntry,NumericEntry</li>
+ *     <li>SortedSetEntry --&gt; EntryType,BinaryEntry,NumericEntry,NumericEntry</li>
+ *     <li>SortedNumericEntry --&gt; EntryType,NumericEntry,NumericEntry</li>
+ *     <li>FieldNumber,PackedVersion,MinLength,MaxLength,BlockSize,ValueCount --&gt; {@link DataOutput#writeVInt VInt}</li>
+ *     <li>EntryType,CompressionType --&gt; {@link DataOutput#writeByte Byte}</li>
+ *     <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
+ *     <li>MinValue,GCD,MissingOffset,AddressOffset,DataOffset,EndOffset --&gt; {@link DataOutput#writeLong Int64}</li>
+ *     <li>TableSize,BitsPerValue --&gt; {@link DataOutput#writeVInt vInt}</li>
+ *     <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
+ *   </ul>
+ *   <p>Sorted fields have two entries: a BinaryEntry with the value metadata,
+ *      and an ordinary NumericEntry for the document-to-ord metadata.</p>
+ *   <p>SortedSet fields have three entries: a BinaryEntry with the value metadata,
+ *      and two NumericEntries for the document-to-ord-index and ordinal list metadata.</p>
+ *   <p>SortedNumeric fields have two entries: A NumericEntry with the value metadata,
+ *      and a numeric entry with the document-to-value index.</p>
+ *   <p>FieldNumber of -1 indicates the end of metadata.</p>
+ *   <p>EntryType is a 0 (NumericEntry) or 1 (BinaryEntry)</p>
+ *   <p>DataOffset is the pointer to the start of the data in the DocValues data (.dvd)</p>
+ *   <p>EndOffset is the pointer to the end of the data in the DocValues data (.dvd)</p>
+ *   <p>NumericType indicates how Numeric values will be compressed:
+ *      <ul>
+ *         <li>0 --&gt; delta-compressed. For each block of 16k integers, every integer is delta-encoded
+ *             from the minimum value within the block. 
+ *         <li>1 --&gt, gcd-compressed. When all integers share a common divisor, only quotients are stored
+ *             using blocks of delta-encoded ints.
+ *         <li>2 --&gt; table-compressed. When the number of unique numeric values is small and it would save space,
+ *             a lookup table of unique values is written, followed by the ordinal for each document.
+ *      </ul>
+ *   <p>BinaryType indicates how Binary values will be stored:
+ *      <ul>
+ *         <li>0 --&gt; fixed-width. All values have the same length, addressing by multiplication. 
+ *         <li>1 --&gt, variable-width. An address for each value is stored.
+ *         <li>2 --&gt; prefix-compressed. An address to the start of every interval'th value is stored.
+ *      </ul>
+ *   <p>MinLength and MaxLength represent the min and max byte[] value lengths for Binary values.
+ *      If they are equal, then all values are of a fixed size, and can be addressed as DataOffset + (docID * length).
+ *      Otherwise, the binary values are of variable size, and packed integer metadata (PackedVersion,BlockSize)
+ *      is written for the addresses.
+ *   <p>MissingOffset points to a byte[] containing a bitset of all documents that had a value for the field.
+ *      If its -1, then there are no missing values.
+ *   <p>Checksum contains the CRC32 checksum of all bytes in the .dvm file up
+ *      until the checksum. This is used to verify integrity of the file on opening the
+ *      index.
+ *   <li><a name="dvd" id="dvd"></a>
+ *   <p>The DocValues data or .dvd file.</p>
+ *   <p>For DocValues field, this stores the actual per-document data (the heavy-lifting)</p>
+ *   <p>DocValues data (.dvd) --&gt; Header,&lt;NumericData | BinaryData | SortedData&gt;<sup>NumFields</sup>,Footer</p>
+ *   <ul>
+ *     <li>NumericData --&gt; DeltaCompressedNumerics | TableCompressedNumerics | GCDCompressedNumerics</li>
+ *     <li>BinaryData --&gt;  {@link DataOutput#writeByte Byte}<sup>DataLength</sup>,Addresses</li>
+ *     <li>SortedData --&gt; {@link FST FST&lt;Int64&gt;}</li>
+ *     <li>DeltaCompressedNumerics,TableCompressedNumerics,GCDCompressedNumerics --&gt; {@link DirectWriter PackedInts}</li>
+ *     <li>Addresses --&gt; {@link MonotonicBlockPackedWriter MonotonicBlockPackedInts(blockSize=16k)}</li>
+ *     <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
+ *   </ul>
+ * </ol>
+ * @lucene.experimental
+ */
+public final class Lucene410DocValuesFormat extends DocValuesFormat {
+
+  /** Sole Constructor */
+  public Lucene410DocValuesFormat() {
+    super("Lucene410");
+  }
+
+  @Override
+  public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+    return new Lucene410DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
+  }
+
+  @Override
+  public DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException {
+    return new Lucene410DocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
+  }
+  
+  static final String DATA_CODEC = "Lucene410DocValuesData";
+  static final String DATA_EXTENSION = "dvd";
+  static final String META_CODEC = "Lucene410ValuesMetadata";
+  static final String META_EXTENSION = "dvm";
+  static final int VERSION_START = 0;
+  static final int VERSION_CURRENT = VERSION_START;
+  static final byte NUMERIC = 0;
+  static final byte BINARY = 1;
+  static final byte SORTED = 2;
+  static final byte SORTED_SET = 3;
+  static final byte SORTED_NUMERIC = 4;
+}