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/17 08:11:11 UTC

[3/3] lucene-solr:master: LUCENE-7489: Better sparsity support for Lucene70DocValuesFormat.

LUCENE-7489: Better sparsity support for Lucene70DocValuesFormat.


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

Branch: refs/heads/master
Commit: 927fd51d64a6e72843018786daea855847416487
Parents: a4a314d
Author: Adrien Grand <jp...@gmail.com>
Authored: Wed Oct 12 12:55:16 2016 +0200
Committer: Adrien Grand <jp...@gmail.com>
Committed: Mon Oct 17 09:28:20 2016 +0200

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |    3 +
 .../lucene54/Lucene54DocValuesProducer.java     |   14 +-
 .../apache/lucene/codecs/DocValuesConsumer.java |   45 +-
 .../lucene70/Lucene70DocValuesConsumer.java     | 1001 +++-----
 .../lucene70/Lucene70DocValuesFormat.java       |  123 +-
 .../lucene70/Lucene70DocValuesProducer.java     | 2229 ++++++------------
 .../lucene/index/EmptyDocValuesProducer.java    |    4 +-
 .../lucene/index/LegacySortedSetDocValues.java  |    4 +-
 .../index/SingletonSortedSetDocValues.java      |    2 +-
 .../apache/lucene/index/SortedDocValues.java    |    2 +-
 .../apache/lucene/index/SortedSetDocValues.java |    2 +-
 .../lucene/search/DocValuesRewriteMethod.java   |    2 +-
 .../util/packed/DirectMonotonicReader.java      |    2 -
 .../lucene70/TestLucene70DocValuesFormat.java   |  116 +-
 14 files changed, 1240 insertions(+), 2309 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/927fd51d/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 1981c52..54329ab 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -29,6 +29,9 @@ Bug Fixes
 
 Improvements
 
+* LUCENE-7489: Better storage of sparse doc-values fields with the default
+  codec. (Adrien Grand)
+
 Optimizations
 
 * LUCENE-7416: BooleanQuery optimizes queries that have queries that occur both

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/927fd51d/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesProducer.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesProducer.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesProducer.java
index 25f4b5e..8a44c31 100644
--- a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesProducer.java
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesProducer.java
@@ -928,7 +928,7 @@ final class Lucene54DocValuesProducer extends DocValuesProducer implements Close
       }
 
       @Override
-      public TermsEnum termsEnum() {
+      public TermsEnum termsEnum() throws IOException {
         if (binary instanceof CompressedBinaryDocValues) {
           return ((CompressedBinaryDocValues)binary).getTermsEnum();
         } else {
@@ -1233,7 +1233,7 @@ final class Lucene54DocValuesProducer extends DocValuesProducer implements Close
       }
 
       @Override
-      public TermsEnum termsEnum() {
+      public TermsEnum termsEnum() throws IOException {
         if (binary instanceof CompressedBinaryDocValues) {
           return ((CompressedBinaryDocValues)binary).getTermsEnum();
         } else {
@@ -1292,7 +1292,7 @@ final class Lucene54DocValuesProducer extends DocValuesProducer implements Close
       }
 
       @Override
-      public TermsEnum termsEnum() {
+      public TermsEnum termsEnum() throws IOException {
         if (binary instanceof CompressedBinaryDocValues) {
           return ((CompressedBinaryDocValues) binary).getTermsEnum();
         } else {
@@ -1490,12 +1490,8 @@ final class Lucene54DocValuesProducer extends DocValuesProducer implements Close
       }
     }
 
-    TermsEnum getTermsEnum() {
-      try {
-        return getTermsEnum(data.clone());
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
+    TermsEnum getTermsEnum() throws IOException {
+      return getTermsEnum(data.clone());
     }
 
     private CompressedBinaryTermsEnum getTermsEnum(IndexInput input) throws IOException {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/927fd51d/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java b/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java
index a07cc3d..e04d5b9 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java
@@ -361,7 +361,7 @@ public abstract class DocValuesConsumer implements Closeable {
     addSortedNumericField(mergeFieldInfo,
                           new EmptyDocValuesProducer() {
                             @Override
-                            public SortedNumericDocValues getSortedNumeric(FieldInfo fieldInfo) {
+                            public SortedNumericDocValues getSortedNumeric(FieldInfo fieldInfo) throws IOException {
                               if (fieldInfo != mergeFieldInfo) {
                                 throw new IllegalArgumentException("wrong FieldInfo");
                               }
@@ -375,11 +375,7 @@ public abstract class DocValuesConsumer implements Closeable {
                                 if (docValuesProducer != null) {
                                   FieldInfo readerFieldInfo = mergeState.fieldInfos[i].fieldInfo(mergeFieldInfo.name);
                                   if (readerFieldInfo != null && readerFieldInfo.getDocValuesType() == DocValuesType.SORTED_NUMERIC) {
-                                    try {
-                                      values = docValuesProducer.getSortedNumeric(readerFieldInfo);
-                                    } catch (IOException ioe) {
-                                      throw new RuntimeException(ioe);
-                                    }
+                                    values = docValuesProducer.getSortedNumeric(readerFieldInfo);
                                   }
                                 }
                                 if (values == null) {
@@ -391,12 +387,7 @@ public abstract class DocValuesConsumer implements Closeable {
 
                               final long finalCost = cost;
 
-                              final DocIDMerger<SortedNumericDocValuesSub> docIDMerger;
-                              try {
-                                docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
-                              } catch (IOException ioe) {
-                                throw new RuntimeException(ioe);
-                              }
+                              final DocIDMerger<SortedNumericDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
 
                               return new SortedNumericDocValues() {
 
@@ -521,7 +512,7 @@ public abstract class DocValuesConsumer implements Closeable {
     addSortedField(fieldInfo,
                    new EmptyDocValuesProducer() {
                      @Override
-                     public SortedDocValues getSorted(FieldInfo fieldInfoIn) {
+                     public SortedDocValues getSorted(FieldInfo fieldInfoIn) throws IOException {
                        if (fieldInfoIn != fieldInfo) {
                          throw new IllegalArgumentException("wrong FieldInfo");
                        }
@@ -536,11 +527,7 @@ public abstract class DocValuesConsumer implements Closeable {
                          if (docValuesProducer != null) {
                            FieldInfo readerFieldInfo = mergeState.fieldInfos[i].fieldInfo(fieldInfo.name);
                            if (readerFieldInfo != null && readerFieldInfo.getDocValuesType() == DocValuesType.SORTED) {
-                             try {
-                               values = docValuesProducer.getSorted(readerFieldInfo);
-                             } catch (IOException ioe) {
-                               throw new RuntimeException(ioe);
-                             }
+                             values = docValuesProducer.getSorted(readerFieldInfo);
                            }
                          }
                          if (values == null) {
@@ -553,12 +540,7 @@ public abstract class DocValuesConsumer implements Closeable {
 
                        final long finalCost = cost;
 
-                       final DocIDMerger<SortedDocValuesSub> docIDMerger;
-                       try {
-                         docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
-                       } catch (IOException ioe) {
-                         throw new RuntimeException(ioe);
-                       }
+                       final DocIDMerger<SortedDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
                        
                        return new SortedDocValues() {
                          private int docID = -1;
@@ -693,7 +675,7 @@ public abstract class DocValuesConsumer implements Closeable {
     addSortedSetField(mergeFieldInfo,
                       new EmptyDocValuesProducer() {
                         @Override
-                        public SortedSetDocValues getSortedSet(FieldInfo fieldInfo) {
+                        public SortedSetDocValues getSortedSet(FieldInfo fieldInfo) throws IOException {
                           if (fieldInfo != mergeFieldInfo) {
                             throw new IllegalArgumentException("wrong FieldInfo");
                           }
@@ -709,11 +691,7 @@ public abstract class DocValuesConsumer implements Closeable {
                             if (docValuesProducer != null) {
                               FieldInfo readerFieldInfo = mergeState.fieldInfos[i].fieldInfo(mergeFieldInfo.name);
                               if (readerFieldInfo != null && readerFieldInfo.getDocValuesType() == DocValuesType.SORTED_SET) {
-                                try {
-                                  values = docValuesProducer.getSortedSet(readerFieldInfo);
-                                } catch (IOException ioe) {
-                                  throw new RuntimeException(ioe);
-                                }
+                                values = docValuesProducer.getSortedSet(readerFieldInfo);
                               }
                             }
                             if (values == null) {
@@ -723,12 +701,7 @@ public abstract class DocValuesConsumer implements Closeable {
                             subs.add(new SortedSetDocValuesSub(mergeState.docMaps[i], values, map.getGlobalOrds(i)));
                           }
             
-                          final DocIDMerger<SortedSetDocValuesSub> docIDMerger;
-                          try {
-                            docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
-                          } catch (IOException ioe) {
-                            throw new RuntimeException(ioe);
-                          }
+                          final DocIDMerger<SortedSetDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
                           
                           final long finalCost = cost;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/927fd51d/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
index 564db7c..e1b66e1 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesConsumer.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesConsumer.java
@@ -17,56 +17,47 @@
 package org.apache.lucene.codecs.lucene70;
 
 
+import static org.apache.lucene.codecs.lucene70.Lucene70DocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT;
+
 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.BinaryDocValues;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.EmptyDocValuesProducer;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.index.SortedNumericDocValues;
+import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.SortedSetSelector;
 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;
@@ -85,37 +76,53 @@ final class Lucene70DocValuesConsumer extends DocValuesConsumer implements Close
       }
     }
   }
-  
+
+  @Override
+  public void close() throws IOException {
+    boolean success = false;
+    try {
+      if (meta != null) {
+        meta.writeInt(-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;
+    }
+  }
+
   @Override
   public void addNumericField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
-    addNumericField(field, LegacyDocValuesIterables.numericIterable(field, valuesProducer, maxDoc), NumberType.VALUE);
+    meta.writeInt(field.number);
+    meta.writeByte(Lucene70DocValuesFormat.NUMERIC);
+
+    writeValues(field, new EmptyDocValuesProducer() {
+      @Override
+      public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException {
+        return DocValues.singleton(valuesProducer.getNumeric(field));
+      }
+    });
   }
 
-  void addNumericField(FieldInfo field, Iterable<Number> values, NumberType numberType) throws IOException {
-    long count = 0;
-    long minValue = Long.MAX_VALUE;
-    long maxValue = Long.MIN_VALUE;
+  private long[] writeValues(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
+    SortedNumericDocValues values = valuesProducer.getSortedNumeric(field);
+    int numDocsWithValue = 0;
+    long numValues = 0;
+    long min = Long.MAX_VALUE;
+    long max = 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++;
-          }
-        }
+    Set<Long> uniqueValues = new HashSet<>();
+    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
+      for (int i = 0, count = values.docValueCount(); i < count; ++i) {
+        long v = values.nextValue();
 
         if (gcd != 1) {
           if (v < Long.MIN_VALUE / 2 || v > Long.MAX_VALUE / 2) {
@@ -123,675 +130,393 @@ final class Lucene70DocValuesConsumer extends DocValuesConsumer implements Close
             // 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);
+          } else if (numValues != 0) { // minValue needs to be set first
+            gcd = MathUtil.gcd(gcd, v - min);
           }
         }
 
-        minValue = Math.min(minValue, v);
-        maxValue = Math.max(maxValue, v);
+        min = Math.min(min, v);
+        max = Math.max(max, v);
 
-        if (uniqueValues != null) {
-          if (uniqueValues.add(v)) {
-            if (uniqueValues.size() > 256) {
-              uniqueValues = null;
-            }
-          }
+        if (uniqueValues != null
+            && uniqueValues.add(v)
+            && 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;
+        numValues++;
       }
-    }
-    
-    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();
+
+      numDocsWithValue++;
     }
 
-    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;
+    if (numDocsWithValue == 0) {
+      meta.writeLong(-2);
+      meta.writeLong(0L);
+    } else if (numDocsWithValue == maxDoc) {
+      meta.writeLong(-1);
+      meta.writeLong(0L);
     } else {
-      format = DELTA_COMPRESSED;
+      long offset = data.getFilePointer();
+      meta.writeLong(offset);
+      values = valuesProducer.getSortedNumeric(field);
+      IndexedDISI.writeBitSet(values, data);
+      meta.writeLong(data.getFilePointer() - offset);
     }
-    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);
+
+    meta.writeLong(numValues);
+    final int numBitsPerValue;
+    Map<Long, Integer> encode = null;
+    if (min >= max) {
+      numBitsPerValue = 0;
+      meta.writeInt(-1);
     } 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);
+      if (uniqueValues != null
+          && uniqueValues.size() > 1
+          && DirectWriter.unsignedBitsRequired(uniqueValues.size() - 1) < DirectWriter.unsignedBitsRequired((max - min) / gcd)) {
+        numBitsPerValue = DirectWriter.unsignedBitsRequired(uniqueValues.size() - 1);
+        final Long[] sortedUniqueValues = uniqueValues.toArray(new Long[0]);
+        Arrays.sort(sortedUniqueValues);
+        meta.writeInt(sortedUniqueValues.length);
+        for (Long v : sortedUniqueValues) {
+          meta.writeLong(v);
         }
-        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);
+        encode = new HashMap<>();
+        for (int i = 0; i < sortedUniqueValues.length; ++i) {
+          encode.put(sortedUniqueValues[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();
+        min = 0;
+        gcd = 1;
+      } else {
+        uniqueValues = null;
+        numBitsPerValue = DirectWriter.unsignedBitsRequired((max - min) / gcd);
+        if (gcd == 1 && min > 0
+            && DirectWriter.unsignedBitsRequired(max) == DirectWriter.unsignedBitsRequired(max - min)) {
+          min = 0;
         }
-        // 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;
+        meta.writeInt(-1);
       }
-      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);
+    meta.writeByte((byte) numBitsPerValue);
+    meta.writeLong(min);
+    meta.writeLong(gcd);
+    long startOffset = data.getFilePointer();
+    meta.writeLong(startOffset);
+    if (numBitsPerValue != 0) {
+      values = valuesProducer.getSortedNumeric(field);
+      DirectWriter writer = DirectWriter.getInstance(data, numValues, numBitsPerValue);
+      for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
+        for (int i = 0, count = values.docValueCount(); i < count; ++i) {
+          long v = values.nextValue();
+          if (encode == null) {
+            writer.add((v - min) / gcd);
+          } else {
+            writer.add(encode.get(v));
           }
-          break;
-        default:
-          throw new AssertionError();
+        }
       }
-      docID++;
+      writer.finish();
     }
-    docIdsWriter.finish();
-    return docID;
+    meta.writeLong(data.getFilePointer() - startOffset);
+
+    return new long[] {numDocsWithValue, numValues};
   }
 
   @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.writeInt(field.number);
     meta.writeByte(Lucene70DocValuesFormat.BINARY);
+
+    BinaryDocValues values = valuesProducer.getBinary(field);
+    long start = data.getFilePointer();
+    meta.writeLong(start);
+    int numDocsWithField = 0;
     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++;
+    int maxLength = 0;
+    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
+      numDocsWithField++;
+      BytesRef v = values.binaryValue();
+      int length = v.length;
+      data.writeBytes(v.bytes, v.offset, v.length);
+      minLength = Math.min(length, minLength);
+      maxLength = Math.max(length, maxLength);
     }
-    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);
+    assert numDocsWithField <= maxDoc;
+    meta.writeLong(data.getFilePointer() - start);
+
+    if (numDocsWithField == 0) {
+      meta.writeLong(-2);
+      meta.writeLong(0L);
+    } else if (numDocsWithField == maxDoc) {
+      meta.writeLong(-1);
+      meta.writeLong(0L);
     } else {
-      meta.writeLong(data.getFilePointer());
-      writeMissingBitset(values);
+      long offset = data.getFilePointer();
+      meta.writeLong(offset);
+      values = valuesProducer.getBinary(field);
+      IndexedDISI.writeBitSet(values, data);
+      meta.writeLong(data.getFilePointer() - offset);
     }
-    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.writeInt(numDocsWithField);
+    meta.writeInt(minLength);
+    meta.writeInt(maxLength);
+    if (maxLength > minLength) {
+      start = data.getFilePointer();
+      meta.writeLong(start);
       meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
 
-      final DirectMonotonicWriter writer = DirectMonotonicWriter.getInstance(meta, data, count + 1, DIRECT_MONOTONIC_BLOCK_SHIFT);
+      final DirectMonotonicWriter writer = DirectMonotonicWriter.getInstance(meta, data, numDocsWithField + 1, DIRECT_MONOTONIC_BLOCK_SHIFT);
       long addr = 0;
       writer.add(addr);
-      for (BytesRef v : values) {
-        if (v != null) {
-          addr += v.length;
-        }
+      values = valuesProducer.getBinary(field);
+      for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
+        addr += values.binaryValue().length;
         writer.add(addr);
       }
       writer.finish();
-      meta.writeLong(data.getFilePointer());
+      meta.writeLong(data.getFilePointer() - start);
     }
   }
-  
-  /** 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++;
+
+  @Override
+  public void addSortedField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
+    meta.writeInt(field.number);
+    meta.writeByte(Lucene70DocValuesFormat.SORTED);
+    doAddSortedField(field, valuesProducer);
+  }
+
+  private void doAddSortedField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
+    SortedDocValues values = valuesProducer.getSorted(field);
+    int numDocsWithField = 0;
+    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
+      numDocsWithField++;
     }
-    // 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);
+
+    if (numDocsWithField == 0) {
+      meta.writeLong(-2);
+      meta.writeLong(0L);
+    } else if (numDocsWithField == maxDoc) {
+      meta.writeLong(-1);
+      meta.writeLong(0L);
     } 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;
-      }
+      long offset = data.getFilePointer();
+      meta.writeLong(offset);
+      values = valuesProducer.getSorted(field);
+      IndexedDISI.writeBitSet(values, data);
+      meta.writeLong(data.getFilePointer() - offset);
     }
-    if (twoByte) {
-      headerBuffer.writeByte((byte)255);
-      for (int i = 1; i < suffixDeltas.length; i++) {
-        headerBuffer.writeShort((short) suffixDeltas[i]);
-      }
+
+    meta.writeInt(numDocsWithField);
+    if (values.getValueCount() <= 1) {
+      meta.writeByte((byte) 0);
+      meta.writeLong(0L);
+      meta.writeLong(0L);
     } else {
-      for (int i = 1; i < suffixDeltas.length; i++) {
-        headerBuffer.writeByte((byte) suffixDeltas[i]);
+      int numberOfBitsPerOrd = DirectWriter.unsignedBitsRequired(values.getValueCount() - 1);
+      meta.writeByte((byte) numberOfBitsPerOrd);
+      long start = data.getFilePointer();
+      meta.writeLong(start);
+      DirectWriter writer = DirectWriter.getInstance(data, numDocsWithField, numberOfBitsPerOrd);
+      values = valuesProducer.getSorted(field);
+      for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
+        writer.add(values.ordValue());
       }
+      writer.finish();
+      meta.writeLong(data.getFilePointer() - start);
     }
-    headerBuffer.writeTo(data);
-    headerBuffer.reset();
-    bytesBuffer.writeTo(data);
-    bytesBuffer.reset();
+
+    addTermsDict(DocValues.singleton(valuesProducer.getSorted(field)));
   }
-  
-  // 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);
+
+  private void addTermsDict(SortedSetDocValues values) throws IOException {
+    final long size = values.getValueCount();
+    meta.writeVLong(size);
+    meta.writeInt(Lucene70DocValuesFormat.TERMS_DICT_BLOCK_SHIFT);
+
+    RAMOutputStream addressBuffer = new RAMOutputStream();
+    meta.writeInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
+    long numBlocks = (size + Lucene70DocValuesFormat.TERMS_DICT_BLOCK_MASK) >>> Lucene70DocValuesFormat.TERMS_DICT_BLOCK_SHIFT;
+    DirectMonotonicWriter writer = DirectMonotonicWriter.getInstance(meta, addressBuffer, numBlocks, DIRECT_MONOTONIC_BLOCK_SHIFT);
+
+    BytesRefBuilder previous = new BytesRefBuilder();
+    long ord = 0;
+    long start = data.getFilePointer();
+    int maxLength = 0;
+    TermsEnum iterator = values.termsEnum();
+    for (BytesRef term = iterator.next(); term != null; term = iterator.next()) {
+      if ((ord & Lucene70DocValuesFormat.TERMS_DICT_BLOCK_MASK) == 0) {
+        writer.add(data.getFilePointer() - start);
+        data.writeVInt(term.length);
+        data.writeBytes(term.bytes, term.offset, term.length);
+      } else {
+        final int prefixLength = StringHelper.bytesDifference(previous.get(), term);
+        final int suffixLength = term.length - prefixLength;
+        assert suffixLength > 0; // terms are unique
+
+        data.writeByte((byte) (Math.min(prefixLength, 15) | (Math.min(15, suffixLength - 1) << 4)));
+        if (prefixLength >= 15) {
+          data.writeVInt(prefixLength - 15);
+        }
+        if (suffixLength >= 16) {
+          data.writeVInt(suffixLength - 16);
+        }
+        data.writeBytes(term.bytes, term.offset + prefixLength, term.length - prefixLength);
       }
-      count++;
+      maxLength = Math.max(maxLength, term.length);
+      previous.copyBytes(term);
+      ++ord;
     }
-    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);
+    writer.finish();
+    meta.writeInt(maxLength);
+    meta.writeLong(start);
+    meta.writeLong(data.getFilePointer() - start);
+    start = data.getFilePointer();
+    addressBuffer.writeTo(data);
+    meta.writeLong(start);
+    meta.writeLong(data.getFilePointer() - start);
+
+    // Now write the reverse terms index
+    writeTermsIndex(values);
   }
 
-  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);
+  private void writeTermsIndex(SortedSetDocValues values) throws IOException {
+    final long size = values.getValueCount();
+    meta.writeInt(Lucene70DocValuesFormat.TERMS_DICT_REVERSE_INDEX_SHIFT);
+    long start = data.getFilePointer();
+
+    long numBlocks = 1L + ((size + Lucene70DocValuesFormat.TERMS_DICT_REVERSE_INDEX_MASK) >>> Lucene70DocValuesFormat.TERMS_DICT_REVERSE_INDEX_SHIFT);
+    RAMOutputStream addressBuffer = new RAMOutputStream();
+    DirectMonotonicWriter writer = DirectMonotonicWriter.getInstance(meta, addressBuffer, numBlocks, DIRECT_MONOTONIC_BLOCK_SHIFT);
+
+    TermsEnum iterator = values.termsEnum();
+    BytesRefBuilder previous = new BytesRefBuilder();
+    long offset = 0;
+    long ord = 0;
+    for (BytesRef term = iterator.next(); term != null; term = iterator.next()) {
+      if ((ord & Lucene70DocValuesFormat.TERMS_DICT_REVERSE_INDEX_MASK) == 0) {
+        writer.add(offset);
+        int sortKeyLength = StringHelper.sortKeyLength(previous.get(), term);
+        offset += sortKeyLength;
+        data.writeBytes(term.bytes, term.offset, sortKeyLength);
+      } else if ((ord & Lucene70DocValuesFormat.TERMS_DICT_REVERSE_INDEX_MASK) == Lucene70DocValuesFormat.TERMS_DICT_REVERSE_INDEX_MASK) {
+        previous.copyBytes(term);
+      }
+      ++ord;
+    }
+    writer.add(offset);
+    writer.finish();
+    meta.writeLong(start);
+    meta.writeLong(data.getFilePointer() - start);
+    start = data.getFilePointer();
+    addressBuffer.writeTo(data);
+    meta.writeLong(start);
+    meta.writeLong(data.getFilePointer() - start);
   }
 
   @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);
+  public void addSortedNumericField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
+    meta.writeInt(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);
+    long[] stats = writeValues(field, valuesProducer);
+    int numDocsWithField = Math.toIntExact(stats[0]);
+    long numValues = stats[1];
+    assert numValues >= numDocsWithField;
 
-        // 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);
+    meta.writeInt(numDocsWithField);
+    if (numValues > numDocsWithField) {
+      long start = data.getFilePointer();
+      meta.writeLong(start);
+      meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
+
+      final DirectMonotonicWriter addressesWriter = DirectMonotonicWriter.getInstance(meta, data, numDocsWithField + 1L, DIRECT_MONOTONIC_BLOCK_SHIFT);
+      long addr = 0;
+      addressesWriter.add(addr);
+      SortedNumericDocValues values = valuesProducer.getSortedNumeric(field);
+      for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
+        addr += values.docValueCount();
+        addressesWriter.add(addr);
       }
+      addressesWriter.finish();
+      meta.writeLong(data.getFilePointer() - start);
     }
   }
 
   @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.writeInt(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);
+    SortedSetDocValues values = valuesProducer.getSortedSet(field);
+    int numDocsWithField = 0;
+    long numOrds = 0;
+    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
+      numDocsWithField++;
+      for (long ord = values.nextOrd(); ord != SortedSetDocValues.NO_MORE_ORDS; ord = values.nextOrd()) {
+        numOrds++;
       }
     }
-  }
 
-  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));
+    if (numDocsWithField == numOrds) {
+      meta.writeByte((byte) 0);
+      doAddSortedField(field, new EmptyDocValuesProducer() {
+        @Override
+        public SortedDocValues getSorted(FieldInfo field) throws IOException {
+          return SortedSetSelector.wrap(valuesProducer.getSortedSet(field), SortedSetSelector.Type.MIN);
+        }
+      });
+      return;
     }
-    assert valueIterator.hasNext() == false;
-    return new TreeSet<>(uniqueValueSet);
-  }
+    meta.writeByte((byte) 1);
 
-  private void writeDictionary(SortedSet<LongsRef> uniqueValueSets) throws IOException {
-    int lengthSum = 0;
-    for (LongsRef longs : uniqueValueSets) {
-      lengthSum += longs.length;
+    assert numDocsWithField != 0;
+    if (numDocsWithField == maxDoc) {
+      meta.writeLong(-1);
+      meta.writeLong(0L);
+    } else {
+      long offset = data.getFilePointer();
+      meta.writeLong(offset);
+      values = valuesProducer.getSortedSet(field);
+      IndexedDISI.writeBitSet(values, data);
+      meta.writeLong(data.getFilePointer() - offset);
     }
 
-    meta.writeInt(lengthSum);
-    for (LongsRef valueSet : uniqueValueSets) {
-      for (int  i = 0; i < valueSet.length; ++i) {
-        meta.writeLong(valueSet.longs[valueSet.offset + i]);
+    int numberOfBitsPerOrd = DirectWriter.unsignedBitsRequired(values.getValueCount() - 1);
+    meta.writeByte((byte) numberOfBitsPerOrd);
+    long start = data.getFilePointer();
+    meta.writeLong(start);
+    DirectWriter writer = DirectWriter.getInstance(data, numOrds, numberOfBitsPerOrd);
+    values = valuesProducer.getSortedSet(field);
+    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
+      for (long ord = values.nextOrd(); ord != SortedSetDocValues.NO_MORE_ORDS; ord = values.nextOrd()) {
+        writer.add(ord);
       }
     }
+    writer.finish();
+    meta.writeLong(data.getFilePointer() - start);
 
-    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.writeInt(numDocsWithField);
+    start = data.getFilePointer();
+    meta.writeLong(start);
     meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
 
-    final DirectMonotonicWriter writer = DirectMonotonicWriter.getInstance(meta, data, maxDoc + 1, DIRECT_MONOTONIC_BLOCK_SHIFT);
+    final DirectMonotonicWriter addressesWriter = DirectMonotonicWriter.getInstance(meta, data, numDocsWithField + 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);
+    addressesWriter.add(addr);
+    values = valuesProducer.getSortedSet(field);
+    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
+      values.nextOrd();
+      addr++;
+      while (values.nextOrd() != SortedSetDocValues.NO_MORE_ORDS) {
+        addr++;
       }
-      meta = data = null;
+      addressesWriter.add(addr);
     }
+    addressesWriter.finish();
+    meta.writeLong(data.getFilePointer() - start);
+
+    addTermsDict(values);
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/927fd51d/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
index 2112341..ee477d6 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesFormat.java
@@ -23,39 +23,64 @@ 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.IndexWriterConfig;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.store.DataOutput;
 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:
+ * Documents that have a value for the field are encoded in a way that it is always possible to
+ * know the ordinal of the current document in the set of documents that have a value. For instance,
+ * say the set of documents that have a value for the field is <tt>{1, 5, 6, 11}</tt>. When the
+ * iterator is on <tt>6</tt>, it knows that this is the 3rd item of the set. This way, values can
+ * be stored densely and accessed based on their index at search time. If all documents in a segment
+ * have a value for the field, the index is the same as the doc ID, so this case is encoded implicitly
+ * and is very fast at query time. On the other hand if some documents are missing a value for the
+ * field then the set of documents that have a value is encoded into blocks. All doc IDs that share
+ * the same upper 16 bits are encoded into the same block with the following strategies:
+ * <ul>
+ *     <li>SPARSE: This strategy is used when a block contains at most 4095 documents. The lower 16
+ *         bits of doc IDs are stored as {@link DataOutput#writeShort(short) shorts} while the upper
+ *         16 bits are given by the block ID.
+ *     <li>DENSE: This strategy is used when a block contains between 4096 and 65535 documents. The
+ *         lower bits of doc IDs are stored in a bit set. Advancing is performed using
+ *         {@link Long#numberOfTrailingZeros(long) ntz} operations while the index is computed by
+ *         accumulating the {@link Long#bitCount(long) bit counts} of the visited longs.
+ *     <li>ALL: This strategy is used when a block contains exactly 65536 documents, meaning that
+ *         the block is full. In that case doc IDs do not need to be stored explicitly. This is
+ *         typically faster than both SPARSE and DENSE which is a reason why it is preferable to have
+ *         all documents that have a value for a field using contiguous doc IDs, for instance by
+ *         using {@link IndexWriterConfig#setIndexSort(org.apache.lucene.search.Sort) index sorting}.
+ * </ul>
+ * <p>
+ * Then the five per-document value types (Numeric,Binary,Sorted,SortedSet,SortedNumeric) are
+ * encoded using the following 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}). 
+ *        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.
+ *    <li>Const-compressed: when there is only one possible value, no per-document data is needed and
+ *        this value is encoded alone.
  * </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 
+ *        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
@@ -64,27 +89,21 @@ import org.apache.lucene.util.packed.DirectWriter;
  * <p>
  * {@link DocValuesType#SORTED SORTED}:
  * <ul>
- *    <li>Sorted: a mapping of ordinals to deduplicated terms is written as Binary, 
+ *    <li>Sorted: a mapping of ordinals to deduplicated terms is written as Prefix-compressed 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 
+ *    <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>
@@ -112,72 +131,30 @@ public final class Lucene70DocValuesFormat extends DocValuesFormat {
   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_CODEC = "Lucene70DocValuesData";
   static final String DATA_EXTENSION = "dvd";
-  static final String META_CODEC = "Lucene54DocValuesMetadata";
+  static final String META_CODEC = "Lucene70DocValuesMetadata";
   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;
+
+  static final int TERMS_DICT_BLOCK_SHIFT = 4;
+  static final int TERMS_DICT_BLOCK_SIZE = 1 << TERMS_DICT_BLOCK_SHIFT;
+  static final int TERMS_DICT_BLOCK_MASK = TERMS_DICT_BLOCK_SIZE - 1;
+
+  static final int TERMS_DICT_REVERSE_INDEX_SHIFT = 10;
+  static final int TERMS_DICT_REVERSE_INDEX_SIZE = 1 << TERMS_DICT_REVERSE_INDEX_SHIFT;
+  static final int TERMS_DICT_REVERSE_INDEX_MASK = TERMS_DICT_REVERSE_INDEX_SIZE - 1;
 }