You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jp...@apache.org on 2016/10/04 08:52:02 UTC

[3/3] lucene-solr:master: LUCENE-7453: Create a Lucene70Codec.

LUCENE-7453: Create a Lucene70Codec.


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

Branch: refs/heads/master
Commit: 32446e9205679fb94b247f0fa2aa97ecd54a49ff
Parents: 2ad0082
Author: Adrien Grand <jp...@gmail.com>
Authored: Mon Oct 3 17:11:07 2016 +0200
Committer: Adrien Grand <jp...@gmail.com>
Committed: Tue Oct 4 10:50:34 2016 +0200

----------------------------------------------------------------------
 .../benchmark/byTask/tasks/CreateIndexTask.java |    4 +-
 .../java/org/apache/lucene/codecs/Codec.java    |    2 +-
 .../lucene54/Lucene54DocValuesFormat.java       |    3 +
 .../lucene/codecs/lucene70/Lucene70Codec.java   |  177 ++
 .../lucene70/Lucene70DocValuesConsumer.java     |  797 ++++++++
 .../lucene70/Lucene70DocValuesFormat.java       |  183 ++
 .../lucene70/Lucene70DocValuesProducer.java     | 1816 ++++++++++++++++++
 .../lucene/codecs/lucene70/package-info.java    |  392 ++++
 .../services/org.apache.lucene.codecs.Codec     |    1 +
 .../org.apache.lucene.codecs.DocValuesFormat    |    1 +
 ...cene50StoredFieldsFormatHighCompression.java |    8 +-
 .../lucene53/TestLucene53NormsFormat.java       |    4 +-
 .../lucene70/TestLucene70DocValuesFormat.java   |  613 ++++++
 .../org/apache/lucene/index/Test2BPoints.java   |    2 +-
 .../apache/lucene/index/TestPointValues.java    |    8 +-
 .../org/apache/lucene/search/TestBoolean2.java  |    8 +-
 .../apache/lucene/search/TestPointQueries.java  |    4 +-
 .../org/apache/lucene/document/TestNearest.java |    2 +-
 .../apache/lucene/spatial3d/TestGeo3DPoint.java |    4 +-
 .../suggest/document/TestSuggestField.java      |    4 +-
 .../apache/lucene/geo/BaseGeoPointTestCase.java |    2 +-
 .../util/TestRuleSetupAndRestoreClassEnv.java   |    6 +-
 .../java/org/apache/lucene/util/TestUtil.java   |    8 +-
 .../apache/solr/core/SchemaCodecFactory.java    |    4 +-
 .../solr/collection1/conf/schema_codec.xml      |    2 +-
 25 files changed, 4019 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/32446e92/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java
----------------------------------------------------------------------
diff --git a/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java b/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java
index df8a1b4..c2c145b 100644
--- a/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java
+++ b/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java
@@ -29,7 +29,7 @@ import org.apache.lucene.benchmark.byTask.PerfRunData;
 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.lucene62.Lucene62Codec;
+import org.apache.lucene.codecs.lucene70.Lucene70Codec;
 import org.apache.lucene.index.ConcurrentMergeScheduler;
 import org.apache.lucene.index.IndexCommit;
 import org.apache.lucene.index.IndexDeletionPolicy;
@@ -139,7 +139,7 @@ public class CreateIndexTask extends PerfTask {
     if (defaultCodec == null && postingsFormat != null) {
       try {
         final PostingsFormat postingsFormatChosen = PostingsFormat.forName(postingsFormat);
-        iwConf.setCodec(new Lucene62Codec() {
+        iwConf.setCodec(new Lucene70Codec() {
           @Override
           public PostingsFormat getPostingsFormatForField(String field) {
             return postingsFormatChosen;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/32446e92/lucene/core/src/java/org/apache/lucene/codecs/Codec.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/Codec.java b/lucene/core/src/java/org/apache/lucene/codecs/Codec.java
index 442445c..d864710 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/Codec.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/Codec.java
@@ -57,7 +57,7 @@ public abstract class Codec implements NamedSPILoader.NamedSPI {
     }
     
     // TODO: should we use this, or maybe a system property is better?
-    static Codec defaultCodec = LOADER.lookup("Lucene62");
+    static Codec defaultCodec = LOADER.lookup("Lucene70");
   }
 
   private final String name;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/32446e92/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesFormat.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesFormat.java
index 800649c..91ccfe2 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesFormat.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.DocValuesProducer;
+import org.apache.lucene.codecs.lucene70.Lucene70DocValuesFormat;
 import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
@@ -95,7 +96,9 @@ import org.apache.lucene.util.packed.DirectWriter;
  *   <li><tt>.dvm</tt>: DocValues metadata</li>
  * </ol>
  * @lucene.experimental
+ * @deprecated Use {@link Lucene70DocValuesFormat}.
  */
+@Deprecated
 public final class Lucene54DocValuesFormat extends DocValuesFormat {
 
   /** Sole Constructor */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/32446e92/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70Codec.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70Codec.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70Codec.java
new file mode 100644
index 0000000..8d86649
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70Codec.java
@@ -0,0 +1,177 @@
+/*
+ * 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.
+ */
+package org.apache.lucene.codecs.lucene70;
+
+import java.util.Objects;
+
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.CompoundFormat;
+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.PointsFormat;
+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.Lucene50StoredFieldsFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
+import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat;
+import org.apache.lucene.codecs.lucene53.Lucene53NormsFormat;
+import org.apache.lucene.codecs.lucene60.Lucene60FieldInfosFormat;
+import org.apache.lucene.codecs.lucene60.Lucene60PointsFormat;
+import org.apache.lucene.codecs.lucene62.Lucene62SegmentInfoFormat;
+import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
+import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
+
+/**
+ * Implements the Lucene 7.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.lucene70 package documentation for file format details.
+ *
+ * @lucene.experimental
+ */
+public class Lucene70Codec extends Codec {
+  private final TermVectorsFormat vectorsFormat = new Lucene50TermVectorsFormat();
+  private final FieldInfosFormat fieldInfosFormat = new Lucene60FieldInfosFormat();
+  private final SegmentInfoFormat segmentInfosFormat = new Lucene62SegmentInfoFormat();
+  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 Lucene70Codec.this.getPostingsFormatForField(field);
+    }
+  };
+  
+  private final DocValuesFormat docValuesFormat = new PerFieldDocValuesFormat() {
+    @Override
+    public DocValuesFormat getDocValuesFormatForField(String field) {
+      return Lucene70Codec.this.getDocValuesFormatForField(field);
+    }
+  };
+  
+  private final StoredFieldsFormat storedFieldsFormat;
+
+  /** 
+   * Instantiates a new codec.
+   */
+  public Lucene70Codec() {
+    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 Lucene70Codec(Mode mode) {
+    super("Lucene70");
+    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 PointsFormat pointsFormat() {
+    return new Lucene60PointsFormat();
+  }
+
+  /** 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 "Lucene70".
+   *  <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("Lucene70");
+
+  private final NormsFormat normsFormat = new Lucene53NormsFormat();
+
+  @Override
+  public final NormsFormat normsFormat() {
+    return normsFormat;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/32446e92/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesConsumer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesConsumer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesConsumer.java
new file mode 100644
index 0000000..564db7c
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesConsumer.java
@@ -0,0 +1,797 @@
+/*
+ * 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.
+ */
+package org.apache.lucene.codecs.lucene70;
+
+
+import java.io.Closeable; // javadocs
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.stream.StreamSupport;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.DocValuesConsumer;
+import org.apache.lucene.codecs.DocValuesProducer;
+import org.apache.lucene.codecs.LegacyDocValuesIterables;
+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.LongsRef;
+import org.apache.lucene.util.MathUtil;
+import org.apache.lucene.util.PagedBytes.PagedBytesDataInput;
+import org.apache.lucene.util.PagedBytes;
+import org.apache.lucene.util.StringHelper;
+import org.apache.lucene.util.packed.DirectMonotonicWriter;
+import org.apache.lucene.util.packed.DirectWriter;
+import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
+import org.apache.lucene.util.packed.PackedInts;
+
+import static org.apache.lucene.codecs.lucene70.Lucene70DocValuesFormat.*;
+
+/** writer for {@link Lucene70DocValuesFormat} */
+final class Lucene70DocValuesConsumer extends DocValuesConsumer implements Closeable {
+
+  enum NumberType {
+    /** Dense ordinals */
+    ORDINAL,
+    /** Random long values */
+    VALUE;
+  }
+
+  IndexOutput data, meta;
+  final int maxDoc;
+  
+  /** expert: Creates a new writer */
+  public Lucene70DocValuesConsumer(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.writeIndexHeader(data, dataCodec, Lucene70DocValuesFormat.VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
+      String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
+      meta = state.directory.createOutput(metaName, state.context);
+      CodecUtil.writeIndexHeader(meta, metaCodec, Lucene70DocValuesFormat.VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
+      maxDoc = state.segmentInfo.maxDoc();
+      success = true;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(this);
+      }
+    }
+  }
+  
+  @Override
+  public void addNumericField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
+    addNumericField(field, LegacyDocValuesIterables.numericIterable(field, valuesProducer, maxDoc), NumberType.VALUE);
+  }
+
+  void addNumericField(FieldInfo field, Iterable<Number> values, NumberType numberType) throws IOException {
+    long count = 0;
+    long minValue = Long.MAX_VALUE;
+    long maxValue = Long.MIN_VALUE;
+    long gcd = 0;
+    long missingCount = 0;
+    long zeroCount = 0;
+    // TODO: more efficient?
+    HashSet<Long> uniqueValues = null;
+    long missingOrdCount = 0;
+    if (numberType == NumberType.VALUE) {
+      uniqueValues = new HashSet<>();
+
+      for (Number nv : values) {
+        final long v;
+        if (nv == null) {
+          v = 0;
+          missingCount++;
+          zeroCount++;
+        } else {
+          v = nv.longValue();
+          if (v == 0) {
+            zeroCount++;
+          }
+        }
+
+        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();
+        if (v == -1L) {
+          missingOrdCount++;
+        }
+        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 boolean sparse; // 10% of docs or less have a value
+    switch (numberType) {
+      case VALUE:
+        sparse = (double) missingCount / count >= 0.90;
+        break;
+      case ORDINAL:
+        sparse = (double) missingOrdCount / count >= 0.90;
+        break;
+      default:
+        throw new AssertionError();
+    }
+
+    final int format;
+    if (uniqueValues != null 
+        && count <= Integer.MAX_VALUE
+        && (uniqueValues.size() == 1
+           || (uniqueValues.size() == 2 && missingCount > 0 && zeroCount == missingCount))) {
+      // either one unique value C or two unique values: "missing" and C
+      format = CONST_COMPRESSED;
+    } else if (sparse && count >= 1024) {
+      // require at least 1024 docs to avoid flipping back and forth when doing NRT search
+      format = SPARSE_COMPRESSED;
+    } else 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(Lucene70DocValuesFormat.NUMERIC);
+    meta.writeVInt(format);
+    if (format == SPARSE_COMPRESSED) {
+      meta.writeLong(data.getFilePointer());
+      final long numDocsWithValue;
+      switch (numberType) {
+        case VALUE:
+          numDocsWithValue = count - missingCount;
+          break;
+        case ORDINAL:
+          numDocsWithValue = count - missingOrdCount;
+          break;
+        default:
+          throw new AssertionError();
+      }
+      final long maxDoc = writeSparseMissingBitset(values, numberType, numDocsWithValue);
+      assert maxDoc == count;
+    } else if (missingCount == 0) {
+      meta.writeLong(ALL_LIVE);
+    } else if (missingCount == count) {
+      meta.writeLong(ALL_MISSING);
+    } else {
+      meta.writeLong(data.getFilePointer());
+      writeMissingBitset(values);
+    }
+    meta.writeLong(data.getFilePointer());
+    meta.writeVLong(count);
+
+    switch (format) {
+      case CONST_COMPRESSED:
+        // write the constant (nonzero value in the n=2 case, singleton value otherwise)
+        meta.writeLong(minValue < 0 ? Collections.min(uniqueValues) : Collections.max(uniqueValues));
+        break;
+      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;
+      case SPARSE_COMPRESSED:
+        final Iterable<Number> filteredMissingValues;
+        switch (numberType) {
+          case VALUE:
+            meta.writeByte((byte) 0);
+            filteredMissingValues = new Iterable<Number>() {
+              @Override
+              public Iterator<Number> iterator() {
+                return StreamSupport
+                    .stream(values.spliterator(), false)
+                    .filter(value -> value != null)
+                    .iterator();
+              }
+            };
+            break;
+          case ORDINAL:
+            meta.writeByte((byte) 1);
+            filteredMissingValues = new Iterable<Number>() {
+              @Override
+              public Iterator<Number> iterator() {
+                return StreamSupport
+                    .stream(values.spliterator(), false)
+                    .filter(value -> value.longValue() != -1L)
+                    .iterator();
+              }
+            };
+            break;
+          default:
+            throw new AssertionError();
+        }
+        // Write non-missing values as a numeric field
+        addNumericField(field, filteredMissingValues, numberType);
+        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 {
+    long bits = 0;
+    int count = 0;
+    for (Object v : values) {
+      if (count == 64) {
+        data.writeLong(bits);
+        count = 0;
+        bits = 0;
+      }
+      if (v != null) {
+        bits |= 1L << count;
+      }
+      count++;
+    }
+    if (count > 0) {
+      data.writeLong(bits);
+    }
+  }
+
+  long writeSparseMissingBitset(Iterable<Number> values, NumberType numberType, long numDocsWithValue) throws IOException {
+    meta.writeVLong(numDocsWithValue);
+
+    // Write doc IDs that have a value
+    meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
+    final DirectMonotonicWriter docIdsWriter = DirectMonotonicWriter.getInstance(meta, data, numDocsWithValue, DIRECT_MONOTONIC_BLOCK_SHIFT);
+    long docID = 0;
+    for (Number nv : values) {
+      switch (numberType) {
+        case VALUE:
+          if (nv != null) {
+            docIdsWriter.add(docID);
+          }
+          break;
+        case ORDINAL:
+          if (nv.longValue() != -1L) {
+            docIdsWriter.add(docID);
+          }
+          break;
+        default:
+          throw new AssertionError();
+      }
+      docID++;
+    }
+    docIdsWriter.finish();
+    return docID;
+  }
+
+  @Override
+  public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
+    addBinaryField(field, LegacyDocValuesIterables.binaryIterable(field, valuesProducer, maxDoc));
+  }
+  
+  private void addBinaryField(FieldInfo field, Iterable<BytesRef> values) throws IOException {
+    // write the byte[] data
+    meta.writeVInt(field.number);
+    meta.writeByte(Lucene70DocValuesFormat.BINARY);
+    int minLength = Integer.MAX_VALUE;
+    int maxLength = Integer.MIN_VALUE;
+    final long startFP = data.getFilePointer();
+    long count = 0;
+    long missingCount = 0;
+    for(BytesRef v : values) {
+      final int length;
+      if (v == null) {
+        length = 0;
+        missingCount++;
+      } 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 (missingCount == 0) {
+      meta.writeLong(ALL_LIVE);
+    } else if (missingCount == count) {
+      meta.writeLong(ALL_MISSING);
+    } else {
+      meta.writeLong(data.getFilePointer());
+      writeMissingBitset(values);
+    }
+    meta.writeVInt(minLength);
+    meta.writeVInt(maxLength);
+    meta.writeVLong(count);
+    meta.writeLong(startFP);
+    
+    // if minLength == maxLength, it's 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(DIRECT_MONOTONIC_BLOCK_SHIFT);
+
+      final DirectMonotonicWriter writer = DirectMonotonicWriter.getInstance(meta, data, count + 1, DIRECT_MONOTONIC_BLOCK_SHIFT);
+      long addr = 0;
+      writer.add(addr);
+      for (BytesRef v : values) {
+        if (v != null) {
+          addr += v.length;
+        }
+        writer.add(addr);
+      }
+      writer.finish();
+      meta.writeLong(data.getFilePointer());
+    }
+  }
+  
+  /** expert: writes a value dictionary for a sorted/sortedset field */
+  private void addTermsDict(FieldInfo field, final Iterable<BytesRef> values) throws IOException {
+    // first check if it's a "fixed-length" terms dict, and compressibility if so
+    int minLength = Integer.MAX_VALUE;
+    int maxLength = Integer.MIN_VALUE;
+    long numValues = 0;
+    BytesRefBuilder previousValue = new BytesRefBuilder();
+    long prefixSum = 0; // only valid for fixed-width data, as we have a choice there
+    for (BytesRef v : values) {
+      minLength = Math.min(minLength, v.length);
+      maxLength = Math.max(maxLength, v.length);
+      if (minLength == maxLength) {
+        int termPosition = (int) (numValues & INTERVAL_MASK);
+        if (termPosition == 0) {
+          // first term in block, save it away to compare against the last term later
+          previousValue.copyBytes(v);
+        } else if (termPosition == INTERVAL_COUNT - 1) {
+          // last term in block, accumulate shared prefix against first term
+          prefixSum += StringHelper.bytesDifference(previousValue.get(), v);
+        }
+      }
+      numValues++;
+    }
+    // for fixed width data, look at the avg(shared prefix) before deciding how to encode:
+    // prefix compression "costs" worst case 2 bytes per term because we must store suffix lengths.
+    // so if we share at least 3 bytes on average, always compress.
+    if (minLength == maxLength && prefixSum <= 3*(numValues >> INTERVAL_SHIFT)) {
+      // no index needed: not very compressible, 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(Lucene70DocValuesFormat.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 it's not much and less overall RAM than the previous approach!
+      RAMOutputStream addressBuffer = new RAMOutputStream();
+      MonotonicBlockPackedWriter termAddresses = new MonotonicBlockPackedWriter(addressBuffer, MONOTONIC_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(MONOTONIC_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, MONOTONIC_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, DocValuesProducer valuesProducer) throws IOException {
+    meta.writeVInt(field.number);
+    meta.writeByte(Lucene70DocValuesFormat.SORTED);
+    addTermsDict(field, LegacyDocValuesIterables.valuesIterable(valuesProducer.getSorted(field)));
+    addNumericField(field, LegacyDocValuesIterables.sortedOrdIterable(valuesProducer, field, maxDoc), NumberType.ORDINAL);
+  }
+
+  private void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> ords) throws IOException {
+    meta.writeVInt(field.number);
+    meta.writeByte(Lucene70DocValuesFormat.SORTED);
+    addTermsDict(field, values);
+    addNumericField(field, ords, NumberType.ORDINAL);
+  }
+
+  @Override
+  public void addSortedNumericField(FieldInfo field, final DocValuesProducer valuesProducer) throws IOException {
+
+    final Iterable<Number> docToValueCount = LegacyDocValuesIterables.sortedNumericToDocCount(valuesProducer, field, maxDoc);
+    final Iterable<Number> values = LegacyDocValuesIterables.sortedNumericToValues(valuesProducer, field);
+    
+    meta.writeVInt(field.number);
+    meta.writeByte(Lucene70DocValuesFormat.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), NumberType.VALUE);
+    } else {
+      final SortedSet<LongsRef> uniqueValueSets = uniqueValueSets(docToValueCount, values);
+      if (uniqueValueSets != null) {
+        meta.writeVInt(SORTED_SET_TABLE);
+
+        // write the set_id -> values mapping
+        writeDictionary(uniqueValueSets);
+
+        // write the doc -> set_id as a numeric field
+        addNumericField(field, docToSetId(uniqueValueSets, docToValueCount, values), NumberType.ORDINAL);
+      } else {
+        meta.writeVInt(SORTED_WITH_ADDRESSES);
+        // write the stream of values as a numeric field
+        addNumericField(field, values, NumberType.VALUE);
+        // write the doc -> ord count as a absolute index to the stream
+        addOrdIndex(field, docToValueCount);
+      }
+    }
+  }
+
+  @Override
+  public void addSortedSetField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
+
+    Iterable<BytesRef> values = LegacyDocValuesIterables.valuesIterable(valuesProducer.getSortedSet(field));
+    Iterable<Number> docToOrdCount = LegacyDocValuesIterables.sortedSetOrdCountIterable(valuesProducer, field, maxDoc);
+    Iterable<Number> ords = LegacyDocValuesIterables.sortedSetOrdsIterable(valuesProducer, field);
+
+    meta.writeVInt(field.number);
+    meta.writeByte(Lucene70DocValuesFormat.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 {
+      final SortedSet<LongsRef> uniqueValueSets = uniqueValueSets(docToOrdCount, ords);
+      if (uniqueValueSets != null) {
+        meta.writeVInt(SORTED_SET_TABLE);
+
+        // write the set_id -> ords mapping
+        writeDictionary(uniqueValueSets);
+
+        // write the ord -> byte[] as a binary field
+        addTermsDict(field, values);
+
+        // write the doc -> set_id as a numeric field
+        addNumericField(field, docToSetId(uniqueValueSets, docToOrdCount, ords), NumberType.ORDINAL);
+      } 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, NumberType.ORDINAL);
+
+        // write the doc -> ord count as a absolute index to the stream
+        addOrdIndex(field, docToOrdCount);
+      }
+    }
+  }
+
+  private SortedSet<LongsRef> uniqueValueSets(Iterable<Number> docToValueCount, Iterable<Number> values) {
+    Set<LongsRef> uniqueValueSet = new HashSet<>();
+    LongsRef docValues = new LongsRef(256);
+
+    Iterator<Number> valueCountIterator = docToValueCount.iterator();
+    Iterator<Number> valueIterator = values.iterator();
+    int totalDictSize = 0;
+    while (valueCountIterator.hasNext()) {
+      docValues.length = valueCountIterator.next().intValue();
+      if (docValues.length > 256) {
+        return null;
+      }
+      for (int i = 0; i < docValues.length; ++i) {
+        docValues.longs[i] = valueIterator.next().longValue();
+      }
+      if (uniqueValueSet.contains(docValues)) {
+        continue;
+      }
+      totalDictSize += docValues.length;
+      if (totalDictSize > 256) {
+        return null;
+      }
+      uniqueValueSet.add(new LongsRef(Arrays.copyOf(docValues.longs, docValues.length), 0, docValues.length));
+    }
+    assert valueIterator.hasNext() == false;
+    return new TreeSet<>(uniqueValueSet);
+  }
+
+  private void writeDictionary(SortedSet<LongsRef> uniqueValueSets) throws IOException {
+    int lengthSum = 0;
+    for (LongsRef longs : uniqueValueSets) {
+      lengthSum += longs.length;
+    }
+
+    meta.writeInt(lengthSum);
+    for (LongsRef valueSet : uniqueValueSets) {
+      for (int  i = 0; i < valueSet.length; ++i) {
+        meta.writeLong(valueSet.longs[valueSet.offset + i]);
+      }
+    }
+
+    meta.writeInt(uniqueValueSets.size());
+    for (LongsRef valueSet : uniqueValueSets) {
+      meta.writeInt(valueSet.length);
+    }
+  }
+
+  private Iterable<Number> docToSetId(SortedSet<LongsRef> uniqueValueSets, Iterable<Number> docToValueCount, Iterable<Number> values) {
+    final Map<LongsRef, Integer> setIds = new HashMap<>();
+    int i = 0;
+    for (LongsRef set : uniqueValueSets) {
+      setIds.put(set, i++);
+    }
+    assert i == uniqueValueSets.size();
+
+    return new Iterable<Number>() {
+
+      @Override
+      public Iterator<Number> iterator() {
+        final Iterator<Number> valueCountIterator = docToValueCount.iterator();
+        final Iterator<Number> valueIterator = values.iterator();
+        final LongsRef docValues = new LongsRef(256);
+        return new Iterator<Number>() {
+
+          @Override
+          public boolean hasNext() {
+            return valueCountIterator.hasNext();
+          }
+
+          @Override
+          public Number next() {
+            docValues.length = valueCountIterator.next().intValue();
+            for (int i = 0; i < docValues.length; ++i) {
+              docValues.longs[i] = valueIterator.next().longValue();
+            }
+            final Integer id = setIds.get(docValues);
+            assert id != null;
+            return id;
+          }
+
+        };
+
+      }
+    };
+  }
+
+  // writes addressing information as MONOTONIC_COMPRESSED integer
+  private void addOrdIndex(FieldInfo field, Iterable<Number> values) throws IOException {
+    meta.writeVInt(field.number);
+    meta.writeByte(Lucene70DocValuesFormat.NUMERIC);
+    meta.writeVInt(MONOTONIC_COMPRESSED);
+    meta.writeLong(-1L);
+    meta.writeLong(data.getFilePointer());
+    meta.writeVLong(maxDoc);
+    meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
+
+    final DirectMonotonicWriter writer = DirectMonotonicWriter.getInstance(meta, data, maxDoc + 1, DIRECT_MONOTONIC_BLOCK_SHIFT);
+    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;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/32446e92/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesFormat.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesFormat.java
new file mode 100644
index 0000000..2112341
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesFormat.java
@@ -0,0 +1,183 @@
+/*
+ * 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.
+ */
+package org.apache.lucene.codecs.lucene70;
+
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.DocValuesConsumer;
+import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.DocValuesProducer;
+import org.apache.lucene.index.DocValuesType;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.util.SmallFloat;
+import org.apache.lucene.util.packed.DirectWriter;
+
+/**
+ * Lucene 7.0 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.
+ *    <li>Const-compressed: when there is only one possible non-missing value, only the missing
+ *        bitset is encoded.
+ *    <li>Sparse-compressed: only documents with a value are stored, and lookups are performed
+ *        using binary search.
+ * </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>Single: if all documents have 0 or 1 value, then data are written like SORTED.
+ *    <li>SortedSet table: when there are few unique sets of values (&lt; 256) then each set is assigned
+ *        an id, a lookup table is written and the mapping from document to set id is written using the
+ *        numeric strategies above.
+ *    <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>Single: if all documents have 0 or 1 value, then data are written like NUMERIC.
+ *    <li>SortedSet table: when there are few unique sets of values (&lt; 256) then each set is assigned
+ *        an id, a lookup table is written and the mapping from document to set id is written using the
+ *        numeric strategies above.
+ *    <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>
+ * @lucene.experimental
+ */
+public final class Lucene70DocValuesFormat extends DocValuesFormat {
+
+  /** Sole Constructor */
+  public Lucene70DocValuesFormat() {
+    super("Lucene70");
+  }
+
+  @Override
+  public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+    return new Lucene70DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
+  }
+
+  @Override
+  public DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException {
+    return new Lucene70DocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
+  }
+  
+  static final String DATA_CODEC = "Lucene54DocValuesData";
+  static final String DATA_EXTENSION = "dvd";
+  static final String META_CODEC = "Lucene54DocValuesMetadata";
+  static final String META_EXTENSION = "dvm";
+  static final int VERSION_START = 0;
+  static final int VERSION_CURRENT = VERSION_START;
+  
+  // indicates docvalues type
+  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;
+  
+  // 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. */
+  static final int DELTA_COMPRESSED = 0;
+  /** Compressed by computing the GCD. */
+  static final int GCD_COMPRESSED = 1;
+  /** Compressed by giving IDs to unique values. */
+  static final int TABLE_COMPRESSED = 2;
+  /** Compressed with monotonically increasing values */
+  static final int MONOTONIC_COMPRESSED = 3;
+  /** Compressed with constant value (uses only missing bitset) */
+  static final int CONST_COMPRESSED = 4;
+  /** Compressed with sparse arrays. */
+  static final int SPARSE_COMPRESSED = 5;
+
+  /** Uncompressed binary, written directly (fixed length). */
+  static final int BINARY_FIXED_UNCOMPRESSED = 0;
+  /** Uncompressed binary, written directly (variable length). */
+  static final int BINARY_VARIABLE_UNCOMPRESSED = 1;
+  /** Compressed binary with shared prefixes */
+  static final int BINARY_PREFIX_COMPRESSED = 2;
+
+  /** Standard storage for sorted set values with 1 level of indirection:
+   *  {@code docId -> address -> ord}. */
+  static final int SORTED_WITH_ADDRESSES = 0;
+  /** Single-valued sorted set values, encoded as sorted values, so no level
+   *  of indirection: {@code docId -> ord}. */
+  static final int SORTED_SINGLE_VALUED = 1;
+  /** Compressed giving IDs to unique sets of values:
+   * {@code docId -> setId -> ords} */
+  static final int SORTED_SET_TABLE = 2;
+  
+  /** placeholder for missing offset that means there are no missing values */
+  static final int ALL_LIVE = -1;
+  /** placeholder for missing offset that means all values are missing */
+  static final int ALL_MISSING = -2;
+  
+  // addressing uses 16k blocks
+  static final int MONOTONIC_BLOCK_SIZE = 16384;
+  static final int DIRECT_MONOTONIC_BLOCK_SHIFT = 16;
+}