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 15:49:04 UTC

svn commit: r1515416 - in /lucene/dev/branches/lucene5178/lucene: core/src/java/org/apache/lucene/codecs/lucene42/ core/src/test/org/apache/lucene/codecs/lucene42/ core/src/test/org/apache/lucene/index/ facet/src/java/org/apache/lucene/facet/codecs/fac...

Author: rmuir
Date: Mon Aug 19 13:49:03 2013
New Revision: 1515416

URL: http://svn.apache.org/r1515416
Log:
make 4.2 read only / setup impersonator

Added:
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsConsumer.java   (with props)
    lucene/dev/branches/lucene5178/lucene/facet/src/java/org/apache/lucene/facet/codecs/facet45/
    lucene/dev/branches/lucene5178/lucene/facet/src/java/org/apache/lucene/facet/codecs/facet45/Facet45Codec.java   (with props)
    lucene/dev/branches/lucene5178/lucene/facet/src/java/org/apache/lucene/facet/codecs/facet45/package.html   (with props)
    lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene42/
    lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java
      - copied, changed from r1515388, lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java
    lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene42/Lucene42RWCodec.java   (with props)
    lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene42/Lucene42RWDocValuesFormat.java   (with props)
    lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene42/package.html   (with props)
Removed:
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java
    lucene/dev/branches/lucene5178/lucene/facet/src/java/org/apache/lucene/facet/codecs/facet42/Facet42Codec.java
Modified:
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42Codec.java
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesFormat.java
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsFormat.java
    lucene/dev/branches/lucene5178/lucene/core/src/test/org/apache/lucene/codecs/lucene42/TestLucene42DocValuesFormat.java
    lucene/dev/branches/lucene5178/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
    lucene/dev/branches/lucene5178/lucene/facet/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
    lucene/dev/branches/lucene5178/lucene/facet/src/test/org/apache/lucene/facet/FacetTestCase.java
    lucene/dev/branches/lucene5178/lucene/facet/src/test/org/apache/lucene/facet/search/TestDemoFacets.java
    lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/FastCompressingCodec.java
    lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/FastDecompressionCompressingCodec.java
    lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/HighCompressionCompressingCodec.java
    lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java
    lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java
    lucene/dev/branches/lucene5178/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
    lucene/dev/branches/lucene5178/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat

Modified: lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42Codec.java?rev=1515416&r1=1515415&r2=1515416&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42Codec.java (original)
+++ lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42Codec.java Mon Aug 19 13:49:03 2013
@@ -17,7 +17,10 @@ package org.apache.lucene.codecs.lucene4
  * limitations under the License.
  */
 
+import java.io.IOException;
+
 import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.FieldInfosFormat;
 import org.apache.lucene.codecs.FilterCodec;
@@ -32,6 +35,7 @@ import org.apache.lucene.codecs.lucene40
 import org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat;
 import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
 import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
+import org.apache.lucene.index.SegmentWriteState;
 
 /**
  * Implements the Lucene 4.2 index format, with configurable per-field postings
@@ -42,10 +46,12 @@ import org.apache.lucene.codecs.perfield
  *
  * @see org.apache.lucene.codecs.lucene42 package documentation for file format details.
  * @lucene.experimental
+ * @deprecated Only for reading old 4.2 segments
  */
 // NOTE: if we make largish changes in a minor release, easier to just make Lucene43Codec or whatever
 // if they are backwards compatible or smallish we can probably do the backwards in the postingsreader
 // (it writes a minor version, etc).
+@Deprecated
 public class Lucene42Codec extends Codec {
   private final StoredFieldsFormat fieldsFormat = new Lucene41StoredFieldsFormat();
   private final TermVectorsFormat vectorsFormat = new Lucene42TermVectorsFormat();
@@ -129,10 +135,15 @@ public class Lucene42Codec extends Codec
   private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
   private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene42");
 
-  private final NormsFormat normsFormat = new Lucene42NormsFormat();
+  private final NormsFormat normsFormat = new Lucene42NormsFormat() {
+    @Override
+    public DocValuesConsumer normsConsumer(SegmentWriteState state) throws IOException {
+      throw new UnsupportedOperationException("this codec can only be used for reading");
+    }
+  };
 
   @Override
-  public final NormsFormat normsFormat() {
+  public NormsFormat normsFormat() {
     return normsFormat;
   }
 }

Modified: lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesFormat.java?rev=1515416&r1=1515415&r2=1515416&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesFormat.java (original)
+++ lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesFormat.java Mon Aug 19 13:49:03 2013
@@ -123,8 +123,10 @@ import org.apache.lucene.util.packed.Blo
  * <ul>
  *   <li> Binary doc values can be at most {@link #MAX_BINARY_FIELD_LENGTH} in length.
  * </ul>
+ * @deprecated Only for reading old 4.2 segments
  */
-public final class Lucene42DocValuesFormat extends DocValuesFormat {
+@Deprecated
+public class Lucene42DocValuesFormat extends DocValuesFormat {
 
   /** Maximum length for each binary doc values field. */
   public static final int MAX_BINARY_FIELD_LENGTH = (1 << 15) - 2;
@@ -154,8 +156,7 @@ public final class Lucene42DocValuesForm
 
   @Override
   public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
-    // note: we choose DEFAULT here (its reasonably fast, and for small bpv has tiny waste)
-    return new Lucene42DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION, acceptableOverheadRatio);
+    throw new UnsupportedOperationException("this codec can only be used for reading");
   }
   
   @Override
@@ -163,8 +164,8 @@ public final class Lucene42DocValuesForm
     return new Lucene42DocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
   }
   
-  private static final String DATA_CODEC = "Lucene42DocValuesData";
-  private static final String DATA_EXTENSION = "dvd";
-  private static final String METADATA_CODEC = "Lucene42DocValuesMetadata";
-  private static final String METADATA_EXTENSION = "dvm";
+  static final String DATA_CODEC = "Lucene42DocValuesData";
+  static final String DATA_EXTENSION = "dvd";
+  static final String METADATA_CODEC = "Lucene42DocValuesMetadata";
+  static final String METADATA_EXTENSION = "dvm";
 }

Modified: lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java?rev=1515416&r1=1515415&r2=1515416&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java (original)
+++ lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java Mon Aug 19 13:49:03 2013
@@ -17,11 +17,6 @@ package org.apache.lucene.codecs.lucene4
  * limitations under the License.
  */
 
-import static org.apache.lucene.codecs.lucene42.Lucene42DocValuesConsumer.DELTA_COMPRESSED;
-import static org.apache.lucene.codecs.lucene42.Lucene42DocValuesConsumer.GCD_COMPRESSED;
-import static org.apache.lucene.codecs.lucene42.Lucene42DocValuesConsumer.TABLE_COMPRESSED;
-import static org.apache.lucene.codecs.lucene42.Lucene42DocValuesConsumer.UNCOMPRESSED;
-
 import java.io.IOException;
 import java.util.Comparator;
 import java.util.HashMap;
@@ -78,6 +73,22 @@ class Lucene42DocValuesProducer extends 
       new HashMap<Integer,FST<Long>>();
   
   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;
     
   Lucene42DocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
     maxDoc = state.segmentInfo.getDocCount();
@@ -88,8 +99,8 @@ class Lucene42DocValuesProducer extends 
     final int version;
     try {
       version = CodecUtil.checkHeader(in, metaCodec, 
-                                      Lucene42DocValuesConsumer.VERSION_START,
-                                      Lucene42DocValuesConsumer.VERSION_CURRENT);
+                                      VERSION_START,
+                                      VERSION_CURRENT);
       numerics = new HashMap<Integer,NumericEntry>();
       binaries = new HashMap<Integer,BinaryEntry>();
       fsts = new HashMap<Integer,FSTEntry>();
@@ -109,8 +120,8 @@ class Lucene42DocValuesProducer extends 
       String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
       data = state.directory.openInput(dataName, state.context);
       final int version2 = CodecUtil.checkHeader(data, dataCodec, 
-                                                 Lucene42DocValuesConsumer.VERSION_START,
-                                                 Lucene42DocValuesConsumer.VERSION_CURRENT);
+                                                 VERSION_START,
+                                                 VERSION_CURRENT);
       if (version != version2) {
         throw new CorruptIndexException("Format versions mismatch");
       }
@@ -127,7 +138,7 @@ class Lucene42DocValuesProducer extends 
     int fieldNumber = meta.readVInt();
     while (fieldNumber != -1) {
       int fieldType = meta.readByte();
-      if (fieldType == Lucene42DocValuesConsumer.NUMBER) {
+      if (fieldType == NUMBER) {
         NumericEntry entry = new NumericEntry();
         entry.offset = meta.readLong();
         entry.format = meta.readByte();
@@ -140,11 +151,11 @@ class Lucene42DocValuesProducer extends 
           default:
                throw new CorruptIndexException("Unknown format: " + entry.format + ", input=" + meta);
         }
-        if (entry.format != Lucene42DocValuesConsumer.UNCOMPRESSED) {
+        if (entry.format != UNCOMPRESSED) {
           entry.packedIntsVersion = meta.readVInt();
         }
         numerics.put(fieldNumber, entry);
-      } else if (fieldType == Lucene42DocValuesConsumer.BYTES) {
+      } else if (fieldType == BYTES) {
         BinaryEntry entry = new BinaryEntry();
         entry.offset = meta.readLong();
         entry.numBytes = meta.readLong();
@@ -155,7 +166,7 @@ class Lucene42DocValuesProducer extends 
           entry.blockSize = meta.readVInt();
         }
         binaries.put(fieldNumber, entry);
-      } else if (fieldType == Lucene42DocValuesConsumer.FST) {
+      } else if (fieldType == FST) {
         FSTEntry entry = new FSTEntry();
         entry.offset = meta.readLong();
         entry.numOrds = meta.readVLong();

Added: lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsConsumer.java?rev=1515416&view=auto
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsConsumer.java (added)
+++ lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsConsumer.java Mon Aug 19 13:49:03 2013
@@ -0,0 +1,209 @@
+package org.apache.lucene.codecs.lucene42;
+
+/*
+ * 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 org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.DocValuesConsumer;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.MathUtil;
+import org.apache.lucene.util.packed.BlockPackedWriter;
+import org.apache.lucene.util.packed.PackedInts.FormatAndBits;
+import org.apache.lucene.util.packed.PackedInts;
+
+/**
+ * Writer for {@link Lucene42NormsFormat}
+ */
+class Lucene42NormsConsumer extends DocValuesConsumer {
+  static final int VERSION_START = 0;
+  static final int VERSION_GCD_COMPRESSION = 1;
+  static final int VERSION_CURRENT = VERSION_GCD_COMPRESSION;
+  
+  static final byte NUMBER = 0;
+
+  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;
+
+  final IndexOutput data, meta;
+  final int maxDoc;
+  final float acceptableOverheadRatio;
+  
+  Lucene42NormsConsumer(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 {
+    meta.writeVInt(field.number);
+    meta.writeByte(NUMBER);
+    meta.writeLong(data.getFilePointer());
+    long minValue = Long.MAX_VALUE;
+    long maxValue = Long.MIN_VALUE;
+    long gcd = 0;
+    // TODO: more efficient?
+    HashSet<Long> uniqueValues = null;
+    if (true) {
+      uniqueValues = new HashSet<>();
+
+      long count = 0;
+      for (Number nv : values) {
+        assert nv != null;
+        final long 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 (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 {
+    throw new UnsupportedOperationException();
+  }
+  
+  @Override
+  public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, final Iterable<Number> docToOrdCount, final Iterable<Number> ords) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+}

Modified: lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsFormat.java?rev=1515416&r1=1515415&r2=1515416&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsFormat.java (original)
+++ lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsFormat.java Mon Aug 19 13:49:03 2013
@@ -41,7 +41,7 @@ import org.apache.lucene.util.packed.Pac
  * </ul>
  * @see Lucene42DocValuesFormat
  */
-public final class Lucene42NormsFormat extends NormsFormat {
+public class Lucene42NormsFormat extends NormsFormat {
   final float acceptableOverheadRatio;
 
   /** 
@@ -67,7 +67,7 @@ public final class Lucene42NormsFormat e
   
   @Override
   public DocValuesConsumer normsConsumer(SegmentWriteState state) throws IOException {
-    return new Lucene42DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION, acceptableOverheadRatio);
+    return new Lucene42NormsConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION, acceptableOverheadRatio);
   }
   
   @Override

Modified: lucene/dev/branches/lucene5178/lucene/core/src/test/org/apache/lucene/codecs/lucene42/TestLucene42DocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/core/src/test/org/apache/lucene/codecs/lucene42/TestLucene42DocValuesFormat.java?rev=1515416&r1=1515415&r2=1515416&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/core/src/test/org/apache/lucene/codecs/lucene42/TestLucene42DocValuesFormat.java (original)
+++ lucene/dev/branches/lucene5178/lucene/core/src/test/org/apache/lucene/codecs/lucene42/TestLucene42DocValuesFormat.java Mon Aug 19 13:49:03 2013
@@ -24,7 +24,7 @@ import org.apache.lucene.index.BaseCompr
  * Tests Lucene42DocValuesFormat
  */
 public class TestLucene42DocValuesFormat extends BaseCompressingDocValuesFormatTestCase {
-  private final Codec codec = new Lucene42Codec();
+  private final Codec codec = new Lucene42RWCodec();
 
   @Override
   protected Codec getCodec() {

Modified: lucene/dev/branches/lucene5178/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java?rev=1515416&r1=1515415&r2=1515416&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java (original)
+++ lucene/dev/branches/lucene5178/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java Mon Aug 19 13:49:03 2013
@@ -74,7 +74,7 @@ import org.junit.Ignore;
 // we won't even be running the actual code, only the impostor
 // @SuppressCodecs("Lucene4x")
 // Sep codec cannot yet handle the offsets in our 4.x index!
-@SuppressCodecs({"MockFixedIntBlock", "MockVariableIntBlock", "MockSep", "MockRandom", "Lucene40", "Lucene41"})
+@SuppressCodecs({"MockFixedIntBlock", "MockVariableIntBlock", "MockSep", "MockRandom", "Lucene40", "Lucene41", "Lucene42"})
 public class TestBackwardsCompatibility extends LuceneTestCase {
 
   // Uncomment these cases & run them on an older Lucene version,

Added: lucene/dev/branches/lucene5178/lucene/facet/src/java/org/apache/lucene/facet/codecs/facet45/Facet45Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/facet/src/java/org/apache/lucene/facet/codecs/facet45/Facet45Codec.java?rev=1515416&view=auto
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/facet/src/java/org/apache/lucene/facet/codecs/facet45/Facet45Codec.java (added)
+++ lucene/dev/branches/lucene5178/lucene/facet/src/java/org/apache/lucene/facet/codecs/facet45/Facet45Codec.java Mon Aug 19 13:49:03 2013
@@ -0,0 +1,80 @@
+package org.apache.lucene.facet.codecs.facet45;
+
+/*
+ * 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.HashSet;
+import java.util.Set;
+
+import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.lucene45.Lucene45Codec;
+import org.apache.lucene.facet.codecs.facet42.Facet42DocValuesFormat;
+import org.apache.lucene.facet.params.CategoryListParams;
+import org.apache.lucene.facet.params.FacetIndexingParams;
+
+/**
+ * Same as {@link Lucene45Codec} except it uses {@link Facet42DocValuesFormat}
+ * for facet fields (faster-but-more-RAM-consuming doc values).
+ * 
+ * <p>
+ * <b>NOTE</b>: this codec does not support facet partitions (see
+ * {@link FacetIndexingParams#getPartitionSize()}).
+ *
+ * <p>
+ * <b>NOTE</b>: this format cannot handle more than 2 GB
+ * of facet data in a single segment.  If your usage may hit
+ * this limit, you can either use Lucene's default
+ * DocValuesFormat, limit the maximum segment size in your
+ * MergePolicy, or send us a patch fixing the limitation.
+ * 
+ * @lucene.experimental
+ */
+public class Facet45Codec extends Lucene45Codec {
+
+  private final Set<String> facetFields;
+  private final DocValuesFormat facetsDVFormat = DocValuesFormat.forName("Facet42");
+
+  // must have that for SPI purposes
+  /** Default constructor, uses {@link FacetIndexingParams#DEFAULT}. */
+  public Facet45Codec() {
+    this(FacetIndexingParams.DEFAULT);
+  }
+
+  /**
+   * Initializes with the given {@link FacetIndexingParams}. Returns the proper
+   * {@link DocValuesFormat} for the fields that are returned by
+   * {@link FacetIndexingParams#getAllCategoryListParams()}.
+   */
+  public Facet45Codec(FacetIndexingParams fip) {
+    if (fip.getPartitionSize() != Integer.MAX_VALUE) {
+      throw new IllegalArgumentException("this Codec does not support partitions");
+    }
+    this.facetFields = new HashSet<String>();
+    for (CategoryListParams clp : fip.getAllCategoryListParams()) {
+      facetFields.add(clp.field);
+    }
+  }
+  
+  @Override
+  public DocValuesFormat getDocValuesFormatForField(String field) {
+    if (facetFields.contains(field)) {
+      return facetsDVFormat;
+    } else {
+      return super.getDocValuesFormatForField(field);
+    }
+  }
+}

Added: lucene/dev/branches/lucene5178/lucene/facet/src/java/org/apache/lucene/facet/codecs/facet45/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/facet/src/java/org/apache/lucene/facet/codecs/facet45/package.html?rev=1515416&view=auto
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/facet/src/java/org/apache/lucene/facet/codecs/facet45/package.html (added)
+++ lucene/dev/branches/lucene5178/lucene/facet/src/java/org/apache/lucene/facet/codecs/facet45/package.html Mon Aug 19 13:49:03 2013
@@ -0,0 +1,22 @@
+<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
+<!--
+ 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.
+-->
+<html>
+<body>
+Codec + DocValuesFormat that are optimized for facets.
+</body>
+</html>

Modified: lucene/dev/branches/lucene5178/lucene/facet/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/facet/src/resources/META-INF/services/org.apache.lucene.codecs.Codec?rev=1515416&r1=1515415&r2=1515416&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/facet/src/resources/META-INF/services/org.apache.lucene.codecs.Codec (original)
+++ lucene/dev/branches/lucene5178/lucene/facet/src/resources/META-INF/services/org.apache.lucene.codecs.Codec Mon Aug 19 13:49:03 2013
@@ -13,4 +13,4 @@
 #  See the License for the specific language governing permissions and
 #  limitations under the License.
 
-org.apache.lucene.facet.codecs.facet42.Facet42Codec
+org.apache.lucene.facet.codecs.facet45.Facet45Codec

Modified: lucene/dev/branches/lucene5178/lucene/facet/src/test/org/apache/lucene/facet/FacetTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/facet/src/test/org/apache/lucene/facet/FacetTestCase.java?rev=1515416&r1=1515415&r2=1515416&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/facet/src/test/org/apache/lucene/facet/FacetTestCase.java (original)
+++ lucene/dev/branches/lucene5178/lucene/facet/src/test/org/apache/lucene/facet/FacetTestCase.java Mon Aug 19 13:49:03 2013
@@ -3,7 +3,7 @@ package org.apache.lucene.facet;
 import java.util.Random;
 
 import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.facet.codecs.facet42.Facet42Codec;
+import org.apache.lucene.facet.codecs.facet45.Facet45Codec;
 import org.apache.lucene.facet.encoding.DGapIntEncoder;
 import org.apache.lucene.facet.encoding.DGapVInt8IntEncoder;
 import org.apache.lucene.facet.encoding.EightFlagsIntEncoder;
@@ -53,7 +53,7 @@ public abstract class FacetTestCase exte
   public static void beforeClassFacetTestCase() throws Exception {
     if (random().nextDouble() < 0.3) {
       savedDefault = Codec.getDefault(); // save to restore later
-      Codec.setDefault(new Facet42Codec());
+      Codec.setDefault(new Facet45Codec());
     }
   }
   

Modified: lucene/dev/branches/lucene5178/lucene/facet/src/test/org/apache/lucene/facet/search/TestDemoFacets.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/facet/src/test/org/apache/lucene/facet/search/TestDemoFacets.java?rev=1515416&r1=1515415&r2=1515416&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/facet/src/test/org/apache/lucene/facet/search/TestDemoFacets.java (original)
+++ lucene/dev/branches/lucene5178/lucene/facet/src/test/org/apache/lucene/facet/search/TestDemoFacets.java Mon Aug 19 13:49:03 2013
@@ -31,7 +31,7 @@ import org.apache.lucene.document.Docume
 import org.apache.lucene.document.Field;
 import org.apache.lucene.facet.FacetTestCase;
 import org.apache.lucene.facet.FacetTestUtils;
-import org.apache.lucene.facet.codecs.facet42.Facet42Codec;
+import org.apache.lucene.facet.codecs.facet45.Facet45Codec;
 import org.apache.lucene.facet.index.FacetFields;
 import org.apache.lucene.facet.params.CategoryListParams;
 import org.apache.lucene.facet.params.FacetIndexingParams;
@@ -260,7 +260,7 @@ public class TestDemoFacets extends Face
     Directory dir = newDirectory();
     Directory taxoDir = newDirectory();
     IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
-    iwc.setCodec(new Facet42Codec());
+    iwc.setCodec(new Facet45Codec());
     RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
     DirectoryTaxonomyWriter taxoWriter = new DirectoryTaxonomyWriter(taxoDir, IndexWriterConfig.OpenMode.CREATE);
 

Modified: lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/FastCompressingCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/FastCompressingCodec.java?rev=1515416&r1=1515415&r2=1515416&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/FastCompressingCodec.java (original)
+++ lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/FastCompressingCodec.java Mon Aug 19 13:49:03 2013
@@ -1,9 +1,7 @@
 package org.apache.lucene.codecs.compressing;
 
-import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.NormsFormat;
-import org.apache.lucene.codecs.lucene42.Lucene42DocValuesFormat; // nocommit
-import org.apache.lucene.codecs.lucene42.Lucene42NormsFormat; // nocommit
+import org.apache.lucene.codecs.lucene42.Lucene42NormsFormat;
 import org.apache.lucene.util.packed.PackedInts;
 
 /*
@@ -42,9 +40,4 @@ public class FastCompressingCodec extend
   public NormsFormat normsFormat() {
     return new Lucene42NormsFormat(PackedInts.FAST);
   }
-
-  @Override
-  public DocValuesFormat docValuesFormat() {
-    return new Lucene42DocValuesFormat(PackedInts.FAST);
-  }
 }

Modified: lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/FastDecompressionCompressingCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/FastDecompressionCompressingCodec.java?rev=1515416&r1=1515415&r2=1515416&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/FastDecompressionCompressingCodec.java (original)
+++ lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/FastDecompressionCompressingCodec.java Mon Aug 19 13:49:03 2013
@@ -1,12 +1,9 @@
 package org.apache.lucene.codecs.compressing;
 
-import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.NormsFormat;
-import org.apache.lucene.codecs.lucene42.Lucene42DocValuesFormat;
 import org.apache.lucene.codecs.lucene42.Lucene42NormsFormat;
 import org.apache.lucene.util.packed.PackedInts;
 
-// nocommit
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -43,9 +40,4 @@ public class FastDecompressionCompressin
   public NormsFormat normsFormat() {
     return new Lucene42NormsFormat(PackedInts.DEFAULT);
   }
-
-  @Override
-  public DocValuesFormat docValuesFormat() {
-    return new Lucene42DocValuesFormat(PackedInts.DEFAULT);
-  }
 }

Modified: lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/HighCompressionCompressingCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/HighCompressionCompressingCodec.java?rev=1515416&r1=1515415&r2=1515416&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/HighCompressionCompressingCodec.java (original)
+++ lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/HighCompressionCompressingCodec.java Mon Aug 19 13:49:03 2013
@@ -21,7 +21,6 @@ import org.apache.lucene.util.packed.Pac
  * limitations under the License.
  */
 
-// nocommit
 /** CompressionCodec that uses {@link CompressionMode#HIGH_COMPRESSION} */
 public class HighCompressionCompressingCodec extends CompressingCodec {
 

Copied: lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java (from r1515388, lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java?p2=lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java&p1=lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java&r1=1515388&r2=1515416&rev=1515416&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java (original)
+++ lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java Mon Aug 19 13:49:03 2013
@@ -46,25 +46,20 @@ import org.apache.lucene.util.packed.Mon
 import org.apache.lucene.util.packed.PackedInts.FormatAndBits;
 import org.apache.lucene.util.packed.PackedInts;
 
+import static org.apache.lucene.codecs.lucene42.Lucene42DocValuesProducer.VERSION_CURRENT;
+import static org.apache.lucene.codecs.lucene42.Lucene42DocValuesProducer.BLOCK_SIZE;
+import static org.apache.lucene.codecs.lucene42.Lucene42DocValuesProducer.BYTES;
+import static org.apache.lucene.codecs.lucene42.Lucene42DocValuesProducer.NUMBER;
+import static org.apache.lucene.codecs.lucene42.Lucene42DocValuesProducer.FST;
+import static org.apache.lucene.codecs.lucene42.Lucene42DocValuesProducer.DELTA_COMPRESSED;
+import static org.apache.lucene.codecs.lucene42.Lucene42DocValuesProducer.GCD_COMPRESSED;
+import static org.apache.lucene.codecs.lucene42.Lucene42DocValuesProducer.TABLE_COMPRESSED;
+import static org.apache.lucene.codecs.lucene42.Lucene42DocValuesProducer.UNCOMPRESSED;
+
 /**
  * Writer for {@link Lucene42DocValuesFormat}
  */
 class Lucene42DocValuesConsumer extends DocValuesConsumer {
-  static final int VERSION_START = 0;
-  static final int VERSION_GCD_COMPRESSION = 1;
-  static final int VERSION_CURRENT = VERSION_GCD_COMPRESSION;
-  
-  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;
-
   final IndexOutput data, meta;
   final int maxDoc;
   final float acceptableOverheadRatio;

Added: lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene42/Lucene42RWCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene42/Lucene42RWCodec.java?rev=1515416&view=auto
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene42/Lucene42RWCodec.java (added)
+++ lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene42/Lucene42RWCodec.java Mon Aug 19 13:49:03 2013
@@ -0,0 +1,39 @@
+package org.apache.lucene.codecs.lucene42;
+
+/*
+ * 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.DocValuesFormat;
+import org.apache.lucene.codecs.NormsFormat;
+
+/**
+ * Read-write version of {@link Lucene42Codec} for testing.
+ */
+public class Lucene42RWCodec extends Lucene42Codec {
+  private static final DocValuesFormat dv = new Lucene42RWDocValuesFormat();
+  private static final NormsFormat norms = new Lucene42NormsFormat();
+
+  @Override
+  public DocValuesFormat getDocValuesFormatForField(String field) {
+    return dv;
+  }
+
+  @Override
+  public NormsFormat normsFormat() {
+    return norms;
+  }
+}

Added: lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene42/Lucene42RWDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene42/Lucene42RWDocValuesFormat.java?rev=1515416&view=auto
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene42/Lucene42RWDocValuesFormat.java (added)
+++ lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene42/Lucene42RWDocValuesFormat.java Mon Aug 19 13:49:03 2013
@@ -0,0 +1,35 @@
+package org.apache.lucene.codecs.lucene42;
+
+/*
+ * 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.index.SegmentWriteState;
+
+/**
+ * Read-write version of {@link Lucene42DocValuesFormat} for testing.
+ */
+public class Lucene42RWDocValuesFormat extends Lucene42DocValuesFormat {
+  
+  @Override
+  public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+    // note: we choose DEFAULT here (its reasonably fast, and for small bpv has tiny waste)
+    return new Lucene42DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION, acceptableOverheadRatio);
+  }
+}

Added: lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene42/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene42/package.html?rev=1515416&view=auto
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene42/package.html (added)
+++ lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene42/package.html Mon Aug 19 13:49:03 2013
@@ -0,0 +1,25 @@
+<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
+<!--
+ 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.
+-->
+<html>
+<head>
+   <meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
+</head>
+<body>
+Support for testing {@link org.apache.lucene.codecs.lucene42.Lucene42Codec}.
+</body>
+</html>
\ No newline at end of file

Modified: lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java?rev=1515416&r1=1515415&r2=1515416&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java (original)
+++ lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java Mon Aug 19 13:49:03 2013
@@ -2498,6 +2498,7 @@ public abstract class BaseDocValuesForma
     d.close();
   }
 
+  // nocommit: get this out of here and into the deprecated codecs (4.0, 4.2)
   public void testHugeBinaryValueLimit() throws Exception {
     // We only test DVFormats that have a limit
     assumeFalse("test requires codec with limits on max binary field length", codecAcceptsHugeBinaryValues("field"));

Modified: lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java?rev=1515416&r1=1515415&r2=1515416&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java (original)
+++ lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java Mon Aug 19 13:49:03 2013
@@ -40,6 +40,7 @@ import org.apache.lucene.codecs.lucene40
 import org.apache.lucene.codecs.lucene40.Lucene40RWPostingsFormat;
 import org.apache.lucene.codecs.lucene41.Lucene41RWCodec;
 import org.apache.lucene.codecs.lucene42.Lucene42Codec;
+import org.apache.lucene.codecs.lucene42.Lucene42RWCodec;
 import org.apache.lucene.codecs.lucene45.Lucene45Codec;
 import org.apache.lucene.codecs.simpletext.SimpleTextCodec;
 import org.apache.lucene.index.RandomCodec;
@@ -147,7 +148,6 @@ final class TestRuleSetupAndRestoreClass
     
     savedCodec = Codec.getDefault();
     int randomVal = random.nextInt(10);
-    // nocommit: 4.2 impersonator
     if ("Lucene40".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) &&
                                           "random".equals(TEST_POSTINGSFORMAT) &&
                                           "random".equals(TEST_DOCVALUESFORMAT) &&
@@ -163,6 +163,13 @@ final class TestRuleSetupAndRestoreClass
                                                  !shouldAvoidCodec("Lucene41"))) { 
       codec = Codec.forName("Lucene41");
       assert codec instanceof Lucene41RWCodec : "fix your classpath to have tests-framework.jar before lucene-core.jar";
+    } else if ("Lucene42".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) &&
+                                                 "random".equals(TEST_POSTINGSFORMAT) &&
+                                                 "random".equals(TEST_DOCVALUESFORMAT) &&
+                                                  randomVal == 2 &&
+                                                  !shouldAvoidCodec("Lucene42"))) { 
+      codec = Codec.forName("Lucene42");
+      assert codec instanceof Lucene42RWCodec : "fix your classpath to have tests-framework.jar before lucene-core.jar";
     } else if (("random".equals(TEST_POSTINGSFORMAT) == false) || ("random".equals(TEST_DOCVALUESFORMAT) == false)) {
       // the user wired postings or DV: this is messy
       // refactor into RandomCodec....

Modified: lucene/dev/branches/lucene5178/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.Codec?rev=1515416&r1=1515415&r2=1515416&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.Codec (original)
+++ lucene/dev/branches/lucene5178/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.Codec Mon Aug 19 13:49:03 2013
@@ -21,3 +21,4 @@ org.apache.lucene.codecs.compressing.Hig
 org.apache.lucene.codecs.compressing.dummy.DummyCompressingCodec
 org.apache.lucene.codecs.lucene40.Lucene40RWCodec
 org.apache.lucene.codecs.lucene41.Lucene41RWCodec
+org.apache.lucene.codecs.lucene42.Lucene42RWCodec

Modified: lucene/dev/branches/lucene5178/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat?rev=1515416&r1=1515415&r2=1515416&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat (original)
+++ lucene/dev/branches/lucene5178/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat Mon Aug 19 13:49:03 2013
@@ -14,3 +14,4 @@
 #  limitations under the License.
 
 org.apache.lucene.codecs.asserting.AssertingDocValuesFormat
+org.apache.lucene.codecs.lucene42.Lucene42RWDocValuesFormat