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/19 17:26:42 UTC

svn commit: r1515469 - in /lucene/dev/branches/lucene3069/lucene: codecs/src/java/org/apache/lucene/codecs/temp/ codecs/src/resources/META-INF/services/ core/src/java/org/apache/lucene/codecs/ core/src/java/org/apache/lucene/codecs/temp/ core/src/java/...

Author: han
Date: Mon Aug 19 15:26:42 2013
New Revision: 1515469

URL: http://svn.apache.org/r1515469
Log:
LUCENE-3069: API refactoring on Pulsing PF

Added:
    lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/temp/TempFSTOrdPulsing41PostingsFormat.java
    lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/temp/TempFSTPulsing41PostingsFormat.java
    lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/temp/TempPulsing41PostingsFormat.java
      - copied, changed from r1515266, lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/Pulsing41PostingsFormat.java
    lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/temp/TempPulsingPostingsFormat.java
      - copied, changed from r1515266, lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsFormat.java
    lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/temp/TempPulsingPostingsReader.java
      - copied, changed from r1515266, lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsReader.java
    lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/temp/TempPulsingPostingsWriter.java
      - copied, changed from r1515266, lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsWriter.java
    lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/TempPostingsBaseFormat.java
      - copied, changed from r1515266, lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/PostingsBaseFormat.java
    lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/temp/TempPostingsBaseFormat.java
      - copied, changed from r1515266, lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsBaseFormat.java
Modified:
    lucene/dev/branches/lucene3069/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat
    lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/temp/TempFSTOrdTermsReader.java
    lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/temp/TempFSTOrdTermsWriter.java
    lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/temp/TempFSTTermsReader.java
    lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/temp/TempPostingsWriter.java
    lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java

Added: lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/temp/TempFSTOrdPulsing41PostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/temp/TempFSTOrdPulsing41PostingsFormat.java?rev=1515469&view=auto
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/temp/TempFSTOrdPulsing41PostingsFormat.java (added)
+++ lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/temp/TempFSTOrdPulsing41PostingsFormat.java Mon Aug 19 15:26:42 2013
@@ -0,0 +1,78 @@
+package org.apache.lucene.codecs.temp;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.FieldsConsumer;
+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.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.util.IOUtils;
+
+/** TempFSTOrd + Pulsing41
+ *  @lucene.experimental */
+
+public class TempFSTOrdPulsing41PostingsFormat extends PostingsFormat {
+  private final TempPostingsBaseFormat wrappedPostingsBaseFormat;
+  
+  public TempFSTOrdPulsing41PostingsFormat() {
+    super("TempFSTOrdPulsing41");
+    this.wrappedPostingsBaseFormat = new TempPostingsBaseFormat();
+  }
+
+  @Override
+  public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+    TempPostingsWriterBase docsWriter = null;
+    TempPostingsWriterBase pulsingWriter = null;
+
+    boolean success = false;
+    try {
+      docsWriter = wrappedPostingsBaseFormat.postingsWriterBase(state);
+      pulsingWriter = new TempPulsingPostingsWriter(state, 1, docsWriter);
+      FieldsConsumer ret = new TempFSTOrdTermsWriter(state, pulsingWriter);
+      success = true;
+      return ret;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(docsWriter, pulsingWriter);
+      }
+    }
+  }
+
+  @Override
+  public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
+    TempPostingsReaderBase docsReader = null;
+    TempPostingsReaderBase pulsingReader = null;
+    boolean success = false;
+    try {
+      docsReader = wrappedPostingsBaseFormat.postingsReaderBase(state);
+      pulsingReader = new TempPulsingPostingsReader(state, docsReader);
+      FieldsProducer ret = new TempFSTOrdTermsReader(state, pulsingReader);
+      success = true;
+      return ret;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(docsReader, pulsingReader);
+      }
+    }
+  }
+}

Added: lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/temp/TempFSTPulsing41PostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/temp/TempFSTPulsing41PostingsFormat.java?rev=1515469&view=auto
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/temp/TempFSTPulsing41PostingsFormat.java (added)
+++ lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/temp/TempFSTPulsing41PostingsFormat.java Mon Aug 19 15:26:42 2013
@@ -0,0 +1,79 @@
+package org.apache.lucene.codecs.temp;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.FieldsConsumer;
+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.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.util.IOUtils;
+
+/** TempFST + Pulsing41, test only, since
+ *  FST does no delta encoding here!
+ *  @lucene.experimental */
+
+public class TempFSTPulsing41PostingsFormat extends PostingsFormat {
+  private final TempPostingsBaseFormat wrappedPostingsBaseFormat;
+  
+  public TempFSTPulsing41PostingsFormat() {
+    super("TempFSTPulsing41");
+    this.wrappedPostingsBaseFormat = new TempPostingsBaseFormat();
+  }
+
+  @Override
+  public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+    TempPostingsWriterBase docsWriter = null;
+    TempPostingsWriterBase pulsingWriter = null;
+
+    boolean success = false;
+    try {
+      docsWriter = wrappedPostingsBaseFormat.postingsWriterBase(state);
+      pulsingWriter = new TempPulsingPostingsWriter(state, 1, docsWriter);
+      FieldsConsumer ret = new TempFSTTermsWriter(state, pulsingWriter);
+      success = true;
+      return ret;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(docsWriter, pulsingWriter);
+      }
+    }
+  }
+
+  @Override
+  public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
+    TempPostingsReaderBase docsReader = null;
+    TempPostingsReaderBase pulsingReader = null;
+    boolean success = false;
+    try {
+      docsReader = wrappedPostingsBaseFormat.postingsReaderBase(state);
+      pulsingReader = new TempPulsingPostingsReader(state, docsReader);
+      FieldsProducer ret = new TempFSTTermsReader(state, pulsingReader);
+      success = true;
+      return ret;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(docsReader, pulsingReader);
+      }
+    }
+  }
+}

Copied: lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/temp/TempPulsing41PostingsFormat.java (from r1515266, lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/Pulsing41PostingsFormat.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/temp/TempPulsing41PostingsFormat.java?p2=lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/temp/TempPulsing41PostingsFormat.java&p1=lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/Pulsing41PostingsFormat.java&r1=1515266&r2=1515469&rev=1515469&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/Pulsing41PostingsFormat.java (original)
+++ lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/temp/TempPulsing41PostingsFormat.java Mon Aug 19 15:26:42 2013
@@ -1,4 +1,4 @@
-package org.apache.lucene.codecs.pulsing;
+package org.apache.lucene.codecs.temp;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -20,26 +20,27 @@ package org.apache.lucene.codecs.pulsing
 import org.apache.lucene.codecs.BlockTreeTermsWriter;
 import org.apache.lucene.codecs.lucene41.Lucene41PostingsBaseFormat;
 import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat; // javadocs
+import org.apache.lucene.codecs.temp.TempPostingsBaseFormat;
 
 /**
  * Concrete pulsing implementation over {@link Lucene41PostingsFormat}.
  * 
  * @lucene.experimental
  */
-public class Pulsing41PostingsFormat extends PulsingPostingsFormat {
+public class TempPulsing41PostingsFormat extends TempPulsingPostingsFormat {
 
   /** Inlines docFreq=1 terms, otherwise uses the normal "Lucene41" format. */
-  public Pulsing41PostingsFormat() {
+  public TempPulsing41PostingsFormat() {
     this(1);
   }
 
   /** Inlines docFreq=<code>freqCutoff</code> terms, otherwise uses the normal "Lucene41" format. */
-  public Pulsing41PostingsFormat(int freqCutoff) {
+  public TempPulsing41PostingsFormat(int freqCutoff) {
     this(freqCutoff, BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
   }
 
   /** Inlines docFreq=<code>freqCutoff</code> terms, otherwise uses the normal "Lucene41" format. */
-  public Pulsing41PostingsFormat(int freqCutoff, int minBlockSize, int maxBlockSize) {
-    super("Pulsing41", new Lucene41PostingsBaseFormat(), freqCutoff, minBlockSize, maxBlockSize);
+  public TempPulsing41PostingsFormat(int freqCutoff, int minBlockSize, int maxBlockSize) {
+    super("TempPulsing41", new TempPostingsBaseFormat(), freqCutoff, minBlockSize, maxBlockSize);
   }
 }

Copied: lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/temp/TempPulsingPostingsFormat.java (from r1515266, lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsFormat.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/temp/TempPulsingPostingsFormat.java?p2=lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/temp/TempPulsingPostingsFormat.java&p1=lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsFormat.java&r1=1515266&r2=1515469&rev=1515469&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsFormat.java (original)
+++ lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/temp/TempPulsingPostingsFormat.java Mon Aug 19 15:26:42 2013
@@ -1,4 +1,4 @@
-package org.apache.lucene.codecs.pulsing;
+package org.apache.lucene.codecs.temp;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -19,14 +19,14 @@ package org.apache.lucene.codecs.pulsing
 
 import java.io.IOException;
 
-import org.apache.lucene.codecs.BlockTreeTermsReader;
-import org.apache.lucene.codecs.BlockTreeTermsWriter;
+import org.apache.lucene.codecs.temp.TempBlockTreeTermsReader;
+import org.apache.lucene.codecs.temp.TempBlockTreeTermsWriter;
 import org.apache.lucene.codecs.FieldsConsumer;
 import org.apache.lucene.codecs.FieldsProducer;
-import org.apache.lucene.codecs.PostingsBaseFormat;
+import org.apache.lucene.codecs.TempPostingsBaseFormat;
 import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.PostingsReaderBase;
-import org.apache.lucene.codecs.PostingsWriterBase;
+import org.apache.lucene.codecs.TempPostingsReaderBase;
+import org.apache.lucene.codecs.TempPostingsWriterBase;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.util.IOUtils;
@@ -37,20 +37,20 @@ import org.apache.lucene.util.IOUtils;
  *
  *  @lucene.experimental */
 
-public abstract class PulsingPostingsFormat extends PostingsFormat {
+public abstract class TempPulsingPostingsFormat extends PostingsFormat {
 
   private final int freqCutoff;
   private final int minBlockSize;
   private final int maxBlockSize;
-  private final PostingsBaseFormat wrappedPostingsBaseFormat;
+  private final TempPostingsBaseFormat wrappedPostingsBaseFormat;
   
-  public PulsingPostingsFormat(String name, PostingsBaseFormat wrappedPostingsBaseFormat, int freqCutoff) {
-    this(name, wrappedPostingsBaseFormat, freqCutoff, BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
+  public TempPulsingPostingsFormat(String name, TempPostingsBaseFormat wrappedPostingsBaseFormat, int freqCutoff) {
+    this(name, wrappedPostingsBaseFormat, freqCutoff, TempBlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, TempBlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
   }
 
   /** Terms with freq <= freqCutoff are inlined into terms
    *  dict. */
-  public PulsingPostingsFormat(String name, PostingsBaseFormat wrappedPostingsBaseFormat, int freqCutoff, int minBlockSize, int maxBlockSize) {
+  public TempPulsingPostingsFormat(String name, TempPostingsBaseFormat wrappedPostingsBaseFormat, int freqCutoff, int minBlockSize, int maxBlockSize) {
     super(name);
     this.freqCutoff = freqCutoff;
     this.minBlockSize = minBlockSize;
@@ -66,11 +66,11 @@ public abstract class PulsingPostingsFor
 
   @Override
   public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
-    PostingsWriterBase docsWriter = null;
+    TempPostingsWriterBase docsWriter = null;
 
     // Terms that have <= freqCutoff number of docs are
     // "pulsed" (inlined):
-    PostingsWriterBase pulsingWriter = null;
+    TempPostingsWriterBase pulsingWriter = null;
 
     // Terms dict
     boolean success = false;
@@ -79,8 +79,8 @@ public abstract class PulsingPostingsFor
 
       // Terms that have <= freqCutoff number of docs are
       // "pulsed" (inlined):
-      pulsingWriter = new PulsingPostingsWriter(freqCutoff, docsWriter);
-      FieldsConsumer ret = new BlockTreeTermsWriter(state, pulsingWriter, minBlockSize, maxBlockSize);
+      pulsingWriter = new TempPulsingPostingsWriter(state, freqCutoff, docsWriter);
+      FieldsConsumer ret = new TempBlockTreeTermsWriter(state, pulsingWriter, minBlockSize, maxBlockSize);
       success = true;
       return ret;
     } finally {
@@ -92,14 +92,14 @@ public abstract class PulsingPostingsFor
 
   @Override
   public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
-    PostingsReaderBase docsReader = null;
-    PostingsReaderBase pulsingReader = null;
+    TempPostingsReaderBase docsReader = null;
+    TempPostingsReaderBase pulsingReader = null;
 
     boolean success = false;
     try {
       docsReader = wrappedPostingsBaseFormat.postingsReaderBase(state);
-      pulsingReader = new PulsingPostingsReader(docsReader);
-      FieldsProducer ret = new BlockTreeTermsReader(
+      pulsingReader = new TempPulsingPostingsReader(state, docsReader);
+      FieldsProducer ret = new TempBlockTreeTermsReader(
                                                     state.directory, state.fieldInfos, state.segmentInfo,
                                                     pulsingReader,
                                                     state.context,

Copied: lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/temp/TempPulsingPostingsReader.java (from r1515266, lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsReader.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/temp/TempPulsingPostingsReader.java?p2=lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/temp/TempPulsingPostingsReader.java&p1=lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsReader.java&r1=1515266&r2=1515469&rev=1515469&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsReader.java (original)
+++ lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/temp/TempPulsingPostingsReader.java Mon Aug 19 15:26:42 2013
@@ -1,4 +1,4 @@
-package org.apache.lucene.codecs.pulsing;
+package org.apache.lucene.codecs.temp;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -20,16 +20,20 @@ package org.apache.lucene.codecs.pulsing
 import java.io.IOException;
 import java.util.IdentityHashMap;
 import java.util.Map;
+import java.util.TreeMap;
 
 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.IndexFileNames;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.TermState;
 import org.apache.lucene.store.ByteArrayDataInput;
+import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Attribute;
@@ -37,6 +41,7 @@ import org.apache.lucene.util.AttributeI
 import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
 
 /** Concrete class that reads the current doc/freq/skip
  *  postings format 
@@ -46,32 +51,58 @@ import org.apache.lucene.util.BytesRef;
 // create two separate docs readers, one that also reads
 // prox and one that doesn't?
 
-public class PulsingPostingsReader extends PostingsReaderBase {
+public class TempPulsingPostingsReader extends TempPostingsReaderBase {
 
   // Fallback reader for non-pulsed terms:
-  final PostingsReaderBase wrappedPostingsReader;
+  final TempPostingsReaderBase wrappedPostingsReader;
+  final SegmentReadState segmentState;
   int maxPositions;
+  int version;
+  TreeMap<Integer, Integer> fields;
 
-  public PulsingPostingsReader(PostingsReaderBase wrappedPostingsReader) {
+  public TempPulsingPostingsReader(SegmentReadState state, TempPostingsReaderBase wrappedPostingsReader) {
     this.wrappedPostingsReader = wrappedPostingsReader;
+    this.segmentState = state;
   }
 
   @Override
   public void init(IndexInput termsIn) throws IOException {
-    CodecUtil.checkHeader(termsIn, PulsingPostingsWriter.CODEC,
-      PulsingPostingsWriter.VERSION_START, PulsingPostingsWriter.VERSION_START);
+    version = CodecUtil.checkHeader(termsIn, TempPulsingPostingsWriter.CODEC,
+                                    TempPulsingPostingsWriter.VERSION_START, 
+                                    TempPulsingPostingsWriter.VERSION_CURRENT);
+    // nocommit: here open file to load field summary
     maxPositions = termsIn.readVInt();
     wrappedPostingsReader.init(termsIn);
+    if (version >= TempPulsingPostingsWriter.VERSION_META_ARRAY) {
+      fields = new TreeMap<Integer, Integer>();
+      String summaryFileName = IndexFileNames.segmentFileName(segmentState.segmentInfo.name, segmentState.segmentSuffix, TempPulsingPostingsWriter.SUMMARY_EXTENSION);
+      IndexInput in = null;
+      try { 
+        in = segmentState.directory.openInput(summaryFileName, segmentState.context);
+        CodecUtil.checkHeader(in, TempPulsingPostingsWriter.CODEC, version, 
+                              TempPulsingPostingsWriter.VERSION_CURRENT);
+        int numField = in.readVInt();
+        for (int i = 0; i < numField; i++) {
+          int fieldNum = in.readVInt();
+          int longsSize = in.readVInt();
+          fields.put(fieldNum, longsSize);
+        }
+      } finally {
+        IOUtils.closeWhileHandlingException(in);
+      }
+    } else {
+      assert false;
+      fields = null;
+    }
   }
 
   private static class PulsingTermState extends BlockTermState {
+    private boolean absolute = false;
+    private long[] longs;
     private byte[] postings;
     private int postingsSize;                     // -1 if this term was not inlined
     private BlockTermState wrappedTermState;
 
-    ByteArrayDataInput inlinedBytesReader;
-    private byte[] inlinedBytes;
-
     @Override
     public PulsingTermState clone() {
       PulsingTermState clone;
@@ -82,6 +113,11 @@ public class PulsingPostingsReader exten
       } else {
         assert wrappedTermState != null;
         clone.wrappedTermState = (BlockTermState) wrappedTermState.clone();
+        clone.absolute = absolute;
+        if (longs != null) {
+          clone.longs = new long[longs.length];
+          System.arraycopy(longs, 0, clone.longs, 0, longs.length);
+        }
       }
       return clone;
     }
@@ -99,11 +135,6 @@ public class PulsingPostingsReader exten
       } else {
         wrappedTermState.copyFrom(other.wrappedTermState);
       }
-
-      // NOTE: we do not copy the
-      // inlinedBytes/inlinedBytesReader; these are only
-      // stored on the "primary" TermState.  They are
-      // "transient" to cloned term states.
     }
 
     @Override
@@ -117,25 +148,6 @@ public class PulsingPostingsReader exten
   }
 
   @Override
-  public void readTermsBlock(IndexInput termsIn, FieldInfo fieldInfo, BlockTermState _termState) throws IOException {
-    //System.out.println("PR.readTermsBlock state=" + _termState);
-    final PulsingTermState termState = (PulsingTermState) _termState;
-    if (termState.inlinedBytes == null) {
-      termState.inlinedBytes = new byte[128];
-      termState.inlinedBytesReader = new ByteArrayDataInput();
-    }
-    int len = termsIn.readVInt();
-    //System.out.println("  len=" + len + " fp=" + termsIn.getFilePointer());
-    if (termState.inlinedBytes.length < len) {
-      termState.inlinedBytes = new byte[ArrayUtil.oversize(len, 1)];
-    }
-    termsIn.readBytes(termState.inlinedBytes, 0, len);
-    termState.inlinedBytesReader.reset(termState.inlinedBytes);
-    termState.wrappedTermState.termBlockOrd = 0;
-    wrappedPostingsReader.readTermsBlock(termsIn, fieldInfo, termState.wrappedTermState);
-  }
-
-  @Override
   public BlockTermState newTermState() throws IOException {
     PulsingTermState state = new PulsingTermState();
     state.wrappedTermState = wrappedPostingsReader.newTermState();
@@ -143,20 +155,25 @@ public class PulsingPostingsReader exten
   }
 
   @Override
-  public void nextTerm(FieldInfo fieldInfo, BlockTermState _termState) throws IOException {
+  public void decodeTerm(long[] empty, DataInput in, FieldInfo fieldInfo, BlockTermState _termState, boolean absolute) throws IOException {
     //System.out.println("PR nextTerm");
     PulsingTermState termState = (PulsingTermState) _termState;
-
+    assert empty.length == 0;
+    termState.absolute = termState.absolute || absolute;
     // if we have positions, its total TF, otherwise its computed based on docFreq.
     long count = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0 ? termState.totalTermFreq : termState.docFreq;
     //System.out.println("  count=" + count + " threshold=" + maxPositions);
 
+    // term dict have no chance to init this
+    // nocommit: nuke this?
+    if (termState.termBlockOrd == 0) {  
+      termState.wrappedTermState.termBlockOrd = 0;
+    }
     if (count <= maxPositions) {
-
       // Inlined into terms dict -- just read the byte[] blob in,
       // but don't decode it now (we only decode when a DocsEnum
       // or D&PEnum is pulled):
-      termState.postingsSize = termState.inlinedBytesReader.readVInt();
+      termState.postingsSize = in.readVInt();
       if (termState.postings == null || termState.postings.length < termState.postingsSize) {
         termState.postings = new byte[ArrayUtil.oversize(termState.postingsSize, 1)];
       }
@@ -164,16 +181,26 @@ public class PulsingPostingsReader exten
       // (the blob holding all inlined terms' blobs for
       // current term block) into another byte[] (just the
       // blob for this term)...
-      termState.inlinedBytesReader.readBytes(termState.postings, 0, termState.postingsSize);
+      in.readBytes(termState.postings, 0, termState.postingsSize);
       //System.out.println("  inlined bytes=" + termState.postingsSize);
+      termState.absolute = absolute ? true : termState.absolute;
     } else {
       //System.out.println("  not inlined");
+      final int longsSize = fields.get(fieldInfo.number);
+      if (termState.longs == null) {
+        termState.longs = new long[longsSize];
+      } else {
+        assert termState.longs.length == longsSize;
+      }
+      for (int i = 0; i < longsSize; i++) {
+        termState.longs[i] = in.readVLong();
+      }
       termState.postingsSize = -1;
-      // TODO: should we do full copyFrom?  much heavier...?
       termState.wrappedTermState.docFreq = termState.docFreq;
       termState.wrappedTermState.totalTermFreq = termState.totalTermFreq;
-      wrappedPostingsReader.nextTerm(fieldInfo, termState.wrappedTermState);
+      wrappedPostingsReader.decodeTerm(termState.longs, in, fieldInfo, termState.wrappedTermState, termState.absolute);
       termState.wrappedTermState.termBlockOrd++;
+      termState.absolute = false;
     }
   }
 
@@ -594,7 +621,7 @@ public class PulsingPostingsReader exten
    * 
    * @lucene.internal */
   public static interface PulsingEnumAttribute extends Attribute {
-    public Map<PulsingPostingsReader,DocsEnum> enums();
+    public Map<TempPulsingPostingsReader,DocsEnum> enums();
   }
     
   /** 
@@ -609,11 +636,11 @@ public class PulsingPostingsReader exten
     // you don't reuse? and maybe pulsingPostingsReader should throw an exc if it wraps
     // another pulsing, because this is just stupid and wasteful. 
     // we still have to be careful in case someone does Pulsing(Stomping(Pulsing(...
-    private final Map<PulsingPostingsReader,DocsEnum> enums = 
-      new IdentityHashMap<PulsingPostingsReader,DocsEnum>();
+    private final Map<TempPulsingPostingsReader,DocsEnum> enums = 
+      new IdentityHashMap<TempPulsingPostingsReader,DocsEnum>();
       
     @Override
-    public Map<PulsingPostingsReader,DocsEnum> enums() {
+    public Map<TempPulsingPostingsReader,DocsEnum> enums() {
       return enums;
     }
 

Copied: lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/temp/TempPulsingPostingsWriter.java (from r1515266, lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsWriter.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/temp/TempPulsingPostingsWriter.java?p2=lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/temp/TempPulsingPostingsWriter.java&p1=lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsWriter.java&r1=1515266&r2=1515469&rev=1515469&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsWriter.java (original)
+++ lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/temp/TempPulsingPostingsWriter.java Mon Aug 19 15:26:42 2013
@@ -1,4 +1,4 @@
-package org.apache.lucene.codecs.pulsing;
+package org.apache.lucene.codecs.temp;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -21,14 +21,19 @@ import java.io.IOException;
 import java.util.List;
 import java.util.ArrayList;
 
+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.FieldInfo;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
+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;
+import org.apache.lucene.util.IOUtils;
 
 // TODO: we now inline based on total TF of the term,
 // but it might be better to inline by "net bytes used"
@@ -45,30 +50,47 @@ import org.apache.lucene.util.BytesRef;
  * or passed through to the wrapped writer.
  *
  * @lucene.experimental */
-public final class PulsingPostingsWriter extends PostingsWriterBase {
+public final class TempPulsingPostingsWriter extends TempPostingsWriterBase {
 
-  final static String CODEC = "PulsedPostingsWriter";
+  final static String CODEC = "TempPulsedPostingsWriter";
+
+  // recording field summary
+  final static String SUMMARY_EXTENSION = "smy";
 
   // To add a new version, increment from the last one, and
   // change VERSION_CURRENT to point to your new version:
   final static int VERSION_START = 0;
 
-  final static int VERSION_CURRENT = VERSION_START;
+  final static int VERSION_META_ARRAY = 0;
+
+  final static int VERSION_CURRENT = VERSION_META_ARRAY;
 
+  private SegmentWriteState segmentState;
   private IndexOutput termsOut;
 
+  private List<FieldMetaData> fields;
+
   private IndexOptions indexOptions;
   private boolean storePayloads;
 
-  private static class PendingTerm {
-    private final byte[] bytes;
-    public PendingTerm(byte[] bytes) {
-      this.bytes = bytes;
+  // information for wrapped PF, in current field
+  private int longsSize;
+  private long[] longs;
+  boolean absolute;
+
+  private static class PulsingTermState extends BlockTermState {
+    private byte[] bytes;
+    private BlockTermState wrappedState;
+    @Override
+    public String toString() {
+      if (bytes != null) {
+        return "inlined";
+      } else {
+        return "not inlined wrapped=" + wrappedState;
+      }
     }
   }
 
-  private final List<PendingTerm> pendingTerms = new ArrayList<PendingTerm>();
-
   // one entry per position
   private final Position[] pending;
   private int pendingCount = 0;                           // -1 once we've hit too many positions
@@ -83,32 +105,51 @@ public final class PulsingPostingsWriter
     int endOffset;
   }
 
+  private static final class FieldMetaData {
+    int fieldNumber;
+    int longsSize;
+    FieldMetaData(int number, int size) {
+      fieldNumber = number;
+      longsSize = size;
+    }
+  }
+
   // TODO: -- lazy init this?  ie, if every single term
   // was inlined (eg for a "primary key" field) then we
   // never need to use this fallback?  Fallback writer for
   // non-inlined terms:
-  final PostingsWriterBase wrappedPostingsWriter;
+  final TempPostingsWriterBase wrappedPostingsWriter;
 
   /** If the total number of positions (summed across all docs
    *  for this term) is <= maxPositions, then the postings are
    *  inlined into terms dict */
-  public PulsingPostingsWriter(int maxPositions, PostingsWriterBase wrappedPostingsWriter) {
+  public TempPulsingPostingsWriter(SegmentWriteState state, int maxPositions, TempPostingsWriterBase wrappedPostingsWriter) {
+
     pending = new Position[maxPositions];
     for(int i=0;i<maxPositions;i++) {
       pending[i] = new Position();
     }
+    fields = new ArrayList<FieldMetaData>();
 
     // We simply wrap another postings writer, but only call
     // on it when tot positions is >= the cutoff:
     this.wrappedPostingsWriter = wrappedPostingsWriter;
+    this.segmentState = state;
   }
 
   @Override
-  public void start(IndexOutput termsOut) throws IOException {
+  public void init(IndexOutput termsOut) throws IOException {
     this.termsOut = termsOut;
     CodecUtil.writeHeader(termsOut, CODEC, VERSION_CURRENT);
     termsOut.writeVInt(pending.length); // encode maxPositions in header
-    wrappedPostingsWriter.start(termsOut);
+    wrappedPostingsWriter.init(termsOut);
+  }
+
+  @Override
+  public BlockTermState newTermState() throws IOException {
+    PulsingTermState state = new PulsingTermState();
+    state.wrappedState = wrappedPostingsWriter.newTermState();
+    return state;
   }
 
   @Override
@@ -123,11 +164,15 @@ public final class PulsingPostingsWriter
   // 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) {
     this.indexOptions = fieldInfo.getIndexOptions();
     //if (DEBUG) System.out.println("PW field=" + fieldInfo.name + " indexOptions=" + indexOptions);
     storePayloads = fieldInfo.hasPayloads();
-    wrappedPostingsWriter.setField(fieldInfo);
+    absolute = false;
+    longsSize = wrappedPostingsWriter.setField(fieldInfo);
+    longs = new long[longsSize];
+    fields.add(new FieldMetaData(fieldInfo.number, longsSize));
+    return 0;
     //DEBUG = BlockTreeTermsWriter.DEBUG;
   }
 
@@ -219,18 +264,19 @@ public final class PulsingPostingsWriter
 
   /** 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 {
+    PulsingTermState state = (PulsingTermState) _state;
+
     // if (DEBUG) System.out.println("PW   finishTerm docCount=" + stats.docFreq + " pendingCount=" + pendingCount + " pendingTerms.size()=" + pendingTerms.size());
 
     assert pendingCount > 0 || pendingCount == -1;
 
     if (pendingCount == -1) {
-      wrappedPostingsWriter.finishTerm(stats);
-      // Must add null entry to record terms that our
-      // wrapped postings impl added
-      pendingTerms.add(null);
+      state.wrappedState.docFreq = state.docFreq;
+      state.wrappedState.totalTermFreq = state.totalTermFreq;
+      state.bytes = null;
+      wrappedPostingsWriter.finishTerm(state.wrappedState);
     } else {
-
       // There were few enough total occurrences for this
       // term, so we fully inline our postings data into
       // terms dict, now:
@@ -325,61 +371,52 @@ public final class PulsingPostingsWriter
         }
       }
 
-      final byte[] bytes = new byte[(int) buffer.getFilePointer()];
-      buffer.writeTo(bytes, 0);
-      pendingTerms.add(new PendingTerm(bytes));
+      state.bytes = new byte[(int) buffer.getFilePointer()];
+      buffer.writeTo(state.bytes, 0);
       buffer.reset();
     }
-
     pendingCount = 0;
   }
 
   @Override
-  public void close() throws IOException {
-    wrappedPostingsWriter.close();
-  }
-
-  @Override
-  public void flushTermsBlock(int start, int count) throws IOException {
-    // if (DEBUG) System.out.println("PW: flushTermsBlock start=" + start + " count=" + count + " pendingTerms.size()=" + pendingTerms.size());
-    int wrappedCount = 0;
-    assert buffer.getFilePointer() == 0;
-    assert start >= count;
-
-    final int limit = pendingTerms.size() - start + count;
-
-    for(int idx=pendingTerms.size()-start; idx<limit; idx++) {
-      final PendingTerm term = pendingTerms.get(idx);
-      if (term == null) {
-        wrappedCount++;
-      } else {
-        buffer.writeVInt(term.bytes.length);
-        buffer.writeBytes(term.bytes, 0, term.bytes.length);
+  public void encodeTerm(long[] empty, DataOutput out, FieldInfo fieldInfo, BlockTermState _state, boolean absolute) throws IOException {
+    PulsingTermState state = (PulsingTermState)_state;
+    assert empty.length == 0;
+    this.absolute = this.absolute || absolute;
+    if (state.bytes == null) {
+      assert longsSize > 0;
+      wrappedPostingsWriter.encodeTerm(longs, buffer, fieldInfo, state.wrappedState, this.absolute);
+      for (int i = 0; i < longsSize; i++) {
+        out.writeVLong(longs[i]);
       }
+      buffer.writeTo(out);
+      buffer.reset();
+      this.absolute = false;
+    } else {
+      out.writeVInt(state.bytes.length);
+      out.writeBytes(state.bytes, 0, state.bytes.length);
+      this.absolute = absolute ? true : this.absolute;
     }
+  }
 
-    termsOut.writeVInt((int) buffer.getFilePointer());
-    buffer.writeTo(termsOut);
-    buffer.reset();
-
-    // TDOO: this could be somewhat costly since
-    // pendingTerms.size() could be biggish?
-    int futureWrappedCount = 0;
-    final int limit2 = pendingTerms.size();
-    for(int idx=limit;idx<limit2;idx++) {
-      if (pendingTerms.get(idx) == null) {
-        futureWrappedCount++;
+  @Override
+  public void close() throws IOException {
+    wrappedPostingsWriter.close();
+    assert (VERSION_CURRENT >= VERSION_META_ARRAY);
+    String summaryFileName = IndexFileNames.segmentFileName(segmentState.segmentInfo.name, segmentState.segmentSuffix, SUMMARY_EXTENSION);
+    IndexOutput out = null;
+    try {
+      out = segmentState.directory.createOutput(summaryFileName, segmentState.context);
+      CodecUtil.writeHeader(out, CODEC, VERSION_CURRENT);
+      out.writeVInt(fields.size());
+      for (FieldMetaData field : fields) {
+        out.writeVInt(field.fieldNumber);
+        out.writeVInt(field.longsSize);
       }
+      out.close();
+    } finally {
+      IOUtils.closeWhileHandlingException(out);
     }
-
-    // Remove the terms we just wrote:
-    pendingTerms.subList(pendingTerms.size()-start, limit).clear();
-
-    // if (DEBUG) System.out.println("PW:   len=" + buffer.getFilePointer() + " fp=" + termsOut.getFilePointer() + " futureWrappedCount=" + futureWrappedCount + " wrappedCount=" + wrappedCount);
-    // TODO: can we avoid calling this if all terms
-    // were inlined...?  Eg for a "primary key" field, the
-    // wrapped codec is never invoked...
-    wrappedPostingsWriter.flushTermsBlock(futureWrappedCount+wrappedCount, wrappedCount);
   }
 
   // Pushes pending positions to the wrapped codec

Modified: lucene/dev/branches/lucene3069/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat?rev=1515469&r1=1515468&r2=1515469&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat (original)
+++ lucene/dev/branches/lucene3069/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat Mon Aug 19 15:26:42 2013
@@ -19,3 +19,6 @@ org.apache.lucene.codecs.memory.MemoryPo
 org.apache.lucene.codecs.bloom.BloomFilteringPostingsFormat
 org.apache.lucene.codecs.memory.DirectPostingsFormat
 org.apache.lucene.codecs.temp.TempBlockPostingsFormat
+org.apache.lucene.codecs.temp.TempPulsing41PostingsFormat
+org.apache.lucene.codecs.temp.TempFSTPulsing41PostingsFormat
+org.apache.lucene.codecs.temp.TempFSTOrdPulsing41PostingsFormat

Copied: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/TempPostingsBaseFormat.java (from r1515266, lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/PostingsBaseFormat.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/TempPostingsBaseFormat.java?p2=lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/TempPostingsBaseFormat.java&p1=lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/PostingsBaseFormat.java&r1=1515266&r2=1515469&rev=1515469&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/PostingsBaseFormat.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/TempPostingsBaseFormat.java Mon Aug 19 15:26:42 2013
@@ -34,22 +34,22 @@ import org.apache.lucene.index.SegmentRe
 
 // can we clean this up and do this some other way? 
 // refactor some of these classes and use covariant return?
-public abstract class PostingsBaseFormat {
+public abstract class TempPostingsBaseFormat {
 
   /** Unique name that's used to retrieve this codec when
    *  reading the index */
   public final String name;
   
   /** Sole constructor. */
-  protected PostingsBaseFormat(String name) {
+  protected TempPostingsBaseFormat(String name) {
     this.name = name;
   }
 
   /** Creates the {@link PostingsReaderBase} for this
    *  format. */
-  public abstract PostingsReaderBase postingsReaderBase(SegmentReadState state) throws IOException;
+  public abstract TempPostingsReaderBase postingsReaderBase(SegmentReadState state) throws IOException;
 
   /** Creates the {@link PostingsWriterBase} for this
    *  format. */
-  public abstract PostingsWriterBase postingsWriterBase(SegmentWriteState state) throws IOException;
+  public abstract TempPostingsWriterBase postingsWriterBase(SegmentWriteState state) throws IOException;
 }

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/temp/TempFSTOrdTermsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/temp/TempFSTOrdTermsReader.java?rev=1515469&r1=1515468&r2=1515469&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/temp/TempFSTOrdTermsReader.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/temp/TempFSTOrdTermsReader.java Mon Aug 19 15:26:42 2013
@@ -100,7 +100,7 @@ public class TempFSTOrdTermsReader exten
         checkFieldSummary(state.segmentInfo, current, previous);
       }
     } finally {
-      IOUtils.close(indexIn, blockIn);
+      IOUtils.closeWhileHandlingException(indexIn, blockIn);
     }
   }
 
@@ -323,7 +323,7 @@ public class TempFSTOrdTermsReader exten
         if (metaBlockOrd != oldBlockOrd) {
           refillMetadata();
         }
-        metaBytesReader.reset(metaBytesBlock, bytesStart[upto], bytesLength[upto]);
+        metaBytesReader.setPosition(bytesStart[upto]);
         postingsReader.decodeTerm(longs[upto], metaBytesReader, fieldInfo, state, true);
       }
 

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/temp/TempFSTOrdTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/temp/TempFSTOrdTermsWriter.java?rev=1515469&r1=1515468&r2=1515469&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/temp/TempFSTOrdTermsWriter.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/temp/TempFSTOrdTermsWriter.java Mon Aug 19 15:26:42 2013
@@ -223,10 +223,10 @@ public class TempFSTOrdTermsWriter exten
       state.docFreq = stats.docFreq;
       state.totalTermFreq = stats.totalTermFreq;
       postingsWriter.finishTerm(state);
-      postingsWriter.encodeTerm(longs, metaBytesOut, fieldInfo, state, false);
+      postingsWriter.encodeTerm(longs, metaBytesOut, fieldInfo, state, true);
       for (int i = 0; i < longsSize; i++) {
-        metaLongsOut.writeVLong(longs[i]);
-        lastLongs[i] += longs[i];
+        metaLongsOut.writeVLong(longs[i] - lastLongs[i]);
+        lastLongs[i] = longs[i];
       }
       metaLongsOut.writeVLong(metaBytesOut.getFilePointer() - lastMetaBytesFP);
 

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/temp/TempFSTTermsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/temp/TempFSTTermsReader.java?rev=1515469&r1=1515468&r2=1515469&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/temp/TempFSTTermsReader.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/temp/TempFSTTermsReader.java Mon Aug 19 15:26:42 2013
@@ -94,7 +94,7 @@ public class TempFSTTermsReader extends 
       success = true;
     } finally {
       if (!success) {
-        in.close();
+        IOUtils.closeWhileHandlingException(in);
       }
     }
   }

Copied: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/temp/TempPostingsBaseFormat.java (from r1515266, lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsBaseFormat.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/temp/TempPostingsBaseFormat.java?p2=lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/temp/TempPostingsBaseFormat.java&p1=lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsBaseFormat.java&r1=1515266&r2=1515469&rev=1515469&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsBaseFormat.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/temp/TempPostingsBaseFormat.java Mon Aug 19 15:26:42 2013
@@ -1,4 +1,4 @@
-package org.apache.lucene.codecs.lucene41;
+package org.apache.lucene.codecs.temp;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -19,9 +19,8 @@ package org.apache.lucene.codecs.lucene4
 
 import java.io.IOException;
 
-import org.apache.lucene.codecs.PostingsBaseFormat;
-import org.apache.lucene.codecs.PostingsReaderBase;
-import org.apache.lucene.codecs.PostingsWriterBase;
+import org.apache.lucene.codecs.TempPostingsReaderBase;
+import org.apache.lucene.codecs.TempPostingsWriterBase;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
 
@@ -32,20 +31,20 @@ import org.apache.lucene.index.SegmentWr
  * @lucene.experimental */
 
 // TODO: should these also be named / looked up via SPI?
-public final class Lucene41PostingsBaseFormat extends PostingsBaseFormat {
+public final class TempPostingsBaseFormat extends org.apache.lucene.codecs.TempPostingsBaseFormat {
 
   /** Sole constructor. */
-  public Lucene41PostingsBaseFormat() {
-    super("Lucene41");
+  public TempPostingsBaseFormat() {
+    super("Temp");
   }
 
   @Override
-  public PostingsReaderBase postingsReaderBase(SegmentReadState state) throws IOException {
-    return new Lucene41PostingsReader(state.directory, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
+  public TempPostingsReaderBase postingsReaderBase(SegmentReadState state) throws IOException {
+    return new TempPostingsReader(state.directory, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
   }
 
   @Override
-  public PostingsWriterBase postingsWriterBase(SegmentWriteState state) throws IOException {
-    return new Lucene41PostingsWriter(state);
+  public TempPostingsWriterBase postingsWriterBase(SegmentWriteState state) throws IOException {
+    return new TempPostingsWriter(state);
   }
 }

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/temp/TempPostingsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/temp/TempPostingsWriter.java?rev=1515469&r1=1515468&r2=1515469&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/temp/TempPostingsWriter.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/temp/TempPostingsWriter.java Mon Aug 19 15:26:42 2013
@@ -200,6 +200,10 @@ public final class TempPostingsWriter ex
     long skipOffset = -1;
     long lastPosBlockOffset = -1;
     int singletonDocID = -1;
+    @Override
+    public String toString() {
+      return super.toString() + " docStartFP=" + docTermStartFP + " posStartFP=" + posTermStartFP + " payStartFP=" + payTermStartFP + " lastPosBlockOffset=" + lastPosBlockOffset + " singletonDocID=" + singletonDocID;
+    }
   }
 
   @Override

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java?rev=1515469&r1=1515468&r2=1515469&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java Mon Aug 19 15:26:42 2013
@@ -51,7 +51,7 @@ public class RAMOutputStream extends Ind
   }
 
   /** Copy the current contents of this buffer to the named output. */
-  public void writeTo(IndexOutput out) throws IOException {
+  public void writeTo(DataOutput out) throws IOException {
     flush();
     final long end = file.length;
     long pos = 0;