You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ha...@apache.org on 2013/08/23 10:29:18 UTC

svn commit: r1516742 - in /lucene/dev/branches/lucene3069/lucene/test-framework/src: java/org/apache/lucene/codecs/lucene40/ resources/META-INF/services/

Author: han
Date: Fri Aug 23 08:29:18 2013
New Revision: 1516742

URL: http://svn.apache.org/r1516742
Log:
LUCENE-3069: API refactoring on Lucene40RW

Added:
    lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Temp40PostingsReader.java
      - copied, changed from r1516681, lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java
    lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Temp40PostingsWriter.java
      - copied, changed from r1516681, lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsWriter.java
    lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Temp40RWPostingsFormat.java
      - copied, changed from r1516681, lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40RWPostingsFormat.java
Modified:
    lucene/dev/branches/lucene3069/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat

Copied: lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Temp40PostingsReader.java (from r1516681, lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Temp40PostingsReader.java?p2=lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Temp40PostingsReader.java&p1=lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java&r1=1516681&r2=1516742&rev=1516742&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java (original)
+++ lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Temp40PostingsReader.java Fri Aug 23 08:29:18 2013
@@ -22,7 +22,7 @@ import java.util.Arrays;
 
 import org.apache.lucene.codecs.BlockTermState;
 import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.codecs.PostingsReaderBase;
+import org.apache.lucene.codecs.TempPostingsReaderBase;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
@@ -32,6 +32,7 @@ import org.apache.lucene.index.IndexFile
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.TermState;
 import org.apache.lucene.store.ByteArrayDataInput;
+import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
@@ -44,14 +45,14 @@ import org.apache.lucene.util.IOUtils;
  * Concrete class that reads the 4.0 frq/prox
  * postings format. 
  *  
- *  @see Lucene40PostingsFormat
+ *  @see Temp40RWPostingsFormat
  *  @deprecated Only for reading old 4.0 segments */
 @Deprecated
-public class Lucene40PostingsReader extends PostingsReaderBase {
+public class Temp40PostingsReader extends TempPostingsReaderBase {
 
-  final static String TERMS_CODEC = "Lucene40PostingsWriterTerms";
-  final static String FRQ_CODEC = "Lucene40PostingsWriterFrq";
-  final static String PRX_CODEC = "Lucene40PostingsWriterPrx";
+  final static String TERMS_CODEC = "Temp40PostingsWriterTerms";
+  final static String FRQ_CODEC = "Temp40PostingsWriterFrq";
+  final static String PRX_CODEC = "Temp40PostingsWriterPrx";
 
   //private static boolean DEBUG = BlockTreeTermsWriter.DEBUG;
   
@@ -71,12 +72,12 @@ public class Lucene40PostingsReader exte
   // private String segment;
 
   /** Sole constructor. */
-  public Lucene40PostingsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo segmentInfo, IOContext ioContext, String segmentSuffix) throws IOException {
+  public Temp40PostingsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo segmentInfo, IOContext ioContext, String segmentSuffix) throws IOException {
     boolean success = false;
     IndexInput freqIn = null;
     IndexInput proxIn = null;
     try {
-      freqIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene40PostingsFormat.FREQ_EXTENSION),
+      freqIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Temp40RWPostingsFormat.FREQ_EXTENSION),
                            ioContext);
       CodecUtil.checkHeader(freqIn, FRQ_CODEC, VERSION_START, VERSION_CURRENT);
       // TODO: hasProx should (somehow!) become codec private,
@@ -88,7 +89,7 @@ public class Lucene40PostingsReader exte
       // then somewhere record that we actually wrote it so we
       // know whether to open on read:
       if (fieldInfos.hasProx()) {
-        proxIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene40PostingsFormat.PROX_EXTENSION),
+        proxIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Temp40RWPostingsFormat.PROX_EXTENSION),
                              ioContext);
         CodecUtil.checkHeader(proxIn, PRX_CODEC, VERSION_START, VERSION_CURRENT);
       } else {
@@ -121,11 +122,6 @@ public class Lucene40PostingsReader exte
     long proxOffset;
     long skipOffset;
 
-    // Only used by the "primary" TermState -- clones don't
-    // copy this (basically they are "transient"):
-    ByteArrayDataInput bytesReader;  // TODO: should this NOT be in the TermState...?
-    byte[] bytes;
-
     @Override
     public StandardTermState clone() {
       StandardTermState other = new StandardTermState();
@@ -140,11 +136,6 @@ public class Lucene40PostingsReader exte
       freqOffset = other.freqOffset;
       proxOffset = other.proxOffset;
       skipOffset = other.skipOffset;
-
-      // Do not copy bytes, bytesReader (else TermState is
-      // very heavy, ie drags around the entire block's
-      // byte[]).  On seek back, if next() is in fact used
-      // (rare!), they will be re-read from disk.
     }
 
     @Override
@@ -171,38 +162,18 @@ public class Lucene40PostingsReader exte
     }
   }
 
-  /* Reads but does not decode the byte[] blob holding
-     metadata for the current terms block */
   @Override
-  public void readTermsBlock(IndexInput termsIn, FieldInfo fieldInfo, BlockTermState _termState) throws IOException {
-    final StandardTermState termState = (StandardTermState) _termState;
-
-    final int len = termsIn.readVInt();
-
-    // if (DEBUG) System.out.println("  SPR.readTermsBlock bytes=" + len + " ts=" + _termState);
-    if (termState.bytes == null) {
-      termState.bytes = new byte[ArrayUtil.oversize(len, 1)];
-      termState.bytesReader = new ByteArrayDataInput();
-    } else if (termState.bytes.length < len) {
-      termState.bytes = new byte[ArrayUtil.oversize(len, 1)];
-    }
-
-    termsIn.readBytes(termState.bytes, 0, len);
-    termState.bytesReader.reset(termState.bytes, 0, len);
-  }
-
-  @Override
-  public void nextTerm(FieldInfo fieldInfo, BlockTermState _termState)
+  public void decodeTerm(long[] longs, DataInput in, FieldInfo fieldInfo, BlockTermState _termState, boolean absolute)
     throws IOException {
     final StandardTermState termState = (StandardTermState) _termState;
     // if (DEBUG) System.out.println("SPR: nextTerm seg=" + segment + " tbOrd=" + termState.termBlockOrd + " bytesReader.fp=" + termState.bytesReader.getPosition());
     final boolean isFirstTerm = termState.termBlockOrd == 0;
-
-    if (isFirstTerm) {
-      termState.freqOffset = termState.bytesReader.readVLong();
-    } else {
-      termState.freqOffset += termState.bytesReader.readVLong();
+    if (absolute) {
+      termState.freqOffset = 0;
+      termState.proxOffset = 0;
     }
+
+    termState.freqOffset += in.readVLong();
     /*
     if (DEBUG) {
       System.out.println("  dF=" + termState.docFreq);
@@ -212,7 +183,7 @@ public class Lucene40PostingsReader exte
     assert termState.freqOffset < freqIn.length();
 
     if (termState.docFreq >= skipMinimum) {
-      termState.skipOffset = termState.bytesReader.readVLong();
+      termState.skipOffset = in.readVLong();
       // if (DEBUG) System.out.println("  skipOffset=" + termState.skipOffset + " vs freqIn.length=" + freqIn.length());
       assert termState.freqOffset + termState.skipOffset < freqIn.length();
     } else {
@@ -220,11 +191,7 @@ public class Lucene40PostingsReader exte
     }
 
     if (fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0) {
-      if (isFirstTerm) {
-        termState.proxOffset = termState.bytesReader.readVLong();
-      } else {
-        termState.proxOffset += termState.bytesReader.readVLong();
-      }
+      termState.proxOffset += in.readVLong();
       // if (DEBUG) System.out.println("  proxFP=" + termState.proxOffset);
     }
   }

Copied: lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Temp40PostingsWriter.java (from r1516681, lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsWriter.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Temp40PostingsWriter.java?p2=lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Temp40PostingsWriter.java&p1=lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsWriter.java&r1=1516681&r2=1516742&rev=1516742&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsWriter.java (original)
+++ lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Temp40PostingsWriter.java Fri Aug 23 08:29:18 2013
@@ -24,8 +24,9 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.lucene.codecs.BlockTermState;
 import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.codecs.PostingsWriterBase;
+import org.apache.lucene.codecs.TempPostingsWriterBase;
 import org.apache.lucene.codecs.TermStats;
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.DocsEnum;
@@ -33,6 +34,7 @@ import org.apache.lucene.index.FieldInfo
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.BytesRef;
@@ -41,10 +43,10 @@ import org.apache.lucene.util.IOUtils;
 /**
  * Concrete class that writes the 4.0 frq/prx postings format.
  * 
- * @see Lucene40PostingsFormat
+ * @see Temp40RWPostingsFormat
  * @lucene.experimental 
  */
-public final class Lucene40PostingsWriter extends PostingsWriterBase {
+public final class Temp40PostingsWriter extends TempPostingsWriterBase {
 
   final IndexOutput freqOut;
   final IndexOutput proxOut;
@@ -67,7 +69,6 @@ public final class Lucene40PostingsWrite
    */
   final int maxSkipLevels = 10;
   final int totalNumDocs;
-  IndexOutput termsOut;
 
   IndexOptions indexOptions;
   boolean storePayloads;
@@ -81,27 +82,30 @@ public final class Lucene40PostingsWrite
   int lastPosition;
   int lastOffset;
 
+  final static StandardTermState emptyState = new StandardTermState();
+  StandardTermState lastState;
+
   // private String segment;
 
-  /** Creates a {@link Lucene40PostingsWriter}, with the
+  /** Creates a {@link Temp40PostingsWriter}, with the
    *  {@link #DEFAULT_SKIP_INTERVAL}. */
-  public Lucene40PostingsWriter(SegmentWriteState state) throws IOException {
+  public Temp40PostingsWriter(SegmentWriteState state) throws IOException {
     this(state, DEFAULT_SKIP_INTERVAL);
   }
   
-  /** Creates a {@link Lucene40PostingsWriter}, with the
+  /** Creates a {@link Temp40PostingsWriter}, with the
    *  specified {@code skipInterval}. */
-  public Lucene40PostingsWriter(SegmentWriteState state, int skipInterval) throws IOException {
+  public Temp40PostingsWriter(SegmentWriteState state, int skipInterval) throws IOException {
     super();
     this.skipInterval = skipInterval;
     this.skipMinimum = skipInterval; /* set to the same for now */
     // this.segment = state.segmentName;
-    String fileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Lucene40PostingsFormat.FREQ_EXTENSION);
+    String fileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Temp40RWPostingsFormat.FREQ_EXTENSION);
     freqOut = state.directory.createOutput(fileName, state.context);
     boolean success = false;
     IndexOutput proxOut = null;
     try {
-      CodecUtil.writeHeader(freqOut, Lucene40PostingsReader.FRQ_CODEC, Lucene40PostingsReader.VERSION_CURRENT);
+      CodecUtil.writeHeader(freqOut, Temp40PostingsReader.FRQ_CODEC, Temp40PostingsReader.VERSION_CURRENT);
       // TODO: this is a best effort, if one of these fields has no postings
       // then we make an empty prx file, same as if we are wrapped in 
       // per-field postingsformat. maybe... we shouldn't
@@ -109,9 +113,9 @@ public final class Lucene40PostingsWrite
       if (state.fieldInfos.hasProx()) {
         // At least one field does not omit TF, so create the
         // prox file
-        fileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Lucene40PostingsFormat.PROX_EXTENSION);
+        fileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Temp40RWPostingsFormat.PROX_EXTENSION);
         proxOut = state.directory.createOutput(fileName, state.context);
-        CodecUtil.writeHeader(proxOut, Lucene40PostingsReader.PRX_CODEC, Lucene40PostingsReader.VERSION_CURRENT);
+        CodecUtil.writeHeader(proxOut, Temp40PostingsReader.PRX_CODEC, Temp40PostingsReader.VERSION_CURRENT);
       } else {
         // Every field omits TF so we will write no prox file
         proxOut = null;
@@ -134,15 +138,20 @@ public final class Lucene40PostingsWrite
   }
 
   @Override
-  public void start(IndexOutput termsOut) throws IOException {
-    this.termsOut = termsOut;
-    CodecUtil.writeHeader(termsOut, Lucene40PostingsReader.TERMS_CODEC, Lucene40PostingsReader.VERSION_CURRENT);
+  public void init(IndexOutput termsOut) throws IOException {
+    CodecUtil.writeHeader(termsOut, Temp40PostingsReader.TERMS_CODEC, Temp40PostingsReader.VERSION_CURRENT);
     termsOut.writeInt(skipInterval);                // write skipInterval
     termsOut.writeInt(maxSkipLevels);               // write maxSkipLevels
     termsOut.writeInt(skipMinimum);                 // write skipMinimum
   }
 
   @Override
+  public BlockTermState newTermState() {
+    return new StandardTermState();
+  }
+
+
+  @Override
   public void startTerm() {
     freqStart = freqOut.getFilePointer();
     //if (DEBUG) System.out.println("SPW: startTerm freqOut.fp=" + freqStart);
@@ -159,7 +168,7 @@ public final class Lucene40PostingsWrite
   // Currently, this instance is re-used across fields, so
   // our parent calls setField whenever the field changes
   @Override
-  public void setField(FieldInfo fieldInfo) {
+  public int setField(FieldInfo fieldInfo) {
     //System.out.println("SPW: setField");
     /*
     if (BlockTreeTermsWriter.DEBUG && fieldInfo.name.equals("id")) {
@@ -173,8 +182,10 @@ public final class Lucene40PostingsWrite
     
     storeOffsets = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;        
     storePayloads = fieldInfo.hasPayloads();
+    lastState = emptyState;
     //System.out.println("  set init blockFreqStart=" + freqStart);
     //System.out.println("  set init blockProxStart=" + proxStart);
+    return 0;
   }
 
   int lastDocID;
@@ -265,94 +276,48 @@ public final class Lucene40PostingsWrite
   public void finishDoc() {
   }
 
-  private static class PendingTerm {
-    public final long freqStart;
-    public final long proxStart;
-    public final long skipOffset;
-
-    public PendingTerm(long freqStart, long proxStart, long skipOffset) {
-      this.freqStart = freqStart;
-      this.proxStart = proxStart;
-      this.skipOffset = skipOffset;
-    }
+  private static class StandardTermState extends BlockTermState {
+    public long freqStart;
+    public long proxStart;
+    public long skipOffset;
   }
 
-  private final List<PendingTerm> pendingTerms = new ArrayList<PendingTerm>();
-
   /** Called when we are done adding docs to this term */
   @Override
-  public void finishTerm(TermStats stats) throws IOException {
-
+  public void finishTerm(BlockTermState _state) throws IOException {
+    StandardTermState state = (StandardTermState)_state;
     // if (DEBUG) System.out.println("SPW: finishTerm seg=" + segment + " freqStart=" + freqStart);
-    assert stats.docFreq > 0;
+    assert state.docFreq > 0;
 
     // TODO: wasteful we are counting this (counting # docs
     // for this term) in two places?
-    assert stats.docFreq == df;
-
-    final long skipOffset;
+    assert state.docFreq == df;
+    state.freqStart = freqStart;
+    state.proxStart = proxStart;
     if (df >= skipMinimum) {
-      skipOffset = skipListWriter.writeSkip(freqOut)-freqStart;
+      state.skipOffset = skipListWriter.writeSkip(freqOut)-freqStart;
     } else {
-      skipOffset = -1;
+      state.skipOffset = -1;
     }
-
-    pendingTerms.add(new PendingTerm(freqStart, proxStart, skipOffset));
-
     lastDocID = 0;
     df = 0;
   }
 
-  private final RAMOutputStream bytesWriter = new RAMOutputStream();
-
   @Override
-  public void flushTermsBlock(int start, int count) throws IOException {
-    //if (DEBUG) System.out.println("SPW: flushTermsBlock start=" + start + " count=" + count + " left=" + (pendingTerms.size()-count) + " pendingTerms.size()=" + pendingTerms.size());
-
-    if (count == 0) {
-      termsOut.writeByte((byte) 0);
-      return;
-    }
-
-    assert start <= pendingTerms.size();
-    assert count <= start;
-
-    final int limit = pendingTerms.size() - start + count;
-    final PendingTerm firstTerm = pendingTerms.get(limit - count);
-    // First term in block is abs coded:
-    bytesWriter.writeVLong(firstTerm.freqStart);
-
-    if (firstTerm.skipOffset != -1) {
-      assert firstTerm.skipOffset > 0;
-      bytesWriter.writeVLong(firstTerm.skipOffset);
+  public void encodeTerm(long[] empty, DataOutput out, FieldInfo fieldInfo, BlockTermState _state, boolean absolute) throws IOException {
+    StandardTermState state = (StandardTermState)_state;
+    if (absolute) {
+      lastState = emptyState;
+    }
+    out.writeVLong(state.freqStart - lastState.freqStart);
+    if (state.skipOffset != -1) {
+      assert state.skipOffset > 0;
+      out.writeVLong(state.skipOffset);
     }
     if (indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0) {
-      bytesWriter.writeVLong(firstTerm.proxStart);
+      out.writeVLong(state.proxStart - lastState.proxStart);
     }
-    long lastFreqStart = firstTerm.freqStart;
-    long lastProxStart = firstTerm.proxStart;
-    for(int idx=limit-count+1; idx<limit; idx++) {
-      final PendingTerm term = pendingTerms.get(idx);
-      //if (DEBUG) System.out.println("  write term freqStart=" + term.freqStart);
-      // The rest of the terms term are delta coded:
-      bytesWriter.writeVLong(term.freqStart - lastFreqStart);
-      lastFreqStart = term.freqStart;
-      if (term.skipOffset != -1) {
-        assert term.skipOffset > 0;
-        bytesWriter.writeVLong(term.skipOffset);
-      }
-      if (indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0) {
-        bytesWriter.writeVLong(term.proxStart - lastProxStart);
-        lastProxStart = term.proxStart;
-      }
-    }
-
-    termsOut.writeVInt((int) bytesWriter.getFilePointer());
-    bytesWriter.writeTo(termsOut);
-    bytesWriter.reset();
-
-    // Remove the terms we just wrote:
-    pendingTerms.subList(limit-count, limit).clear();
+    lastState = state;
   }
 
   @Override

Copied: lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Temp40RWPostingsFormat.java (from r1516681, lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40RWPostingsFormat.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Temp40RWPostingsFormat.java?p2=lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Temp40RWPostingsFormat.java&p1=lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40RWPostingsFormat.java&r1=1516681&r2=1516742&rev=1516742&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40RWPostingsFormat.java (original)
+++ lucene/dev/branches/lucene3069/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Temp40RWPostingsFormat.java Fri Aug 23 08:29:18 2013
@@ -2,10 +2,23 @@ package org.apache.lucene.codecs.lucene4
 
 import java.io.IOException;
 
+import org.apache.lucene.codecs.temp.*;
+import org.apache.lucene.codecs.BlockTreeTermsReader;
 import org.apache.lucene.codecs.BlockTreeTermsWriter;
+import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.FieldsConsumer;
-import org.apache.lucene.codecs.PostingsWriterBase;
+import org.apache.lucene.codecs.FieldsProducer;
+import org.apache.lucene.codecs.PostingsFormat;
+import org.apache.lucene.codecs.TempPostingsReaderBase;
+import org.apache.lucene.codecs.TempPostingsWriterBase;
+import org.apache.lucene.index.DocsEnum; // javadocs
+import org.apache.lucene.index.FieldInfo.IndexOptions; // javadocs
+import org.apache.lucene.index.FieldInfos; // javadocs
+import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.store.DataOutput; // javadocs
+import org.apache.lucene.util.fst.FST; // javadocs
+
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -27,10 +40,65 @@ import org.apache.lucene.index.SegmentWr
 /**
  * Read-write version of {@link Lucene40PostingsFormat} for testing.
  */
-public class Lucene40RWPostingsFormat extends Lucene40PostingsFormat {
+public class Temp40RWPostingsFormat extends PostingsFormat {
+  /** minimum items (terms or sub-blocks) per block for BlockTree */
+  protected final int minBlockSize;
+  /** maximum items (terms or sub-blocks) per block for BlockTree */
+  protected final int maxBlockSize;
+
+  /** Creates {@code Lucene40PostingsFormat} with default
+   *  settings. */
+  public Temp40RWPostingsFormat() {
+    this(BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
+  }
+
+  /** Creates {@code Lucene40PostingsFormat} with custom
+   *  values for {@code minBlockSize} and {@code
+   *  maxBlockSize} passed to block terms dictionary.
+   *  @see BlockTreeTermsWriter#BlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int) */
+  private Temp40RWPostingsFormat(int minBlockSize, int maxBlockSize) {
+    super("Temp40RW");
+    this.minBlockSize = minBlockSize;
+    assert minBlockSize > 1;
+    this.maxBlockSize = maxBlockSize;
+  }
+
+  @Override
+  public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
+    TempPostingsReaderBase postings = new Temp40PostingsReader(state.directory, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
+
+    boolean success = false;
+    try {
+      FieldsProducer ret = new TempBlockTreeTermsReader(
+                                                    state.directory,
+                                                    state.fieldInfos,
+                                                    state.segmentInfo,
+                                                    postings,
+                                                    state.context,
+                                                    state.segmentSuffix);
+      success = true;
+      return ret;
+    } finally {
+      if (!success) {
+        postings.close();
+      }
+    }
+  }
+
+  /** Extension of freq postings file */
+  static final String FREQ_EXTENSION = "frq";
+
+  /** Extension of prox postings file */
+  static final String PROX_EXTENSION = "prx";
+
+  @Override
+  public String toString() {
+    return getName() + "(minBlockSize=" + minBlockSize + " maxBlockSize=" + maxBlockSize + ")";
+  }
+
   @Override
   public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
-    PostingsWriterBase docs = new Lucene40PostingsWriter(state);
+    TempPostingsWriterBase docs = new Temp40PostingsWriter(state);
 
     // TODO: should we make the terms index more easily
     // pluggable?  Ie so that this codec would record which
@@ -38,7 +106,7 @@ public class Lucene40RWPostingsFormat ex
     // Or... you must make a new Codec for this?
     boolean success = false;
     try {
-      FieldsConsumer ret = new BlockTreeTermsWriter(state, docs, minBlockSize, maxBlockSize);
+      FieldsConsumer ret = new TempBlockTreeTermsWriter(state, docs, minBlockSize, maxBlockSize);
       success = true;
       return ret;
     } finally {

Modified: lucene/dev/branches/lucene3069/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat?rev=1516742&r1=1516741&r2=1516742&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat (original)
+++ lucene/dev/branches/lucene3069/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat Fri Aug 23 08:29:18 2013
@@ -29,3 +29,4 @@ org.apache.lucene.codecs.temp.TempSepPos
 org.apache.lucene.codecs.temp.TempFixedIntBlockPostingsFormat
 org.apache.lucene.codecs.temp.TempVariableIntBlockPostingsFormat
 org.apache.lucene.codecs.temp.TempRandomPostingsFormat
+org.apache.lucene.codecs.lucene40.Temp40RWPostingsFormat