You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jp...@apache.org on 2015/10/27 20:49:47 UTC

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

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/DirectMonotonicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/DirectMonotonicReader.java?rev=1710876&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/DirectMonotonicReader.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/DirectMonotonicReader.java Tue Oct 27 19:49:47 2015
@@ -0,0 +1,122 @@
+package org.apache.lucene.util.packed;
+
+/*
+ * 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.store.IndexInput;
+import org.apache.lucene.store.RandomAccessInput;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.LongValues;
+import org.apache.lucene.util.RamUsageEstimator;
+
+/**
+ * Retrieves an instance previously written by {@link DirectMonotonicWriter}.
+ * @see DirectMonotonicWriter 
+ */
+public final class DirectMonotonicReader {
+
+  /** An instance that always returns {@code 0}. */
+  private static final LongValues EMPTY = new LongValues() {
+
+    @Override
+    public long get(long index) {
+      return 0;
+    }
+
+  };
+
+  /** In-memory metadata that needs to be kept around for
+   *  {@link DirectMonotonicReader} to read data from disk. */
+  public static class Meta implements Accountable {
+    private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(Meta.class);
+
+    final long numValues;
+    final int blockShift;
+    final int numBlocks;
+    final long[] mins;
+    final float[] avgs;
+    final byte[] bpvs;
+    final long[] offsets;
+
+    Meta(long numValues, int blockShift) {
+      this.numValues = numValues;
+      this.blockShift = blockShift;
+      long numBlocks = numValues >>> blockShift;
+      if ((numBlocks << blockShift) < numValues) {
+        numBlocks += 1;
+      }
+      this.numBlocks = (int) numBlocks;
+      this.mins = new long[this.numBlocks];
+      this.avgs = new float[this.numBlocks];
+      this.bpvs = new byte[this.numBlocks];
+      this.offsets = new long[this.numBlocks];
+    }
+
+    @Override
+    public long ramBytesUsed() {
+      return BASE_RAM_BYTES_USED
+          + RamUsageEstimator.sizeOf(mins)
+          + RamUsageEstimator.sizeOf(avgs)
+          + RamUsageEstimator.sizeOf(bpvs)
+          + RamUsageEstimator.sizeOf(offsets);
+    }
+  }
+
+  /** Load metadata from the given {@link IndexInput}.
+   *  @see DirectMonotonicReader#getInstance(Meta, RandomAccessInput) */
+  public static Meta loadMeta(IndexInput metaIn, long numValues, int blockShift) throws IOException {
+    Meta meta = new Meta(numValues, blockShift);
+    for (int i = 0; i < meta.numBlocks; ++i) {
+      meta.mins[i] = metaIn.readLong();
+      meta.avgs[i] = Float.intBitsToFloat(metaIn.readInt());
+      meta.offsets[i] = metaIn.readLong();
+      meta.bpvs[i] = metaIn.readByte();
+    }
+    return meta;
+  }
+
+  /**
+   * Retrieves an instance from the specified slice.
+   */
+  public static LongValues getInstance(Meta meta, RandomAccessInput data) throws IOException {
+    final LongValues[] readers = new LongValues[meta.numBlocks];
+    for (int i = 0; i < meta.mins.length; ++i) {
+      if (meta.bpvs[i] == 0) {
+        readers[i] = EMPTY;
+      } else {
+        readers[i] = DirectReader.getInstance(data, meta.bpvs[i], meta.offsets[i]);
+      }
+    }
+    final int blockShift = meta.blockShift;
+
+    final long[] mins = meta.mins;
+    final float[] avgs = meta.avgs;
+    return new LongValues() {
+
+      @Override
+      public long get(long index) {
+        final int block = (int) (index >>> blockShift);
+        final long blockIndex = index & ((1 << blockShift) - 1);
+        final long delta = readers[block].get(blockIndex);
+        return mins[block] + (long) (avgs[block] * blockIndex) + delta;
+      }
+
+    };
+  }
+}

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/DirectMonotonicWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/DirectMonotonicWriter.java?rev=1710876&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/DirectMonotonicWriter.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/DirectMonotonicWriter.java Tue Oct 27 19:49:47 2015
@@ -0,0 +1,137 @@
+package org.apache.lucene.util.packed;
+
+/*
+ * 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.store.IndexOutput;
+
+/**
+ * Write monotonically-increasing sequences of integers. This writer splits
+ * data into blocks and then for each block, computes the average slope, the
+ * minimum value and only encode the delta from the expected value using a
+ * {@link DirectWriter}.
+ * 
+ * @see DirectMonotonicReader
+ * @lucene.internal 
+ */
+public final class DirectMonotonicWriter {
+
+  public static final int MIN_BLOCK_SHIFT = 2;
+  public static final int MAX_BLOCK_SHIFT = 22;
+
+  final IndexOutput meta;
+  final IndexOutput data;
+  final long numValues;
+  final long baseDataPointer;
+  final long[] buffer;
+  int bufferSize;
+  long count;
+  boolean finished;
+
+  DirectMonotonicWriter(IndexOutput metaOut, IndexOutput dataOut, long numValues, int blockShift) {
+    this.meta = metaOut;
+    this.data = dataOut;
+    this.numValues = numValues;
+    if (blockShift < 2 || blockShift > 30) {
+      throw new IllegalArgumentException("blockShift must be in [3-30], got " + blockShift);
+    }
+    final int blockSize = 1 << blockShift;
+    this.buffer = new long[blockSize];
+    this.bufferSize = 0;
+    this.baseDataPointer = dataOut.getFilePointer();
+  }
+
+  private void flush() throws IOException {
+    assert bufferSize != 0;
+
+    final float avgInc = (float) ((double) (buffer[bufferSize-1] - buffer[0]) / Math.max(1, bufferSize - 1));
+    for (int i = 0; i < bufferSize; ++i) {
+      final long expected = (long) (avgInc * (long) i);
+      buffer[i] -= expected;
+    }
+
+    long min = buffer[0];
+    for (int i = 1; i < bufferSize; ++i) {
+      min = Math.min(buffer[i], min);
+    }
+
+    long maxDelta = 0;
+    for (int i = 0; i < bufferSize; ++i) {
+      buffer[i] -= min;
+      // use | will change nothing when it comes to computing required bits
+      // but has the benefit of working fine with negative values too
+      // (in case of overflow)
+      maxDelta |= buffer[i];
+    }
+
+    meta.writeLong(min);
+    meta.writeInt(Float.floatToIntBits(avgInc));
+    meta.writeLong(data.getFilePointer() - baseDataPointer);
+    if (maxDelta == 0) {
+      meta.writeByte((byte) 0);
+    } else {
+      final int bitsRequired = DirectWriter.unsignedBitsRequired(maxDelta);
+      DirectWriter writer = DirectWriter.getInstance(data, bufferSize, bitsRequired);
+      for (int i = 0; i < bufferSize; ++i) {
+        writer.add(buffer[i]);
+      }
+      writer.finish();
+      meta.writeByte((byte) bitsRequired);
+    }
+    bufferSize = 0;
+  }
+
+  long previous = Long.MIN_VALUE;
+
+  /** Write a new value. Note that data might not make it to storage until
+   * {@link #finish()} is called.
+   *  @throws IllegalArgumentException if values don't come in order */
+  public void add(long v) throws IOException {
+    if (v < previous) {
+      throw new IllegalArgumentException("Values do not come in order: " + previous + ", " + v);
+    }
+    if (bufferSize == buffer.length) {
+      flush();
+    }
+    buffer[bufferSize++] = v;
+    previous = v;
+    count++;
+  }
+
+  /** This must be called exactly once after all values have been {@link #add(long) added}. */
+  public void finish() throws IOException {
+    if (count != numValues) {
+      throw new IllegalStateException("Wrong number of values added, expected: " + numValues + ", got: " + count);
+    }
+    if (finished) {
+      throw new IllegalStateException("#finish has been called already");
+    }
+    flush();
+    finished = true;
+  }
+
+  /** Returns an instance suitable for encoding {@code numValues} into monotonic
+   *  blocks of 2<sup>{@code blockShift}</sup> values. Metadata will be written
+   *  to {@code metaOut} and actual data to {@code dataOut}. */
+  public static DirectMonotonicWriter getInstance(IndexOutput metaOut, IndexOutput dataOut, long numValues, int blockShift) {
+    return new DirectMonotonicWriter(metaOut, dataOut, numValues, blockShift);
+  }
+
+}

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/DirectReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/DirectReader.java?rev=1710876&r1=1710875&r2=1710876&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/DirectReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/DirectReader.java Tue Oct 27 19:49:47 2015
@@ -43,37 +43,47 @@ public class DirectReader {
    * {@code bitsPerValue} for each value 
    */
   public static LongValues getInstance(RandomAccessInput slice, int bitsPerValue) {
+    return getInstance(slice, bitsPerValue, 0);
+  }
+
+  /** 
+   * Retrieves an instance from the specified {@code offset} of the given slice
+   * decoding {@code bitsPerValue} for each value 
+   */
+  public static LongValues getInstance(RandomAccessInput slice, int bitsPerValue, long offset) {
     switch (bitsPerValue) {
-      case 1: return new DirectPackedReader1(slice);
-      case 2: return new DirectPackedReader2(slice);
-      case 4: return new DirectPackedReader4(slice);
-      case 8: return new DirectPackedReader8(slice);
-      case 12: return new DirectPackedReader12(slice);
-      case 16: return new DirectPackedReader16(slice);
-      case 20: return new DirectPackedReader20(slice);
-      case 24: return new DirectPackedReader24(slice);
-      case 28: return new DirectPackedReader28(slice);
-      case 32: return new DirectPackedReader32(slice);
-      case 40: return new DirectPackedReader40(slice);
-      case 48: return new DirectPackedReader48(slice);
-      case 56: return new DirectPackedReader56(slice);
-      case 64: return new DirectPackedReader64(slice);
+      case 1: return new DirectPackedReader1(slice, offset);
+      case 2: return new DirectPackedReader2(slice, offset);
+      case 4: return new DirectPackedReader4(slice, offset);
+      case 8: return new DirectPackedReader8(slice, offset);
+      case 12: return new DirectPackedReader12(slice, offset);
+      case 16: return new DirectPackedReader16(slice, offset);
+      case 20: return new DirectPackedReader20(slice, offset);
+      case 24: return new DirectPackedReader24(slice, offset);
+      case 28: return new DirectPackedReader28(slice, offset);
+      case 32: return new DirectPackedReader32(slice, offset);
+      case 40: return new DirectPackedReader40(slice, offset);
+      case 48: return new DirectPackedReader48(slice, offset);
+      case 56: return new DirectPackedReader56(slice, offset);
+      case 64: return new DirectPackedReader64(slice, offset);
       default: throw new IllegalArgumentException("unsupported bitsPerValue: " + bitsPerValue);
     }
   }
   
   static final class DirectPackedReader1 extends LongValues {
     final RandomAccessInput in;
+    final long offset;
     
-    DirectPackedReader1(RandomAccessInput in) {
+    DirectPackedReader1(RandomAccessInput in, long offset) {
       this.in = in;
+      this.offset = offset;
     }
 
     @Override
     public long get(long index) {
       try {
         int shift = 7 - (int) (index & 7);
-        return (in.readByte(index >>> 3) >>> shift) & 0x1;
+        return (in.readByte(offset + (index >>> 3)) >>> shift) & 0x1;
       } catch (IOException e) {
         throw new RuntimeException(e);
       }
@@ -82,16 +92,18 @@ public class DirectReader {
   
   static final class DirectPackedReader2 extends LongValues {
     final RandomAccessInput in;
+    final long offset;
     
-    DirectPackedReader2(RandomAccessInput in) {
+    DirectPackedReader2(RandomAccessInput in, long offset) {
       this.in = in;
+      this.offset = offset;
     }
 
     @Override
     public long get(long index) {
       try {
         int shift = (3 - (int)(index & 3)) << 1;
-        return (in.readByte(index >>> 2) >>> shift) & 0x3;
+        return (in.readByte(offset + (index >>> 2)) >>> shift) & 0x3;
       } catch (IOException e) {
         throw new RuntimeException(e);
       }
@@ -100,16 +112,18 @@ public class DirectReader {
   
   static final class DirectPackedReader4 extends LongValues {
     final RandomAccessInput in;
+    final long offset;
 
-    DirectPackedReader4(RandomAccessInput in) {
+    DirectPackedReader4(RandomAccessInput in, long offset) {
       this.in = in;
+      this.offset = offset;
     }
 
     @Override
     public long get(long index) {
       try {
         int shift = (int) ((index + 1) & 1) << 2;
-        return (in.readByte(index >>> 1) >>> shift) & 0xF;
+        return (in.readByte(offset + (index >>> 1)) >>> shift) & 0xF;
       } catch (IOException e) {
         throw new RuntimeException(e);
       }
@@ -118,15 +132,17 @@ public class DirectReader {
     
   static final class DirectPackedReader8 extends LongValues {
     final RandomAccessInput in;
+    final long offset;
 
-    DirectPackedReader8(RandomAccessInput in) {
+    DirectPackedReader8(RandomAccessInput in, long offset) {
       this.in = in;
+      this.offset = offset;
     }
 
     @Override
     public long get(long index) {
       try {
-        return in.readByte(index) & 0xFF;
+        return in.readByte(offset + index) & 0xFF;
       } catch (IOException e) {
         throw new RuntimeException(e);
       }
@@ -135,9 +151,11 @@ public class DirectReader {
   
   static final class DirectPackedReader12 extends LongValues {
     final RandomAccessInput in;
+    final long offset;
     
-    DirectPackedReader12(RandomAccessInput in) {
+    DirectPackedReader12(RandomAccessInput in, long offset) {
       this.in = in;
+      this.offset = offset;
     }
 
     @Override
@@ -145,7 +163,7 @@ public class DirectReader {
       try {
         long offset = (index * 12) >>> 3;
         int shift = (int) ((index + 1) & 1) << 2;
-        return (in.readShort(offset) >>> shift) & 0xFFF;
+        return (in.readShort(this.offset + offset) >>> shift) & 0xFFF;
       } catch (IOException e) {
         throw new RuntimeException(e);
       }
@@ -154,15 +172,17 @@ public class DirectReader {
   
   static final class DirectPackedReader16 extends LongValues {
     final RandomAccessInput in;
+    final long offset;
     
-    DirectPackedReader16(RandomAccessInput in) {
+    DirectPackedReader16(RandomAccessInput in, long offset) {
       this.in = in;
+      this.offset = offset;
     }
 
     @Override
     public long get(long index) {
       try {
-        return in.readShort(index << 1) & 0xFFFF;
+        return in.readShort(offset + (index << 1)) & 0xFFFF;
       } catch (IOException e) {
         throw new RuntimeException(e);
       }
@@ -171,9 +191,11 @@ public class DirectReader {
   
   static final class DirectPackedReader20 extends LongValues {
     final RandomAccessInput in;
+    final long offset;
 
-    DirectPackedReader20(RandomAccessInput in) {
+    DirectPackedReader20(RandomAccessInput in, long offset) {
       this.in = in;
+      this.offset = offset;
     }
 
     @Override
@@ -181,7 +203,7 @@ public class DirectReader {
       try {
         long offset = (index * 20) >>> 3;
         // TODO: clean this up...
-        int v = in.readInt(offset) >>> 8;
+        int v = in.readInt(this.offset + offset) >>> 8;
         int shift = (int) ((index + 1) & 1) << 2;
         return (v >>> shift) & 0xFFFFF;
       } catch (IOException e) {
@@ -192,15 +214,17 @@ public class DirectReader {
   
   static final class DirectPackedReader24 extends LongValues {
     final RandomAccessInput in;
+    final long offset;
     
-    DirectPackedReader24(RandomAccessInput in) {
+    DirectPackedReader24(RandomAccessInput in, long offset) {
       this.in = in;
+      this.offset = offset;
     }
 
     @Override
     public long get(long index) {
       try {
-        return in.readInt(index * 3) >>> 8;
+        return in.readInt(offset + index * 3) >>> 8;
       } catch (IOException e) {
         throw new RuntimeException(e);
       }
@@ -209,9 +233,11 @@ public class DirectReader {
   
   static final class DirectPackedReader28 extends LongValues {
     final RandomAccessInput in;
+    final long offset;
     
-    DirectPackedReader28(RandomAccessInput in) {
+    DirectPackedReader28(RandomAccessInput in, long offset) {
       this.in = in;
+      this.offset = offset;
     }
     
     @Override
@@ -219,7 +245,7 @@ public class DirectReader {
       try {
         long offset = (index * 28) >>> 3;
         int shift = (int) ((index + 1) & 1) << 2;
-        return (in.readInt(offset) >>> shift) & 0xFFFFFFFL;
+        return (in.readInt(this.offset + offset) >>> shift) & 0xFFFFFFFL;
       } catch (IOException e) {
         throw new RuntimeException(e);
       }
@@ -228,15 +254,17 @@ public class DirectReader {
   
   static final class DirectPackedReader32 extends LongValues {
     final RandomAccessInput in;
+    final long offset;
     
-    DirectPackedReader32(RandomAccessInput in) {
+    DirectPackedReader32(RandomAccessInput in, long offset) {
       this.in = in;
+      this.offset = offset;
     }
     
     @Override
     public long get(long index) {
       try {
-        return in.readInt(index << 2) & 0xFFFFFFFFL;
+        return in.readInt(this.offset + (index << 2)) & 0xFFFFFFFFL;
       } catch (IOException e) {
         throw new RuntimeException(e);
       }
@@ -245,15 +273,17 @@ public class DirectReader {
   
   static final class DirectPackedReader40 extends LongValues {
     final RandomAccessInput in;
+    final long offset;
     
-    DirectPackedReader40(RandomAccessInput in) {
+    DirectPackedReader40(RandomAccessInput in, long offset) {
       this.in = in;
+      this.offset = offset;
     }
     
     @Override
     public long get(long index) {
       try {
-        return in.readLong(index * 5) >>> 24;
+        return in.readLong(this.offset + index * 5) >>> 24;
       } catch (IOException e) {
         throw new RuntimeException(e);
       }
@@ -262,15 +292,17 @@ public class DirectReader {
   
   static final class DirectPackedReader48 extends LongValues {
     final RandomAccessInput in;
+    final long offset;
     
-    DirectPackedReader48(RandomAccessInput in) {
+    DirectPackedReader48(RandomAccessInput in, long offset) {
       this.in = in;
+      this.offset = offset;
     }
     
     @Override
     public long get(long index) {
       try {
-        return in.readLong(index * 6) >>> 16;
+        return in.readLong(this.offset + index * 6) >>> 16;
       } catch (IOException e) {
         throw new RuntimeException(e);
       }
@@ -279,15 +311,17 @@ public class DirectReader {
   
   static final class DirectPackedReader56 extends LongValues {
     final RandomAccessInput in;
+    final long offset;
     
-    DirectPackedReader56(RandomAccessInput in) {
+    DirectPackedReader56(RandomAccessInput in, long offset) {
       this.in = in;
+      this.offset = offset;
     }
     
     @Override
     public long get(long index) {
       try {
-        return in.readLong(index * 7) >>> 8;
+        return in.readLong(this.offset + index * 7) >>> 8;
       } catch (IOException e) {
         throw new RuntimeException(e);
       }
@@ -296,15 +330,17 @@ public class DirectReader {
   
   static final class DirectPackedReader64 extends LongValues {
     final RandomAccessInput in;
+    final long offset;
     
-    DirectPackedReader64(RandomAccessInput in) {
+    DirectPackedReader64(RandomAccessInput in, long offset) {
       this.in = in;
+      this.offset = offset;
     }
     
     @Override
     public long get(long index) {
       try {
-        return in.readLong(index << 3);
+        return in.readLong(offset + (index << 3));
       } catch (IOException e) {
         throw new RuntimeException(e);
       }

Modified: lucene/dev/trunk/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec?rev=1710876&r1=1710875&r2=1710876&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec (original)
+++ lucene/dev/trunk/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec Tue Oct 27 19:49:47 2015
@@ -13,4 +13,4 @@
 #  See the License for the specific language governing permissions and
 #  limitations under the License.
 
-org.apache.lucene.codecs.lucene53.Lucene53Codec
+org.apache.lucene.codecs.lucene54.Lucene54Codec

Modified: lucene/dev/trunk/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat?rev=1710876&r1=1710875&r2=1710876&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat (original)
+++ lucene/dev/trunk/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat Tue Oct 27 19:49:47 2015
@@ -13,4 +13,4 @@
 #  See the License for the specific language governing permissions and
 #  limitations under the License.
 
-org.apache.lucene.codecs.lucene50.Lucene50DocValuesFormat
+org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50StoredFieldsFormatHighCompression.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50StoredFieldsFormatHighCompression.java?rev=1710876&r1=1710875&r2=1710876&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50StoredFieldsFormatHighCompression.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50StoredFieldsFormatHighCompression.java Tue Oct 27 19:49:47 2015
@@ -19,7 +19,7 @@ package org.apache.lucene.codecs.lucene5
 
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
-import org.apache.lucene.codecs.lucene53.Lucene53Codec;
+import org.apache.lucene.codecs.lucene54.Lucene54Codec;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.StoredField;
 import org.apache.lucene.index.BaseStoredFieldsFormatTestCase;
@@ -34,7 +34,7 @@ import com.carrotsearch.randomizedtestin
 public class TestLucene50StoredFieldsFormatHighCompression extends BaseStoredFieldsFormatTestCase {
   @Override
   protected Codec getCodec() {
-    return new Lucene53Codec(Mode.BEST_COMPRESSION);
+    return new Lucene54Codec(Mode.BEST_COMPRESSION);
   }
   
   /**
@@ -45,7 +45,7 @@ public class TestLucene50StoredFieldsFor
     Directory dir = newDirectory();
     for (int i = 0; i < 10; i++) {
       IndexWriterConfig iwc = newIndexWriterConfig();
-      iwc.setCodec(new Lucene53Codec(RandomPicks.randomFrom(random(), Mode.values())));
+      iwc.setCodec(new Lucene54Codec(RandomPicks.randomFrom(random(), Mode.values())));
       IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig());
       Document doc = new Document();
       doc.add(new StoredField("field1", "value1"));
@@ -72,7 +72,7 @@ public class TestLucene50StoredFieldsFor
   
   public void testInvalidOptions() throws Exception {
     try {
-      new Lucene53Codec(null);
+      new Lucene54Codec(null);
       fail("didn't hit exception");
     } catch (NullPointerException expected) {
       // expected

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene53/TestLucene53NormsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene53/TestLucene53NormsFormat.java?rev=1710876&r1=1710875&r2=1710876&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene53/TestLucene53NormsFormat.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene53/TestLucene53NormsFormat.java Tue Oct 27 19:49:47 2015
@@ -18,13 +18,14 @@ package org.apache.lucene.codecs.lucene5
  */
 
 import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.lucene54.Lucene54Codec;
 import org.apache.lucene.index.BaseNormsFormatTestCase;
 
 /**
  * Tests Lucene53NormsFormat
  */
 public class TestLucene53NormsFormat extends BaseNormsFormatTestCase {
-  private final Codec codec = new Lucene53Codec();
+  private final Codec codec = new Lucene54Codec();
   
   @Override
   protected Codec getCodec() {

Added: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene54/TestLucene54DocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene54/TestLucene54DocValuesFormat.java?rev=1710876&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene54/TestLucene54DocValuesFormat.java (added)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene54/TestLucene54DocValuesFormat.java Tue Oct 27 19:49:47 2015
@@ -0,0 +1,281 @@
+package org.apache.lucene.codecs.lucene54;
+
+/*
+ * 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.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.BaseCompressingDocValuesFormatTestCase;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.SerialMergeScheduler;
+import org.apache.lucene.index.SortedSetDocValues;
+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 Lucene54DocValuesFormat
+ */
+public class TestLucene54DocValuesFormat extends BaseCompressingDocValuesFormatTestCase {
+  private final Codec codec = TestUtil.alwaysDocValuesFormat(new Lucene54DocValuesFormat());
+
+  @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.
+  
+  @Slow
+  public void testSortedSetVariableLengthBigVsStoredFields() throws Exception {
+    int numIterations = atLeast(1);
+    for (int i = 0; i < numIterations; i++) {
+      doTestSortedSetVsStoredFields(atLeast(300), 1, 32766, 16, 100);
+    }
+  }
+  
+  @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, 100);
+    }
+  }
+  
+  @Slow
+  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);
+    }
+  }
+  
+  @Slow
+  public void testTermsEnumFixedWidth() throws Exception {
+    int numIterations = atLeast(1);
+    for (int i = 0; i < numIterations; i++) {
+      doTestTermsEnumRandom(TestUtil.nextInt(random(), 1025, 5121), 10, 10);
+    }
+  }
+  
+  @Slow
+  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 = TestUtil.getPostingsFormatWithOrds(random());
+    final DocValuesFormat dv = new Lucene54DocValuesFormat();
+    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) {
+        SortedSetDocValues ssdv = r.getSortedSetDocValues("dv");
+        assertEquals(terms.size(), ssdv.getValueCount());
+        TermsEnum expected = terms.iterator();
+        TermsEnum actual = r.getSortedSetDocValues("dv").termsEnum();
+        assertEquals(terms.size(), expected, actual);
+
+        doTestSortedSetEnumAdvanceIndependently(ssdv);
+      }
+    }
+    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();
+      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());
+      }
+    }
+  }
+}

Added: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/packed/TestDirectMonotonic.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/packed/TestDirectMonotonic.java?rev=1710876&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/packed/TestDirectMonotonic.java (added)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/packed/TestDirectMonotonic.java Tue Oct 27 19:49:47 2015
@@ -0,0 +1,137 @@
+package org.apache.lucene.util.packed;
+
+/*
+ * 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.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.LongValues;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+public class TestDirectMonotonic extends LuceneTestCase {
+
+  public void testSimple() throws IOException {
+    Directory dir = newDirectory();
+    final int blockShift = 2;
+
+    List<Long> actualValues = Arrays.asList(1L, 2L, 5L, 7L, 8L, 100L);
+    final int numValues = actualValues.size();
+
+    final long dataLength;
+    try (IndexOutput metaOut = dir.createOutput("meta", IOContext.DEFAULT);
+        IndexOutput dataOut = dir.createOutput("data", IOContext.DEFAULT)) {
+      DirectMonotonicWriter w = DirectMonotonicWriter.getInstance(metaOut, dataOut, numValues, blockShift);
+      for (long v : actualValues) {
+        w.add(v);
+      }
+      w.finish();
+      dataLength = dataOut.getFilePointer();
+    }
+
+    try (IndexInput metaIn = dir.openInput("meta", IOContext.READONCE);
+        IndexInput dataIn = dir.openInput("data", IOContext.DEFAULT)) {
+      DirectMonotonicReader.Meta meta = DirectMonotonicReader.loadMeta(metaIn, numValues, blockShift);
+      LongValues values = DirectMonotonicReader.getInstance(meta, dataIn.randomAccessSlice(0, dataLength));
+      for (int i = 0; i < numValues; ++i) {
+        final long v = values.get(i);
+        assertEquals(actualValues.get(i).longValue(), v);
+      }
+    }
+
+    dir.close();
+  }
+
+  public void testConstantSlope() throws IOException {
+    Directory dir = newDirectory();
+    final int blockShift = TestUtil.nextInt(random(), DirectMonotonicWriter.MIN_BLOCK_SHIFT, DirectMonotonicWriter.MAX_BLOCK_SHIFT);
+    final int numValues = TestUtil.nextInt(random(), 1, 1 << 20);
+    final long min = random().nextLong();
+    final long inc = random().nextInt(1 << random().nextInt(20));
+
+    List<Long> actualValues = new ArrayList<>();
+    for (int i = 0; i < numValues; ++i) {
+      actualValues.add(min + inc * i);
+    }
+
+    final long dataLength;
+    try (IndexOutput metaOut = dir.createOutput("meta", IOContext.DEFAULT);
+        IndexOutput dataOut = dir.createOutput("data", IOContext.DEFAULT)) {
+      DirectMonotonicWriter w = DirectMonotonicWriter.getInstance(metaOut, dataOut, numValues, blockShift);
+      for (long v : actualValues) {
+        w.add(v);
+      }
+      w.finish();
+      dataLength = dataOut.getFilePointer();
+    }
+
+    try (IndexInput metaIn = dir.openInput("meta", IOContext.READONCE);
+        IndexInput dataIn = dir.openInput("data", IOContext.DEFAULT)) {
+      DirectMonotonicReader.Meta meta = DirectMonotonicReader.loadMeta(metaIn, numValues, blockShift);
+      LongValues values = DirectMonotonicReader.getInstance(meta, dataIn.randomAccessSlice(0, dataLength));
+      for (int i = 0; i < numValues; ++i) {
+        assertEquals(actualValues.get(i).longValue(), values.get(i));
+      }
+      assertEquals(0, dataIn.getFilePointer());
+    }
+
+    dir.close();
+  }
+
+  public void testRandom() throws IOException {
+    Directory dir = newDirectory();
+    final int blockShift = TestUtil.nextInt(random(), DirectMonotonicWriter.MIN_BLOCK_SHIFT, DirectMonotonicWriter.MAX_BLOCK_SHIFT);
+    final int numValues = TestUtil.nextInt(random(), 1, 1 << 20);
+    List<Long> actualValues = new ArrayList<>();
+    long previous = random().nextLong();
+    actualValues.add(previous);
+    for (int i = 1; i < numValues; ++i) {
+      previous += random().nextInt(1 << random().nextInt(20));
+      actualValues.add(previous);
+    }
+
+    final long dataLength;
+    try (IndexOutput metaOut = dir.createOutput("meta", IOContext.DEFAULT);
+        IndexOutput dataOut = dir.createOutput("data", IOContext.DEFAULT)) {
+      DirectMonotonicWriter w = DirectMonotonicWriter.getInstance(metaOut, dataOut, numValues, blockShift);
+      for (long v : actualValues) {
+        w.add(v);
+      }
+      w.finish();
+      dataLength = dataOut.getFilePointer();
+    }
+
+    try (IndexInput metaIn = dir.openInput("meta", IOContext.READONCE);
+        IndexInput dataIn = dir.openInput("data", IOContext.DEFAULT)) {
+      DirectMonotonicReader.Meta meta = DirectMonotonicReader.loadMeta(metaIn, numValues, blockShift);
+      LongValues values = DirectMonotonicReader.getInstance(meta, dataIn.randomAccessSlice(0, dataLength));
+      for (int i = 0; i < numValues; ++i) {
+        assertEquals(actualValues.get(i).longValue(), values.get(i));
+      }
+    }
+
+    dir.close();
+  }
+
+}

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/packed/TestDirectPacked.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/packed/TestDirectPacked.java?rev=1710876&r1=1710875&r2=1710876&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/packed/TestDirectPacked.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/packed/TestDirectPacked.java Tue Oct 27 19:49:47 2015
@@ -26,6 +26,7 @@ import org.apache.lucene.store.IOContext
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
 import org.apache.lucene.util.packed.DirectReader;
 import org.apache.lucene.util.packed.DirectWriter;
 
@@ -45,7 +46,7 @@ public class TestDirectPacked extends Lu
     writer.finish();
     output.close();
     IndexInput input = dir.openInput("foo", IOContext.DEFAULT);
-    NumericDocValues reader = DirectReader.getInstance(input.randomAccessSlice(0, input.length()), bitsPerValue);
+    NumericDocValues reader = DirectReader.getInstance(input.randomAccessSlice(0, input.length()), bitsPerValue, 0);
     assertEquals(1, reader.get(0));
     assertEquals(0, reader.get(1));
     assertEquals(2, reader.get(2));
@@ -78,12 +79,21 @@ public class TestDirectPacked extends Lu
   public void testRandom() throws Exception {
     Directory dir = newDirectory();
     for (int bpv = 1; bpv <= 64; bpv++) {
-      doTestBpv(dir, bpv);
+      doTestBpv(dir, bpv, 0);
     }
     dir.close();
   }
-    
-  private void doTestBpv(Directory directory, int bpv) throws Exception {
+
+  public void testRandomWithOffset() throws Exception {
+    Directory dir = newDirectory();
+    final int offset = TestUtil.nextInt(random(), 1, 100);
+    for (int bpv = 1; bpv <= 64; bpv++) {
+      doTestBpv(dir, bpv, offset);
+    }
+    dir.close();
+  }
+
+  private void doTestBpv(Directory directory, int bpv, long offset) throws Exception {
     MyRandom random = new MyRandom(random().nextLong());
     int numIters = TEST_NIGHTLY ? 100 : 10;
     for (int i = 0; i < numIters; i++) {
@@ -91,6 +101,9 @@ public class TestDirectPacked extends Lu
       int bitsRequired = bpv == 64 ? 64 : DirectWriter.bitsRequired(1L<<(bpv-1));
       String name = "bpv" + bpv + "_" + i;
       IndexOutput output = directory.createOutput(name, IOContext.DEFAULT);
+      for (long j = 0; j < offset; ++j) {
+        output.writeByte((byte) random().nextInt());
+      }
       DirectWriter writer = DirectWriter.getInstance(output, original.length, bitsRequired);
       for (int j = 0; j < original.length; j++) {
         writer.add(original[j]);
@@ -98,7 +111,7 @@ public class TestDirectPacked extends Lu
       writer.finish();
       output.close();
       IndexInput input = directory.openInput(name, IOContext.DEFAULT);
-      NumericDocValues reader = DirectReader.getInstance(input.randomAccessSlice(0, input.length()), bitsRequired);
+      NumericDocValues reader = DirectReader.getInstance(input.randomAccessSlice(0, input.length()), bitsRequired, offset);
       for (int j = 0; j < original.length; j++) {
         assertEquals("bpv=" + bpv, original[j], reader.get(j));
       }

Modified: lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDTreeDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDTreeDocValuesFormat.java?rev=1710876&r1=1710875&r2=1710876&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDTreeDocValuesFormat.java (original)
+++ lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDTreeDocValuesFormat.java Tue Oct 27 19:49:47 2015
@@ -22,7 +22,7 @@ import java.io.IOException;
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.DocValuesProducer;
-import org.apache.lucene.codecs.lucene50.Lucene50DocValuesFormat;
+import org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
 
@@ -31,7 +31,7 @@ import org.apache.lucene.index.SegmentWr
  * from {@link BKDPointField} for fast bounding-box ({@link BKDPointInBBoxQuery})
  * and polygon ({@link BKDPointInPolygonQuery}) queries.
  *
- * <p>This wraps {@link Lucene50DocValuesFormat}, but saves its own BKD tree
+ * <p>This wraps {@link Lucene54DocValuesFormat}, but saves its own BKD tree
  * structures to disk for fast query-time intersection. See <a
  * href="https://www.cs.duke.edu/~pankaj/publications/papers/bkd-sstd.pdf">this paper</a>
  * for details.
@@ -50,7 +50,7 @@ import org.apache.lucene.index.SegmentWr
  * <p>The index is also quite compact, because docs only appear once in
  * the tree (no "prefix terms").
  *
- * <p>In addition to the files written by {@link Lucene50DocValuesFormat}, this format writes:
+ * <p>In addition to the files written by {@link Lucene54DocValuesFormat}, this format writes:
  * <ol>
  *   <li><tt>.kdd</tt>: BKD leaf data and index</li>
  *   <li><tt>.kdm</tt>: BKD metadata</li>
@@ -75,7 +75,7 @@ public class BKDTreeDocValuesFormat exte
   private final int maxPointsInLeafNode;
   private final int maxPointsSortInHeap;
   
-  private final DocValuesFormat delegate = new Lucene50DocValuesFormat();
+  private final DocValuesFormat delegate = new Lucene54DocValuesFormat();
 
   /** Default constructor */
   public BKDTreeDocValuesFormat() {

Modified: lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/rangetree/RangeTreeDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/rangetree/RangeTreeDocValuesFormat.java?rev=1710876&r1=1710875&r2=1710876&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/rangetree/RangeTreeDocValuesFormat.java (original)
+++ lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/rangetree/RangeTreeDocValuesFormat.java Tue Oct 27 19:49:47 2015
@@ -20,7 +20,7 @@ package org.apache.lucene.rangetree;
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.DocValuesProducer;
-import org.apache.lucene.codecs.lucene50.Lucene50DocValuesFormat;
+import org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat;
 import org.apache.lucene.document.SortedNumericDocValuesField;
 import org.apache.lucene.document.SortedSetDocValuesField; // javadocs
 import org.apache.lucene.index.SegmentReadState;
@@ -34,7 +34,7 @@ import java.io.IOException;
  * for numeric range queries using ({@link NumericRangeTreeQuery}) and arbitrary binary
  * range queries using {@link SortedSetRangeTreeQuery}.
  *
- * <p>This wraps {@link Lucene50DocValuesFormat}, but saves its own numeric tree
+ * <p>This wraps {@link Lucene54DocValuesFormat}, but saves its own numeric tree
  * structures to disk for fast query-time intersection. See <a
  * href="https://www.cs.duke.edu/~pankaj/publications/papers/bkd-sstd.pdf">this paper</a>
  * for details.
@@ -53,7 +53,7 @@ import java.io.IOException;
  * <p>The index is also quite compact, because docs only appear once in
  * the tree (no "prefix terms").
  *
- * <p>In addition to the files written by {@link Lucene50DocValuesFormat}, this format writes:
+ * <p>In addition to the files written by {@link Lucene54DocValuesFormat}, this format writes:
  * <ol>
  *   <li><tt>.ndd</tt>: numeric tree leaf data and index</li>
  *   <li><tt>.ndm</tt>: numeric tree metadata</li>
@@ -78,7 +78,7 @@ public class RangeTreeDocValuesFormat ex
   private final int maxPointsInLeafNode;
   private final int maxPointsSortInHeap;
   
-  private final DocValuesFormat delegate = new Lucene50DocValuesFormat();
+  private final DocValuesFormat delegate = new Lucene54DocValuesFormat();
 
   /** Default constructor */
   public RangeTreeDocValuesFormat() {

Modified: lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/bkdtree/TestBKDTree.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/bkdtree/TestBKDTree.java?rev=1710876&r1=1710875&r2=1710876&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/bkdtree/TestBKDTree.java (original)
+++ lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/bkdtree/TestBKDTree.java Tue Oct 27 19:49:47 2015
@@ -17,18 +17,9 @@ package org.apache.lucene.bkdtree;
  * limitations under the License.
  */
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicBoolean;
-
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.DocValuesFormat;
-import org.apache.lucene.codecs.lucene53.Lucene53Codec;
+import org.apache.lucene.codecs.lucene54.Lucene54Codec;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriterConfig;
@@ -37,7 +28,6 @@ import org.apache.lucene.search.IndexSea
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.Accountables;
 import org.apache.lucene.util.BaseGeoPointTestCase;
@@ -79,7 +69,7 @@ public class TestBKDTree extends BaseGeo
   @Override
   protected void initIndexWriterConfig(final String fieldName, IndexWriterConfig iwc) {
     final DocValuesFormat dvFormat = getDocValuesFormat();
-    Codec codec = new Lucene53Codec() {
+    Codec codec = new Lucene54Codec() {
         @Override
         public DocValuesFormat getDocValuesFormatForField(String field) {
           if (field.equals(fieldName)) {

Modified: lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/rangetree/TestRangeTree.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/rangetree/TestRangeTree.java?rev=1710876&r1=1710875&r2=1710876&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/rangetree/TestRangeTree.java (original)
+++ lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/rangetree/TestRangeTree.java Tue Oct 27 19:49:47 2015
@@ -27,7 +27,7 @@ import java.util.concurrent.atomic.Atomi
 
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.DocValuesFormat;
-import org.apache.lucene.codecs.lucene53.Lucene53Codec;
+import org.apache.lucene.codecs.lucene54.Lucene54Codec;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.NumericDocValuesField;
@@ -356,7 +356,7 @@ public class TestRangeTree extends Lucen
       iwc.setMaxBufferedDocs(values.length/100);
     }
     final DocValuesFormat dvFormat = getDocValuesFormat();
-    Codec codec = new Lucene53Codec() {
+    Codec codec = new Lucene54Codec() {
         @Override
         public DocValuesFormat getDocValuesFormatForField(String field) {
           if (field.equals("sn_value") || field.equals("ss_value")) {

Modified: lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DDocValuesFormat.java?rev=1710876&r1=1710875&r2=1710876&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DDocValuesFormat.java (original)
+++ lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DDocValuesFormat.java Tue Oct 27 19:49:47 2015
@@ -20,7 +20,7 @@ package org.apache.lucene.bkdtree3d;
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.DocValuesProducer;
-import org.apache.lucene.codecs.lucene50.Lucene50DocValuesFormat;
+import org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat;
 import org.apache.lucene.geo3d.PlanetModel;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
@@ -32,7 +32,7 @@ import java.io.IOException;
  * from {@link Geo3DPointField} for fast shape intersection queries using
  * ({@link PointInGeo3DShapeQuery})
  *
- * <p>This wraps {@link Lucene50DocValuesFormat}, but saves its own BKD tree
+ * <p>This wraps {@link Lucene54DocValuesFormat}, but saves its own BKD tree
  * structures to disk for fast query-time intersection. See <a
  * href="https://www.cs.duke.edu/~pankaj/publications/papers/bkd-sstd.pdf">this paper</a>
  * for details.
@@ -51,7 +51,7 @@ import java.io.IOException;
  * <p>The index is also quite compact, because docs only appear once in
  * the tree (no "prefix terms").
  *
- * <p>In addition to the files written by {@link Lucene50DocValuesFormat}, this format writes:
+ * <p>In addition to the files written by {@link Lucene54DocValuesFormat}, this format writes:
  * <ol>
  *   <li><tt>.kd3d</tt>: BKD leaf data and index</li>
  *   <li><tt>.kd3m</tt>: BKD metadata</li>
@@ -77,7 +77,7 @@ public class Geo3DDocValuesFormat extend
   private final int maxPointsInLeafNode;
   private final int maxPointsSortInHeap;
   
-  private final DocValuesFormat delegate = new Lucene50DocValuesFormat();
+  private final DocValuesFormat delegate = new Lucene54DocValuesFormat();
 
   private final PlanetModel planetModel;
 

Modified: lucene/dev/trunk/lucene/spatial3d/src/test/org/apache/lucene/bkdtree3d/TestGeo3DPointField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/spatial3d/src/test/org/apache/lucene/bkdtree3d/TestGeo3DPointField.java?rev=1710876&r1=1710875&r2=1710876&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/spatial3d/src/test/org/apache/lucene/bkdtree3d/TestGeo3DPointField.java (original)
+++ lucene/dev/trunk/lucene/spatial3d/src/test/org/apache/lucene/bkdtree3d/TestGeo3DPointField.java Tue Oct 27 19:49:47 2015
@@ -29,7 +29,7 @@ import java.util.concurrent.atomic.Atomi
 
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.DocValuesFormat;
-import org.apache.lucene.codecs.lucene53.Lucene53Codec;
+import org.apache.lucene.codecs.lucene54.Lucene54Codec;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.NumericDocValuesField;
@@ -910,7 +910,7 @@ public class TestGeo3DPointField extends
       iwc.setMaxBufferedDocs(lats.length/100);
     }
     final DocValuesFormat dvFormat = new Geo3DDocValuesFormat(planetModel, maxPointsInLeaf, maxPointsSortInHeap);
-    Codec codec = new Lucene53Codec() {
+    Codec codec = new Lucene54Codec() {
         @Override
         public DocValuesFormat getDocValuesFormatForField(String field) {
           if (field.equals("point")) {

Modified: lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java?rev=1710876&r1=1710875&r2=1710876&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java Tue Oct 27 19:49:47 2015
@@ -31,7 +31,7 @@ import org.apache.lucene.analysis.Analyz
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.lucene53.Lucene53Codec;
+import org.apache.lucene.codecs.lucene54.Lucene54Codec;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.IntField;
@@ -631,7 +631,7 @@ public class TestSuggestField extends Lu
   static IndexWriterConfig iwcWithSuggestField(Analyzer analyzer, final Set<String> suggestFields) {
     IndexWriterConfig iwc = newIndexWriterConfig(random(), analyzer);
     iwc.setMergePolicy(newLogMergePolicy());
-    Codec filterCodec = new Lucene53Codec() {
+    Codec filterCodec = new Lucene54Codec() {
       PostingsFormat postingsFormat = new Completion50PostingsFormat();
 
       @Override

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java?rev=1710876&r1=1710875&r2=1710876&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java Tue Oct 27 19:49:47 2015
@@ -33,7 +33,7 @@ import org.apache.lucene.codecs.assertin
 import org.apache.lucene.codecs.cheapbastard.CheapBastardCodec;
 import org.apache.lucene.codecs.compressing.CompressingCodec;
 import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat;
-import org.apache.lucene.codecs.lucene53.Lucene53Codec;
+import org.apache.lucene.codecs.lucene54.Lucene54Codec;
 import org.apache.lucene.codecs.mockrandom.MockRandomPostingsFormat;
 import org.apache.lucene.codecs.simpletext.SimpleTextCodec;
 import org.apache.lucene.index.RandomCodec;
@@ -182,8 +182,8 @@ final class TestRuleSetupAndRestoreClass
       codec = new AssertingCodec();
     } else if ("Compressing".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 6 && !shouldAvoidCodec("Compressing"))) {
       codec = CompressingCodec.randomInstance(random);
-    } else if ("Lucene53".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 5 && !shouldAvoidCodec("Lucene53"))) {
-      codec = new Lucene53Codec(RandomPicks.randomFrom(random, Lucene50StoredFieldsFormat.Mode.values()));
+    } else if ("Lucene54".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 5 && !shouldAvoidCodec("Lucene54"))) {
+      codec = new Lucene54Codec(RandomPicks.randomFrom(random, Lucene50StoredFieldsFormat.Mode.values()));
     } else if (!"random".equals(TEST_CODEC)) {
       codec = Codec.forName(TEST_CODEC);
     } else if ("random".equals(TEST_POSTINGSFORMAT)) {

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java?rev=1710876&r1=1710875&r2=1710876&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java Tue Oct 27 19:49:47 2015
@@ -54,9 +54,9 @@ import org.apache.lucene.codecs.Postings
 import org.apache.lucene.codecs.asserting.AssertingCodec;
 import org.apache.lucene.codecs.blockterms.LuceneFixedGap;
 import org.apache.lucene.codecs.blocktreeords.BlockTreeOrdsPostingsFormat;
-import org.apache.lucene.codecs.lucene50.Lucene50DocValuesFormat;
 import org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat;
-import org.apache.lucene.codecs.lucene53.Lucene53Codec;
+import org.apache.lucene.codecs.lucene54.Lucene54Codec;
+import org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat;
 import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
 import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
 import org.apache.lucene.document.BinaryDocValuesField;
@@ -879,7 +879,7 @@ public final class TestUtil {
    * This may be different than {@link Codec#getDefault()} because that is randomized. 
    */
   public static Codec getDefaultCodec() {
-    return new Lucene53Codec();
+    return new Lucene54Codec();
   }
   
   /** 
@@ -912,7 +912,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 Lucene50DocValuesFormat();
+    return new Lucene54DocValuesFormat();
   }
 
   // TODO: generalize all 'test-checks-for-crazy-codecs' to

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java?rev=1710876&r1=1710875&r2=1710876&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java Tue Oct 27 19:49:47 2015
@@ -3,7 +3,7 @@ package org.apache.solr.core;
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.lucene53.Lucene53Codec;
+import org.apache.lucene.codecs.lucene54.Lucene54Codec;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.util.plugin.SolrCoreAware;
@@ -51,7 +51,7 @@ public class SchemaCodecFactory extends
   @Override
   public void init(NamedList args) {
     super.init(args);
-    codec = new Lucene53Codec() {
+    codec = new Lucene54Codec() {
       @Override
       public PostingsFormat getPostingsFormatForField(String field) {
         final SchemaField schemaField = core.getLatestSchema().getFieldOrNull(field);

Modified: lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/schema_codec.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/schema_codec.xml?rev=1710876&r1=1710875&r2=1710876&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/schema_codec.xml (original)
+++ lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/schema_codec.xml Tue Oct 27 19:49:47 2015
@@ -21,7 +21,7 @@
   <fieldType name="string_simpletext" class="solr.StrField" postingsFormat="SimpleText"/>
   <fieldType name="string_standard" class="solr.StrField" postingsFormat="Lucene50"/>
 
-  <fieldType name="string_disk" class="solr.StrField" docValuesFormat="Lucene50" />
+  <fieldType name="string_disk" class="solr.StrField" docValuesFormat="Lucene54" />
   <fieldType name="string_memory" class="solr.StrField" docValuesFormat="Memory" />
 
   <fieldType name="string" class="solr.StrField" />