You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2014/10/16 03:20:59 UTC

svn commit: r1632200 [2/2] - in /lucene/dev/branches/lucene5969/lucene: backward-codecs/src/java/org/apache/lucene/codecs/lucene410/ backward-codecs/src/resources/META-INF/services/ backward-codecs/src/test/org/apache/lucene/codecs/lucene410/ backward-...

Added: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesProducer.java?rev=1632200&view=auto
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesProducer.java (added)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesProducer.java Thu Oct 16 01:20:58 2014
@@ -0,0 +1,1153 @@
+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 static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.BINARY_FIXED_UNCOMPRESSED;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.BINARY_PREFIX_COMPRESSED;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.BINARY_VARIABLE_UNCOMPRESSED;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.CONST_COMPRESSED;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.DELTA_COMPRESSED;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.GCD_COMPRESSED;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.MONOTONIC_COMPRESSED;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.SORTED_SINGLE_VALUED;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.SORTED_WITH_ADDRESSES;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.TABLE_COMPRESSED;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.INTERVAL_SHIFT;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.INTERVAL_COUNT;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.INTERVAL_MASK;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.REVERSE_INTERVAL_SHIFT;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.REVERSE_INTERVAL_MASK;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.BLOCK_INTERVAL_SHIFT;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.BLOCK_INTERVAL_MASK;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.ALL_LIVE;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.ALL_MISSING;
+
+import java.io.Closeable; // javadocs
+import java.io.IOException;
+import java.util.ArrayList;
+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.DocsAndPositionsEnum;
+import org.apache.lucene.index.DocsEnum;
+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;
+
+/** 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.getDocCount();
+    merging = false;
+    
+    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.checkSegmentHeader(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.checkSegmentHeader(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);
+      }
+    }
+    
+    ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOfInstance(getClass()));
+  }
+
+  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 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_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 (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 (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);
+          }
+          NumericEntry ordIndex = readNumericEntry(meta);
+          ordIndexes.put(info.name, ordIndex);
+        } else if (ss.format != SORTED_SINGLE_VALUED) {
+          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();
+        }
+        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_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 Iterable<? extends 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);
+    NumericEntry numericEntry = numerics.get(field.name);
+    final LongValues values = getNumeric(numericEntry);
+    if (ss.format == SORTED_SINGLE_VALUED) {
+      final Bits docsWithField = getLiveBits(numericEntry.missingOffset, maxDoc);
+      return DocValues.singleton(values, docsWithField);
+    } else if (ss.format == SORTED_WITH_ADDRESSES) {
+      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 {
+      throw new AssertionError();
+    }
+  }
+
+  @Override
+  public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
+    SortedSetEntry ss = sortedSets.get(field.name);
+    if (ss.format == SORTED_SINGLE_VALUED) {
+      final SortedDocValues values = getSorted(field);
+      return DocValues.singleton(values);
+    } else if (ss.format != SORTED_WITH_ADDRESSES) {
+      throw new AssertionError();
+    }
+
+    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 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;
+  }
+
+  // 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 Iterable<? extends 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 DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
+        throw new UnsupportedOperationException();
+      }
+      
+      @Override
+      public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
+        throw new UnsupportedOperationException();
+      }
+    }
+  }
+}

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/package.html?rev=1632200&r1=1632199&r2=1632200&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/package.html (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/package.html Thu Oct 16 01:20:58 2014
@@ -185,7 +185,7 @@ term frequency. To add Term Vectors to y
 {@link org.apache.lucene.document.Field Field} constructors
 </li>
 <li>
-{@link org.apache.lucene.codecs.lucene410.Lucene410DocValuesFormat Per-document values}. 
+{@link org.apache.lucene.codecs.lucene50.Lucene50DocValuesFormat Per-document values}. 
 Like stored values, these are also keyed by document
 number, but are generally intended to be loaded into main memory for fast
 access. Whereas stored values are generally intended for summary results from
@@ -294,7 +294,7 @@ systems that frequently run out of file 
 <td>Encodes length and boost factors for docs and fields</td>
 </tr>
 <tr>
-<td>{@link org.apache.lucene.codecs.lucene410.Lucene410DocValuesFormat Per-Document Values}</td>
+<td>{@link org.apache.lucene.codecs.lucene50.Lucene50DocValuesFormat Per-Document Values}</td>
 <td>.dvd, .dvm</td>
 <td>Encodes additional scoring factors or other per-document information.</td>
 </tr>

Modified: lucene/dev/branches/lucene5969/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat?rev=1632200&r1=1632199&r2=1632200&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat Thu Oct 16 01:20:58 2014
@@ -13,4 +13,4 @@
 #  See the License for the specific language governing permissions and
 #  limitations under the License.
 
-org.apache.lucene.codecs.lucene410.Lucene410DocValuesFormat
+org.apache.lucene.codecs.lucene50.Lucene50DocValuesFormat

Added: lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50DocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50DocValuesFormat.java?rev=1632200&view=auto
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50DocValuesFormat.java (added)
+++ lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50DocValuesFormat.java Thu Oct 16 01:20:58 2014
@@ -0,0 +1,285 @@
+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.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.PostingsFormat;
+import org.apache.lucene.codecs.asserting.AssertingCodec;
+import org.apache.lucene.codecs.blocktreeords.Ords41PostingsFormat;
+import org.apache.lucene.codecs.lucene41ords.Lucene41WithOrds;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.SortedSetDocValuesField;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.BaseCompressingDocValuesFormatTestCase;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.SerialMergeScheduler;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.index.TermsEnum.SeekStatus;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.TestUtil;
+
+/**
+ * Tests Lucene50DocValuesFormat
+ */
+public class TestLucene50DocValuesFormat extends BaseCompressingDocValuesFormatTestCase {
+  private final Codec codec = TestUtil.alwaysDocValuesFormat(new Lucene50DocValuesFormat());
+
+  @Override
+  protected Codec getCodec() {
+    return codec;
+  }
+  
+  // TODO: these big methods can easily blow up some of the other ram-hungry codecs...
+  // for now just keep them here, as we want to test this for this format.
+  
+  public void testSortedSetVariableLengthBigVsStoredFields() throws Exception {
+    int numIterations = atLeast(1);
+    for (int i = 0; i < numIterations; i++) {
+      doTestSortedSetVsStoredFields(atLeast(300), 1, 32766, 16);
+    }
+  }
+  
+  @Nightly
+  public void testSortedSetVariableLengthManyVsStoredFields() throws Exception {
+    int numIterations = atLeast(1);
+    for (int i = 0; i < numIterations; i++) {
+      doTestSortedSetVsStoredFields(TestUtil.nextInt(random(), 1024, 2049), 1, 500, 16);
+    }
+  }
+  
+  public void testSortedVariableLengthBigVsStoredFields() throws Exception {
+    int numIterations = atLeast(1);
+    for (int i = 0; i < numIterations; i++) {
+      doTestSortedVsStoredFields(atLeast(300), 1, 32766);
+    }
+  }
+  
+  @Nightly
+  public void testSortedVariableLengthManyVsStoredFields() throws Exception {
+    int numIterations = atLeast(1);
+    for (int i = 0; i < numIterations; i++) {
+      doTestSortedVsStoredFields(TestUtil.nextInt(random(), 1024, 2049), 1, 500);
+    }
+  }
+  
+  public void testTermsEnumFixedWidth() throws Exception {
+    int numIterations = atLeast(1);
+    for (int i = 0; i < numIterations; i++) {
+      doTestTermsEnumRandom(TestUtil.nextInt(random(), 1025, 5121), 10, 10);
+    }
+  }
+  
+  public void testTermsEnumVariableWidth() throws Exception {
+    int numIterations = atLeast(1);
+    for (int i = 0; i < numIterations; i++) {
+      doTestTermsEnumRandom(TestUtil.nextInt(random(), 1025, 5121), 1, 500);
+    }
+  }
+  
+  @Nightly
+  public void testTermsEnumRandomMany() throws Exception {
+    int numIterations = atLeast(1);
+    for (int i = 0; i < numIterations; i++) {
+      doTestTermsEnumRandom(TestUtil.nextInt(random(), 1025, 8121), 1, 500);
+    }
+  }
+  
+  // TODO: try to refactor this and some termsenum tests into the base class.
+  // to do this we need to fix the test class to get a DVF not a Codec so we can setup
+  // the postings format correctly.
+  private void doTestTermsEnumRandom(int numDocs, int minLength, int maxLength) throws Exception {
+    Directory dir = newFSDirectory(createTempDir());
+    IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
+    conf.setMergeScheduler(new SerialMergeScheduler());
+    // set to duel against a codec which has ordinals:
+    final PostingsFormat pf;
+    switch (random().nextInt(2)) {
+      case 0: pf = new Lucene41WithOrds();
+              break;
+      case 1: pf = new Ords41PostingsFormat();
+              break;
+      // TODO: these don't actually support ords!
+      //case 2: pf = new FSTOrdPostingsFormat();
+      //        break;
+      default: throw new AssertionError();
+    }
+    final DocValuesFormat dv = new Lucene50DocValuesFormat();
+    conf.setCodec(new AssertingCodec() {
+      @Override
+      public PostingsFormat getPostingsFormatForField(String field) {
+        return pf;
+      }
+
+      @Override
+      public DocValuesFormat getDocValuesFormatForField(String field) {
+        return dv;
+      }
+    });
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf);
+    
+    // index some docs
+    for (int i = 0; i < numDocs; i++) {
+      Document doc = new Document();
+      Field idField = new StringField("id", Integer.toString(i), Field.Store.NO);
+      doc.add(idField);
+      final int length = TestUtil.nextInt(random(), minLength, maxLength);
+      int numValues = random().nextInt(17);
+      // create a random list of strings
+      List<String> values = new ArrayList<>();
+      for (int v = 0; v < numValues; v++) {
+        values.add(TestUtil.randomSimpleString(random(), minLength, length));
+      }
+      
+      // add in any order to the indexed field
+      ArrayList<String> unordered = new ArrayList<>(values);
+      Collections.shuffle(unordered, random());
+      for (String v : values) {
+        doc.add(newStringField("indexed", v, Field.Store.NO));
+      }
+
+      // add in any order to the dv field
+      ArrayList<String> unordered2 = new ArrayList<>(values);
+      Collections.shuffle(unordered2, random());
+      for (String v : unordered2) {
+        doc.add(new SortedSetDocValuesField("dv", new BytesRef(v)));
+      }
+
+      writer.addDocument(doc);
+      if (random().nextInt(31) == 0) {
+        writer.commit();
+      }
+    }
+    
+    // delete some docs
+    int numDeletions = random().nextInt(numDocs/10);
+    for (int i = 0; i < numDeletions; i++) {
+      int id = random().nextInt(numDocs);
+      writer.deleteDocuments(new Term("id", Integer.toString(id)));
+    }
+    
+    // compare per-segment
+    DirectoryReader ir = writer.getReader();
+    for (LeafReaderContext context : ir.leaves()) {
+      LeafReader r = context.reader();
+      Terms terms = r.terms("indexed");
+      if (terms != null) {
+        assertEquals(terms.size(), r.getSortedSetDocValues("dv").getValueCount());
+        TermsEnum expected = terms.iterator(null);
+        TermsEnum actual = r.getSortedSetDocValues("dv").termsEnum();
+        assertEquals(terms.size(), expected, actual);
+      }
+    }
+    ir.close();
+    
+    writer.forceMerge(1);
+    
+    // now compare again after the merge
+    ir = writer.getReader();
+    LeafReader ar = getOnlySegmentReader(ir);
+    Terms terms = ar.terms("indexed");
+    if (terms != null) {
+      assertEquals(terms.size(), ar.getSortedSetDocValues("dv").getValueCount());
+      TermsEnum expected = terms.iterator(null);
+      TermsEnum actual = ar.getSortedSetDocValues("dv").termsEnum();
+      assertEquals(terms.size(), expected, actual);
+    }
+    ir.close();
+    
+    writer.close();
+    dir.close();
+  }
+  
+  private void assertEquals(long numOrds, TermsEnum expected, TermsEnum actual) throws Exception {
+    BytesRef ref;
+    
+    // sequential next() through all terms
+    while ((ref = expected.next()) != null) {
+      assertEquals(ref, actual.next());
+      assertEquals(expected.ord(), actual.ord());
+      assertEquals(expected.term(), actual.term());
+    }
+    assertNull(actual.next());
+    
+    // sequential seekExact(ord) through all terms
+    for (long i = 0; i < numOrds; i++) {
+      expected.seekExact(i);
+      actual.seekExact(i);
+      assertEquals(expected.ord(), actual.ord());
+      assertEquals(expected.term(), actual.term());
+    }
+    
+    // sequential seekExact(BytesRef) through all terms
+    for (long i = 0; i < numOrds; i++) {
+      expected.seekExact(i);
+      assertTrue(actual.seekExact(expected.term()));
+      assertEquals(expected.ord(), actual.ord());
+      assertEquals(expected.term(), actual.term());
+    }
+    
+    // sequential seekCeil(BytesRef) through all terms
+    for (long i = 0; i < numOrds; i++) {
+      expected.seekExact(i);
+      assertEquals(SeekStatus.FOUND, actual.seekCeil(expected.term()));
+      assertEquals(expected.ord(), actual.ord());
+      assertEquals(expected.term(), actual.term());
+    }
+    
+    // random seekExact(ord)
+    for (long i = 0; i < numOrds; i++) {
+      long randomOrd = TestUtil.nextLong(random(), 0, numOrds - 1);
+      expected.seekExact(randomOrd);
+      actual.seekExact(randomOrd);
+      assertEquals(expected.ord(), actual.ord());
+      assertEquals(expected.term(), actual.term());
+    }
+    
+    // random seekExact(BytesRef)
+    for (long i = 0; i < numOrds; i++) {
+      long randomOrd = TestUtil.nextLong(random(), 0, numOrds - 1);
+      expected.seekExact(randomOrd);
+      actual.seekExact(expected.term());
+      assertEquals(expected.ord(), actual.ord());
+      assertEquals(expected.term(), actual.term());
+    }
+    
+    // random seekCeil(BytesRef)
+    for (long i = 0; i < numOrds; i++) {
+      BytesRef target = new BytesRef(TestUtil.randomUnicodeString(random()));
+      SeekStatus expectedStatus = expected.seekCeil(target);
+      assertEquals(expectedStatus, actual.seekCeil(target));
+      if (expectedStatus != SeekStatus.END) {
+        assertEquals(expected.ord(), actual.ord());
+        assertEquals(expected.term(), actual.term());
+      }
+    }
+  }
+}

Modified: lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java?rev=1632200&r1=1632199&r2=1632200&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java (original)
+++ lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java Thu Oct 16 01:20:58 2014
@@ -2883,4 +2883,6 @@ public abstract class BaseDocValuesForma
   protected boolean codecSupportsSortedNumeric() {
     return true;
   }
+  
+  // nocommit: ensure every type has a *mergeAwayAllValues test, and add simple constant tests
 }

Modified: lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java?rev=1632200&r1=1632199&r2=1632200&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java (original)
+++ lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java Thu Oct 16 01:20:58 2014
@@ -46,8 +46,8 @@ import org.apache.lucene.codecs.DocValue
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.asserting.AssertingCodec;
 import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
-import org.apache.lucene.codecs.lucene410.Lucene410DocValuesFormat;
 import org.apache.lucene.codecs.lucene50.Lucene50Codec;
+import org.apache.lucene.codecs.lucene50.Lucene50DocValuesFormat;
 import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
 import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
 import org.apache.lucene.document.BinaryDocValuesField;
@@ -761,7 +761,7 @@ public final class TestUtil {
    * Returns the actual default docvalues format (e.g. LuceneMNDocValuesFormat for this version of Lucene.
    */
   public static DocValuesFormat getDefaultDocValuesFormat() {
-    return new Lucene410DocValuesFormat();
+    return new Lucene50DocValuesFormat();
   }
 
   // TODO: generalize all 'test-checks-for-crazy-codecs' to