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 2015/10/27 20:49:47 UTC

svn commit: r1710876 [1/4] - in /lucene/dev/trunk: lucene/ lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/ lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/ lucene/backward-codecs/src/resources/META-INF/services/ luc...

Author: jpountz
Date: Tue Oct 27 19:49:47 2015
New Revision: 1710876

URL: http://svn.apache.org/viewvc?rev=1710876&view=rev
Log:
LUCENE-6840: Put ord indexes on disk.

Added:
    lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesConsumer.java   (with props)
    lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesFormat.java   (with props)
    lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesProducer.java   (with props)
    lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/
    lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/Lucene53Codec.java   (with props)
    lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/package.html   (with props)
    lucene/dev/trunk/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene50/TestLucene50DocValuesFormat.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene54/
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54Codec.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesConsumer.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesFormat.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesProducer.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene54/package-info.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/DirectMonotonicReader.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/DirectMonotonicWriter.java   (with props)
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene54/
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene54/TestLucene54DocValuesFormat.java   (with props)
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/packed/TestDirectMonotonic.java   (with props)
Removed:
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesConsumer.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesFormat.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesProducer.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene53/Lucene53Codec.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50DocValuesFormat.java
Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
    lucene/dev/trunk/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
    lucene/dev/trunk/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/Codec.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50StoredFieldsFormat.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene53/package-info.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/DirectReader.java
    lucene/dev/trunk/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
    lucene/dev/trunk/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50StoredFieldsFormatHighCompression.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene53/TestLucene53NormsFormat.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/packed/TestDirectPacked.java
    lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDTreeDocValuesFormat.java
    lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/rangetree/RangeTreeDocValuesFormat.java
    lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/bkdtree/TestBKDTree.java
    lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/rangetree/TestRangeTree.java
    lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DDocValuesFormat.java
    lucene/dev/trunk/lucene/spatial3d/src/test/org/apache/lucene/bkdtree3d/TestGeo3DPointField.java
    lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java
    lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/schema_codec.xml

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1710876&r1=1710875&r2=1710876&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Tue Oct 27 19:49:47 2015
@@ -162,6 +162,10 @@ Optimizations
 
 * LUCENE-6850: Optimize BooleanScorer for sparse clauses. (Adrien Grand)
 
+* LUCENE-6840: Ordinal indexes for SORTED_SET/SORTED_NUMERIC fields and
+  addresses for BINARY fields are now stored on disk instead of in memory.
+  (Adrien Grand)
+
 Bug Fixes
 
 * LUCENE-6817: ComplexPhraseQueryParser.ComplexPhraseQuery does not display 

Added: lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesConsumer.java?rev=1710876&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesConsumer.java (added)
+++ lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesConsumer.java Tue Oct 27 19:49:47 2015
@@ -0,0 +1,658 @@
+package org.apache.lucene.codecs.lucene50;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.Closeable; // javadocs
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.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 org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.DocValuesConsumer;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.RAMOutputStream;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LongsRef;
+import org.apache.lucene.util.MathUtil;
+import org.apache.lucene.util.PagedBytes;
+import org.apache.lucene.util.PagedBytes.PagedBytesDataInput;
+import org.apache.lucene.util.StringHelper;
+import org.apache.lucene.util.packed.DirectWriter;
+import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
+import org.apache.lucene.util.packed.PackedInts;
+
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesFormat.*;
+
+/** writer for {@link Lucene50DocValuesFormat} */
+class Lucene50DocValuesConsumer extends DocValuesConsumer implements Closeable {
+
+  IndexOutput data, meta;
+  final int maxDoc;
+  
+  /** expert: Creates a new writer */
+  public Lucene50DocValuesConsumer(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, Lucene50DocValuesFormat.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, Lucene50DocValuesFormat.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, Iterable<Number> values) throws IOException {
+    addNumericField(field, values, true);
+  }
+
+  void addNumericField(FieldInfo field, Iterable<Number> values, boolean optimizeStorage) throws IOException {
+    long count = 0;
+    long minValue = Long.MAX_VALUE;
+    long maxValue = Long.MIN_VALUE;
+    long gcd = 0;
+    long missingCount = 0;
+    long zeroCount = 0;
+    // TODO: more efficient?
+    HashSet<Long> uniqueValues = null;
+    if (optimizeStorage) {
+      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();
+        minValue = Math.min(minValue, v);
+        maxValue = Math.max(maxValue, v);
+        ++count;
+      }
+    }
+    
+    final long delta = maxValue - minValue;
+    final int deltaBitsRequired = DirectWriter.unsignedBitsRequired(delta);
+    final int tableBitsRequired = uniqueValues == null
+        ? Integer.MAX_VALUE
+        : DirectWriter.bitsRequired(uniqueValues.size() - 1);
+
+    final int format;
+    if (uniqueValues != null 
+        && 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 (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(Lucene50DocValuesFormat.NUMERIC);
+    meta.writeVInt(format);
+    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;
+      default:
+        throw new AssertionError();
+    }
+    meta.writeLong(data.getFilePointer());
+  }
+  
+  // TODO: in some cases representing missing with minValue-1 wouldn't take up additional space and so on,
+  // but this is very simple, and algorithms only check this for values of 0 anyway (doesnt slow down normal decode)
+  void writeMissingBitset(Iterable<?> values) throws IOException {
+    byte bits = 0;
+    int count = 0;
+    for (Object v : values) {
+      if (count == 8) {
+        data.writeByte(bits);
+        count = 0;
+        bits = 0;
+      }
+      if (v != null) {
+        bits |= 1 << (count & 7);
+      }
+      count++;
+    }
+    if (count > 0) {
+      data.writeByte(bits);
+    }
+  }
+
+  @Override
+  public void addBinaryField(FieldInfo field, Iterable<BytesRef> values) throws IOException {
+    // write the byte[] data
+    meta.writeVInt(field.number);
+    meta.writeByte(Lucene50DocValuesFormat.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(PackedInts.VERSION_CURRENT);
+      meta.writeVInt(MONOTONIC_BLOCK_SIZE);
+
+      final MonotonicBlockPackedWriter writer = new MonotonicBlockPackedWriter(data, MONOTONIC_BLOCK_SIZE);
+      long addr = 0;
+      writer.add(addr);
+      for (BytesRef v : values) {
+        if (v != null) {
+          addr += v.length;
+        }
+        writer.add(addr);
+      }
+      writer.finish();
+    }
+  }
+  
+  /** expert: writes a value dictionary for a sorted/sortedset field */
+  private void addTermsDict(FieldInfo field, final Iterable<BytesRef> values) throws IOException {
+    // first check if it's a "fixed-length" terms dict
+    int minLength = Integer.MAX_VALUE;
+    int maxLength = Integer.MIN_VALUE;
+    long numValues = 0;
+    for (BytesRef v : values) {
+      minLength = Math.min(minLength, v.length);
+      maxLength = Math.max(maxLength, v.length);
+      numValues++;
+    }
+    if (minLength == maxLength) {
+      // no index needed: direct addressing by mult
+      addBinaryField(field, values);
+    } else if (numValues < REVERSE_INTERVAL_COUNT) {
+      // low cardinality: waste a few KB of ram, but can't really use fancy index etc
+      addBinaryField(field, values);
+    } else {
+      assert numValues > 0; // we don't have to handle the empty case
+      // header
+      meta.writeVInt(field.number);
+      meta.writeByte(Lucene50DocValuesFormat.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, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
+    meta.writeVInt(field.number);
+    meta.writeByte(Lucene50DocValuesFormat.SORTED);
+    addTermsDict(field, values);
+    addNumericField(field, docToOrd, false);
+  }
+
+  @Override
+  public void addSortedNumericField(FieldInfo field, final Iterable<Number> docToValueCount, final Iterable<Number> values) throws IOException {
+    meta.writeVInt(field.number);
+    meta.writeByte(Lucene50DocValuesFormat.SORTED_NUMERIC);
+    if (isSingleValued(docToValueCount)) {
+      meta.writeVInt(SORTED_SINGLE_VALUED);
+      // The field is single-valued, we can encode it as NUMERIC
+      addNumericField(field, singletonView(docToValueCount, values, null));
+    } else {
+      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), false);
+      } else {
+        meta.writeVInt(SORTED_WITH_ADDRESSES);
+        // write the stream of values as a numeric field
+        addNumericField(field, values, true);
+        // write the doc -> ord count as a absolute index to the stream
+        addAddresses(field, docToValueCount);
+      }
+    }
+  }
+
+  @Override
+  public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, final Iterable<Number> docToOrdCount, final Iterable<Number> ords) throws IOException {
+    meta.writeVInt(field.number);
+    meta.writeByte(Lucene50DocValuesFormat.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), false);
+      } else {
+        meta.writeVInt(SORTED_WITH_ADDRESSES);
+
+        // write the ord -> byte[] as a binary field
+        addTermsDict(field, values);
+
+        // write the stream of ords as a numeric field
+        // NOTE: we could return an iterator that delta-encodes these within a doc
+        addNumericField(field, ords, false);
+
+        // write the doc -> ord count as a absolute index to the stream
+        addAddresses(field, docToOrdCount);
+      }
+    }
+  }
+
+  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 addAddresses(FieldInfo field, Iterable<Number> values) throws IOException {
+    meta.writeVInt(field.number);
+    meta.writeByte(Lucene50DocValuesFormat.NUMERIC);
+    meta.writeVInt(MONOTONIC_COMPRESSED);
+    meta.writeLong(-1L);
+    meta.writeLong(data.getFilePointer());
+    meta.writeVLong(maxDoc);
+    meta.writeVInt(PackedInts.VERSION_CURRENT);
+    meta.writeVInt(MONOTONIC_BLOCK_SIZE);
+
+    final MonotonicBlockPackedWriter writer = new MonotonicBlockPackedWriter(data, MONOTONIC_BLOCK_SIZE);
+    long addr = 0;
+    writer.add(addr);
+    for (Number v : values) {
+      addr += v.longValue();
+      writer.add(addr);
+    }
+    writer.finish();
+    meta.writeLong(data.getFilePointer());
+  }
+
+  @Override
+  public void close() throws IOException {
+    boolean success = false;
+    try {
+      if (meta != null) {
+        meta.writeVInt(-1); // write EOF marker
+        CodecUtil.writeFooter(meta); // write checksum
+      }
+      if (data != null) {
+        CodecUtil.writeFooter(data); // write checksum
+      }
+      success = true;
+    } finally {
+      if (success) {
+        IOUtils.close(data, meta);
+      } else {
+        IOUtils.closeWhileHandlingException(data, meta);
+      }
+      meta = data = null;
+    }
+  }
+}

Added: lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesFormat.java?rev=1710876&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesFormat.java (added)
+++ lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesFormat.java Tue Oct 27 19:49:47 2015
@@ -0,0 +1,115 @@
+package org.apache.lucene.codecs.lucene50;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.DocValuesConsumer;
+import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.DocValuesProducer;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+
+/**
+ * Lucene 5.0 Doc values format.
+ * @deprecated Only for reading old 5.0-5.3 segments
+ */
+@Deprecated
+public class Lucene50DocValuesFormat extends DocValuesFormat {
+
+  /** Sole Constructor */
+  public Lucene50DocValuesFormat() {
+    super("Lucene50");
+  }
+
+  @Override
+  public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+    return new Lucene50DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
+  }
+
+  @Override
+  public DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException {
+    return new Lucene50DocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
+  }
+  
+  static final String DATA_CODEC = "Lucene50DocValuesData";
+  static final String DATA_EXTENSION = "dvd";
+  static final String META_CODEC = "Lucene50DocValuesMetadata";
+  static final String META_EXTENSION = "dvm";
+  static final int VERSION_START = 0;
+  static final int VERSION_SORTEDSET_TABLE = 1;
+  static final int VERSION_CURRENT = VERSION_SORTEDSET_TABLE;
+  
+  // 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;
+  
+  /** 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;
+}

Added: lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesProducer.java?rev=1710876&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesProducer.java (added)
+++ lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesProducer.java Tue Oct 27 19:49:47 2015
@@ -0,0 +1,1299 @@
+package org.apache.lucene.codecs.lucene50;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.DocValuesProducer;
+import org.apache.lucene.index.BinaryDocValues;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.PostingsEnum;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.RandomAccessOrds;
+import org.apache.lucene.index.SegmentReadState;
+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.store.ChecksumIndexInput;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.RandomAccessInput;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.Accountables;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LongValues;
+import org.apache.lucene.util.PagedBytes;
+import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.packed.DirectReader;
+import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
+
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesFormat.*;
+
+/** reader for {@link Lucene50DocValuesFormat} */
+class Lucene50DocValuesProducer extends DocValuesProducer implements Closeable {
+  private final Map<String,NumericEntry> numerics = new HashMap<>();
+  private final Map<String,BinaryEntry> binaries = new HashMap<>();
+  private final Map<String,SortedSetEntry> sortedSets = new HashMap<>();
+  private final Map<String,SortedSetEntry> sortedNumerics = new HashMap<>();
+  private final Map<String,NumericEntry> ords = new HashMap<>();
+  private final Map<String,NumericEntry> ordIndexes = new HashMap<>();
+  private final int numFields;
+  private final AtomicLong ramBytesUsed;
+  private final IndexInput data;
+  private final int maxDoc;
+
+  // memory-resident structures
+  private final Map<String,MonotonicBlockPackedReader> addressInstances = new HashMap<>();
+  private final Map<String,MonotonicBlockPackedReader> ordIndexInstances = new HashMap<>();
+  private final Map<String,ReverseTermsIndex> reverseIndexInstances = new HashMap<>();
+  
+  private final boolean merging;
+  
+  // clone for merge: when merging we don't do any instances.put()s
+  Lucene50DocValuesProducer(Lucene50DocValuesProducer original) throws IOException {
+    assert Thread.holdsLock(original);
+    numerics.putAll(original.numerics);
+    binaries.putAll(original.binaries);
+    sortedSets.putAll(original.sortedSets);
+    sortedNumerics.putAll(original.sortedNumerics);
+    ords.putAll(original.ords);
+    ordIndexes.putAll(original.ordIndexes);
+    numFields = original.numFields;
+    ramBytesUsed = new AtomicLong(original.ramBytesUsed.get());
+    data = original.data.clone();
+    maxDoc = original.maxDoc;
+    
+    addressInstances.putAll(original.addressInstances);
+    ordIndexInstances.putAll(original.ordIndexInstances);
+    reverseIndexInstances.putAll(original.reverseIndexInstances);
+    merging = true;
+  }
+  
+  /** expert: instantiates a new reader */
+  Lucene50DocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
+    String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
+    this.maxDoc = state.segmentInfo.maxDoc();
+    merging = false;
+    ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOfInstance(getClass()));
+    
+    int version = -1;
+    int numFields = -1;
+    
+    // read in the entries from the metadata file.
+    try (ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context)) {
+      Throwable priorE = null;
+      try {
+        version = CodecUtil.checkIndexHeader(in, metaCodec, 
+                                        Lucene50DocValuesFormat.VERSION_START,
+                                        Lucene50DocValuesFormat.VERSION_CURRENT,
+                                        state.segmentInfo.getId(),
+                                        state.segmentSuffix);
+        numFields = readFields(in, state.fieldInfos);
+      } catch (Throwable exception) {
+        priorE = exception;
+      } finally {
+        CodecUtil.checkFooter(in, priorE);
+      }
+    }
+    
+    this.numFields = numFields;
+    String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
+    this.data = state.directory.openInput(dataName, state.context);
+    boolean success = false;
+    try {
+      final int version2 = CodecUtil.checkIndexHeader(data, dataCodec, 
+                                                 Lucene50DocValuesFormat.VERSION_START,
+                                                 Lucene50DocValuesFormat.VERSION_CURRENT,
+                                                 state.segmentInfo.getId(),
+                                                 state.segmentSuffix);
+      if (version != version2) {
+        throw new CorruptIndexException("Format versions mismatch: meta=" + version + ", data=" + version2, data);
+      }
+      
+      // NOTE: data file is too costly to verify checksum against all the bytes on open,
+      // but for now we at least verify proper structure of the checksum footer: which looks
+      // for FOOTER_MAGIC + algorithmID. This is cheap and can detect some forms of corruption
+      // such as file truncation.
+      CodecUtil.retrieveChecksum(data);
+
+      success = true;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(this.data);
+      }
+    }
+  }
+
+  private void readSortedField(FieldInfo info, IndexInput meta) throws IOException {
+    // sorted = binary + numeric
+    if (meta.readVInt() != info.number) {
+      throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt", meta);
+    }
+    if (meta.readByte() != Lucene50DocValuesFormat.BINARY) {
+      throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt", meta);
+    }
+    BinaryEntry b = readBinaryEntry(meta);
+    binaries.put(info.name, b);
+    
+    if (meta.readVInt() != info.number) {
+      throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt", meta);
+    }
+    if (meta.readByte() != Lucene50DocValuesFormat.NUMERIC) {
+      throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt", meta);
+    }
+    NumericEntry n = readNumericEntry(meta);
+    ords.put(info.name, n);
+  }
+
+  private void readSortedSetFieldWithAddresses(FieldInfo info, IndexInput meta) throws IOException {
+    // sortedset = binary + numeric (addresses) + ordIndex
+    if (meta.readVInt() != info.number) {
+      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+    }
+    if (meta.readByte() != Lucene50DocValuesFormat.BINARY) {
+      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+    }
+    BinaryEntry b = readBinaryEntry(meta);
+    binaries.put(info.name, b);
+
+    if (meta.readVInt() != info.number) {
+      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+    }
+    if (meta.readByte() != Lucene50DocValuesFormat.NUMERIC) {
+      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+    }
+    NumericEntry n1 = readNumericEntry(meta);
+    ords.put(info.name, n1);
+
+    if (meta.readVInt() != info.number) {
+      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+    }
+    if (meta.readByte() != Lucene50DocValuesFormat.NUMERIC) {
+      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+    }
+    NumericEntry n2 = readNumericEntry(meta);
+    ordIndexes.put(info.name, n2);
+  }
+
+  private void readSortedSetFieldWithTable(FieldInfo info, IndexInput meta) throws IOException {
+    // sortedset table = binary + ordset table + ordset index
+    if (meta.readVInt() != info.number) {
+      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+    }
+    if (meta.readByte() != Lucene50DocValuesFormat.BINARY) {
+      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+    }
+
+    BinaryEntry b = readBinaryEntry(meta);
+    binaries.put(info.name, b);
+
+    if (meta.readVInt() != info.number) {
+      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+    }
+    if (meta.readByte() != Lucene50DocValuesFormat.NUMERIC) {
+      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+    }
+    NumericEntry n = readNumericEntry(meta);
+    ords.put(info.name, n);
+  }
+
+  private int readFields(IndexInput meta, FieldInfos infos) throws IOException {
+    int numFields = 0;
+    int fieldNumber = meta.readVInt();
+    while (fieldNumber != -1) {
+      numFields++;
+      FieldInfo info = infos.fieldInfo(fieldNumber);
+      if (info == null) {
+        // trickier to validate more: because we use multiple entries for "composite" types like sortedset, etc.
+        throw new CorruptIndexException("Invalid field number: " + fieldNumber, meta);
+      }
+      byte type = meta.readByte();
+      if (type == Lucene50DocValuesFormat.NUMERIC) {
+        numerics.put(info.name, readNumericEntry(meta));
+      } else if (type == Lucene50DocValuesFormat.BINARY) {
+        BinaryEntry b = readBinaryEntry(meta);
+        binaries.put(info.name, b);
+      } else if (type == Lucene50DocValuesFormat.SORTED) {
+        readSortedField(info, meta);
+      } else if (type == Lucene50DocValuesFormat.SORTED_SET) {
+        SortedSetEntry ss = readSortedSetEntry(meta);
+        sortedSets.put(info.name, ss);
+        if (ss.format == SORTED_WITH_ADDRESSES) {
+          readSortedSetFieldWithAddresses(info, meta);
+        } else if (ss.format == SORTED_SET_TABLE) {
+          readSortedSetFieldWithTable(info, meta);
+        } else if (ss.format == SORTED_SINGLE_VALUED) {
+          if (meta.readVInt() != fieldNumber) {
+            throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+          }
+          if (meta.readByte() != Lucene50DocValuesFormat.SORTED) {
+            throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+          }
+          readSortedField(info, meta);
+        } else {
+          throw new AssertionError();
+        }
+      } else if (type == Lucene50DocValuesFormat.SORTED_NUMERIC) {
+        SortedSetEntry ss = readSortedSetEntry(meta);
+        sortedNumerics.put(info.name, ss);
+        if (ss.format == SORTED_WITH_ADDRESSES) {
+          if (meta.readVInt() != fieldNumber) {
+            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
+          }
+          if (meta.readByte() != Lucene50DocValuesFormat.NUMERIC) {
+            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
+          }
+          numerics.put(info.name, readNumericEntry(meta));
+          if (meta.readVInt() != fieldNumber) {
+            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
+          }
+          if (meta.readByte() != Lucene50DocValuesFormat.NUMERIC) {
+            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
+          }
+          NumericEntry ordIndex = readNumericEntry(meta);
+          ordIndexes.put(info.name, ordIndex);
+        } else if (ss.format == SORTED_SET_TABLE) {
+          if (meta.readVInt() != info.number) {
+            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
+          }
+          if (meta.readByte() != Lucene50DocValuesFormat.NUMERIC) {
+            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
+          }
+          NumericEntry n = readNumericEntry(meta);
+          ords.put(info.name, n);
+        } else if (ss.format == SORTED_SINGLE_VALUED) {
+          if (meta.readVInt() != fieldNumber) {
+            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
+          }
+          if (meta.readByte() != Lucene50DocValuesFormat.NUMERIC) {
+            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
+          }
+          numerics.put(info.name, readNumericEntry(meta));
+        } else {
+          throw new AssertionError();
+        }
+      } else {
+        throw new CorruptIndexException("invalid type: " + type, meta);
+      }
+      fieldNumber = meta.readVInt();
+    }
+    return numFields;
+  }
+  
+  private NumericEntry readNumericEntry(IndexInput meta) throws IOException {
+    NumericEntry entry = new NumericEntry();
+    entry.format = meta.readVInt();
+    entry.missingOffset = meta.readLong();
+    entry.offset = meta.readLong();
+    entry.count = meta.readVLong();
+    switch(entry.format) {
+      case CONST_COMPRESSED:
+        entry.minValue = meta.readLong();
+        if (entry.count > Integer.MAX_VALUE) {
+          // currently just a limitation e.g. of bits interface and so on.
+          throw new CorruptIndexException("illegal CONST_COMPRESSED count: " + entry.count, meta);
+        }
+        break;
+      case GCD_COMPRESSED:
+        entry.minValue = meta.readLong();
+        entry.gcd = meta.readLong();
+        entry.bitsPerValue = meta.readVInt();
+        break;
+      case TABLE_COMPRESSED:
+        final int uniqueValues = meta.readVInt();
+        if (uniqueValues > 256) {
+          throw new CorruptIndexException("TABLE_COMPRESSED cannot have more than 256 distinct values, got=" + uniqueValues, meta);
+        }
+        entry.table = new long[uniqueValues];
+        for (int i = 0; i < uniqueValues; ++i) {
+          entry.table[i] = meta.readLong();
+        }
+        ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(entry.table));
+        entry.bitsPerValue = meta.readVInt();
+        break;
+      case DELTA_COMPRESSED:
+        entry.minValue = meta.readLong();
+        entry.bitsPerValue = meta.readVInt();
+        break;
+      case MONOTONIC_COMPRESSED:
+        entry.packedIntsVersion = meta.readVInt();
+        entry.blockSize = meta.readVInt();
+        break;
+      default:
+        throw new CorruptIndexException("Unknown format: " + entry.format + ", input=", meta);
+    }
+    entry.endOffset = meta.readLong();
+    return entry;
+  }
+  
+  static BinaryEntry readBinaryEntry(IndexInput meta) throws IOException {
+    BinaryEntry entry = new BinaryEntry();
+    entry.format = meta.readVInt();
+    entry.missingOffset = meta.readLong();
+    entry.minLength = meta.readVInt();
+    entry.maxLength = meta.readVInt();
+    entry.count = meta.readVLong();
+    entry.offset = meta.readLong();
+    switch(entry.format) {
+      case BINARY_FIXED_UNCOMPRESSED:
+        break;
+      case BINARY_PREFIX_COMPRESSED:
+        entry.addressesOffset = meta.readLong();
+        entry.packedIntsVersion = meta.readVInt();
+        entry.blockSize = meta.readVInt();
+        entry.reverseIndexOffset = meta.readLong();
+        break;
+      case BINARY_VARIABLE_UNCOMPRESSED:
+        entry.addressesOffset = meta.readLong();
+        entry.packedIntsVersion = meta.readVInt();
+        entry.blockSize = meta.readVInt();
+        break;
+      default:
+        throw new CorruptIndexException("Unknown format: " + entry.format, meta);
+    }
+    return entry;
+  }
+
+  SortedSetEntry readSortedSetEntry(IndexInput meta) throws IOException {
+    SortedSetEntry entry = new SortedSetEntry();
+    entry.format = meta.readVInt();
+    if (entry.format == SORTED_SET_TABLE) {
+      final int totalTableLength = meta.readInt();
+      if (totalTableLength > 256) {
+        throw new CorruptIndexException("SORTED_SET_TABLE cannot have more than 256 values in its dictionary, got=" + totalTableLength, meta);
+      }
+      entry.table = new long[totalTableLength];
+      for (int i = 0; i < totalTableLength; ++i) {
+        entry.table[i] = meta.readLong();
+      }
+      ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(entry.table));
+      final int tableSize = meta.readInt();
+      if (tableSize > totalTableLength + 1) { // +1 because of the empty set
+        throw new CorruptIndexException("SORTED_SET_TABLE cannot have more set ids than ords in its dictionary, got " + totalTableLength + " ords and " + tableSize + " sets", meta);
+      }
+      entry.tableOffsets = new int[tableSize + 1];
+      for (int i = 1; i < entry.tableOffsets.length; ++i) {
+        entry.tableOffsets[i] = entry.tableOffsets[i - 1] + meta.readInt();
+      }
+      ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(entry.tableOffsets));
+    } else if (entry.format != SORTED_SINGLE_VALUED && entry.format != SORTED_WITH_ADDRESSES) {
+      throw new CorruptIndexException("Unknown format: " + entry.format, meta);
+    }
+    return entry;
+  }
+
+  @Override
+  public NumericDocValues getNumeric(FieldInfo field) throws IOException {
+    NumericEntry entry = numerics.get(field.name);
+    return getNumeric(entry);
+  }
+  
+  @Override
+  public long ramBytesUsed() {
+    return ramBytesUsed.get();
+  }
+  
+  @Override
+  public synchronized Collection<Accountable> getChildResources() {
+    List<Accountable> resources = new ArrayList<>();
+    resources.addAll(Accountables.namedAccountables("addresses field", addressInstances));
+    resources.addAll(Accountables.namedAccountables("ord index field", ordIndexInstances));
+    resources.addAll(Accountables.namedAccountables("reverse index field", reverseIndexInstances));
+    return Collections.unmodifiableList(resources);
+  }
+  
+  @Override
+  public void checkIntegrity() throws IOException {
+    CodecUtil.checksumEntireFile(data);
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(fields=" + numFields + ")";
+  }
+
+  LongValues getNumeric(NumericEntry entry) throws IOException {
+    switch (entry.format) {
+      case CONST_COMPRESSED: {
+        final long constant = entry.minValue;
+        final Bits live = getLiveBits(entry.missingOffset, (int)entry.count);
+        return new LongValues() {
+          @Override
+          public long get(long index) {
+            return live.get((int)index) ? constant : 0;
+          }
+        };
+      }
+      case DELTA_COMPRESSED: {
+        RandomAccessInput slice = this.data.randomAccessSlice(entry.offset, entry.endOffset - entry.offset);
+        final long delta = entry.minValue;
+        final LongValues values = DirectReader.getInstance(slice, entry.bitsPerValue);
+        return new LongValues() {
+          @Override
+          public long get(long id) {
+            return delta + values.get(id);
+          }
+        };
+      }
+      case GCD_COMPRESSED: {
+        RandomAccessInput slice = this.data.randomAccessSlice(entry.offset, entry.endOffset - entry.offset);
+        final long min = entry.minValue;
+        final long mult = entry.gcd;
+        final LongValues quotientReader = DirectReader.getInstance(slice, entry.bitsPerValue);
+        return new LongValues() {
+          @Override
+          public long get(long id) {
+            return min + mult * quotientReader.get(id);
+          }
+        };
+      }
+      case TABLE_COMPRESSED: {
+        RandomAccessInput slice = this.data.randomAccessSlice(entry.offset, entry.endOffset - entry.offset);
+        final long table[] = entry.table;
+        final LongValues ords = DirectReader.getInstance(slice, entry.bitsPerValue);
+        return new LongValues() {
+          @Override
+          public long get(long id) {
+            return table[(int) ords.get(id)];
+          }
+        };
+      }
+      default:
+        throw new AssertionError();
+    }
+  }
+
+  @Override
+  public BinaryDocValues getBinary(FieldInfo field) throws IOException {
+    BinaryEntry bytes = binaries.get(field.name);
+    switch(bytes.format) {
+      case BINARY_FIXED_UNCOMPRESSED:
+        return getFixedBinary(field, bytes);
+      case BINARY_VARIABLE_UNCOMPRESSED:
+        return getVariableBinary(field, bytes);
+      case BINARY_PREFIX_COMPRESSED:
+        return getCompressedBinary(field, bytes);
+      default:
+        throw new AssertionError();
+    }
+  }
+  
+  private BinaryDocValues getFixedBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
+    final IndexInput data = this.data.slice("fixed-binary", bytes.offset, bytes.count * bytes.maxLength);
+
+    final BytesRef term = new BytesRef(bytes.maxLength);
+    final byte[] buffer = term.bytes;
+    final int length = term.length = bytes.maxLength;
+    
+    return new LongBinaryDocValues() {
+      @Override
+      public BytesRef get(long id) {
+        try {
+          data.seek(id * length);
+          data.readBytes(buffer, 0, buffer.length);
+          return term;
+        } catch (IOException e) {
+          throw new RuntimeException(e);
+        }
+      }
+    };
+  }
+  
+  /** returns an address instance for variable-length binary values. */
+  private synchronized MonotonicBlockPackedReader getAddressInstance(FieldInfo field, BinaryEntry bytes) throws IOException {
+    MonotonicBlockPackedReader addresses = addressInstances.get(field.name);
+    if (addresses == null) {
+      data.seek(bytes.addressesOffset);
+      addresses = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, bytes.count+1, false);
+      if (!merging) {
+        addressInstances.put(field.name, addresses);
+        ramBytesUsed.addAndGet(addresses.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
+      }
+    }
+    return addresses;
+  }
+  
+  private BinaryDocValues getVariableBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
+    final MonotonicBlockPackedReader addresses = getAddressInstance(field, bytes);
+
+    final IndexInput data = this.data.slice("var-binary", bytes.offset, bytes.addressesOffset - bytes.offset);
+    final BytesRef term = new BytesRef(Math.max(0, bytes.maxLength));
+    final byte buffer[] = term.bytes;
+    
+    return new LongBinaryDocValues() {      
+      @Override
+      public BytesRef get(long id) {
+        long startAddress = addresses.get(id);
+        long endAddress = addresses.get(id+1);
+        int length = (int) (endAddress - startAddress);
+        try {
+          data.seek(startAddress);
+          data.readBytes(buffer, 0, length);
+          term.length = length;
+          return term;
+        } catch (IOException e) {
+          throw new RuntimeException(e);
+        }
+      }
+    };
+  }
+  
+  /** returns an address instance for prefix-compressed binary values. */
+  private synchronized MonotonicBlockPackedReader getIntervalInstance(FieldInfo field, BinaryEntry bytes) throws IOException {
+    MonotonicBlockPackedReader addresses = addressInstances.get(field.name);
+    if (addresses == null) {
+      data.seek(bytes.addressesOffset);
+      final long size = (bytes.count + INTERVAL_MASK) >>> INTERVAL_SHIFT;
+      addresses = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
+      if (!merging) {
+        addressInstances.put(field.name, addresses);
+        ramBytesUsed.addAndGet(addresses.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
+      }
+    }
+    return addresses;
+  }
+  
+  /** returns a reverse lookup instance for prefix-compressed binary values. */
+  private synchronized ReverseTermsIndex getReverseIndexInstance(FieldInfo field, BinaryEntry bytes) throws IOException {
+    ReverseTermsIndex index = reverseIndexInstances.get(field.name);
+    if (index == null) {
+      index = new ReverseTermsIndex();
+      data.seek(bytes.reverseIndexOffset);
+      long size = (bytes.count + REVERSE_INTERVAL_MASK) >>> REVERSE_INTERVAL_SHIFT;
+      index.termAddresses = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
+      long dataSize = data.readVLong();
+      PagedBytes pagedBytes = new PagedBytes(15);
+      pagedBytes.copy(data, dataSize);
+      index.terms = pagedBytes.freeze(true);
+      if (!merging) {
+        reverseIndexInstances.put(field.name, index);
+        ramBytesUsed.addAndGet(index.ramBytesUsed());
+      }
+    }
+    return index;
+  }
+
+  private BinaryDocValues getCompressedBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
+    final MonotonicBlockPackedReader addresses = getIntervalInstance(field, bytes);
+    final ReverseTermsIndex index = getReverseIndexInstance(field, bytes);
+    assert addresses.size() > 0; // we don't have to handle empty case
+    IndexInput slice = data.slice("terms", bytes.offset, bytes.addressesOffset - bytes.offset);
+    return new CompressedBinaryDocValues(bytes, addresses, index, slice);
+  }
+
+  @Override
+  public SortedDocValues getSorted(FieldInfo field) throws IOException {
+    final int valueCount = (int) binaries.get(field.name).count;
+    final BinaryDocValues binary = getBinary(field);
+    NumericEntry entry = ords.get(field.name);
+    final LongValues ordinals = getNumeric(entry);
+    return new SortedDocValues() {
+
+      @Override
+      public int getOrd(int docID) {
+        return (int) ordinals.get(docID);
+      }
+
+      @Override
+      public BytesRef lookupOrd(int ord) {
+        return binary.get(ord);
+      }
+
+      @Override
+      public int getValueCount() {
+        return valueCount;
+      }
+
+      @Override
+      public int lookupTerm(BytesRef key) {
+        if (binary instanceof CompressedBinaryDocValues) {
+          return (int) ((CompressedBinaryDocValues)binary).lookupTerm(key);
+        } else {
+          return super.lookupTerm(key);
+        }
+      }
+
+      @Override
+      public TermsEnum termsEnum() {
+        if (binary instanceof CompressedBinaryDocValues) {
+          return ((CompressedBinaryDocValues)binary).getTermsEnum();
+        } else {
+          return super.termsEnum();
+        }
+      }
+    };
+  }
+  
+  /** returns an address instance for sortedset ordinal lists */
+  private synchronized MonotonicBlockPackedReader getOrdIndexInstance(FieldInfo field, NumericEntry entry) throws IOException {
+    MonotonicBlockPackedReader instance = ordIndexInstances.get(field.name);
+    if (instance == null) {
+      data.seek(entry.offset);
+      instance = MonotonicBlockPackedReader.of(data, entry.packedIntsVersion, entry.blockSize, entry.count+1, false);
+      if (!merging) {
+        ordIndexInstances.put(field.name, instance);
+        ramBytesUsed.addAndGet(instance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
+      }
+    }
+    return instance;
+  }
+  
+  @Override
+  public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException {
+    SortedSetEntry ss = sortedNumerics.get(field.name);
+    if (ss.format == SORTED_SINGLE_VALUED) {
+      NumericEntry numericEntry = numerics.get(field.name);
+      final LongValues values = getNumeric(numericEntry);
+      final Bits docsWithField = getLiveBits(numericEntry.missingOffset, maxDoc);
+      return DocValues.singleton(values, docsWithField);
+    } else if (ss.format == SORTED_WITH_ADDRESSES) {
+      NumericEntry numericEntry = numerics.get(field.name);
+      final LongValues values = getNumeric(numericEntry);
+      final MonotonicBlockPackedReader ordIndex = getOrdIndexInstance(field, ordIndexes.get(field.name));
+      
+      return new SortedNumericDocValues() {
+        long startOffset;
+        long endOffset;
+        
+        @Override
+        public void setDocument(int doc) {
+          startOffset = ordIndex.get(doc);
+          endOffset = ordIndex.get(doc+1L);
+        }
+
+        @Override
+        public long valueAt(int index) {
+          return values.get(startOffset + index);
+        }
+
+        @Override
+        public int count() {
+          return (int) (endOffset - startOffset);
+        }
+      };
+    } else if (ss.format == SORTED_SET_TABLE) {
+      NumericEntry entry = ords.get(field.name);
+      final LongValues ordinals = getNumeric(entry);
+
+      final long[] table = ss.table;
+      final int[] offsets = ss.tableOffsets;
+      return new SortedNumericDocValues() {
+        int startOffset;
+        int endOffset;
+        
+        @Override
+        public void setDocument(int doc) {
+          final int ord = (int) ordinals.get(doc);
+          startOffset = offsets[ord];
+          endOffset = offsets[ord + 1];
+        }
+
+        @Override
+        public long valueAt(int index) {
+          return table[startOffset + index];
+        }
+
+        @Override
+        public int count() {
+          return endOffset - startOffset;
+        }
+      };
+    } else {
+      throw new AssertionError();
+    }
+  }
+
+  @Override
+  public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
+    SortedSetEntry ss = sortedSets.get(field.name);
+    switch (ss.format) {
+      case SORTED_SINGLE_VALUED:
+        final SortedDocValues values = getSorted(field);
+        return DocValues.singleton(values);
+      case SORTED_WITH_ADDRESSES:
+        return getSortedSetWithAddresses(field);
+      case SORTED_SET_TABLE:
+        return getSortedSetTable(field, ss);
+      default:
+        throw new AssertionError();
+    }
+  }
+
+  private SortedSetDocValues getSortedSetWithAddresses(FieldInfo field) throws IOException {
+    final long valueCount = binaries.get(field.name).count;
+    // we keep the byte[]s and list of ords on disk, these could be large
+    final LongBinaryDocValues binary = (LongBinaryDocValues) getBinary(field);
+    final LongValues ordinals = getNumeric(ords.get(field.name));
+    // but the addresses to the ord stream are in RAM
+    final MonotonicBlockPackedReader ordIndex = getOrdIndexInstance(field, ordIndexes.get(field.name));
+    
+    return new RandomAccessOrds() {
+      long startOffset;
+      long offset;
+      long endOffset;
+      
+      @Override
+      public long nextOrd() {
+        if (offset == endOffset) {
+          return NO_MORE_ORDS;
+        } else {
+          long ord = ordinals.get(offset);
+          offset++;
+          return ord;
+        }
+      }
+
+      @Override
+      public void setDocument(int docID) {
+        startOffset = offset = ordIndex.get(docID);
+        endOffset = ordIndex.get(docID+1L);
+      }
+
+      @Override
+      public BytesRef lookupOrd(long ord) {
+        return binary.get(ord);
+      }
+
+      @Override
+      public long getValueCount() {
+        return valueCount;
+      }
+      
+      @Override
+      public long lookupTerm(BytesRef key) {
+        if (binary instanceof CompressedBinaryDocValues) {
+          return ((CompressedBinaryDocValues)binary).lookupTerm(key);
+        } else {
+          return super.lookupTerm(key);
+        }
+      }
+
+      @Override
+      public TermsEnum termsEnum() {
+        if (binary instanceof CompressedBinaryDocValues) {
+          return ((CompressedBinaryDocValues)binary).getTermsEnum();
+        } else {
+          return super.termsEnum();
+        }
+      }
+
+      @Override
+      public long ordAt(int index) {
+        return ordinals.get(startOffset + index);
+      }
+
+      @Override
+      public int cardinality() {
+        return (int) (endOffset - startOffset);
+      }
+    };
+  }
+
+  private SortedSetDocValues getSortedSetTable(FieldInfo field, SortedSetEntry ss) throws IOException {
+    final long valueCount = binaries.get(field.name).count;
+    final LongBinaryDocValues binary = (LongBinaryDocValues) getBinary(field);
+    final LongValues ordinals = getNumeric(ords.get(field.name));
+
+    final long[] table = ss.table;
+    final int[] offsets = ss.tableOffsets;
+
+    return new RandomAccessOrds() {
+
+      int offset, startOffset, endOffset;
+
+      @Override
+      public void setDocument(int docID) {
+        final int ord = (int) ordinals.get(docID);
+        offset = startOffset = offsets[ord];
+        endOffset = offsets[ord + 1];
+      }
+
+      @Override
+      public long ordAt(int index) {
+        return table[startOffset + index];
+      }
+
+      @Override
+      public long nextOrd() {
+        if (offset == endOffset) {
+          return NO_MORE_ORDS;
+        } else {
+          return table[offset++];
+        }
+      }
+
+      @Override
+      public int cardinality() {
+        return endOffset - startOffset;
+      }
+
+      @Override
+      public BytesRef lookupOrd(long ord) {
+        return binary.get(ord);
+      }
+
+      @Override
+      public long getValueCount() {
+        return valueCount;
+      }
+
+      @Override
+      public long lookupTerm(BytesRef key) {
+        if (binary instanceof CompressedBinaryDocValues) {
+          return ((CompressedBinaryDocValues) binary).lookupTerm(key);
+        } else {
+          return super.lookupTerm(key);
+        }
+      }
+
+      @Override
+      public TermsEnum termsEnum() {
+        if (binary instanceof CompressedBinaryDocValues) {
+          return ((CompressedBinaryDocValues) binary).getTermsEnum();
+        } else {
+          return super.termsEnum();
+        }
+      }
+
+    };
+  }
+
+  private Bits getLiveBits(final long offset, final int count) throws IOException {
+    if (offset == ALL_MISSING) {
+      return new Bits.MatchNoBits(count);
+    } else if (offset == ALL_LIVE) {
+      return new Bits.MatchAllBits(count);
+    } else {
+      int length = (int) ((count + 7L) >>> 3);
+      final RandomAccessInput in = data.randomAccessSlice(offset, length);
+      return new Bits() {
+        @Override
+        public boolean get(int index) {
+          try {
+            return (in.readByte(index >> 3) & (1 << (index & 7))) != 0;
+          } catch (IOException e) {
+            throw new RuntimeException(e);
+          }
+        }
+
+        @Override
+        public int length() {
+          return count;
+        }
+      };
+    }
+  }
+
+  @Override
+  public Bits getDocsWithField(FieldInfo field) throws IOException {
+    switch(field.getDocValuesType()) {
+      case SORTED_SET:
+        return DocValues.docsWithValue(getSortedSet(field), maxDoc);
+      case SORTED_NUMERIC:
+        return DocValues.docsWithValue(getSortedNumeric(field), maxDoc);
+      case SORTED:
+        return DocValues.docsWithValue(getSorted(field), maxDoc);
+      case BINARY:
+        BinaryEntry be = binaries.get(field.name);
+        return getLiveBits(be.missingOffset, maxDoc);
+      case NUMERIC:
+        NumericEntry ne = numerics.get(field.name);
+        return getLiveBits(ne.missingOffset, maxDoc);
+      default:
+        throw new AssertionError();
+    }
+  }
+
+  @Override
+  public synchronized DocValuesProducer getMergeInstance() throws IOException {
+    return new Lucene50DocValuesProducer(this);
+  }
+
+  @Override
+  public void close() throws IOException {
+    data.close();
+  }
+  
+  /** metadata entry for a numeric docvalues field */
+  static class NumericEntry {
+    private NumericEntry() {}
+    /** offset to the bitset representing docsWithField, or -1 if no documents have missing values */
+    long missingOffset;
+    /** offset to the actual numeric values */
+    public long offset;
+    /** end offset to the actual numeric values */
+    public long endOffset;
+    /** bits per value used to pack the numeric values */
+    public int bitsPerValue;
+
+    int format;
+    /** packed ints version used to encode these numerics */
+    public int packedIntsVersion;
+    /** count of values written */
+    public long count;
+    /** packed ints blocksize */
+    public int blockSize;
+    
+    long minValue;
+    long gcd;
+    long table[];
+  }
+  
+  /** metadata entry for a binary docvalues field */
+  static class BinaryEntry {
+    private BinaryEntry() {}
+    /** offset to the bitset representing docsWithField, or -1 if no documents have missing values */
+    long missingOffset;
+    /** offset to the actual binary values */
+    long offset;
+
+    int format;
+    /** count of values written */
+    public long count;
+    int minLength;
+    int maxLength;
+    /** offset to the addressing data that maps a value to its slice of the byte[] */
+    public long addressesOffset;
+    /** offset to the reverse index */
+    public long reverseIndexOffset;
+    /** packed ints version used to encode addressing information */
+    public int packedIntsVersion;
+    /** packed ints blocksize */
+    public int blockSize;
+  }
+
+  /** metadata entry for a sorted-set docvalues field */
+  static class SortedSetEntry {
+    private SortedSetEntry() {}
+    int format;
+
+    long[] table;
+    int[] tableOffsets;
+  }
+
+  // internally we compose complex dv (sorted/sortedset) from other ones
+  static abstract class LongBinaryDocValues extends BinaryDocValues {
+    @Override
+    public final BytesRef get(int docID) {
+      return get((long)docID);
+    }
+    
+    abstract BytesRef get(long id);
+  }
+  
+  // used for reverse lookup to a small range of blocks
+  static class ReverseTermsIndex implements Accountable {
+    public MonotonicBlockPackedReader termAddresses;
+    public PagedBytes.Reader terms;
+    
+    @Override
+    public long ramBytesUsed() {
+      return termAddresses.ramBytesUsed() + terms.ramBytesUsed();
+    }
+    
+    @Override
+    public Collection<Accountable> getChildResources() {
+      List<Accountable> resources = new ArrayList<>();
+      resources.add(Accountables.namedAccountable("term bytes", terms));
+      resources.add(Accountables.namedAccountable("term addresses", termAddresses));
+      return Collections.unmodifiableList(resources);
+    }
+
+    @Override
+    public String toString() {
+      return getClass().getSimpleName() + "(size=" + termAddresses.size() + ")";
+    }
+  }
+  
+  //in the compressed case, we add a few additional operations for
+  //more efficient reverse lookup and enumeration
+  static final class CompressedBinaryDocValues extends LongBinaryDocValues {    
+    final long numValues;
+    final long numIndexValues;
+    final int maxTermLength;
+    final MonotonicBlockPackedReader addresses;
+    final IndexInput data;
+    final CompressedBinaryTermsEnum termsEnum;
+    final PagedBytes.Reader reverseTerms;
+    final MonotonicBlockPackedReader reverseAddresses;
+    final long numReverseIndexValues;
+    
+    public CompressedBinaryDocValues(BinaryEntry bytes, MonotonicBlockPackedReader addresses, ReverseTermsIndex index, IndexInput data) throws IOException {
+      this.maxTermLength = bytes.maxLength;
+      this.numValues = bytes.count;
+      this.addresses = addresses;
+      this.numIndexValues = addresses.size();
+      this.data = data;
+      this.reverseTerms = index.terms;
+      this.reverseAddresses = index.termAddresses;
+      this.numReverseIndexValues = reverseAddresses.size();
+      this.termsEnum = getTermsEnum(data);
+    }
+    
+    @Override
+    public BytesRef get(long id) {
+      try {
+        termsEnum.seekExact(id);
+        return termsEnum.term();
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
+    
+    long lookupTerm(BytesRef key) {
+      try {
+        switch (termsEnum.seekCeil(key)) {
+          case FOUND: return termsEnum.ord();
+          case NOT_FOUND: return -termsEnum.ord()-1;
+          default: return -numValues-1;
+        }
+      } catch (IOException bogus) {
+        throw new RuntimeException(bogus);
+      }
+    }
+    
+    TermsEnum getTermsEnum() {
+      try {
+        return getTermsEnum(data.clone());
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
+    
+    private CompressedBinaryTermsEnum getTermsEnum(IndexInput input) throws IOException {
+      return new CompressedBinaryTermsEnum(input);
+    }
+    
+    class CompressedBinaryTermsEnum extends TermsEnum {
+      private long currentOrd = -1;
+      // offset to the start of the current block 
+      private long currentBlockStart;
+      private final IndexInput input;
+      // delta from currentBlockStart to start of each term
+      private final int offsets[] = new int[INTERVAL_COUNT];
+      private final byte buffer[] = new byte[2*INTERVAL_COUNT-1];
+      
+      private final BytesRef term = new BytesRef(maxTermLength);
+      private final BytesRef firstTerm = new BytesRef(maxTermLength);
+      private final BytesRef scratch = new BytesRef();
+      
+      CompressedBinaryTermsEnum(IndexInput input) throws IOException {
+        this.input = input;
+        input.seek(0);
+      }
+      
+      private void readHeader() throws IOException {
+        firstTerm.length = input.readVInt();
+        input.readBytes(firstTerm.bytes, 0, firstTerm.length);
+        input.readBytes(buffer, 0, INTERVAL_COUNT-1);
+        if (buffer[0] == -1) {
+          readShortAddresses();
+        } else {
+          readByteAddresses();
+        }
+        currentBlockStart = input.getFilePointer();
+      }
+      
+      // read single byte addresses: each is delta - 2
+      // (shared prefix byte and length > 0 are both implicit)
+      private void readByteAddresses() throws IOException {
+        int addr = 0;
+        for (int i = 1; i < offsets.length; i++) {
+          addr += 2 + (buffer[i-1] & 0xFF);
+          offsets[i] = addr;
+        }
+      }
+      
+      // read double byte addresses: each is delta - 2
+      // (shared prefix byte and length > 0 are both implicit)
+      private void readShortAddresses() throws IOException {
+        input.readBytes(buffer, INTERVAL_COUNT-1, INTERVAL_COUNT);
+        int addr = 0;
+        for (int i = 1; i < offsets.length; i++) {
+          int x = i<<1;
+          addr += 2 + ((buffer[x-1] << 8) | (buffer[x] & 0xFF));
+          offsets[i] = addr;
+        }
+      }
+      
+      // set term to the first term
+      private void readFirstTerm() throws IOException {
+        term.length = firstTerm.length;
+        System.arraycopy(firstTerm.bytes, firstTerm.offset, term.bytes, 0, term.length);
+      }
+      
+      // read term at offset, delta encoded from first term
+      private void readTerm(int offset) throws IOException {
+        int start = input.readByte() & 0xFF;
+        System.arraycopy(firstTerm.bytes, firstTerm.offset, term.bytes, 0, start);
+        int suffix = offsets[offset] - offsets[offset-1] - 1;
+        input.readBytes(term.bytes, start, suffix);
+        term.length = start + suffix;
+      }
+      
+      @Override
+      public BytesRef next() throws IOException {
+        currentOrd++;
+        if (currentOrd >= numValues) {
+          return null;
+        } else { 
+          int offset = (int) (currentOrd & INTERVAL_MASK);
+          if (offset == 0) {
+            // switch to next block
+            readHeader();
+            readFirstTerm();
+          } else {
+            readTerm(offset);
+          }
+          return term;
+        }
+      }
+      
+      // binary search reverse index to find smaller 
+      // range of blocks to search
+      long binarySearchIndex(BytesRef text) throws IOException {
+        long low = 0;
+        long high = numReverseIndexValues - 1;
+        while (low <= high) {
+          long mid = (low + high) >>> 1;
+          reverseTerms.fill(scratch, reverseAddresses.get(mid));
+          int cmp = scratch.compareTo(text);
+          
+          if (cmp < 0) {
+            low = mid + 1;
+          } else if (cmp > 0) {
+            high = mid - 1;
+          } else {
+            return mid;
+          }
+        }
+        return high;
+      }
+      
+      // binary search against first term in block range 
+      // to find term's block
+      long binarySearchBlock(BytesRef text, long low, long high) throws IOException {       
+        while (low <= high) {
+          long mid = (low + high) >>> 1;
+          input.seek(addresses.get(mid));
+          term.length = input.readVInt();
+          input.readBytes(term.bytes, 0, term.length);
+          int cmp = term.compareTo(text);
+          
+          if (cmp < 0) {
+            low = mid + 1;
+          } else if (cmp > 0) {
+            high = mid - 1;
+          } else {
+            return mid;
+          }
+        }
+        return high;
+      }
+      
+      @Override
+      public SeekStatus seekCeil(BytesRef text) throws IOException {
+        // locate block: narrow to block range with index, then search blocks
+        final long block;
+        long indexPos = binarySearchIndex(text);
+        if (indexPos < 0) {
+          block = 0;
+        } else {
+          long low = indexPos << BLOCK_INTERVAL_SHIFT;
+          long high = Math.min(numIndexValues - 1, low + BLOCK_INTERVAL_MASK);
+          block = Math.max(low, binarySearchBlock(text, low, high));
+        }
+        
+        // position before block, then scan to term.
+        input.seek(addresses.get(block));
+        currentOrd = (block << INTERVAL_SHIFT) - 1;
+        
+        while (next() != null) {
+          int cmp = term.compareTo(text);
+          if (cmp == 0) {
+            return SeekStatus.FOUND;
+          } else if (cmp > 0) {
+            return SeekStatus.NOT_FOUND;
+          }
+        }
+        return SeekStatus.END;
+      }
+      
+      @Override
+      public void seekExact(long ord) throws IOException {
+        long block = ord >>> INTERVAL_SHIFT;
+        if (block != currentOrd >>> INTERVAL_SHIFT) {
+          // switch to different block
+          input.seek(addresses.get(block));
+          readHeader();
+        }
+        
+        currentOrd = ord;
+        
+        int offset = (int) (ord & INTERVAL_MASK);
+        if (offset == 0) {
+          readFirstTerm();
+        } else {
+          input.seek(currentBlockStart + offsets[offset-1]);
+          readTerm(offset);
+        }
+      }
+      
+      @Override
+      public BytesRef term() throws IOException {
+        return term;
+      }
+      
+      @Override
+      public long ord() throws IOException {
+        return currentOrd;
+      }
+      
+      @Override
+      public int docFreq() throws IOException {
+        throw new UnsupportedOperationException();
+      }
+      
+      @Override
+      public long totalTermFreq() throws IOException {
+        return -1;
+      }
+      
+      @Override
+      public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
+        throw new UnsupportedOperationException();
+      }
+
+    }
+  }
+}