You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2010/08/08 16:25:46 UTC

svn commit: r983417 - in /lucene/dev/trunk/lucene: ./ src/java/org/apache/lucene/index/codecs/ src/java/org/apache/lucene/index/codecs/intblock/ src/java/org/apache/lucene/index/codecs/pulsing/ src/java/org/apache/lucene/index/codecs/sep/ src/java/org/...

Author: mikemccand
Date: Sun Aug  8 14:25:45 2010
New Revision: 983417

URL: http://svn.apache.org/viewvc?rev=983417&view=rev
Log:
LUCENE-2589: add support for variable sized int block codecs

Added:
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/intblock/VariableIntBlockIndexInput.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/intblock/VariableIntBlockIndexOutput.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java
Removed:
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockFactory.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockIndexInput.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockIndexOutput.java
Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/Codec.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexInput.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexOutput.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/CodecUtil.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestCodecs.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/codecs/intblock/TestIntBlockCodec.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/LuceneTestCase.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/LuceneTestCaseJ4.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=983417&r1=983416&r2=983417&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Sun Aug  8 14:25:45 2010
@@ -548,6 +548,10 @@ New features
 
 * LUCENE-2526: Don't throw NPE from MultiPhraseQuery.toString when
   it's empty.  (Ross Woolf via Mike McCandless)
+
+* LUCENE-2589: Add a VariableSizedIntIndexInput, which, when used w/
+  Sep*, makes it simple to take any variable sized int block coders
+  (like Simple9/16) and use them in a codec.  (Mike McCandless)
   
 Optimizations
 

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/Codec.java?rev=983417&r1=983416&r2=983417&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/Codec.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/Codec.java Sun Aug  8 14:25:45 2010
@@ -56,4 +56,9 @@ public abstract class Codec {
 
   /** Records all file extensions this codec uses */
   public abstract void getExtensions(Set<String> extensions);
+
+  @Override
+  public String toString() {
+    return name;
+  }
 }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java?rev=983417&r1=983416&r2=983417&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java Sun Aug  8 14:25:45 2010
@@ -115,7 +115,7 @@ class DefaultCodecProvider extends Codec
   DefaultCodecProvider() {
     register(new StandardCodec());
     register(new PreFlexCodec());
-    register(new PulsingCodec());
+    register(new PulsingCodec(1));
   }
 
   @Override

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexInput.java?rev=983417&r1=983416&r2=983417&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexInput.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexInput.java Sun Aug  8 14:25:45 2010
@@ -37,10 +37,10 @@ import org.apache.lucene.util.IntsRef;
  */
 public abstract class FixedIntBlockIndexInput extends IntIndexInput {
 
-  private IndexInput in;
-  protected int blockSize;
-
-  protected void init(final IndexInput in) throws IOException {
+  private final IndexInput in;
+  protected final int blockSize;
+  
+  public FixedIntBlockIndexInput(final IndexInput in) throws IOException {
     this.in = in;
     blockSize = in.readVInt();
   }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexOutput.java?rev=983417&r1=983416&r2=983417&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexOutput.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexOutput.java Sun Aug  8 14:25:45 2010
@@ -36,19 +36,19 @@ import org.apache.lucene.store.IndexOutp
  */
 public abstract class FixedIntBlockIndexOutput extends IntIndexOutput {
 
-  private IndexOutput out;
-  private int blockSize;
-  private int[] pending;
+  protected final IndexOutput out;
+  private final int blockSize;
+  protected final int[] buffer;
   private int upto;
 
-  protected void init(IndexOutput out, int fixedBlockSize) throws IOException {
+  protected FixedIntBlockIndexOutput(IndexOutput out, int fixedBlockSize) throws IOException {
     blockSize = fixedBlockSize;
-    out.writeVInt(blockSize);
     this.out = out;
-    pending = new int[blockSize];
+    out.writeVInt(blockSize);
+    buffer = new int[blockSize];
   }
 
-  protected abstract void flushBlock(int[] buffer, IndexOutput out) throws IOException;
+  protected abstract void flushBlock() throws IOException;
 
   @Override
   public Index index() throws IOException {
@@ -96,9 +96,9 @@ public abstract class FixedIntBlockIndex
 
   @Override
   public void write(int v) throws IOException {
-    pending[upto++] = v;
+    buffer[upto++] = v;
     if (upto == blockSize) {
-      flushBlock(pending, out);
+      flushBlock();
       upto = 0;
     }
   }
@@ -107,9 +107,9 @@ public abstract class FixedIntBlockIndex
   public void close() throws IOException {
     try {
       if (upto > 0) {
-      // NOTE: entries in the block after current upto are
-      // invalid
-        flushBlock(pending, out);
+        // NOTE: entries in the block after current upto are
+        // invalid
+        flushBlock();
       }
     } finally {
       out.close();

Added: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/intblock/VariableIntBlockIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/intblock/VariableIntBlockIndexInput.java?rev=983417&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/intblock/VariableIntBlockIndexInput.java (added)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/intblock/VariableIntBlockIndexInput.java Sun Aug  8 14:25:45 2010
@@ -0,0 +1,216 @@
+package org.apache.lucene.index.codecs.intblock;
+
+/**
+ * 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.
+ */
+
+/** Naive int block API that writes vInts.  This is
+ *  expected to give poor performance; it's really only for
+ *  testing the pluggability.  One should typically use pfor instead. */
+
+import java.io.IOException;
+
+import org.apache.lucene.index.codecs.sep.IntIndexInput;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.IntsRef;
+
+// TODO: much of this can be shared code w/ the fixed case
+
+/** Abstract base class that reads variable-size blocks of ints
+ *  from an IndexInput.  While this is a simple approach, a
+ *  more performant approach would directly create an impl
+ *  of IntIndexInput inside Directory.  Wrapping a generic
+ *  IndexInput will likely cost performance.
+ *
+ * @lucene.experimental
+ */
+public abstract class VariableIntBlockIndexInput extends IntIndexInput {
+
+  protected final IndexInput in;
+  protected final int maxBlockSize;
+
+  protected VariableIntBlockIndexInput(final IndexInput in) throws IOException {
+    this.in = in;
+    maxBlockSize = in.readInt();
+  }
+
+  @Override
+  public Reader reader() throws IOException {
+    final int[] buffer = new int[maxBlockSize];
+    final IndexInput clone = (IndexInput) in.clone();
+    // TODO: can this be simplified?
+    return new Reader(clone, buffer, this.getBlockReader(clone, buffer));
+  }
+
+  @Override
+  public void close() throws IOException {
+    in.close();
+  }
+
+  @Override
+  public Index index() {
+    return new Index();
+  }
+
+  protected abstract BlockReader getBlockReader(IndexInput in, int[] buffer) throws IOException;
+
+  public interface BlockReader {
+    public int readBlock() throws IOException;
+    public void seek(long pos) throws IOException;
+  }
+
+  public static class Reader extends IntIndexInput.Reader {
+    private final IndexInput in;
+
+    public final int[] pending;
+    int upto;
+
+    private boolean seekPending;
+    private long pendingFP;
+    private int pendingUpto;
+    private long lastBlockFP;
+    private int blockSize;
+    private final BlockReader blockReader;
+    private final IntsRef bulkResult = new IntsRef();
+
+    public Reader(final IndexInput in, final int[] pending, final BlockReader blockReader)
+      throws IOException {
+      this.in = in;
+      this.pending = pending;
+      bulkResult.ints = pending;
+      this.blockReader = blockReader;
+    }
+
+    void seek(final long fp, final int upto) throws IOException {
+      // TODO: should we do this in real-time, not lazy?
+      pendingFP = fp;
+      pendingUpto = upto;
+      assert pendingUpto >= 0: "pendingUpto=" + pendingUpto;
+      seekPending = true;
+    }
+
+    private final void maybeSeek() throws IOException {
+      if (seekPending) {
+        if (pendingFP != lastBlockFP) {
+          // need new block
+          in.seek(pendingFP);
+          blockReader.seek(pendingFP);
+          lastBlockFP = pendingFP;
+          blockSize = blockReader.readBlock();
+        }
+        upto = pendingUpto;
+
+        // TODO: if we were more clever when writing the
+        // index, such that a seek point wouldn't be written
+        // until the int encoder "committed", we could avoid
+        // this (likely minor) inefficiency:
+
+        // This is necessary for int encoders that are
+        // non-causal, ie must see future int values to
+        // encode the current ones.
+        while(upto >= blockSize) {
+          upto -= blockSize;
+          lastBlockFP = in.getFilePointer();
+          blockSize = blockReader.readBlock();
+        }
+        seekPending = false;
+      }
+    }
+
+    @Override
+    public int next() throws IOException {
+      this.maybeSeek();
+      if (upto == blockSize) {
+        lastBlockFP = in.getFilePointer();
+        blockSize = blockReader.readBlock();
+        upto = 0;
+      }
+
+      return pending[upto++];
+    }
+
+    @Override
+    public IntsRef read(final int count) throws IOException {
+      this.maybeSeek();
+      if (upto == blockSize) {
+        lastBlockFP = in.getFilePointer();
+        blockSize = blockReader.readBlock();
+        upto = 0;
+      }
+      bulkResult.offset = upto;
+      if (upto + count < blockSize) {
+        bulkResult.length = count;
+        upto += count;
+      } else {
+        bulkResult.length = blockSize - upto;
+        upto = blockSize;
+      }
+
+      return bulkResult;
+    }
+  }
+
+  private class Index extends IntIndexInput.Index {
+    private long fp;
+    private int upto;
+
+    @Override
+    public void read(final IndexInput indexIn, final boolean absolute) throws IOException {
+      if (absolute) {
+        fp = indexIn.readVLong();
+        upto = indexIn.readByte()&0xFF;
+      } else {
+        final long delta = indexIn.readVLong();
+        if (delta == 0) {
+          // same block
+          upto = indexIn.readByte()&0xFF;
+        } else {
+          // new block
+          fp += delta;
+          upto = indexIn.readByte()&0xFF;
+        }
+      }
+      // TODO: we can't do this assert because non-causal
+      // int encoders can have upto over the buffer size
+      //assert upto < maxBlockSize: "upto=" + upto + " max=" + maxBlockSize;
+    }
+
+    @Override
+    public String toString() {
+      return "VarIntBlock.Index fp=" + fp + " upto=" + upto + " maxBlock=" + maxBlockSize;
+    }
+
+    @Override
+    public void seek(final IntIndexInput.Reader other) throws IOException {
+      ((Reader) other).seek(fp, upto);
+    }
+
+    @Override
+    public void set(final IntIndexInput.Index other) {
+      final Index idx = (Index) other;
+      fp = idx.fp;
+      upto = idx.upto;
+    }
+
+    @Override
+    public Object clone() {
+      Index other = new Index();
+      other.fp = fp;
+      other.upto = upto;
+      return other;
+    }
+  }
+}

Added: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/intblock/VariableIntBlockIndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/intblock/VariableIntBlockIndexOutput.java?rev=983417&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/intblock/VariableIntBlockIndexOutput.java (added)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/intblock/VariableIntBlockIndexOutput.java Sun Aug  8 14:25:45 2010
@@ -0,0 +1,128 @@
+package org.apache.lucene.index.codecs.intblock;
+
+/**
+ * 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.
+ */
+
+/** Naive int block API that writes vInts.  This is
+ *  expected to give poor performance; it's really only for
+ *  testing the pluggability.  One should typically use pfor instead. */
+
+import java.io.IOException;
+
+import org.apache.lucene.index.codecs.sep.IntIndexOutput;
+import org.apache.lucene.store.IndexOutput;
+
+// TODO: much of this can be shared code w/ the fixed case
+
+/** Abstract base class that writes variable-size blocks of ints
+ *  to an IndexOutput.  While this is a simple approach, a
+ *  more performant approach would directly create an impl
+ *  of IntIndexOutput inside Directory.  Wrapping a generic
+ *  IndexInput will likely cost performance.
+ *
+ * @lucene.experimental
+ */
+public abstract class VariableIntBlockIndexOutput extends IntIndexOutput {
+
+  protected final IndexOutput out;
+
+  private int upto;
+
+  private static final int MAX_BLOCK_SIZE = 1 << 8;
+
+  /** NOTE: maxBlockSize plus the max non-causal lookahead
+   *  of your codec must be less than 256.  EG Simple9
+   *  requires lookahead=1 because on seeing the Nth value
+   *  it knows it must now encode the N-1 values before it. */
+  protected VariableIntBlockIndexOutput(IndexOutput out, int maxBlockSize) throws IOException {
+    if (maxBlockSize > MAX_BLOCK_SIZE) {
+      throw new IllegalArgumentException("maxBlockSize must be <= " + MAX_BLOCK_SIZE + "; got " + maxBlockSize);
+    }
+    this.out = out;
+    out.writeInt(maxBlockSize);
+  }
+
+  /** Called one value at a time.  Return the number of
+   *  buffered input values that have been written to out. */
+  protected abstract int add(int value) throws IOException;
+
+  @Override
+  public Index index() throws IOException {
+    return new Index();
+  }
+
+  private class Index extends IntIndexOutput.Index {
+    long fp;
+    int upto;
+    long lastFP;
+    int lastUpto;
+
+    @Override
+    public void mark() throws IOException {
+      fp = out.getFilePointer();
+      upto = VariableIntBlockIndexOutput.this.upto;
+    }
+
+    @Override
+    public void set(IntIndexOutput.Index other) throws IOException {
+      Index idx = (Index) other;
+      lastFP = fp = idx.fp;
+      lastUpto = upto = idx.upto;
+    }
+
+    @Override
+    public void write(IndexOutput indexOut, boolean absolute) throws IOException {
+      assert upto >= 0;
+      if (absolute) {
+        indexOut.writeVLong(fp);
+        indexOut.writeByte((byte) upto);
+      } else if (fp == lastFP) {
+        // same block
+        indexOut.writeVLong(0);
+        assert upto >= lastUpto;
+        indexOut.writeByte((byte) upto);
+      } else {      
+        // new block
+        indexOut.writeVLong(fp - lastFP);
+        indexOut.writeByte((byte) upto);
+      }
+      lastUpto = upto;
+      lastFP = fp;
+    }
+  }
+
+  @Override
+  public void write(int v) throws IOException {
+    upto -= add(v)-1;
+    assert upto >= 0;
+  }
+
+  @Override
+  public void close() throws IOException {
+    try {
+      // stuff 0s in until the "real" data is flushed:
+      int stuffed = 0;
+      while(upto > stuffed) {
+        upto -= add(0)-1;
+        assert upto >= 0;
+        stuffed += 1;
+      }
+    } finally {
+      out.close();
+    }
+  }
+}

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java?rev=983417&r1=983416&r2=983417&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java Sun Aug  8 14:25:45 2010
@@ -50,8 +50,18 @@ import org.apache.lucene.util.BytesRef;
 
 public class PulsingCodec extends Codec {
 
-  public PulsingCodec() {
+  private final int freqCutoff;
+
+  /** Terms with freq <= freqCutoff are inlined into terms
+   *  dict. */
+  public PulsingCodec(int freqCutoff) {
     name = "Pulsing";
+    this.freqCutoff = freqCutoff;
+  }
+
+  @Override
+  public String toString() {
+    return name + "(freqCutoff=" + freqCutoff + ")";
   }
 
   @Override
@@ -62,7 +72,6 @@ public class PulsingCodec extends Codec 
 
     // Terms that have <= freqCutoff number of docs are
     // "pulsed" (inlined):
-    final int freqCutoff = 1;
     StandardPostingsWriter pulsingWriter = new PulsingPostingsWriterImpl(freqCutoff, docsWriter);
 
     // Terms dict index

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java?rev=983417&r1=983416&r2=983417&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java Sun Aug  8 14:25:45 2010
@@ -169,6 +169,11 @@ public class SepPostingsReaderImpl exten
       skipOffset = other.skipOffset;
       payloadOffset = other.payloadOffset;
     }
+
+    @Override
+    public String toString() {
+      return "tis.fp=" + filePointer + " docFreq=" + docFreq + " ord=" + ord + " docIndex=" + docIndex;
+    }
   }
 
   @Override
@@ -629,12 +634,10 @@ public class SepPostingsReaderImpl exten
       // positions
       while (pendingPosCount > freq) {
         final int code = posReader.next();
-        if (storePayloads) {
-          if ((code & 1) != 0) {
-            // Payload length has changed
-            payloadLength = posReader.next();
-            assert payloadLength >= 0;
-          }
+        if (storePayloads && (code & 1) != 0) {
+          // Payload length has changed
+          payloadLength = posReader.next();
+          assert payloadLength >= 0;
         }
         pendingPosCount--;
         payloadPending = true;

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/CodecUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/CodecUtil.java?rev=983417&r1=983416&r2=983417&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/CodecUtil.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/CodecUtil.java Sun Aug  8 14:25:45 2010
@@ -33,7 +33,7 @@ import java.io.IOException;
 public final class CodecUtil {
   private final static int CODEC_MAGIC = 0x3fd76c17;
 
-  public static void writeHeader(IndexOutput out, String codec, int version)
+  public static IndexOutput writeHeader(IndexOutput out, String codec, int version)
     throws IOException {
     final long start = out.getFilePointer();
     out.writeInt(CODEC_MAGIC);
@@ -44,6 +44,8 @@ public final class CodecUtil {
     if (out.getFilePointer()-start != codec.length()+9) {
       throw new IllegalArgumentException("codec must be simple ASCII, less than 128 characters in length [got " + codec + "]");
     }
+
+    return out;
   }
 
   public static int headerLength(String codec) {

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestCodecs.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestCodecs.java?rev=983417&r1=983416&r2=983417&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestCodecs.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestCodecs.java Sun Aug  8 14:25:45 2010
@@ -268,7 +268,7 @@ public class TestCodecs extends MultiCod
     final int NUM_TERMS = 100;
     final TermData[] terms = new TermData[NUM_TERMS];
     for(int i=0;i<NUM_TERMS;i++) {
-      final int[] docs = new int[] {1};
+      final int[] docs = new int[] {i};
       final String text = Integer.toString(i, Character.MAX_RADIX);
       terms[i] = new TermData(text, docs, null);
     }
@@ -288,10 +288,21 @@ public class TestCodecs extends MultiCod
     final FieldsEnum fieldsEnum = reader.iterator();
     assertNotNull(fieldsEnum.next());
     final TermsEnum termsEnum = fieldsEnum.terms();
+
+    DocsEnum docsEnum = null;
     for(int i=0;i<NUM_TERMS;i++) {
       final BytesRef term = termsEnum.next();
       assertNotNull(term);
       assertEquals(terms[i].text2, term.utf8ToString());
+
+      // do this twice to stress test the codec's reuse, ie,
+      // make sure it properly fully resets (rewinds) its
+      // internal state:
+      for(int iter=0;iter<2;iter++) {
+        docsEnum = termsEnum.docs(null,  docsEnum);
+        assertEquals(terms[i].docs[0], docsEnum.nextDoc());
+        assertEquals(DocsEnum.NO_MORE_DOCS, docsEnum.nextDoc());
+      }
     }
     assertNull(termsEnum.next());
 

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/codecs/intblock/TestIntBlockCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/codecs/intblock/TestIntBlockCodec.java?rev=983417&r1=983416&r2=983417&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/codecs/intblock/TestIntBlockCodec.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/codecs/intblock/TestIntBlockCodec.java Sun Aug  8 14:25:45 2010
@@ -27,13 +27,15 @@ public class TestIntBlockCodec extends L
   public void testSimpleIntBlocks() throws Exception {
     Directory dir = new MockRAMDirectory();
 
-    IntIndexOutput out = new MockFixedIntBlockIndexOutput(dir, "test", 128);
+    IntStreamFactory f = new MockFixedIntBlockCodec(128).getIntFactory();
+
+    IntIndexOutput out = f.createOutput(dir, "test");
     for(int i=0;i<11777;i++) {
       out.write(i);
     }
     out.close();
 
-    IntIndexInput in = new MockFixedIntBlockIndexInput(dir, "test", 128);
+    IntIndexInput in = f.openInput(dir, "test");
     IntIndexInput.Reader r = in.reader();
 
     for(int i=0;i<11777;i++) {
@@ -47,11 +49,13 @@ public class TestIntBlockCodec extends L
   public void testEmptySimpleIntBlocks() throws Exception {
     Directory dir = new MockRAMDirectory();
 
-    IntIndexOutput out = new MockFixedIntBlockIndexOutput(dir, "test", 128);
+    IntStreamFactory f = new MockFixedIntBlockCodec(128).getIntFactory();
+    IntIndexOutput out = f.createOutput(dir, "test");
+
     // write no ints
     out.close();
 
-    IntIndexInput in = new MockFixedIntBlockIndexInput(dir, "test", 128);
+    IntIndexInput in = f.openInput(dir, "test");
     in.reader();
     // read no ints
     in.close();

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java?rev=983417&r1=983416&r2=983417&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java Sun Aug  8 14:25:45 2010
@@ -26,8 +26,13 @@ import org.apache.lucene.index.SegmentRe
 import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.index.codecs.FieldsConsumer;
 import org.apache.lucene.index.codecs.FieldsProducer;
+import org.apache.lucene.index.codecs.sep.IntStreamFactory;
+import org.apache.lucene.index.codecs.sep.IntIndexInput;
+import org.apache.lucene.index.codecs.sep.IntIndexOutput;
 import org.apache.lucene.index.codecs.sep.SepPostingsReaderImpl;
 import org.apache.lucene.index.codecs.sep.SepPostingsWriterImpl;
+import org.apache.lucene.index.codecs.intblock.FixedIntBlockIndexInput;
+import org.apache.lucene.index.codecs.intblock.FixedIntBlockIndexOutput;
 import org.apache.lucene.index.codecs.standard.SimpleStandardTermsIndexReader;
 import org.apache.lucene.index.codecs.standard.SimpleStandardTermsIndexWriter;
 import org.apache.lucene.index.codecs.standard.StandardPostingsWriter;
@@ -37,24 +42,70 @@ import org.apache.lucene.index.codecs.st
 import org.apache.lucene.index.codecs.standard.StandardTermsIndexReader;
 import org.apache.lucene.index.codecs.standard.StandardTermsIndexWriter;
 import org.apache.lucene.index.codecs.standard.StandardCodec;
-import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.*;
 import org.apache.lucene.util.BytesRef;
 
 /**
- * A silly codec that simply writes each block as a series
- * of vInts.  Don't use this (performance will be poor)!
- * This is here just to test the core intblock codec
- * classes.
+ * A silly test codec to verify core support for fixed
+ * sized int block encoders is working.  The int encoder
+ * used here just writes each block as a series of vInt.
  */
+
 public class MockFixedIntBlockCodec extends Codec {
 
-  public MockFixedIntBlockCodec() {
+  private final int blockSize;
+
+  public MockFixedIntBlockCodec(int blockSize) {
+    this.blockSize = blockSize;
     name = "MockFixedIntBlock";
   }
 
   @Override
+  public String toString() {
+    return name + "(blockSize=" + blockSize + ")";
+  }
+
+  // only for testing
+  public IntStreamFactory getIntFactory() {
+    return new MockIntFactory();
+  }
+
+  private class MockIntFactory extends IntStreamFactory {
+
+    @Override
+    public IntIndexInput openInput(Directory dir, String fileName, int readBufferSize) throws IOException {
+      return new FixedIntBlockIndexInput(dir.openInput(fileName, readBufferSize)) {
+
+        @Override
+        protected BlockReader getBlockReader(final IndexInput in, final int[] buffer) throws IOException {
+          return new BlockReader() {
+            public void seek(long pos) {}
+            public void readBlock() throws IOException {
+              for(int i=0;i<buffer.length;i++) {
+                buffer[i] = in.readVInt();
+              }
+            }
+          };
+        }
+      };
+    }
+
+    @Override
+    public IntIndexOutput createOutput(Directory dir, String fileName) throws IOException {
+      return new FixedIntBlockIndexOutput(dir.createOutput(fileName), blockSize) {
+        @Override
+        protected void flushBlock() throws IOException {
+          for(int i=0;i<buffer.length;i++) {
+            out.writeVInt(buffer[i]);
+          }
+        }
+      };
+    }
+  }
+
+  @Override
   public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
-    StandardPostingsWriter postingsWriter = new SepPostingsWriterImpl(state, new MockFixedIntBlockFactory(1024));
+    StandardPostingsWriter postingsWriter = new SepPostingsWriterImpl(state, new MockIntFactory());
 
     boolean success = false;
     StandardTermsIndexWriter indexWriter;
@@ -88,7 +139,7 @@ public class MockFixedIntBlockCodec exte
     StandardPostingsReader postingsReader = new SepPostingsReaderImpl(state.dir,
                                                                       state.segmentInfo,
                                                                       state.readBufferSize,
-                                                                      new MockFixedIntBlockFactory(1024));
+                                                                      new MockIntFactory());
 
     StandardTermsIndexReader indexReader;
     boolean success = false;

Added: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java?rev=983417&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java (added)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java Sun Aug  8 14:25:45 2010
@@ -0,0 +1,218 @@
+package org.apache.lucene.index.codecs.mockintblock;
+
+/**
+ * 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.Set;
+
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.FieldsConsumer;
+import org.apache.lucene.index.codecs.FieldsProducer;
+import org.apache.lucene.index.codecs.sep.IntStreamFactory;
+import org.apache.lucene.index.codecs.sep.IntIndexInput;
+import org.apache.lucene.index.codecs.sep.IntIndexOutput;
+import org.apache.lucene.index.codecs.sep.SepPostingsReaderImpl;
+import org.apache.lucene.index.codecs.sep.SepPostingsWriterImpl;
+import org.apache.lucene.index.codecs.intblock.VariableIntBlockIndexInput;
+import org.apache.lucene.index.codecs.intblock.VariableIntBlockIndexOutput;
+import org.apache.lucene.index.codecs.standard.SimpleStandardTermsIndexReader;
+import org.apache.lucene.index.codecs.standard.SimpleStandardTermsIndexWriter;
+import org.apache.lucene.index.codecs.standard.StandardPostingsWriter;
+import org.apache.lucene.index.codecs.standard.StandardPostingsReader;
+import org.apache.lucene.index.codecs.standard.StandardTermsDictReader;
+import org.apache.lucene.index.codecs.standard.StandardTermsDictWriter;
+import org.apache.lucene.index.codecs.standard.StandardTermsIndexReader;
+import org.apache.lucene.index.codecs.standard.StandardTermsIndexWriter;
+import org.apache.lucene.index.codecs.standard.StandardCodec;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * A silly test codec to verify core support for variable
+ * sized int block encoders is working.  The int encoder
+ * used here writes baseBlockSize ints at once, if the first
+ * int is <= 3, else 2*baseBlockSize.
+ */
+
+public class MockVariableIntBlockCodec extends Codec {
+  private final int baseBlockSize;
+
+  public MockVariableIntBlockCodec(int baseBlockSize) {
+    name = "MockVariableIntBlock";
+    this.baseBlockSize = baseBlockSize;
+  }
+
+  @Override
+  public String toString() {
+    return name + "(baseBlockSize="+ baseBlockSize + ")";
+  }
+
+  private class MockIntFactory extends IntStreamFactory {
+
+    @Override
+    public IntIndexInput openInput(Directory dir, String fileName, int readBufferSize) throws IOException {
+      final IndexInput in = dir.openInput(fileName, readBufferSize);
+      final int baseBlockSize = in.readInt();
+      return new VariableIntBlockIndexInput(in) {
+
+        @Override
+        protected BlockReader getBlockReader(final IndexInput in, final int[] buffer) throws IOException {
+          return new BlockReader() {
+            public void seek(long pos) {}
+            public int readBlock() throws IOException {
+              buffer[0] = in.readVInt();
+              final int count = buffer[0] <= 3 ? baseBlockSize-1 : 2*baseBlockSize-1;
+              assert buffer.length >= count: "buffer.length=" + buffer.length + " count=" + count;
+              for(int i=0;i<count;i++) {
+                buffer[i+1] = in.readVInt();
+              }
+              return 1+count;
+            }
+          };
+        }
+      };
+    }
+
+    @Override
+    public IntIndexOutput createOutput(Directory dir, String fileName) throws IOException {
+      final IndexOutput out = dir.createOutput(fileName);
+      out.writeInt(baseBlockSize);
+      return new VariableIntBlockIndexOutput(out, 2*baseBlockSize) {
+
+        int pendingCount;
+        final int[] buffer = new int[2+2*baseBlockSize];
+
+        @Override
+        protected int add(int value) throws IOException {
+          buffer[pendingCount++] = value;
+          // silly variable block length int encoder: if
+          // first value <= 3, we write N vints at once;
+          // else, 2*N
+          final int flushAt = buffer[0] <= 3 ? baseBlockSize : 2*baseBlockSize;
+
+          // intentionally be non-causal here:
+          if (pendingCount == flushAt+1) {
+            for(int i=0;i<flushAt;i++) {
+              out.writeVInt(buffer[i]);
+            }
+            buffer[0] = buffer[flushAt];
+            pendingCount = 1;
+            return flushAt;
+          } else {
+            return 0;
+          }
+        }
+      };
+    }
+  }
+
+  @Override
+  public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+    StandardPostingsWriter postingsWriter = new SepPostingsWriterImpl(state, new MockIntFactory());
+
+    boolean success = false;
+    StandardTermsIndexWriter indexWriter;
+    try {
+      indexWriter = new SimpleStandardTermsIndexWriter(state);
+      success = true;
+    } finally {
+      if (!success) {
+        postingsWriter.close();
+      }
+    }
+
+    success = false;
+    try {
+      FieldsConsumer ret = new StandardTermsDictWriter(indexWriter, state, postingsWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
+      success = true;
+      return ret;
+    } finally {
+      if (!success) {
+        try {
+          postingsWriter.close();
+        } finally {
+          indexWriter.close();
+        }
+      }
+    }
+  }
+
+  @Override
+  public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
+    StandardPostingsReader postingsReader = new SepPostingsReaderImpl(state.dir,
+                                                                      state.segmentInfo,
+                                                                      state.readBufferSize,
+                                                                      new MockIntFactory());
+
+    StandardTermsIndexReader indexReader;
+    boolean success = false;
+    try {
+      indexReader = new SimpleStandardTermsIndexReader(state.dir,
+                                                       state.fieldInfos,
+                                                       state.segmentInfo.name,
+                                                       state.termsIndexDivisor,
+                                                       BytesRef.getUTF8SortedAsUnicodeComparator());
+      success = true;
+    } finally {
+      if (!success) {
+        postingsReader.close();
+      }
+    }
+
+    success = false;
+    try {
+      FieldsProducer ret = new StandardTermsDictReader(indexReader,
+                                                       state.dir,
+                                                       state.fieldInfos,
+                                                       state.segmentInfo.name,
+                                                       postingsReader,
+                                                       state.readBufferSize,
+                                                       BytesRef.getUTF8SortedAsUnicodeComparator(),
+                                                       StandardCodec.TERMS_CACHE_SIZE);
+      success = true;
+      return ret;
+    } finally {
+      if (!success) {
+        try {
+          postingsReader.close();
+        } finally {
+          indexReader.close();
+        }
+      }
+    }
+  }
+
+  @Override
+  public void files(Directory dir, SegmentInfo segmentInfo, Set<String> files) {
+    SepPostingsReaderImpl.files(segmentInfo, files);
+    StandardTermsDictReader.files(dir, segmentInfo, files);
+    SimpleStandardTermsIndexReader.files(dir, segmentInfo, files);
+  }
+
+  @Override
+  public void getExtensions(Set<String> extensions) {
+    SepPostingsWriterImpl.getExtensions(extensions);
+    StandardTermsDictReader.getExtensions(extensions);
+    SimpleStandardTermsIndexReader.getIndexExtensions(extensions);
+  }
+}

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/LuceneTestCase.java?rev=983417&r1=983416&r2=983417&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/LuceneTestCase.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/LuceneTestCase.java Sun Aug  8 14:25:45 2010
@@ -32,6 +32,7 @@ import junit.framework.TestCase;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.index.ConcurrentMergeScheduler;
 import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.FieldCache;
 import org.apache.lucene.search.FieldCache.CacheEntry;
@@ -83,7 +84,7 @@ public abstract class LuceneTestCase ext
   
   private volatile Thread.UncaughtExceptionHandler savedUncaughtExceptionHandler = null;
   
-  private String codec;
+  private Codec codec;
 
   /** Used to track if setUp and tearDown are called correctly from subclasses */
   private boolean setup;
@@ -307,9 +308,7 @@ public abstract class LuceneTestCase ext
       seed = null;
       super.runBare();
     } catch (Throwable e) {
-      if (TEST_CODEC.equals("random")) {
-        System.out.println("NOTE: random codec of testcase '" + getName() + "' was: " + codec);
-      }
+      System.out.println("NOTE: random codec of testcase '" + getName() + "' was: " + codec);
       if (seed != null) {
         System.out.println("NOTE: random seed of testcase '" + getName() + "' was: " + seed);
       }

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/LuceneTestCaseJ4.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/LuceneTestCaseJ4.java?rev=983417&r1=983416&r2=983417&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/LuceneTestCaseJ4.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/LuceneTestCaseJ4.java Sun Aug  8 14:25:45 2010
@@ -31,8 +31,10 @@ import org.apache.lucene.index.codecs.Co
 import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.index.codecs.preflexrw.PreFlexRWCodec;
 import org.apache.lucene.index.codecs.preflex.PreFlexCodec;
+import org.apache.lucene.index.codecs.pulsing.PulsingCodec;
 import org.apache.lucene.index.codecs.mocksep.MockSepCodec;
 import org.apache.lucene.index.codecs.mockintblock.MockFixedIntBlockCodec;
+import org.apache.lucene.index.codecs.mockintblock.MockVariableIntBlockCodec;
 
 import org.junit.After;
 import org.junit.AfterClass;
@@ -154,9 +156,9 @@ public class LuceneTestCaseJ4 {
   
   // saves default codec: we do this statically as many build indexes in @beforeClass
   private static String savedDefaultCodec;
-  private static String codec;
+  private static Codec codec;
   
-  private static final String[] TEST_CODECS = new String[] {"MockSep", "MockFixedIntBlock"};
+  private static final String[] TEST_CODECS = new String[] {"MockSep", "MockFixedIntBlock", "MockVariableIntBlock"};
 
   private static void swapCodec(Codec c) {
     final CodecProvider cp = CodecProvider.getDefault();
@@ -172,7 +174,7 @@ public class LuceneTestCaseJ4 {
   }
 
   // returns current default codec
-  static String installTestCodecs() {
+  static Codec installTestCodecs() {
     final CodecProvider cp = CodecProvider.getDefault();
 
     savedDefaultCodec = CodecProvider.getDefaultCodec();
@@ -190,15 +192,18 @@ public class LuceneTestCaseJ4 {
     }
 
     swapCodec(new MockSepCodec());
-    swapCodec(new MockFixedIntBlockCodec());
+    swapCodec(new PulsingCodec(_TestUtil.nextInt(seedRnd, 1, 20)));
+    swapCodec(new MockFixedIntBlockCodec(_TestUtil.nextInt(seedRnd, 1, 2000)));
+    // baseBlockSize cannot be over 127:
+    swapCodec(new MockVariableIntBlockCodec(_TestUtil.nextInt(seedRnd, 1, 127)));
 
-    return codec;
+    return cp.lookup(codec);
   }
 
   // returns current PreFlex codec
-  static void removeTestCodecs(String codec) {
+  static void removeTestCodecs(Codec codec) {
     final CodecProvider cp = CodecProvider.getDefault();
-    if (codec.equals("PreFlex")) {
+    if (codec.name.equals("PreFlex")) {
       final Codec preFlex = cp.lookup("PreFlex");
       if (preFlex != null) {
         cp.unregister(preFlex);
@@ -207,6 +212,8 @@ public class LuceneTestCaseJ4 {
     }
     cp.unregister(cp.lookup("MockSep"));
     cp.unregister(cp.lookup("MockFixedIntBlock"));
+    cp.unregister(cp.lookup("MockVariableIntBlock"));
+    swapCodec(new PulsingCodec(1));
     CodecProvider.setDefaultCodec(savedDefaultCodec);
   }
 
@@ -530,9 +537,7 @@ public class LuceneTestCaseJ4 {
       System.out.println("NOTE: random static seed of testclass '" + getName() + "' was: " + staticSeed);
     }
     
-    if (TEST_CODEC.equals("random")) {
-      System.out.println("NOTE: random codec of testcase '" + getName() + "' was: " + codec);
-    }
+    System.out.println("NOTE: random codec of testcase '" + getName() + "' was: " + codec);
 
     if (seed != null) {
       System.out.println("NOTE: random seed of testcase '" + getName() + "' was: " + seed);