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 2012/11/19 16:58:16 UTC

svn commit: r1411262 - in /lucene/dev/trunk/lucene: core/src/java/org/apache/lucene/codecs/compressing/ core/src/java/org/apache/lucene/codecs/lucene41/ test-framework/src/java/org/apache/lucene/codecs/compressing/ test-framework/src/resources/META-INF...

Author: jpountz
Date: Mon Nov 19 15:58:15 2012
New Revision: 1411262

URL: http://svn.apache.org/viewvc?rev=1411262&view=rev
Log:
LUCENE-4558: Make CompressingStoredFieldsFormat more flexible.

Added:
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/DummyCompressingCodec.java   (with props)
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/FastCompressingCodec.java   (with props)
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/FastDecompressionCompressingCodec.java   (with props)
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/HighCompressionCompressingCodec.java   (with props)
Modified:
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsFormat.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressionMode.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsFormat.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/CompressingCodec.java
    lucene/dev/trunk/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.Codec

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsFormat.java?rev=1411262&r1=1411261&r2=1411262&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsFormat.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsFormat.java Mon Nov 19 15:58:15 2012
@@ -19,6 +19,7 @@ package org.apache.lucene.codecs.compres
 
 import java.io.IOException;
 
+import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.StoredFieldsFormat;
 import org.apache.lucene.codecs.StoredFieldsReader;
 import org.apache.lucene.codecs.StoredFieldsWriter;
@@ -29,6 +30,7 @@ import org.apache.lucene.index.SegmentIn
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 
+
 /**
  * A {@link StoredFieldsFormat} that is very similar to
  * {@link Lucene40StoredFieldsFormat} but compresses documents in chunks in
@@ -45,16 +47,23 @@ import org.apache.lucene.store.IOContext
  */
 public class CompressingStoredFieldsFormat extends StoredFieldsFormat {
 
+  private final String formatName;
   private final CompressionMode compressionMode;
   private final int chunkSize;
 
   /**
    * Create a new {@link CompressingStoredFieldsFormat}.
    * <p>
+   * <code>formatName</code> is the name of the format. This name will be used
+   * in the file formats to perform
+   * {@link CodecUtil#checkHeader(org.apache.lucene.store.DataInput, String, int, int) codec header checks}.
+   * <p>
    * The <code>compressionMode</code> parameter allows you to choose between
    * compression algorithms that have various compression and decompression
    * speeds so that you can pick the one that best fits your indexing and
-   * searching throughput.
+   * searching throughput. You should never instantiate two
+   * {@link CompressingStoredFieldsFormat}s that have the same name but
+   * different {@link CompressionMode}s.
    * <p>
    * <code>chunkSize</code> is the minimum byte size of a chunk of documents.
    * A value of <code>1</code> can make sense if there is redundancy across
@@ -67,11 +76,13 @@ public class CompressingStoredFieldsForm
    * loading a little slower (depending on the size of your OS cache compared
    * to the size of your index).
    *
+   * @param formatName the name of the {@link StoredFieldsFormat}
    * @param compressionMode the {@link CompressionMode} to use
    * @param chunkSize the minimum number of bytes of a single chunk of stored documents
    * @see CompressionMode
    */
-  public CompressingStoredFieldsFormat(CompressionMode compressionMode, int chunkSize) {
+  public CompressingStoredFieldsFormat(String formatName, CompressionMode compressionMode, int chunkSize) {
+    this.formatName = formatName;
     this.compressionMode = compressionMode;
     if (chunkSize < 1) {
       throw new IllegalArgumentException("chunkSize must be >= 1");
@@ -79,27 +90,17 @@ public class CompressingStoredFieldsForm
     this.chunkSize = chunkSize;
   }
 
-  /**
-   * Create a new {@link CompressingStoredFieldsFormat} with
-   * {@link CompressionMode#FAST} compression and chunks of <tt>16 KB</tt>.
-   *
-   * @see CompressingStoredFieldsFormat#CompressingStoredFieldsFormat(CompressionMode, int)
-   */
-  public CompressingStoredFieldsFormat() {
-    this(CompressionMode.FAST, 1 << 14);
-  }
-
   @Override
   public StoredFieldsReader fieldsReader(Directory directory, SegmentInfo si,
       FieldInfos fn, IOContext context) throws IOException {
-    return new CompressingStoredFieldsReader(directory, si, fn, context);
+    return new CompressingStoredFieldsReader(directory, si, fn, context, formatName, compressionMode);
   }
 
   @Override
   public StoredFieldsWriter fieldsWriter(Directory directory, SegmentInfo si,
       IOContext context) throws IOException {
     return new CompressingStoredFieldsWriter(directory, si, context,
-        compressionMode, chunkSize);
+        formatName, compressionMode, chunkSize);
   }
 
   @Override

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsReader.java?rev=1411262&r1=1411261&r2=1411262&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsReader.java Mon Nov 19 15:58:15 2012
@@ -18,10 +18,8 @@ package org.apache.lucene.codecs.compres
  */
 
 import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.BYTE_ARR;
-import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.CODEC_NAME_DAT;
-import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.CODEC_NAME_IDX;
-import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.HEADER_LENGTH_DAT;
-import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.HEADER_LENGTH_IDX;
+import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.CODEC_SFX_DAT;
+import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.CODEC_SFX_IDX;
 import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.NUMERIC_DOUBLE;
 import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.NUMERIC_FLOAT;
 import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.NUMERIC_INT;
@@ -81,7 +79,9 @@ final class CompressingStoredFieldsReade
     this.closed = false;
   }
 
-  public CompressingStoredFieldsReader(Directory d, SegmentInfo si, FieldInfos fn, IOContext context) throws IOException {
+  public CompressingStoredFieldsReader( Directory d, SegmentInfo si, FieldInfos fn,
+      IOContext context, String formatName, CompressionMode compressionMode) throws IOException {
+    this.compressionMode = compressionMode;
     final String segment = si.name;
     boolean success = false;
     fieldInfos = fn;
@@ -92,17 +92,17 @@ final class CompressingStoredFieldsReade
       final String indexStreamFN = IndexFileNames.segmentFileName(segment, "", FIELDS_INDEX_EXTENSION);
       indexStream = d.openInput(indexStreamFN, context);
 
-      CodecUtil.checkHeader(indexStream, CODEC_NAME_IDX, VERSION_START, VERSION_CURRENT);
-      CodecUtil.checkHeader(fieldsStream, CODEC_NAME_DAT, VERSION_START, VERSION_CURRENT);
-      assert HEADER_LENGTH_DAT == fieldsStream.getFilePointer();
-      assert HEADER_LENGTH_IDX == indexStream.getFilePointer();
+      final String codecNameIdx = formatName + CODEC_SFX_IDX;
+      final String codecNameDat = formatName + CODEC_SFX_DAT;
+      CodecUtil.checkHeader(indexStream, codecNameIdx, VERSION_START, VERSION_CURRENT);
+      CodecUtil.checkHeader(fieldsStream, codecNameDat, VERSION_START, VERSION_CURRENT);
+      assert CodecUtil.headerLength(codecNameDat) == fieldsStream.getFilePointer();
+      assert CodecUtil.headerLength(codecNameIdx) == indexStream.getFilePointer();
 
       indexReader = new CompressingStoredFieldsIndexReader(indexStream, si);
       indexStream = null;
 
       packedIntsVersion = fieldsStream.readVInt();
-      final int compressionModeId = fieldsStream.readVInt();
-      compressionMode = CompressionMode.byId(compressionModeId);
       decompressor = compressionMode.newDecompressor();
       this.bytes = new BytesRef();
 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java?rev=1411262&r1=1411261&r2=1411262&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java Mon Nov 19 15:58:15 2012
@@ -59,12 +59,10 @@ final class CompressingStoredFieldsWrite
   static final int TYPE_BITS = PackedInts.bitsRequired(NUMERIC_DOUBLE);
   static final int TYPE_MASK = (int) PackedInts.maxValue(TYPE_BITS);
 
-  static final String CODEC_NAME_IDX = "CompressingStoredFieldsIndex";
-  static final String CODEC_NAME_DAT = "CompressingStoredFieldsData";
+  static final String CODEC_SFX_IDX = "Index";
+  static final String CODEC_SFX_DAT = "Data";
   static final int VERSION_START = 0;
   static final int VERSION_CURRENT = VERSION_START;
-  static final long HEADER_LENGTH_IDX = CodecUtil.headerLength(CODEC_NAME_IDX);
-  static final long HEADER_LENGTH_DAT = CodecUtil.headerLength(CODEC_NAME_DAT);
 
   private final Directory directory;
   private final String segment;
@@ -81,8 +79,8 @@ final class CompressingStoredFieldsWrite
   private int docBase; // doc ID at the beginning of the chunk
   private int numBufferedDocs; // docBase + numBufferedDocs == current doc ID
 
-  public CompressingStoredFieldsWriter(Directory directory, SegmentInfo si,
-      IOContext context, CompressionMode compressionMode, int chunkSize) throws IOException {
+  public CompressingStoredFieldsWriter(Directory directory, SegmentInfo si, IOContext context,
+      String formatName, CompressionMode compressionMode, int chunkSize) throws IOException {
     assert directory != null;
     this.directory = directory;
     this.segment = si.name;
@@ -100,16 +98,17 @@ final class CompressingStoredFieldsWrite
     try {
       fieldsStream = directory.createOutput(IndexFileNames.segmentFileName(segment, "", FIELDS_EXTENSION), context);
 
-      CodecUtil.writeHeader(indexStream, CODEC_NAME_IDX, VERSION_CURRENT);
-      CodecUtil.writeHeader(fieldsStream, CODEC_NAME_DAT, VERSION_CURRENT);
-      assert HEADER_LENGTH_IDX == indexStream.getFilePointer();
-      assert HEADER_LENGTH_DAT == fieldsStream.getFilePointer();
+      final String codecNameIdx = formatName + CODEC_SFX_IDX;
+      final String codecNameDat = formatName + CODEC_SFX_DAT;
+      CodecUtil.writeHeader(indexStream, codecNameIdx, VERSION_CURRENT);
+      CodecUtil.writeHeader(fieldsStream, codecNameDat, VERSION_CURRENT);
+      assert CodecUtil.headerLength(codecNameDat) == fieldsStream.getFilePointer();
+      assert CodecUtil.headerLength(codecNameIdx) == indexStream.getFilePointer();
 
       indexWriter = new CompressingStoredFieldsIndexWriter(indexStream);
       indexStream = null;
 
       fieldsStream.writeVInt(PackedInts.VERSION_CURRENT);
-      fieldsStream.writeVInt(compressionMode.getId());
 
       success = true;
     } finally {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressionMode.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressionMode.java?rev=1411262&r1=1411261&r2=1411262&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressionMode.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressionMode.java Mon Nov 19 15:58:15 2012
@@ -33,7 +33,7 @@ import org.apache.lucene.util.BytesRef;
  * decompression of stored fields.
  * @lucene.experimental
  */
-public enum CompressionMode {
+public abstract class CompressionMode {
 
   /**
    * A compression mode that trades compression ratio for speed. Although the
@@ -41,19 +41,24 @@ public enum CompressionMode {
    * very fast. Use this mode with indices that have a high update rate but
    * should be able to load documents from disk quickly.
    */
-  FAST(0) {
+  public static final CompressionMode FAST = new CompressionMode() {
 
     @Override
-    Compressor newCompressor() {
+    public Compressor newCompressor() {
       return LZ4_FAST_COMPRESSOR;
     }
 
     @Override
-    Decompressor newDecompressor() {
+    public Decompressor newDecompressor() {
       return LZ4_DECOMPRESSOR;
     }
 
-  },
+    @Override
+    public String toString() {
+      return "FAST";
+    }
+
+  };
 
   /**
    * A compression mode that trades speed for compression ratio. Although
@@ -61,19 +66,24 @@ public enum CompressionMode {
    * provide a good compression ratio. This mode might be interesting if/when
    * your index size is much bigger than your OS cache.
    */
-  HIGH_COMPRESSION(1) {
+  public static final CompressionMode HIGH_COMPRESSION = new CompressionMode() {
 
     @Override
-    Compressor newCompressor() {
+    public Compressor newCompressor() {
       return new DeflateCompressor(Deflater.BEST_COMPRESSION);
     }
 
     @Override
-    Decompressor newDecompressor() {
+    public Decompressor newDecompressor() {
       return new DeflateDecompressor();
     }
 
-  },
+    @Override
+    public String toString() {
+      return "HIGH_COMPRESSION";
+    }
+
+  };
 
   /**
    * This compression mode is similar to {@link #FAST} but it spends more time
@@ -81,55 +91,37 @@ public enum CompressionMode {
    * mode is best used with indices that have a low update rate but should be
    * able to load documents from disk quickly.
    */
-  FAST_DECOMPRESSION(2) {
+  public static final CompressionMode FAST_DECOMPRESSION = new CompressionMode() {
 
     @Override
-    Compressor newCompressor() {
+    public Compressor newCompressor() {
       return LZ4_HIGH_COMPRESSOR;
     }
 
     @Override
-    Decompressor newDecompressor() {
+    public Decompressor newDecompressor() {
       return LZ4_DECOMPRESSOR;
     }
 
-  };
-
-  /** Get a {@link CompressionMode} according to its id. */
-  public static CompressionMode byId(int id) {
-    for (CompressionMode mode : CompressionMode.values()) {
-      if (mode.getId() == id) {
-        return mode;
-      }
+    @Override
+    public String toString() {
+      return "FAST_DECOMPRESSION";
     }
-    throw new IllegalArgumentException("Unknown id: " + id);
-  }
 
-  private final int id;
-
-  private CompressionMode(int id) {
-    this.id = id;
-  }
+  };
 
-  /**
-   * Returns an ID for this compression mode. Should be unique across
-   * {@link CompressionMode}s as it is used for serialization and
-   * unserialization.
-   */
-  public final int getId() {
-    return id;
-  }
+  /** Sole constructor. */
+  protected CompressionMode() {}
 
   /**
    * Create a new {@link Compressor} instance.
    */
-  abstract Compressor newCompressor();
+  public abstract Compressor newCompressor();
 
   /**
    * Create a new {@link Decompressor} instance.
    */
-  abstract Decompressor newDecompressor();
-
+  public abstract Decompressor newDecompressor();
 
   private static final Decompressor LZ4_DECOMPRESSOR = new Decompressor() {
 
@@ -264,6 +256,7 @@ public enum CompressionMode {
 
       if (compressor.needsInput()) {
         // no output
+        assert len == 0 : len;
         out.writeVInt(0);
         return;
       }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsFormat.java?rev=1411262&r1=1411261&r2=1411262&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsFormat.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsFormat.java Mon Nov 19 15:58:15 2012
@@ -53,7 +53,6 @@ import org.apache.lucene.util.packed.Pac
  * <li>FieldData (.fdt) --&gt; &lt;Header&gt;, PackedIntsVersion, CompressionFormat, &lt;Chunk&gt;<sup>ChunkCount</sup></li>
  * <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
  * <li>PackedIntsVersion --&gt; {@link PackedInts#VERSION_CURRENT} as a {@link DataOutput#writeVInt VInt}</li>
- * <li>CompressionFormat --&gt; always <tt>0</tt> as a {@link DataOutput#writeVInt VInt}, this may allow for different compression formats in the future</li>
  * <li>ChunkCount is not known in advance and is the number of chunks necessary to store all document of the segment</li>
  * <li>Chunk --&gt; DocBase, ChunkDocs, DocFieldCounts, DocLengths, &lt;CompressedDocs&gt;</li>
  * <li>DocBase --&gt; the ID of the first document of the chunk as a {@link DataOutput#writeVInt VInt}</li>
@@ -147,7 +146,7 @@ public final class Lucene41StoredFieldsF
 
   /** Sole constructor. */
   public Lucene41StoredFieldsFormat() {
-    super(CompressionMode.FAST, 1 << 14);
+    super("Lucene41StoredFields", CompressionMode.FAST, 1 << 14);
   }
 
 }

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/CompressingCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/CompressingCodec.java?rev=1411262&r1=1411261&r2=1411262&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/CompressingCodec.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/CompressingCodec.java Mon Nov 19 15:58:15 2012
@@ -24,35 +24,40 @@ import org.apache.lucene.codecs.StoredFi
 import org.apache.lucene.codecs.lucene41.Lucene41Codec;
 
 import com.carrotsearch.randomizedtesting.generators.RandomInts;
-import com.carrotsearch.randomizedtesting.generators.RandomPicks;
 
 /**
  * A codec that uses {@link CompressingStoredFieldsFormat} for its stored
  * fields and delegates to {@link Lucene41Codec} for everything else.
  */
-public class CompressingCodec extends FilterCodec {
+public abstract class CompressingCodec extends FilterCodec {
 
   /**
    * Create a random instance.
    */
+  public static CompressingCodec randomInstance(Random random, int chunkSize) {
+    switch (random.nextInt(4)) {
+    case 0:
+      return new FastCompressingCodec(chunkSize);
+    case 1:
+      return new FastDecompressionCompressingCodec(chunkSize);
+    case 2:
+      return new HighCompressionCompressingCodec(chunkSize);
+    case 3:
+      return new DummyCompressingCodec(chunkSize);
+    default:
+      throw new AssertionError();
+    }
+  }
+
   public static CompressingCodec randomInstance(Random random) {
-    final CompressionMode mode = RandomPicks.randomFrom(random, CompressionMode.values());
-    final int chunkSize = RandomInts.randomIntBetween(random, 1, 500);
-    return new CompressingCodec(mode, chunkSize);
+    return randomInstance(random, RandomInts.randomIntBetween(random, 1, 500));
   }
 
   private final CompressingStoredFieldsFormat storedFieldsFormat;
 
-  /**
-   * @see CompressingStoredFieldsFormat#CompressingStoredFieldsFormat(CompressionMode, int)
-   */
-  public CompressingCodec(CompressionMode compressionMode, int chunkSize) {
-    super("Compressing", new Lucene41Codec());
-    this.storedFieldsFormat = new CompressingStoredFieldsFormat(compressionMode, chunkSize);
-  }
-
-  public CompressingCodec() {
-    this(CompressionMode.FAST, 1 << 14);
+  public CompressingCodec(String name, CompressionMode compressionMode, int chunkSize) {
+    super(name, new Lucene41Codec());
+    this.storedFieldsFormat = new CompressingStoredFieldsFormat(name, compressionMode, chunkSize);
   }
 
   @Override

Added: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/DummyCompressingCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/DummyCompressingCodec.java?rev=1411262&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/DummyCompressingCodec.java (added)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/DummyCompressingCodec.java Mon Nov 19 15:58:15 2012
@@ -0,0 +1,94 @@
+package org.apache.lucene.codecs.compressing;
+
+/*
+ * 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.DataInput;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+
+/** CompressionCodec that does not compress data, useful for testing. */
+public class DummyCompressingCodec extends CompressingCodec {
+
+  public static final CompressionMode DUMMY = new CompressionMode() {
+
+    @Override
+    public Compressor newCompressor() {
+      return DUMMY_COMPRESSOR;
+    }
+
+    @Override
+    public Decompressor newDecompressor() {
+      return DUMMY_DECOMPRESSOR;
+    }
+
+    @Override
+    public String toString() {
+      return "DUMMY";
+    }
+
+  };
+
+  private static final Decompressor DUMMY_DECOMPRESSOR = new Decompressor() {
+
+    @Override
+    public void decompress(DataInput in, int originalLength,
+        int offset, int length, BytesRef bytes) throws IOException {
+      assert offset + length <= originalLength;
+      if (bytes.bytes.length < originalLength) {
+        bytes.bytes = new byte[ArrayUtil.oversize(originalLength, 1)];
+      }
+      in.readBytes(bytes.bytes, 0, offset + length);
+      bytes.offset = offset;
+      bytes.length = length;
+    }
+
+    @Override
+    public void copyCompressedData(DataInput in, int originalLength, DataOutput out) throws IOException {
+      out.copyBytes(in, originalLength);
+    }
+
+    @Override
+    public Decompressor clone() {
+      return this;
+    }
+
+  };
+
+  private static final Compressor DUMMY_COMPRESSOR = new Compressor() {
+
+    @Override
+    public void compress(byte[] bytes, int off, int len, DataOutput out) throws IOException {
+      out.writeBytes(bytes, off, len);
+    }
+
+  };
+
+  /** Constructor that allows to configure the chunk size. */
+  public DummyCompressingCodec(int chunkSize) {
+    super("DummyCompressingStoredFields", DUMMY, chunkSize);
+  }
+
+  /** Default constructor. */
+  public DummyCompressingCodec() {
+    this(1 << 14);
+  }
+
+}

Added: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/FastCompressingCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/FastCompressingCodec.java?rev=1411262&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/FastCompressingCodec.java (added)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/FastCompressingCodec.java Mon Nov 19 15:58:15 2012
@@ -0,0 +1,33 @@
+package org.apache.lucene.codecs.compressing;
+
+/*
+ * 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.
+ */
+
+/** CompressionCodec that uses {@link CompressionMode#FAST} */
+public class FastCompressingCodec extends CompressingCodec {
+
+  /** Constructor that allows to configure the chunk size. */
+  public FastCompressingCodec(int chunkSize) {
+    super("FastCompressingStoredFields", CompressionMode.FAST, chunkSize);
+  }
+
+  /** Default constructor. */
+  public FastCompressingCodec() {
+    this(1 << 14);
+  }
+
+}

Added: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/FastDecompressionCompressingCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/FastDecompressionCompressingCodec.java?rev=1411262&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/FastDecompressionCompressingCodec.java (added)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/FastDecompressionCompressingCodec.java Mon Nov 19 15:58:15 2012
@@ -0,0 +1,33 @@
+package org.apache.lucene.codecs.compressing;
+
+/*
+ * 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.
+ */
+
+/** CompressionCodec that uses {@link CompressionMode#FAST_DECOMPRESSION} */
+public class FastDecompressionCompressingCodec extends CompressingCodec {
+
+  /** Constructor that allows to configure the chunk size. */
+  public FastDecompressionCompressingCodec(int chunkSize) {
+    super("FastDecompressionCompressingStoredFields", CompressionMode.FAST_DECOMPRESSION, chunkSize);
+  }
+
+  /** Default constructor. */
+  public FastDecompressionCompressingCodec() {
+    this(1 << 14);
+  }
+
+}

Added: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/HighCompressionCompressingCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/HighCompressionCompressingCodec.java?rev=1411262&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/HighCompressionCompressingCodec.java (added)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/HighCompressionCompressingCodec.java Mon Nov 19 15:58:15 2012
@@ -0,0 +1,31 @@
+package org.apache.lucene.codecs.compressing;
+
+/*
+ * 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.
+ */
+
+/** CompressionCodec that uses {@link CompressionMode#HIGH_COMPRESSION} */
+public class HighCompressionCompressingCodec extends CompressingCodec {
+
+  public HighCompressionCompressingCodec(int chunkSize) {
+    super("HighCompressionCompressingStoredFields", CompressionMode.HIGH_COMPRESSION, chunkSize);
+  }
+
+  public HighCompressionCompressingCodec() {
+    this(1 << 14);
+  }
+
+}

Modified: lucene/dev/trunk/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.Codec?rev=1411262&r1=1411261&r2=1411262&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.Codec (original)
+++ lucene/dev/trunk/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.Codec Mon Nov 19 15:58:15 2012
@@ -14,4 +14,7 @@
 #  limitations under the License.
 
 org.apache.lucene.codecs.asserting.AssertingCodec
-org.apache.lucene.codecs.compressing.CompressingCodec
+org.apache.lucene.codecs.compressing.FastCompressingCodec
+org.apache.lucene.codecs.compressing.FastDecompressionCompressingCodec
+org.apache.lucene.codecs.compressing.HighCompressionCompressingCodec
+org.apache.lucene.codecs.compressing.DummyCompressingCodec