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 2013/08/19 22:11:03 UTC

svn commit: r1515582 - in /lucene/dev/branches/lucene5178/lucene: codecs/src/java/org/apache/lucene/codecs/memory/ codecs/src/resources/META-INF/services/ codecs/src/test/org/apache/lucene/codecs/memory/ test-framework/src/java/org/apache/lucene/index/...

Author: rmuir
Date: Mon Aug 19 20:11:03 2013
New Revision: 1515582

URL: http://svn.apache.org/r1515582
Log:
add memorydv with missing support

Added:
    lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesConsumer.java   (with props)
    lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesFormat.java   (with props)
    lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesProducer.java   (with props)
    lucene/dev/branches/lucene5178/lucene/codecs/src/test/org/apache/lucene/codecs/memory/TestMemoryDocValuesFormat.java   (with props)
Modified:
    lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/memory/package.html
    lucene/dev/branches/lucene5178/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
    lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
    lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/util/_TestUtil.java

Added: lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesConsumer.java?rev=1515582&view=auto
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesConsumer.java (added)
+++ lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesConsumer.java Mon Aug 19 20:11:03 2013
@@ -0,0 +1,403 @@
+package org.apache.lucene.codecs.memory;
+
+/*
+ * 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 java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+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.ByteArrayDataOutput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.MathUtil;
+import org.apache.lucene.util.fst.Builder;
+import org.apache.lucene.util.fst.FST.INPUT_TYPE;
+import org.apache.lucene.util.fst.FST;
+import org.apache.lucene.util.fst.PositiveIntOutputs;
+import org.apache.lucene.util.fst.Util;
+import org.apache.lucene.util.packed.BlockPackedWriter;
+import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
+import org.apache.lucene.util.packed.PackedInts.FormatAndBits;
+import org.apache.lucene.util.packed.PackedInts;
+
+import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.VERSION_CURRENT;
+import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.BLOCK_SIZE;
+import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.BYTES;
+import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.NUMBER;
+import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.FST;
+import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.DELTA_COMPRESSED;
+import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.GCD_COMPRESSED;
+import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.TABLE_COMPRESSED;
+import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.UNCOMPRESSED;
+
+/**
+ * Writer for {@link MemoryDocValuesFormat}
+ */
+class MemoryDocValuesConsumer extends DocValuesConsumer {
+  final IndexOutput data, meta;
+  final int maxDoc;
+  final float acceptableOverheadRatio;
+  
+  MemoryDocValuesConsumer(SegmentWriteState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension, float acceptableOverheadRatio) throws IOException {
+    this.acceptableOverheadRatio = acceptableOverheadRatio;
+    maxDoc = state.segmentInfo.getDocCount();
+    boolean success = false;
+    try {
+      String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
+      data = state.directory.createOutput(dataName, state.context);
+      CodecUtil.writeHeader(data, dataCodec, VERSION_CURRENT);
+      String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
+      meta = state.directory.createOutput(metaName, state.context);
+      CodecUtil.writeHeader(meta, metaCodec, VERSION_CURRENT);
+      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 {
+    meta.writeVInt(field.number);
+    meta.writeByte(NUMBER);
+    meta.writeLong(data.getFilePointer());
+    long minValue = Long.MAX_VALUE;
+    long maxValue = Long.MIN_VALUE;
+    long gcd = 0;
+    boolean missing = false;
+    // TODO: more efficient?
+    HashSet<Long> uniqueValues = null;
+    if (optimizeStorage) {
+      uniqueValues = new HashSet<>();
+
+      long count = 0;
+      for (Number nv : values) {
+        final long v;
+        if (nv == null) {
+          v = 0;
+          missing = true;
+        } else {
+          v = nv.longValue();
+        }
+
+        if (gcd != 1) {
+          if (v < Long.MIN_VALUE / 2 || v > Long.MAX_VALUE / 2) {
+            // in that case v - minValue might overflow and make the GCD computation return
+            // wrong results. Since these extreme values are unlikely, we just discard
+            // GCD computation for them
+            gcd = 1;
+          } else if (count != 0) { // minValue needs to be set first
+            gcd = MathUtil.gcd(gcd, v - minValue);
+          }
+        }
+
+        minValue = Math.min(minValue, v);
+        maxValue = Math.max(maxValue, v);
+
+        if (uniqueValues != null) {
+          if (uniqueValues.add(v)) {
+            if (uniqueValues.size() > 256) {
+              uniqueValues = null;
+            }
+          }
+        }
+
+        ++count;
+      }
+      assert count == maxDoc;
+    }
+    
+    if (missing) {
+      long start = data.getFilePointer();
+      writeMissingBitset(values);
+      meta.writeLong(start);
+      meta.writeLong(data.getFilePointer() - start);
+    } else {
+      meta.writeLong(-1L);
+    }
+
+    if (uniqueValues != null) {
+      // small number of unique values
+      final int bitsPerValue = PackedInts.bitsRequired(uniqueValues.size()-1);
+      FormatAndBits formatAndBits = PackedInts.fastestFormatAndBits(maxDoc, bitsPerValue, acceptableOverheadRatio);
+      if (formatAndBits.bitsPerValue == 8 && minValue >= Byte.MIN_VALUE && maxValue <= Byte.MAX_VALUE) {
+        meta.writeByte(UNCOMPRESSED); // uncompressed
+        for (Number nv : values) {
+          data.writeByte(nv == null ? 0 : (byte) nv.longValue());
+        }
+      } else {
+        meta.writeByte(TABLE_COMPRESSED); // table-compressed
+        Long[] decode = uniqueValues.toArray(new Long[uniqueValues.size()]);
+        final HashMap<Long,Integer> encode = new HashMap<Long,Integer>();
+        data.writeVInt(decode.length);
+        for (int i = 0; i < decode.length; i++) {
+          data.writeLong(decode[i]);
+          encode.put(decode[i], i);
+        }
+
+        meta.writeVInt(PackedInts.VERSION_CURRENT);
+        data.writeVInt(formatAndBits.format.getId());
+        data.writeVInt(formatAndBits.bitsPerValue);
+
+        final PackedInts.Writer writer = PackedInts.getWriterNoHeader(data, formatAndBits.format, maxDoc, formatAndBits.bitsPerValue, PackedInts.DEFAULT_BUFFER_SIZE);
+        for(Number nv : values) {
+          writer.add(encode.get(nv == null ? 0 : nv.longValue()));
+        }
+        writer.finish();
+      }
+    } else if (gcd != 0 && gcd != 1) {
+      meta.writeByte(GCD_COMPRESSED);
+      meta.writeVInt(PackedInts.VERSION_CURRENT);
+      data.writeLong(minValue);
+      data.writeLong(gcd);
+      data.writeVInt(BLOCK_SIZE);
+
+      final BlockPackedWriter writer = new BlockPackedWriter(data, BLOCK_SIZE);
+      for (Number nv : values) {
+        long value = nv == null ? 0 : nv.longValue();
+        writer.add((value - minValue) / gcd);
+      }
+      writer.finish();
+    } else {
+      meta.writeByte(DELTA_COMPRESSED); // delta-compressed
+
+      meta.writeVInt(PackedInts.VERSION_CURRENT);
+      data.writeVInt(BLOCK_SIZE);
+
+      final BlockPackedWriter writer = new BlockPackedWriter(data, BLOCK_SIZE);
+      for (Number nv : values) {
+        writer.add(nv == null ? 0 : nv.longValue());
+      }
+      writer.finish();
+    }
+  }
+  
+  @Override
+  public void close() throws IOException {
+    boolean success = false;
+    try {
+      if (meta != null) {
+        meta.writeVInt(-1); // write EOF marker
+      }
+      success = true;
+    } finally {
+      if (success) {
+        IOUtils.close(data, meta);
+      } else {
+        IOUtils.closeWhileHandlingException(data, meta);
+      }
+    }
+  }
+
+  @Override
+  public void addBinaryField(FieldInfo field, final Iterable<BytesRef> values) throws IOException {
+    // write the byte[] data
+    meta.writeVInt(field.number);
+    meta.writeByte(BYTES);
+    int minLength = Integer.MAX_VALUE;
+    int maxLength = Integer.MIN_VALUE;
+    final long startFP = data.getFilePointer();
+    boolean missing = false;
+    for(BytesRef v : values) {
+      final int length;
+      if (v == null) {
+        length = 0;
+        missing = true;
+      } else {
+        length = v.length;
+      }
+      if (length > MemoryDocValuesFormat.MAX_BINARY_FIELD_LENGTH) {
+        throw new IllegalArgumentException("DocValuesField \"" + field.name + "\" is too large, must be <= " + MemoryDocValuesFormat.MAX_BINARY_FIELD_LENGTH);
+      }
+      minLength = Math.min(minLength, length);
+      maxLength = Math.max(maxLength, length);
+      if (v != null) {
+        data.writeBytes(v.bytes, v.offset, v.length);
+      }
+    }
+    meta.writeLong(startFP);
+    meta.writeLong(data.getFilePointer() - startFP);
+    if (missing) {
+      long start = data.getFilePointer();
+      writeMissingBitset(values);
+      meta.writeLong(start);
+      meta.writeLong(data.getFilePointer() - start);
+    } else {
+      meta.writeLong(-1L);
+    }
+    meta.writeVInt(minLength);
+    meta.writeVInt(maxLength);
+    
+    // if minLength == maxLength, its a fixed-length byte[], we are done (the addresses are implicit)
+    // otherwise, we need to record the length fields...
+    if (minLength != maxLength) {
+      meta.writeVInt(PackedInts.VERSION_CURRENT);
+      meta.writeVInt(BLOCK_SIZE);
+
+      final MonotonicBlockPackedWriter writer = new MonotonicBlockPackedWriter(data, BLOCK_SIZE);
+      long addr = 0;
+      for (BytesRef v : values) {
+        if (v != null) {
+          addr += v.length;
+        }
+        writer.add(addr);
+      }
+      writer.finish();
+    }
+  }
+  
+  private void writeFST(FieldInfo field, Iterable<BytesRef> values) throws IOException {
+    meta.writeVInt(field.number);
+    meta.writeByte(FST);
+    meta.writeLong(data.getFilePointer());
+    PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
+    Builder<Long> builder = new Builder<Long>(INPUT_TYPE.BYTE1, outputs);
+    IntsRef scratch = new IntsRef();
+    long ord = 0;
+    for (BytesRef v : values) {
+      builder.add(Util.toIntsRef(v, scratch), ord);
+      ord++;
+    }
+    FST<Long> fst = builder.finish();
+    if (fst != null) {
+      fst.save(data);
+    }
+    meta.writeVLong(ord);
+  }
+  
+  // TODO: in some cases representing missing with minValue-1 wouldn't take up additional space and so on,
+  // but this is very simple, and algorithms only check this for values of 0 anyway (doesnt slow down normal decode)
+  void writeMissingBitset(Iterable<?> values) throws IOException {
+    long bits = 0;
+    int count = 0;
+    for (Object v : values) {
+      if (count == 64) {
+        data.writeLong(bits);
+        count = 0;
+        bits = 0;
+      }
+      if (v != null) {
+        bits |= 1L << (count & 0x3f);
+      }
+      count++;
+    }
+    if (count > 0) {
+      data.writeLong(bits);
+    }
+  }
+
+  @Override
+  public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
+    // write the ordinals as numerics
+    addNumericField(field, docToOrd, false);
+    
+    // write the values as FST
+    writeFST(field, values);
+  }
+
+  // note: this might not be the most efficient... but its fairly simple
+  @Override
+  public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, final Iterable<Number> docToOrdCount, final Iterable<Number> ords) throws IOException {
+    // write the ordinals as a binary field
+    addBinaryField(field, new Iterable<BytesRef>() {
+      @Override
+      public Iterator<BytesRef> iterator() {
+        return new SortedSetIterator(docToOrdCount.iterator(), ords.iterator());
+      }
+    });
+      
+    // write the values as FST
+    writeFST(field, values);
+  }
+  
+  // per-document vint-encoded byte[]
+  static class SortedSetIterator implements Iterator<BytesRef> {
+    byte[] buffer = new byte[10];
+    ByteArrayDataOutput out = new ByteArrayDataOutput();
+    BytesRef ref = new BytesRef();
+    
+    final Iterator<Number> counts;
+    final Iterator<Number> ords;
+    
+    SortedSetIterator(Iterator<Number> counts, Iterator<Number> ords) {
+      this.counts = counts;
+      this.ords = ords;
+    }
+    
+    @Override
+    public boolean hasNext() {
+      return counts.hasNext();
+    }
+
+    @Override
+    public BytesRef next() {
+      if (!hasNext()) {
+        throw new NoSuchElementException();
+      }
+      
+      int count = counts.next().intValue();
+      int maxSize = count*9; // worst case
+      if (maxSize > buffer.length) {
+        buffer = ArrayUtil.grow(buffer, maxSize);
+      }
+      
+      try {
+        encodeValues(count);
+      } catch (IOException bogus) {
+        throw new RuntimeException(bogus);
+      }
+      
+      ref.bytes = buffer;
+      ref.offset = 0;
+      ref.length = out.getPosition();
+
+      return ref;
+    }
+    
+    // encodes count values to buffer
+    private void encodeValues(int count) throws IOException {
+      out.reset(buffer);
+      long lastOrd = 0;
+      for (int i = 0; i < count; i++) {
+        long ord = ords.next().longValue();
+        out.writeVLong(ord - lastOrd);
+        lastOrd = ord;
+      }
+    }
+
+    @Override
+    public void remove() {
+      throw new UnsupportedOperationException();
+    }
+  }
+}

Added: lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesFormat.java?rev=1515582&view=auto
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesFormat.java (added)
+++ lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesFormat.java Mon Aug 19 20:11:03 2013
@@ -0,0 +1,72 @@
+package org.apache.lucene.codecs.memory;
+
+/*
+ * 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.DocValuesProducer;
+import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.util.packed.PackedInts;
+
+/** In-memory docvalues format */
+public class MemoryDocValuesFormat extends DocValuesFormat {
+
+  /** Maximum length for each binary doc values field. */
+  public static final int MAX_BINARY_FIELD_LENGTH = (1 << 15) - 2;
+  
+  final float acceptableOverheadRatio;
+  
+  /** 
+   * Calls {@link #MemoryDocValuesFormat(float) 
+   * MemoryDocValuesFormat(PackedInts.DEFAULT)} 
+   */
+  public MemoryDocValuesFormat() {
+    this(PackedInts.DEFAULT);
+  }
+  
+  /**
+   * Creates a new MemoryDocValuesFormat with the specified
+   * <code>acceptableOverheadRatio</code> for NumericDocValues.
+   * @param acceptableOverheadRatio compression parameter for numerics. 
+   *        Currently this is only used when the number of unique values is small.
+   *        
+   * @lucene.experimental
+   */
+  public MemoryDocValuesFormat(float acceptableOverheadRatio) {
+    super("Memory");
+    this.acceptableOverheadRatio = acceptableOverheadRatio;
+  }
+
+  @Override
+  public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+    return new MemoryDocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION, acceptableOverheadRatio);
+  }
+  
+  @Override
+  public DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException {
+    return new MemoryDocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
+  }
+  
+  static final String DATA_CODEC = "MemoryDocValuesData";
+  static final String DATA_EXTENSION = "mdvd";
+  static final String METADATA_CODEC = "MemoryDocValuesMetadata";
+  static final String METADATA_EXTENSION = "mdvm";
+}

Added: lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesProducer.java?rev=1515582&view=auto
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesProducer.java (added)
+++ lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesProducer.java Mon Aug 19 20:11:03 2013
@@ -0,0 +1,633 @@
+package org.apache.lucene.codecs.memory;
+
+/*
+ * 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 java.util.Comparator;
+import java.util.HashMap;
+import java.util.Map;
+
+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.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.SegmentReadState;
+import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.store.ByteArrayDataInput;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.PagedBytes;
+import org.apache.lucene.util.fst.BytesRefFSTEnum;
+import org.apache.lucene.util.fst.BytesRefFSTEnum.InputOutput;
+import org.apache.lucene.util.fst.FST;
+import org.apache.lucene.util.fst.FST.Arc;
+import org.apache.lucene.util.fst.FST.BytesReader;
+import org.apache.lucene.util.fst.PositiveIntOutputs;
+import org.apache.lucene.util.fst.Util;
+import org.apache.lucene.util.packed.BlockPackedReader;
+import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
+import org.apache.lucene.util.packed.PackedInts;
+
+/**
+ * Reader for {@link MemoryDocValuesFormat}
+ */
+class MemoryDocValuesProducer extends DocValuesProducer {
+  // metadata maps (just file pointers and minimal stuff)
+  private final Map<Integer,NumericEntry> numerics;
+  private final Map<Integer,BinaryEntry> binaries;
+  private final Map<Integer,FSTEntry> fsts;
+  private final IndexInput data;
+  
+  // ram instances we have already loaded
+  private final Map<Integer,NumericDocValues> numericInstances = 
+      new HashMap<Integer,NumericDocValues>();
+  private final Map<Integer,BinaryDocValues> binaryInstances =
+      new HashMap<Integer,BinaryDocValues>();
+  private final Map<Integer,FST<Long>> fstInstances =
+      new HashMap<Integer,FST<Long>>();
+  private final Map<Integer,Bits> docsWithFieldInstances = new HashMap<Integer,Bits>();
+  
+  private final int maxDoc;
+  
+  
+  static final byte NUMBER = 0;
+  static final byte BYTES = 1;
+  static final byte FST = 2;
+
+  static final int BLOCK_SIZE = 4096;
+  
+  static final byte DELTA_COMPRESSED = 0;
+  static final byte TABLE_COMPRESSED = 1;
+  static final byte UNCOMPRESSED = 2;
+  static final byte GCD_COMPRESSED = 3;
+  
+  static final int VERSION_START = 0;
+  static final int VERSION_GCD_COMPRESSION = 1;
+  static final int VERSION_CURRENT = VERSION_GCD_COMPRESSION;
+    
+  MemoryDocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
+    maxDoc = state.segmentInfo.getDocCount();
+    String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
+    // read in the entries from the metadata file.
+    IndexInput in = state.directory.openInput(metaName, state.context);
+    boolean success = false;
+    final int version;
+    try {
+      version = CodecUtil.checkHeader(in, metaCodec, 
+                                      VERSION_START,
+                                      VERSION_CURRENT);
+      numerics = new HashMap<Integer,NumericEntry>();
+      binaries = new HashMap<Integer,BinaryEntry>();
+      fsts = new HashMap<Integer,FSTEntry>();
+      readFields(in, state.fieldInfos);
+
+      success = true;
+    } finally {
+      if (success) {
+        IOUtils.close(in);
+      } else {
+        IOUtils.closeWhileHandlingException(in);
+      }
+    }
+
+    success = false;
+    try {
+      String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
+      data = state.directory.openInput(dataName, state.context);
+      final int version2 = CodecUtil.checkHeader(data, dataCodec, 
+                                                 VERSION_START,
+                                                 VERSION_CURRENT);
+      if (version != version2) {
+        throw new CorruptIndexException("Format versions mismatch");
+      }
+
+      success = true;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(this.data);
+      }
+    }
+  }
+  
+  private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
+    int fieldNumber = meta.readVInt();
+    while (fieldNumber != -1) {
+      int fieldType = meta.readByte();
+      if (fieldType == NUMBER) {
+        NumericEntry entry = new NumericEntry();
+        entry.offset = meta.readLong();
+        entry.missingOffset = meta.readLong();
+        if (entry.missingOffset != -1) {
+          entry.missingBytes = meta.readLong();
+        } else {
+          entry.missingBytes = 0;
+        }
+        entry.format = meta.readByte();
+        switch(entry.format) {
+          case DELTA_COMPRESSED:
+          case TABLE_COMPRESSED:
+          case GCD_COMPRESSED:
+          case UNCOMPRESSED:
+               break;
+          default:
+               throw new CorruptIndexException("Unknown format: " + entry.format + ", input=" + meta);
+        }
+        if (entry.format != UNCOMPRESSED) {
+          entry.packedIntsVersion = meta.readVInt();
+        }
+        numerics.put(fieldNumber, entry);
+      } else if (fieldType == BYTES) {
+        BinaryEntry entry = new BinaryEntry();
+        entry.offset = meta.readLong();
+        entry.numBytes = meta.readLong();
+        entry.missingOffset = meta.readLong();
+        if (entry.missingOffset != -1) {
+          entry.missingBytes = meta.readLong();
+        } else {
+          entry.missingBytes = 0;
+        }
+        entry.minLength = meta.readVInt();
+        entry.maxLength = meta.readVInt();
+        if (entry.minLength != entry.maxLength) {
+          entry.packedIntsVersion = meta.readVInt();
+          entry.blockSize = meta.readVInt();
+        }
+        binaries.put(fieldNumber, entry);
+      } else if (fieldType == FST) {
+        FSTEntry entry = new FSTEntry();
+        entry.offset = meta.readLong();
+        entry.numOrds = meta.readVLong();
+        fsts.put(fieldNumber, entry);
+      } else {
+        throw new CorruptIndexException("invalid entry type: " + fieldType + ", input=" + meta);
+      }
+      fieldNumber = meta.readVInt();
+    }
+  }
+
+  @Override
+  public synchronized NumericDocValues getNumeric(FieldInfo field) throws IOException {
+    NumericDocValues instance = numericInstances.get(field.number);
+    if (instance == null) {
+      instance = loadNumeric(field);
+      numericInstances.put(field.number, instance);
+    }
+    return instance;
+  }
+  
+  private NumericDocValues loadNumeric(FieldInfo field) throws IOException {
+    NumericEntry entry = numerics.get(field.number);
+    data.seek(entry.offset + entry.missingBytes);
+    switch (entry.format) {
+      case TABLE_COMPRESSED:
+        int size = data.readVInt();
+        if (size > 256) {
+          throw new CorruptIndexException("TABLE_COMPRESSED cannot have more than 256 distinct values, input=" + data);
+        }
+        final long decode[] = new long[size];
+        for (int i = 0; i < decode.length; i++) {
+          decode[i] = data.readLong();
+        }
+        final int formatID = data.readVInt();
+        final int bitsPerValue = data.readVInt();
+        final PackedInts.Reader ordsReader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatID), entry.packedIntsVersion, maxDoc, bitsPerValue);
+        return new NumericDocValues() {
+          @Override
+          public long get(int docID) {
+            return decode[(int)ordsReader.get(docID)];
+          }
+        };
+      case DELTA_COMPRESSED:
+        final int blockSize = data.readVInt();
+        final BlockPackedReader reader = new BlockPackedReader(data, entry.packedIntsVersion, blockSize, maxDoc, false);
+        return new NumericDocValues() {
+          @Override
+          public long get(int docID) {
+            return reader.get(docID);
+          }
+        };
+      case UNCOMPRESSED:
+        final byte bytes[] = new byte[maxDoc];
+        data.readBytes(bytes, 0, bytes.length);
+        return new NumericDocValues() {
+          @Override
+          public long get(int docID) {
+            return bytes[docID];
+          }
+        };
+      case GCD_COMPRESSED:
+        final long min = data.readLong();
+        final long mult = data.readLong();
+        final int quotientBlockSize = data.readVInt();
+        final BlockPackedReader quotientReader = new BlockPackedReader(data, entry.packedIntsVersion, quotientBlockSize, maxDoc, false);
+        return new NumericDocValues() {
+          @Override
+          public long get(int docID) {
+            return min + mult * quotientReader.get(docID);
+          }
+        };
+      default:
+        throw new AssertionError();
+    }
+  }
+
+  @Override
+  public synchronized BinaryDocValues getBinary(FieldInfo field) throws IOException {
+    BinaryDocValues instance = binaryInstances.get(field.number);
+    if (instance == null) {
+      instance = loadBinary(field);
+      binaryInstances.put(field.number, instance);
+    }
+    return instance;
+  }
+  
+  private BinaryDocValues loadBinary(FieldInfo field) throws IOException {
+    BinaryEntry entry = binaries.get(field.number);
+    data.seek(entry.offset);
+    PagedBytes bytes = new PagedBytes(16);
+    bytes.copy(data, entry.numBytes);
+    final PagedBytes.Reader bytesReader = bytes.freeze(true);
+    if (entry.minLength == entry.maxLength) {
+      final int fixedLength = entry.minLength;
+      return new BinaryDocValues() {
+        @Override
+        public void get(int docID, BytesRef result) {
+          bytesReader.fillSlice(result, fixedLength * (long)docID, fixedLength);
+        }
+      };
+    } else {
+      data.seek(data.getFilePointer() + entry.missingBytes);
+      final MonotonicBlockPackedReader addresses = new MonotonicBlockPackedReader(data, entry.packedIntsVersion, entry.blockSize, maxDoc, false);
+      return new BinaryDocValues() {
+        @Override
+        public void get(int docID, BytesRef result) {
+          long startAddress = docID == 0 ? 0 : addresses.get(docID-1);
+          long endAddress = addresses.get(docID); 
+          bytesReader.fillSlice(result, startAddress, (int) (endAddress - startAddress));
+        }
+      };
+    }
+  }
+  
+  @Override
+  public SortedDocValues getSorted(FieldInfo field) throws IOException {
+    final FSTEntry entry = fsts.get(field.number);
+    if (entry.numOrds == 0) {
+      return SortedDocValues.EMPTY;
+    }
+    FST<Long> instance;
+    synchronized(this) {
+      instance = fstInstances.get(field.number);
+      if (instance == null) {
+        data.seek(entry.offset);
+        instance = new FST<Long>(data, PositiveIntOutputs.getSingleton());
+        fstInstances.put(field.number, instance);
+      }
+    }
+    final NumericDocValues docToOrd = getNumeric(field);
+    final FST<Long> fst = instance;
+    
+    // per-thread resources
+    final BytesReader in = fst.getBytesReader();
+    final Arc<Long> firstArc = new Arc<Long>();
+    final Arc<Long> scratchArc = new Arc<Long>();
+    final IntsRef scratchInts = new IntsRef();
+    final BytesRefFSTEnum<Long> fstEnum = new BytesRefFSTEnum<Long>(fst); 
+    
+    return new SortedDocValues() {
+      @Override
+      public int getOrd(int docID) {
+        return (int) docToOrd.get(docID);
+      }
+
+      @Override
+      public void lookupOrd(int ord, BytesRef result) {
+        try {
+          in.setPosition(0);
+          fst.getFirstArc(firstArc);
+          IntsRef output = Util.getByOutput(fst, ord, in, firstArc, scratchArc, scratchInts);
+          result.bytes = new byte[output.length];
+          result.offset = 0;
+          result.length = 0;
+          Util.toBytesRef(output, result);
+        } catch (IOException bogus) {
+          throw new RuntimeException(bogus);
+        }
+      }
+
+      @Override
+      public int lookupTerm(BytesRef key) {
+        try {
+          InputOutput<Long> o = fstEnum.seekCeil(key);
+          if (o == null) {
+            return -getValueCount()-1;
+          } else if (o.input.equals(key)) {
+            return o.output.intValue();
+          } else {
+            return (int) -o.output-1;
+          }
+        } catch (IOException bogus) {
+          throw new RuntimeException(bogus);
+        }
+      }
+
+      @Override
+      public int getValueCount() {
+        return (int)entry.numOrds;
+      }
+
+      @Override
+      public TermsEnum termsEnum() {
+        return new FSTTermsEnum(fst);
+      }
+    };
+  }
+  
+  @Override
+  public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
+    final FSTEntry entry = fsts.get(field.number);
+    if (entry.numOrds == 0) {
+      return SortedSetDocValues.EMPTY; // empty FST!
+    }
+    FST<Long> instance;
+    synchronized(this) {
+      instance = fstInstances.get(field.number);
+      if (instance == null) {
+        data.seek(entry.offset);
+        instance = new FST<Long>(data, PositiveIntOutputs.getSingleton());
+        fstInstances.put(field.number, instance);
+      }
+    }
+    final BinaryDocValues docToOrds = getBinary(field);
+    final FST<Long> fst = instance;
+    
+    // per-thread resources
+    final BytesReader in = fst.getBytesReader();
+    final Arc<Long> firstArc = new Arc<Long>();
+    final Arc<Long> scratchArc = new Arc<Long>();
+    final IntsRef scratchInts = new IntsRef();
+    final BytesRefFSTEnum<Long> fstEnum = new BytesRefFSTEnum<Long>(fst); 
+    final BytesRef ref = new BytesRef();
+    final ByteArrayDataInput input = new ByteArrayDataInput();
+    return new SortedSetDocValues() {
+      long currentOrd;
+
+      @Override
+      public long nextOrd() {
+        if (input.eof()) {
+          return NO_MORE_ORDS;
+        } else {
+          currentOrd += input.readVLong();
+          return currentOrd;
+        }
+      }
+      
+      @Override
+      public void setDocument(int docID) {
+        docToOrds.get(docID, ref);
+        input.reset(ref.bytes, ref.offset, ref.length);
+        currentOrd = 0;
+      }
+
+      @Override
+      public void lookupOrd(long ord, BytesRef result) {
+        try {
+          in.setPosition(0);
+          fst.getFirstArc(firstArc);
+          IntsRef output = Util.getByOutput(fst, ord, in, firstArc, scratchArc, scratchInts);
+          result.bytes = new byte[output.length];
+          result.offset = 0;
+          result.length = 0;
+          Util.toBytesRef(output, result);
+        } catch (IOException bogus) {
+          throw new RuntimeException(bogus);
+        }
+      }
+
+      @Override
+      public long lookupTerm(BytesRef key) {
+        try {
+          InputOutput<Long> o = fstEnum.seekCeil(key);
+          if (o == null) {
+            return -getValueCount()-1;
+          } else if (o.input.equals(key)) {
+            return o.output.intValue();
+          } else {
+            return -o.output-1;
+          }
+        } catch (IOException bogus) {
+          throw new RuntimeException(bogus);
+        }
+      }
+
+      @Override
+      public long getValueCount() {
+        return entry.numOrds;
+      }
+
+      @Override
+      public TermsEnum termsEnum() {
+        return new FSTTermsEnum(fst);
+      }
+    };
+  }
+  
+  private Bits getMissingBits(int fieldNumber, final long offset, final long length) throws IOException {
+    if (offset == -1) {
+      return new Bits.MatchAllBits(maxDoc);
+    } else {
+      Bits instance;
+      synchronized(this) {
+        instance = docsWithFieldInstances.get(fieldNumber);
+        if (instance == null) {
+          IndexInput data = this.data.clone();
+          data.seek(offset);
+          assert length % 8 == 0;
+          long bits[] = new long[(int) length >> 3];
+          for (int i = 0; i < bits.length; i++) {
+            bits[i] = data.readLong();
+          }
+          instance = new FixedBitSet(bits, maxDoc);
+          docsWithFieldInstances.put(fieldNumber, instance);
+        }
+      }
+      return instance;
+    }
+  }
+  
+  @Override
+  public Bits getDocsWithField(FieldInfo field) throws IOException {
+    switch(field.getDocValuesType()) {
+      case SORTED_SET:
+        return new SortedSetDocsWithField(getSortedSet(field), maxDoc);
+      case SORTED:
+        return new SortedDocsWithField(getSorted(field), maxDoc);
+      case BINARY:
+        BinaryEntry be = binaries.get(field.number);
+        return getMissingBits(field.number, be.missingOffset, be.missingBytes);
+      case NUMERIC:
+        NumericEntry ne = numerics.get(field.number);
+        return getMissingBits(field.number, ne.missingOffset, ne.missingBytes);
+      default: 
+        throw new AssertionError();
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    data.close();
+  }
+  
+  static class NumericEntry {
+    long offset;
+    long missingOffset;
+    long missingBytes;
+    byte format;
+    int packedIntsVersion;
+  }
+  
+  static class BinaryEntry {
+    long offset;
+    long missingOffset;
+    long missingBytes;
+    long numBytes;
+    int minLength;
+    int maxLength;
+    int packedIntsVersion;
+    int blockSize;
+  }
+  
+  static class FSTEntry {
+    long offset;
+    long numOrds;
+  }
+  
+  // exposes FSTEnum directly as a TermsEnum: avoids binary-search next()
+  static class FSTTermsEnum extends TermsEnum {
+    final BytesRefFSTEnum<Long> in;
+    
+    // this is all for the complicated seek(ord)...
+    // maybe we should add a FSTEnum that supports this operation?
+    final FST<Long> fst;
+    final FST.BytesReader bytesReader;
+    final Arc<Long> firstArc = new Arc<Long>();
+    final Arc<Long> scratchArc = new Arc<Long>();
+    final IntsRef scratchInts = new IntsRef();
+    final BytesRef scratchBytes = new BytesRef();
+    
+    FSTTermsEnum(FST<Long> fst) {
+      this.fst = fst;
+      in = new BytesRefFSTEnum<Long>(fst);
+      bytesReader = fst.getBytesReader();
+    }
+
+    @Override
+    public BytesRef next() throws IOException {
+      InputOutput<Long> io = in.next();
+      if (io == null) {
+        return null;
+      } else {
+        return io.input;
+      }
+    }
+
+    @Override
+    public Comparator<BytesRef> getComparator() {
+      return BytesRef.getUTF8SortedAsUnicodeComparator();
+    }
+
+    @Override
+    public SeekStatus seekCeil(BytesRef text) throws IOException {
+      if (in.seekCeil(text) == null) {
+        return SeekStatus.END;
+      } else if (term().equals(text)) {
+        // TODO: add SeekStatus to FSTEnum like in https://issues.apache.org/jira/browse/LUCENE-3729
+        // to remove this comparision?
+        return SeekStatus.FOUND;
+      } else {
+        return SeekStatus.NOT_FOUND;
+      }
+    }
+
+    @Override
+    public boolean seekExact(BytesRef text) throws IOException {
+      if (in.seekExact(text) == null) {
+        return false;
+      } else {
+        return true;
+      }
+    }
+
+    @Override
+    public void seekExact(long ord) throws IOException {
+      // TODO: would be better to make this simpler and faster.
+      // but we dont want to introduce a bug that corrupts our enum state!
+      bytesReader.setPosition(0);
+      fst.getFirstArc(firstArc);
+      IntsRef output = Util.getByOutput(fst, ord, bytesReader, firstArc, scratchArc, scratchInts);
+      scratchBytes.bytes = new byte[output.length];
+      scratchBytes.offset = 0;
+      scratchBytes.length = 0;
+      Util.toBytesRef(output, scratchBytes);
+      // TODO: we could do this lazily, better to try to push into FSTEnum though?
+      in.seekExact(scratchBytes);
+    }
+
+    @Override
+    public BytesRef term() throws IOException {
+      return in.current().input;
+    }
+
+    @Override
+    public long ord() throws IOException {
+      return in.current().output;
+    }
+
+    @Override
+    public int docFreq() throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public long totalTermFreq() throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @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/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/memory/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/memory/package.html?rev=1515582&r1=1515581&r2=1515582&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/memory/package.html (original)
+++ lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/memory/package.html Mon Aug 19 20:11:03 2013
@@ -20,6 +20,6 @@
    <meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
 </head>
 <body>
-Postings format that is read entirely into memory.
+Postings and DocValues formats that are read entirely into memory.
 </body>
 </html>
\ No newline at end of file

Modified: lucene/dev/branches/lucene5178/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat?rev=1515582&r1=1515581&r2=1515582&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat (original)
+++ lucene/dev/branches/lucene5178/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat Mon Aug 19 20:11:03 2013
@@ -14,4 +14,5 @@
 #  limitations under the License.
 
 org.apache.lucene.codecs.diskdv.DiskDocValuesFormat
+org.apache.lucene.codecs.memory.MemoryDocValuesFormat
 org.apache.lucene.codecs.simpletext.SimpleTextDocValuesFormat

Added: lucene/dev/branches/lucene5178/lucene/codecs/src/test/org/apache/lucene/codecs/memory/TestMemoryDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/codecs/src/test/org/apache/lucene/codecs/memory/TestMemoryDocValuesFormat.java?rev=1515582&view=auto
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/codecs/src/test/org/apache/lucene/codecs/memory/TestMemoryDocValuesFormat.java (added)
+++ lucene/dev/branches/lucene5178/lucene/codecs/src/test/org/apache/lucene/codecs/memory/TestMemoryDocValuesFormat.java Mon Aug 19 20:11:03 2013
@@ -0,0 +1,39 @@
+package org.apache.lucene.codecs.memory;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.index.BaseCompressingDocValuesFormatTestCase;
+import org.apache.lucene.util._TestUtil;
+
+/**
+ * Tests MemoryDocValuesFormat
+ */
+public class TestMemoryDocValuesFormat extends BaseCompressingDocValuesFormatTestCase {
+  private final Codec codec = _TestUtil.alwaysDocValuesFormat(new MemoryDocValuesFormat());
+
+  @Override
+  protected Codec getCodec() {
+    return codec;
+  }
+
+  @Override
+  protected boolean codecAcceptsHugeBinaryValues(String field) {
+    return false;
+  }
+}

Modified: lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java?rev=1515582&r1=1515581&r2=1515582&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java (original)
+++ lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java Mon Aug 19 20:11:03 2013
@@ -40,6 +40,7 @@ import org.apache.lucene.codecs.lucene45
 import org.apache.lucene.codecs.bloom.TestBloomFilteredLucene41Postings;
 import org.apache.lucene.codecs.diskdv.DiskDocValuesFormat;
 import org.apache.lucene.codecs.memory.DirectPostingsFormat;
+import org.apache.lucene.codecs.memory.MemoryDocValuesFormat;
 import org.apache.lucene.codecs.memory.MemoryPostingsFormat;
 import org.apache.lucene.codecs.mockintblock.MockFixedIntBlockPostingsFormat;
 import org.apache.lucene.codecs.mockintblock.MockVariableIntBlockPostingsFormat;
@@ -149,6 +150,7 @@ public class RandomCodec extends Lucene4
     addDocValues(avoidCodecs,
         new Lucene45DocValuesFormat(),
         new DiskDocValuesFormat(),
+        new MemoryDocValuesFormat(),
         new SimpleTextDocValuesFormat(),
         new AssertingDocValuesFormat());
 

Modified: lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/util/_TestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/util/_TestUtil.java?rev=1515582&r1=1515581&r2=1515582&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/util/_TestUtil.java (original)
+++ lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/util/_TestUtil.java Mon Aug 19 20:11:03 2013
@@ -760,7 +760,7 @@ public class _TestUtil {
   // TODO: remove this, push this test to Lucene40/Lucene42 codec tests
   public static boolean fieldSupportsHugeBinaryDocValues(String field) {
     String dvFormat = getDocValuesFormat(field);
-    if (dvFormat.equals("Lucene40") || dvFormat.equals("Lucene42")) {
+    if (dvFormat.equals("Lucene40") || dvFormat.equals("Lucene42") || dvFormat.equals("Memory")) {
       return false;
     }
     return true;