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/10/05 17:14:36 UTC

svn commit: r1394578 [2/2] - in /lucene/dev/trunk/lucene: ./ codecs/src/java/org/apache/lucene/codecs/compressing/ codecs/src/test/org/apache/lucene/codecs/compressing/ core/src/java/org/apache/lucene/util/packed/ test-framework/src/java/org/apache/luc...

Added: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/compressing/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/compressing/package.html?rev=1394578&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/compressing/package.html (added)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/compressing/package.html Fri Oct  5 15:14:35 2012
@@ -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>
+StoredFieldsFormat that allows cross-document and cross-field compression of stored fields.
+</body>
+</html>

Added: lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/compressing/AbstractTestCompressionMode.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/compressing/AbstractTestCompressionMode.java?rev=1394578&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/compressing/AbstractTestCompressionMode.java (added)
+++ lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/compressing/AbstractTestCompressionMode.java Fri Oct  5 15:14:35 2012
@@ -0,0 +1,111 @@
+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 java.util.Arrays;
+
+import org.apache.lucene.store.ByteArrayDataInput;
+import org.apache.lucene.store.ByteArrayDataOutput;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LuceneTestCase;
+
+import com.carrotsearch.randomizedtesting.generators.RandomInts;
+
+public abstract class AbstractTestCompressionMode extends LuceneTestCase {
+
+  CompressionMode mode;
+
+  static byte[] randomArray() {
+    final int max = random().nextBoolean()
+        ? random().nextInt(4)
+        : random().nextInt(256);
+    final int length = random().nextBoolean()
+        ? random().nextInt(20)
+        : random().nextInt(192 * 1024);
+    final byte[] arr = new byte[length];
+    for (int i = 0; i < arr.length; ++i) {
+      arr[i] = (byte) RandomInts.randomIntBetween(random(), 0, max);
+    }
+    return arr;
+  }
+
+  byte[] compress(byte[] uncompressed) throws IOException {
+    Compressor compressor = mode.newCompressor();
+    return compress(compressor, uncompressed);
+  }
+
+  static byte[] compress(Compressor compressor, byte[] uncompressed) throws IOException {
+    byte[] compressed = new byte[uncompressed.length * 2 + 16]; // should be enough
+    ByteArrayDataOutput out = new ByteArrayDataOutput(compressed);
+    compressor.compress(uncompressed, 0, uncompressed.length, out);
+    final int compressedLen = out.getPosition();
+    return Arrays.copyOf(compressed, compressedLen);
+  }
+
+  byte[] uncompress(byte[] compressed) throws IOException {
+    Uncompressor uncompressor = mode.newUncompressor();
+    return uncompress(uncompressor, compressed);
+  }
+
+  static byte[] uncompress(Uncompressor uncompressor, byte[] compressed) throws IOException {
+    final BytesRef bytes = new BytesRef();
+    uncompressor.uncompress(new ByteArrayDataInput(compressed), bytes);
+    return Arrays.copyOfRange(bytes.bytes, bytes.offset, bytes.offset + bytes.length);
+  }
+
+  byte[] uncompress(byte[] compressed, int offset, int length) throws IOException {
+    Uncompressor uncompressor = mode.newUncompressor();
+    final BytesRef bytes = new BytesRef();
+    uncompressor.uncompress(new ByteArrayDataInput(compressed), offset, length, bytes);
+    return Arrays.copyOfRange(bytes.bytes, bytes.offset, bytes.offset + bytes.length);
+  }
+
+  public void testUncompress() throws IOException {
+    final byte[] uncompressed = randomArray();
+    final byte[] compressed = compress(uncompressed);
+    final byte[] restored = uncompress(compressed);
+    assertArrayEquals(uncompressed, restored);
+  }
+
+  public void testPartialUncompress() throws IOException {
+    final int iterations = atLeast(10);
+    for (int i = 0; i < iterations; ++i) {
+      final byte[] uncompressed = randomArray();
+      final byte[] compressed = compress(uncompressed);
+      final int offset, length;
+      if (uncompressed.length == 0) {
+        offset = length = 0;
+      } else {
+        offset = random().nextInt(uncompressed.length);
+        length = random().nextInt(uncompressed.length - offset);
+      }
+      final byte[] restored = uncompress(compressed, offset, length);
+      assertArrayEquals(Arrays.copyOfRange(uncompressed, offset, offset + length), restored);
+    }
+  }
+
+  public void testCopyCompressedData() throws IOException {
+    final byte[] uncompressed = randomArray();
+    final byte[] compressed = compress(uncompressed);
+    GrowableByteArrayDataOutput out = new GrowableByteArrayDataOutput(uncompressed.length);
+    mode.newUncompressor().copyCompressedData(new ByteArrayDataInput(compressed), out);
+    assertArrayEquals(compressed, Arrays.copyOf(out.bytes, out.length));
+  }
+
+}

Added: lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/compressing/TestCompressingStoredFieldsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/compressing/TestCompressingStoredFieldsFormat.java?rev=1394578&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/compressing/TestCompressingStoredFieldsFormat.java (added)
+++ lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/compressing/TestCompressingStoredFieldsFormat.java Fri Oct  5 15:14:35 2012
@@ -0,0 +1,193 @@
+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 java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.codecs.lucene40.Lucene40Codec;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.DoubleField;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.FloatField;
+import org.apache.lucene.document.IntField;
+import org.apache.lucene.document.LongField;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.StorableField;
+import org.apache.lucene.index.StoredDocument;
+import org.apache.lucene.search.NumericRangeQuery;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
+
+import com.carrotsearch.randomizedtesting.generators.RandomInts;
+
+public class TestCompressingStoredFieldsFormat extends LuceneTestCase {
+
+  public void testWriteReadMerge() throws IOException {
+    Directory dir = newDirectory();
+    IndexWriterConfig iwConf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
+    iwConf.setMaxBufferedDocs(RandomInts.randomIntBetween(random(), 2, 30));
+    iwConf.setCodec(CompressingCodec.randomInstance(random()));
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwConf);
+    final int docCount = atLeast(200);
+    final byte[][][] data = new byte [docCount][][];
+    for (int i = 0; i < docCount; ++i) {
+      final int fieldCount = rarely()
+          ? RandomInts.randomIntBetween(random(), 1, 500)
+          : RandomInts.randomIntBetween(random(), 1, 5);
+      data[i] = new byte[fieldCount][];
+      for (int j = 0; j < fieldCount; ++j) {
+        final int length = rarely()
+            ? random().nextInt(1000)
+            : random().nextInt(10);
+        final byte[] arr = new byte[length];
+        final int max = rarely() ? 256 : 2;
+        for (int k = 0; k < length; ++k) {
+          arr[k] = (byte) random().nextInt(max);
+        }
+        data[i][j] = arr;
+      }
+    }
+
+    final FieldType type = new FieldType(StringField.TYPE_STORED);
+    type.setIndexed(false);
+    type.freeze();
+    IntField id = new IntField("id", 0, Store.YES);
+    for (int i = 0; i < data.length; ++i) {
+      Document doc = new Document();
+      doc.add(id);
+      id.setIntValue(i);
+      for (int j = 0; j < data[i].length; ++j) {
+        Field f = new Field("bytes" + j, data[i][j], type);
+        doc.add(f);
+      }
+      iw.w.addDocument(doc);
+      if (random().nextBoolean() && (i % (data.length / 10) == 0)) {
+        iw.w.close();
+        // switch codecs
+        if (iwConf.getCodec() instanceof Lucene40Codec) {
+          iwConf.setCodec(CompressingCodec.randomInstance(random()));
+        } else {
+          iwConf.setCodec(new Lucene40Codec());
+        }
+        iw = new RandomIndexWriter(random(), dir, iwConf);
+      }
+    }
+
+    for (int i = 0; i < 10; ++i) {
+      final int min = random().nextInt(data.length);
+      final int max = min + random().nextInt(20);
+      iw.deleteDocuments(NumericRangeQuery.newIntRange("id", min, max, true, false));
+    }
+
+    iw.forceMerge(2); // force merges with deletions
+
+    iw.commit();
+
+    final DirectoryReader ir = DirectoryReader.open(dir);
+    assertTrue(ir.numDocs() > 0);
+    int numDocs = 0;
+    for (int i = 0; i < ir.maxDoc(); ++i) {
+      final StoredDocument doc = ir.document(i);
+      if (doc == null) {
+        continue;
+      }
+      ++ numDocs;
+      final int docId = doc.getField("id").numericValue().intValue();
+      assertEquals(data[docId].length + 1, doc.getFields().size());
+      for (int j = 0; j < data[docId].length; ++j) {
+        final byte[] arr = data[docId][j];
+        final BytesRef arr2Ref = doc.getBinaryValue("bytes" + j);
+        final byte[] arr2 = Arrays.copyOfRange(arr2Ref.bytes, arr2Ref.offset, arr2Ref.offset + arr2Ref.length);
+        assertArrayEquals(arr, arr2);
+      }
+    }
+    assertTrue(ir.numDocs() <= numDocs);
+    ir.close();
+
+    iw.deleteAll();
+    iw.commit();
+    iw.forceMerge(1);
+    iw.close();
+    dir.close();
+  }
+
+  public void testReadSkip() throws IOException {
+    Directory dir = newDirectory();
+    IndexWriterConfig iwConf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
+    iwConf.setMaxBufferedDocs(RandomInts.randomIntBetween(random(), 2, 30));
+    iwConf.setCodec(CompressingCodec.randomInstance(random()));
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwConf);
+
+    FieldType ft = new FieldType();
+    ft.setStored(true);
+    ft.freeze();
+    
+    final String string = _TestUtil.randomSimpleString(random(), 50);
+    final byte[] bytes = string.getBytes("UTF-8");
+    final long l = random().nextBoolean() ? random().nextInt(42) : random().nextLong();
+    final int i = random().nextBoolean() ? random().nextInt(42) : random().nextInt();
+    final float f = random().nextFloat();
+    final double d = random().nextDouble();
+
+    List<Field> fields = Arrays.asList(
+        new Field("bytes", bytes, ft),
+        new Field("string", string, ft),
+        new LongField("long", l, Store.YES),
+        new IntField("int", i, Store.YES),
+        new FloatField("float", f, Store.YES),
+        new DoubleField("double", d, Store.YES)
+    );
+
+    for (int k = 0; k < 100; ++k) {
+      Document doc = new Document();
+      for (Field fld : fields) {
+        doc.add(fld);
+      }
+      iw.w.addDocument(doc);
+    }
+    iw.close();
+
+    final DirectoryReader reader = DirectoryReader.open(dir);
+    final int docID = random().nextInt(100);
+    for (Field fld : fields) {
+      String fldName = fld.name();
+      final StoredDocument sDoc = reader.document(docID, Collections.singleton(fldName));
+      final StorableField sField = sDoc.getField(fldName);
+      if (Field.class.equals(fld.getClass())) {
+        assertEquals(fld.binaryValue(), sField.binaryValue());
+        assertEquals(fld.stringValue(), sField.stringValue());
+      } else {
+        assertEquals(fld.numericValue(), sField.numericValue());
+      }
+    }
+    reader.close();
+    dir.close();
+  }
+
+}

Added: lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/compressing/TestFastCompressionMode.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/compressing/TestFastCompressionMode.java?rev=1394578&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/compressing/TestFastCompressionMode.java (added)
+++ lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/compressing/TestFastCompressionMode.java Fri Oct  5 15:14:35 2012
@@ -0,0 +1,27 @@
+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.
+ */
+
+public class TestFastCompressionMode extends AbstractTestCompressionMode {
+
+  public void setUp() throws Exception {
+    super.setUp();
+    mode = CompressionMode.FAST;
+  }
+
+}

Added: lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/compressing/TestFastUncompressionMode.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/compressing/TestFastUncompressionMode.java?rev=1394578&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/compressing/TestFastUncompressionMode.java (added)
+++ lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/compressing/TestFastUncompressionMode.java Fri Oct  5 15:14:35 2012
@@ -0,0 +1,28 @@
+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.
+ */
+
+
+public class TestFastUncompressionMode extends AbstractTestCompressionMode {
+
+  public void setUp() throws Exception {
+    super.setUp();
+    mode = CompressionMode.FAST_UNCOMPRESSION;
+  }
+
+}

Added: lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/compressing/TestHighCompressionMode.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/compressing/TestHighCompressionMode.java?rev=1394578&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/compressing/TestHighCompressionMode.java (added)
+++ lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/compressing/TestHighCompressionMode.java Fri Oct  5 15:14:35 2012
@@ -0,0 +1,27 @@
+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.
+ */
+
+
+public class TestHighCompressionMode extends AbstractTestCompressionMode {
+
+  public void setUp() throws Exception {
+    super.setUp();
+    mode = CompressionMode.HIGH_COMPRESSION;
+  }
+}

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/PackedReaderIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/PackedReaderIterator.java?rev=1394578&r1=1394577&r2=1394578&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/PackedReaderIterator.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/PackedReaderIterator.java Fri Oct  5 15:14:35 2012
@@ -37,7 +37,7 @@ final class PackedReaderIterator extends
     this.format = format;
     bulkOperation = BulkOperation.of(format, bitsPerValue);
     iterations = bulkOperation.computeIterations(valueCount, mem);
-    assert iterations > 0;
+    assert valueCount == 0 || iterations > 0;
     nextBlocks = new long[iterations * bulkOperation.blockCount()];
     nextValues = new LongsRef(new long[iterations * bulkOperation.valueCount()], 0, 0);
     assert iterations * bulkOperation.valueCount() == nextValues.longs.length;

Added: 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=1394578&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/CompressingCodec.java (added)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/CompressingCodec.java Fri Oct  5 15:14:35 2012
@@ -0,0 +1,64 @@
+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.util.Random;
+
+import org.apache.lucene.codecs.FilterCodec;
+import org.apache.lucene.codecs.StoredFieldsFormat;
+import org.apache.lucene.codecs.lucene40.Lucene40Codec;
+
+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 Lucene40Codec} for everything else.
+ */
+public class CompressingCodec extends FilterCodec {
+
+  /**
+   * Create a random instance.
+   */
+  public static CompressingCodec randomInstance(Random random) {
+    final CompressionMode mode = RandomPicks.randomFrom(random, CompressionMode.values());
+    final int chunkSize = RandomInts.randomIntBetween(random, 1, 500);
+    final CompressingStoredFieldsIndex index = RandomPicks.randomFrom(random, CompressingStoredFieldsIndex.values());
+    return new CompressingCodec(mode, chunkSize, index);
+  }
+
+  private final CompressingStoredFieldsFormat storedFieldsFormat;
+
+  /**
+   * @see CompressingStoredFieldsFormat#CompressingStoredFieldsFormat(CompressionMode, int, CompressingStoredFieldsIndex)
+   */
+  public CompressingCodec(CompressionMode compressionMode, int chunkSize,
+      CompressingStoredFieldsIndex storedFieldsIndexFormat) {
+    super("Compressing", new Lucene40Codec());
+    this.storedFieldsFormat = new CompressingStoredFieldsFormat(compressionMode, chunkSize, storedFieldsIndexFormat);
+  }
+
+  public CompressingCodec() {
+    this(CompressionMode.FAST, 1 << 14, CompressingStoredFieldsIndex.MEMORY_CHUNK);
+  }
+
+  @Override
+  public StoredFieldsFormat storedFieldsFormat() {
+    return storedFieldsFormat;
+  }
+}

Added: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/package.html?rev=1394578&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/package.html (added)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/package.html Fri Oct  5 15:14:35 2012
@@ -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.compressing.CompressingStoredFieldsFormat}.
+</body>
+</html>

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=1394578&r1=1394577&r2=1394578&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 Fri Oct  5 15:14:35 2012
@@ -31,6 +31,7 @@ import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.appending.AppendingCodec;
 import org.apache.lucene.codecs.asserting.AssertingCodec;
+import org.apache.lucene.codecs.compressing.CompressingCodec;
 import org.apache.lucene.codecs.lucene40.Lucene40Codec;
 import org.apache.lucene.codecs.mockrandom.MockRandomPostingsFormat;
 import org.apache.lucene.codecs.simpletext.SimpleTextCodec;
@@ -165,6 +166,8 @@ final class TestRuleSetupAndRestoreClass
       codec = new AppendingCodec();
     } else if ("Asserting".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 7 && !shouldAvoidCodec("Asserting"))) {
       codec = new AssertingCodec();
+    } else if ("Compressing".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 6 && !shouldAvoidCodec("Compressing"))) {
+      codec = CompressingCodec.randomInstance(random);
     } 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/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=1394578&r1=1394577&r2=1394578&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 Fri Oct  5 15:14:35 2012
@@ -14,3 +14,4 @@
 #  limitations under the License.
 
 org.apache.lucene.codecs.asserting.AssertingCodec
+org.apache.lucene.codecs.compressing.CompressingCodec