You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2012/10/09 15:12:36 UTC

svn commit: r1396020 - in /lucene/dev/trunk/lucene: ./ codecs/src/java/org/apache/lucene/codecs/appending/ codecs/src/java/org/apache/lucene/codecs/block/ codecs/src/java/org/apache/lucene/codecs/blockterms/ codecs/src/resources/META-INF/services/ code...

Author: rmuir
Date: Tue Oct  9 13:12:35 2012
New Revision: 1396020

URL: http://svn.apache.org/viewvc?rev=1396020&view=rev
Log:
LUCENE-4399: Remove AppendingCodec

Removed:
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/appending/
    lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/appending/TestAppendingCodec.java
    lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/appending/TestAppendingPostingsFormat.java
Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/block/BlockPostingsFormat.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsReader.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsWriter.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/FixedGapTermsIndexReader.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/FixedGapTermsIndexWriter.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/VariableGapTermsIndexReader.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/VariableGapTermsIndexWriter.java
    lucene/dev/trunk/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
    lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/block/TestBlockPostingsFormat.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/BufferedIndexOutput.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/ChecksumIndexOutput.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/CompoundFileWriter.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/IndexOutput.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java
    lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/util/SlowRAMDirectory.java
    lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/store/NativeUnixDirectory.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexOutputWrapper.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/ThrottledIndexOutput.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1396020&r1=1396019&r2=1396020&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Tue Oct  9 13:12:35 2012
@@ -44,6 +44,11 @@ New Features
   the suggester to ignore such variations. (Robert Muir, Sudarshan
   Gaikaiwari, Mike McCandless)
 
+API Changes
+
+* LUCENE-4399: Deprecated AppendingCodec. Lucene's term dictionaries
+  no longer seek when writing.  (Adrien Grand, Robert Muir)
+
 Bug Fixes
 
 * LUCENE-1822: BaseFragListBuilder hard-coded 6 char margin is too naive.

Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/block/BlockPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/block/BlockPostingsFormat.java?rev=1396020&r1=1396019&r2=1396020&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/block/BlockPostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/block/BlockPostingsFormat.java Tue Oct  9 13:12:35 2012
@@ -118,8 +118,8 @@ import org.apache.lucene.util.packed.Pac
  *
  * <ul>
  * <!-- TODO: expand on this, its not really correct and doesnt explain sub-blocks etc -->
- *   <li>TermDictionary(.tim) --&gt; Header, DirOffset, PostingsHeader, PackedBlockSize, 
- *                                   &lt;Block&gt;<sup>NumBlocks</sup>, FieldSummary</li>
+ *   <li>TermDictionary(.tim) --&gt; Header, PostingsHeader, PackedBlockSize, 
+ *                                   &lt;Block&gt;<sup>NumBlocks</sup>, FieldSummary, DirOffset</li>
  *   <li>Block --&gt; SuffixBlock, StatsBlock, MetadataBlock</li>
  *   <li>SuffixBlock --&gt; EntryCount, SuffixLength, {@link DataOutput#writeByte byte}<sup>SuffixLength</sup></li>
  *   <li>StatsBlock --&gt; StatsLength, &lt;DocFreq, TotalTermFreq&gt;<sup>EntryCount</sup></li>

Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsReader.java?rev=1396020&r1=1396019&r2=1396020&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsReader.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsReader.java Tue Oct  9 13:12:35 2012
@@ -76,7 +76,9 @@ public class BlockTermsReader extends Fi
   private TermsIndexReaderBase indexReader;
 
   // keeps the dirStart offset
-  protected long dirOffset;
+  private long dirOffset;
+  
+  private final int version; 
 
   // Used as key for the terms cache
   private static class FieldAndTerm extends DoubleBarrelLRUCache.CloneableKey {
@@ -123,7 +125,7 @@ public class BlockTermsReader extends Fi
 
     boolean success = false;
     try {
-      readHeader(in);
+      version = readHeader(in);
 
       // Have PostingsReader init itself
       postingsReader.init(in);
@@ -168,15 +170,21 @@ public class BlockTermsReader extends Fi
     this.indexReader = indexReader;
   }
 
-  protected void readHeader(IndexInput input) throws IOException {
-    CodecUtil.checkHeader(input, BlockTermsWriter.CODEC_NAME,
+  private int readHeader(IndexInput input) throws IOException {
+    int version = CodecUtil.checkHeader(input, BlockTermsWriter.CODEC_NAME,
                           BlockTermsWriter.VERSION_START,
                           BlockTermsWriter.VERSION_CURRENT);
-    dirOffset = input.readLong();
+    if (version < BlockTermsWriter.VERSION_APPEND_ONLY) {
+      dirOffset = input.readLong();
+    }
+    return version;
   }
   
-  protected void seekDir(IndexInput input, long dirOffset)
-      throws IOException {
+  private void seekDir(IndexInput input, long dirOffset) throws IOException {
+    if (version >= BlockTermsWriter.VERSION_APPEND_ONLY) {
+      input.seek(input.length() - 8);
+      dirOffset = input.readLong();
+    }
     input.seek(dirOffset);
   }
   

Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsWriter.java?rev=1396020&r1=1396019&r2=1396020&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsWriter.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsWriter.java Tue Oct  9 13:12:35 2012
@@ -58,8 +58,8 @@ public class BlockTermsWriter extends Fi
 
   // Initial format
   public static final int VERSION_START = 0;
-
-  public static final int VERSION_CURRENT = VERSION_START;
+  public static final int VERSION_APPEND_ONLY = 1;
+  public static final int VERSION_CURRENT = VERSION_APPEND_ONLY;
 
   /** Extension of terms file */
   static final String TERMS_EXTENSION = "tib";
@@ -98,10 +98,8 @@ public class BlockTermsWriter extends Fi
     }
   }
   
-  protected void writeHeader(IndexOutput out) throws IOException {
-    CodecUtil.writeHeader(out, CODEC_NAME, VERSION_CURRENT); 
-
-    out.writeLong(0);                             // leave space for end index pointer    
+  private void writeHeader(IndexOutput out) throws IOException {
+    CodecUtil.writeHeader(out, CODEC_NAME, VERSION_CURRENT);     
   }
 
   @Override
@@ -148,8 +146,7 @@ public class BlockTermsWriter extends Fi
     }
   }
 
-  protected void writeTrailer(long dirStart) throws IOException {
-    out.seek(CodecUtil.headerLength(CODEC_NAME));
+  private void writeTrailer(long dirStart) throws IOException {
     out.writeLong(dirStart);    
   }
   

Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/FixedGapTermsIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/FixedGapTermsIndexReader.java?rev=1396020&r1=1396019&r2=1396020&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/FixedGapTermsIndexReader.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/FixedGapTermsIndexReader.java Tue Oct  9 13:12:35 2012
@@ -70,7 +70,9 @@ public class FixedGapTermsIndexReader ex
   final HashMap<FieldInfo,FieldIndexData> fields = new HashMap<FieldInfo,FieldIndexData>();
   
   // start of the field info data
-  protected long dirOffset;
+  private long dirOffset;
+  
+  private final int version;
 
   public FixedGapTermsIndexReader(Directory dir, FieldInfos fieldInfos, String segment, int indexDivisor, Comparator<BytesRef> termComp, String segmentSuffix, IOContext context)
     throws IOException {
@@ -85,7 +87,7 @@ public class FixedGapTermsIndexReader ex
 
     try {
       
-      readHeader(in);
+      version = readHeader(in);
       indexInterval = in.readInt();
       if (indexInterval < 1) {
         throw new CorruptIndexException("invalid indexInterval: " + indexInterval + " (resource=" + in + ")");
@@ -148,10 +150,13 @@ public class FixedGapTermsIndexReader ex
     return indexDivisor;
   }
 
-  protected void readHeader(IndexInput input) throws IOException {
-    CodecUtil.checkHeader(input, FixedGapTermsIndexWriter.CODEC_NAME,
-      FixedGapTermsIndexWriter.VERSION_START, FixedGapTermsIndexWriter.VERSION_START);
-    dirOffset = input.readLong();
+  private int readHeader(IndexInput input) throws IOException {
+    int version = CodecUtil.checkHeader(input, FixedGapTermsIndexWriter.CODEC_NAME,
+      FixedGapTermsIndexWriter.VERSION_START, FixedGapTermsIndexWriter.VERSION_CURRENT);
+    if (version < FixedGapTermsIndexWriter.VERSION_APPEND_ONLY) {
+      dirOffset = input.readLong();
+    }
+    return version;
   }
 
   private class IndexEnum extends FieldIndexEnum {
@@ -409,7 +414,11 @@ public class FixedGapTermsIndexReader ex
     }
   }
 
-  protected void seekDir(IndexInput input, long dirOffset) throws IOException {
+  private void seekDir(IndexInput input, long dirOffset) throws IOException {
+    if (version >= FixedGapTermsIndexWriter.VERSION_APPEND_ONLY) {
+      input.seek(input.length() - 8);
+      dirOffset = input.readLong();
+    }
     input.seek(dirOffset);
   }
 }

Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/FixedGapTermsIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/FixedGapTermsIndexWriter.java?rev=1396020&r1=1396019&r2=1396020&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/FixedGapTermsIndexWriter.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/FixedGapTermsIndexWriter.java Tue Oct  9 13:12:35 2012
@@ -49,7 +49,8 @@ public class FixedGapTermsIndexWriter ex
 
   final static String CODEC_NAME = "SIMPLE_STANDARD_TERMS_INDEX";
   final static int VERSION_START = 0;
-  final static int VERSION_CURRENT = VERSION_START;
+  final static int VERSION_APPEND_ONLY = 1;
+  final static int VERSION_CURRENT = VERSION_APPEND_ONLY;
 
   final private int termIndexInterval;
 
@@ -74,10 +75,8 @@ public class FixedGapTermsIndexWriter ex
     }
   }
   
-  protected void writeHeader(IndexOutput out) throws IOException {
+  private void writeHeader(IndexOutput out) throws IOException {
     CodecUtil.writeHeader(out, CODEC_NAME, VERSION_CURRENT);
-    // Placeholder for dir offset
-    out.writeLong(0);
   }
 
   @Override
@@ -250,8 +249,7 @@ public class FixedGapTermsIndexWriter ex
     }
   }
 
-  protected void writeTrailer(long dirStart) throws IOException {
-    out.seek(CodecUtil.headerLength(CODEC_NAME));
+  private void writeTrailer(long dirStart) throws IOException {
     out.writeLong(dirStart);
   }
 }

Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/VariableGapTermsIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/VariableGapTermsIndexReader.java?rev=1396020&r1=1396019&r2=1396020&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/VariableGapTermsIndexReader.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/VariableGapTermsIndexReader.java Tue Oct  9 13:12:35 2012
@@ -54,7 +54,9 @@ public class VariableGapTermsIndexReader
   final HashMap<FieldInfo,FieldIndexData> fields = new HashMap<FieldInfo,FieldIndexData>();
   
   // start of the field info data
-  protected long dirOffset;
+  private long dirOffset;
+  
+  private final int version;
 
   final String segment;
   public VariableGapTermsIndexReader(Directory dir, FieldInfos fieldInfos, String segment, int indexDivisor, String segmentSuffix, IOContext context)
@@ -66,7 +68,7 @@ public class VariableGapTermsIndexReader
 
     try {
       
-      readHeader(in);
+      version = readHeader(in);
       this.indexDivisor = indexDivisor;
 
       seekDir(in, dirOffset);
@@ -103,10 +105,13 @@ public class VariableGapTermsIndexReader
     return indexDivisor;
   }
   
-  protected void readHeader(IndexInput input) throws IOException {
-    CodecUtil.checkHeader(input, VariableGapTermsIndexWriter.CODEC_NAME,
-      VariableGapTermsIndexWriter.VERSION_START, VariableGapTermsIndexWriter.VERSION_START);
-    dirOffset = input.readLong();
+  private int readHeader(IndexInput input) throws IOException {
+    int version = CodecUtil.checkHeader(input, VariableGapTermsIndexWriter.CODEC_NAME,
+      VariableGapTermsIndexWriter.VERSION_START, VariableGapTermsIndexWriter.VERSION_CURRENT);
+    if (version < VariableGapTermsIndexWriter.VERSION_APPEND_ONLY) {
+      dirOffset = input.readLong();
+    }
+    return version;
   }
 
   private static class IndexEnum extends FieldIndexEnum {
@@ -229,7 +234,11 @@ public class VariableGapTermsIndexReader
     }
   }
 
-  protected void seekDir(IndexInput input, long dirOffset) throws IOException {
+  private void seekDir(IndexInput input, long dirOffset) throws IOException {
+    if (version >= VariableGapTermsIndexWriter.VERSION_APPEND_ONLY) {
+      input.seek(input.length() - 8);
+      dirOffset = input.readLong();
+    }
     input.seek(dirOffset);
   }
 }

Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/VariableGapTermsIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/VariableGapTermsIndexWriter.java?rev=1396020&r1=1396019&r2=1396020&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/VariableGapTermsIndexWriter.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/VariableGapTermsIndexWriter.java Tue Oct  9 13:12:35 2012
@@ -52,7 +52,8 @@ public class VariableGapTermsIndexWriter
 
   final static String CODEC_NAME = "VARIABLE_GAP_TERMS_INDEX";
   final static int VERSION_START = 0;
-  final static int VERSION_CURRENT = VERSION_START;
+  final static int VERSION_APPEND_ONLY = 1;
+  final static int VERSION_CURRENT = VERSION_APPEND_ONLY;
 
   private final List<FSTFieldWriter> fields = new ArrayList<FSTFieldWriter>();
   
@@ -189,10 +190,8 @@ public class VariableGapTermsIndexWriter
     }
   }
   
-  protected void writeHeader(IndexOutput out) throws IOException {
+  private void writeHeader(IndexOutput out) throws IOException {
     CodecUtil.writeHeader(out, CODEC_NAME, VERSION_CURRENT);
-    // Placeholder for dir offset
-    out.writeLong(0);
   }
 
   @Override
@@ -316,8 +315,7 @@ public class VariableGapTermsIndexWriter
   }
   }
 
-  protected void writeTrailer(long dirStart) throws IOException {
-    out.seek(CodecUtil.headerLength(CODEC_NAME));
+  private void writeTrailer(long dirStart) throws IOException {
     out.writeLong(dirStart);
   }
 }

Modified: lucene/dev/trunk/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec?rev=1396020&r1=1396019&r2=1396020&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec (original)
+++ lucene/dev/trunk/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec Tue Oct  9 13:12:35 2012
@@ -14,4 +14,3 @@
 #  limitations under the License.
 
 org.apache.lucene.codecs.simpletext.SimpleTextCodec
-org.apache.lucene.codecs.appending.AppendingCodec

Modified: lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/block/TestBlockPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/block/TestBlockPostingsFormat.java?rev=1396020&r1=1396019&r2=1396020&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/block/TestBlockPostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/block/TestBlockPostingsFormat.java Tue Oct  9 13:12:35 2012
@@ -19,7 +19,6 @@ package org.apache.lucene.codecs.block;
 
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.appending.AppendingCodec;
 import org.apache.lucene.codecs.block.BlockPostingsFormat;
 import org.apache.lucene.codecs.lucene40.Lucene40Codec;
 import org.apache.lucene.index.BasePostingsFormatTestCase;

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java?rev=1396020&r1=1396019&r2=1396020&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java Tue Oct  9 13:12:35 2012
@@ -100,12 +100,14 @@ public class BlockTreeTermsReader extend
   private final TreeMap<String,FieldReader> fields = new TreeMap<String,FieldReader>();
 
   /** File offset where the directory starts in the terms file. */
-  protected long dirOffset;
+  private long dirOffset;
 
   /** File offset where the directory starts in the index file. */
-  protected long indexDirOffset;
+  private long indexDirOffset;
 
   private String segment;
+  
+  private final int version;
 
   /** Sole constructor. */
   public BlockTreeTermsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo info,
@@ -123,11 +125,14 @@ public class BlockTreeTermsReader extend
     IndexInput indexIn = null;
 
     try {
-      readHeader(in);
+      version = readHeader(in);
       if (indexDivisor != -1) {
         indexIn = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, BlockTreeTermsWriter.TERMS_INDEX_EXTENSION),
                                 ioContext);
-        readIndexHeader(indexIn);
+        int indexVersion = readIndexHeader(indexIn);
+        if (indexVersion != version) {
+          throw new CorruptIndexException("mixmatched version files: " + in + "=" + version + "," + indexIn + "=" + indexVersion);
+        }
       }
 
       // Have PostingsReader init itself
@@ -186,24 +191,34 @@ public class BlockTreeTermsReader extend
   }
 
   /** Reads terms file header. */
-  protected void readHeader(IndexInput input) throws IOException {
-    CodecUtil.checkHeader(input, BlockTreeTermsWriter.TERMS_CODEC_NAME,
+  private int readHeader(IndexInput input) throws IOException {
+    int version = CodecUtil.checkHeader(input, BlockTreeTermsWriter.TERMS_CODEC_NAME,
                           BlockTreeTermsWriter.TERMS_VERSION_START,
                           BlockTreeTermsWriter.TERMS_VERSION_CURRENT);
-    dirOffset = input.readLong();    
+    if (version < BlockTreeTermsWriter.TERMS_VERSION_APPEND_ONLY) {
+      dirOffset = input.readLong();
+    }
+    return version;
   }
 
   /** Reads index file header. */
-  protected void readIndexHeader(IndexInput input) throws IOException {
-    CodecUtil.checkHeader(input, BlockTreeTermsWriter.TERMS_INDEX_CODEC_NAME,
+  private int readIndexHeader(IndexInput input) throws IOException {
+    int version = CodecUtil.checkHeader(input, BlockTreeTermsWriter.TERMS_INDEX_CODEC_NAME,
                           BlockTreeTermsWriter.TERMS_INDEX_VERSION_START,
                           BlockTreeTermsWriter.TERMS_INDEX_VERSION_CURRENT);
-    indexDirOffset = input.readLong();    
+    if (version < BlockTreeTermsWriter.TERMS_INDEX_VERSION_APPEND_ONLY) {
+      indexDirOffset = input.readLong(); 
+    }
+    return version;
   }
 
   /** Seek {@code input} to the directory offset. */
-  protected void seekDir(IndexInput input, long dirOffset)
+  private void seekDir(IndexInput input, long dirOffset)
       throws IOException {
+    if (version >= BlockTreeTermsWriter.TERMS_INDEX_VERSION_APPEND_ONLY) {
+      input.seek(input.length() - 8);
+      dirOffset = input.readLong();
+    }
     input.seek(dirOffset);
   }
 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java?rev=1396020&r1=1396019&r2=1396020&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java Tue Oct  9 13:12:35 2012
@@ -107,9 +107,12 @@ public class BlockTreeTermsWriter extend
 
   /** Initial terms format. */
   public static final int TERMS_VERSION_START = 0;
+  
+  /** Append-only */
+  public static final int TERMS_VERSION_APPEND_ONLY = 1;
 
   /** Current terms format. */
-  public static final int TERMS_VERSION_CURRENT = TERMS_VERSION_START;
+  public static final int TERMS_VERSION_CURRENT = TERMS_VERSION_APPEND_ONLY;
 
   /** Extension of terms index file */
   static final String TERMS_INDEX_EXTENSION = "tip";
@@ -117,9 +120,12 @@ public class BlockTreeTermsWriter extend
 
   /** Initial index format. */
   public static final int TERMS_INDEX_VERSION_START = 0;
+  
+  /** Append-only */
+  public static final int TERMS_INDEX_VERSION_APPEND_ONLY = 1;
 
   /** Current index format. */
-  public static final int TERMS_INDEX_VERSION_CURRENT = TERMS_INDEX_VERSION_START;
+  public static final int TERMS_INDEX_VERSION_CURRENT = TERMS_INDEX_VERSION_APPEND_ONLY;
 
   private final IndexOutput out;
   private final IndexOutput indexOut;
@@ -189,26 +195,22 @@ public class BlockTreeTermsWriter extend
   }
 
   /** Writes the terms file header. */
-  protected void writeHeader(IndexOutput out) throws IOException {
-    CodecUtil.writeHeader(out, TERMS_CODEC_NAME, TERMS_VERSION_CURRENT); 
-    out.writeLong(0);                             // leave space for end index pointer    
+  private void writeHeader(IndexOutput out) throws IOException {
+    CodecUtil.writeHeader(out, TERMS_CODEC_NAME, TERMS_VERSION_CURRENT);   
   }
 
   /** Writes the index file header. */
-  protected void writeIndexHeader(IndexOutput out) throws IOException {
+  private void writeIndexHeader(IndexOutput out) throws IOException {
     CodecUtil.writeHeader(out, TERMS_INDEX_CODEC_NAME, TERMS_INDEX_VERSION_CURRENT); 
-    out.writeLong(0);                             // leave space for end index pointer    
   }
 
   /** Writes the terms file trailer. */
-  protected void writeTrailer(IndexOutput out, long dirStart) throws IOException {
-    out.seek(CodecUtil.headerLength(TERMS_CODEC_NAME));
+  private void writeTrailer(IndexOutput out, long dirStart) throws IOException {
     out.writeLong(dirStart);    
   }
 
   /** Writes the index file trailer. */
-  protected void writeIndexTrailer(IndexOutput indexOut, long dirStart) throws IOException {
-    indexOut.seek(CodecUtil.headerLength(TERMS_INDEX_CODEC_NAME));
+  private void writeIndexTrailer(IndexOutput indexOut, long dirStart) throws IOException {
     indexOut.writeLong(dirStart);    
   }
   

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java?rev=1396020&r1=1396019&r2=1396020&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java Tue Oct  9 13:12:35 2012
@@ -68,9 +68,9 @@ import org.apache.lucene.util.fst.FST; /
  *
  * <ul>
  * <!-- TODO: expand on this, its not really correct and doesnt explain sub-blocks etc -->
- *    <li>TermsDict (.tim) --&gt; Header, DirOffset, PostingsHeader, SkipInterval,
+ *    <li>TermsDict (.tim) --&gt; Header, PostingsHeader, SkipInterval,
  *                               MaxSkipLevels, SkipMinimum, Block<sup>NumBlocks</sup>,
- *                               FieldSummary</li>
+ *                               FieldSummary, DirOffset</li>
  *    <li>Block --&gt; SuffixBlock, StatsBlock, MetadataBlock</li>
  *    <li>SuffixBlock --&gt; EntryCount, SuffixLength, Byte<sup>SuffixLength</sup></li>
  *    <li>StatsBlock --&gt; StatsLength, &lt;DocFreq, TotalTermFreq&gt;<sup>EntryCount</sup></li>
@@ -131,12 +131,13 @@ import org.apache.lucene.util.fst.FST; /
  * accessed randomly.  The index is also used to determine
  * when a given term cannot exist on disk (in the .tim file), saving a disk seek.</p>
  * <ul>
- *   <li>TermsIndex (.tip) --&gt; Header, &lt;IndexStartFP&gt;<sup>NumFields</sup>, 
- *                                FSTIndex<sup>NumFields</sup></li>
+ *   <li>TermsIndex (.tip) --&gt; Header, FSTIndex<sup>NumFields</sup>, 
+ *                                &lt;IndexStartFP&gt;<sup>NumFields</sup>, DirOffset</li>
  *   <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
  *   <li>IndexStartFP --&gt; {@link DataOutput#writeVLong VLong}</li>
  *   <!-- TODO: better describe FST output here -->
  *   <li>FSTIndex --&gt; {@link FST FST&lt;byte[]&gt;}</li>
+ *   <li>DirOffset --&gt; {@link DataOutput#writeLong Uint64}</li>
  * </ul>
  * <p>Notes:</p>
  * <ul>
@@ -145,6 +146,8 @@ import org.apache.lucene.util.fst.FST; /
  *       block that holds all terms starting with that
  *       prefix.  Each field's IndexStartFP points to its
  *       FST.</li>
+ *   <li>DirOffset is a pointer to the start of the IndexStartFPs
+ *       for all fields</li>
  *   <li>It's possible that an on-disk block would contain
  *       too many terms (more than the allowed maximum
  *       (default: 48)).  When this happens, the block is

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/BufferedIndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/BufferedIndexOutput.java?rev=1396020&r1=1396019&r2=1396020&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/BufferedIndexOutput.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/BufferedIndexOutput.java Tue Oct  9 13:12:35 2012
@@ -27,9 +27,6 @@ public abstract class BufferedIndexOutpu
   private long bufferStart = 0;           // position in file of buffer
   private int bufferPosition = 0;         // position in buffer
 
-  /** Writes a single byte.
-   * @see IndexInput#readByte()
-   */
   @Override
   public void writeByte(byte b) throws IOException {
     if (bufferPosition >= BUFFER_SIZE)
@@ -37,11 +34,6 @@ public abstract class BufferedIndexOutpu
     buffer[bufferPosition++] = b;
   }
 
-  /** Writes an array of bytes.
-   * @param b the bytes to write
-   * @param length the number of bytes to write
-   * @see IndexInput#readBytes(byte[],int,int)
-   */
   @Override
   public void writeBytes(byte[] b, int offset, int length) throws IOException {
     int bytesLeft = BUFFER_SIZE - bufferPosition;
@@ -82,7 +74,6 @@ public abstract class BufferedIndexOutpu
     }
   }
 
-  /** Forces any buffered output to be written. */
   @Override
   public void flush() throws IOException {
     flushBuffer(buffer, bufferPosition);
@@ -107,31 +98,16 @@ public abstract class BufferedIndexOutpu
    */
   protected abstract void flushBuffer(byte[] b, int offset, int len) throws IOException;
   
-  /** Closes this stream to further operations. */
   @Override
   public void close() throws IOException {
     flush();
   }
 
-  /** Returns the current position in this file, where the next write will
-   * occur.
-   * @see #seek(long)
-   */
   @Override
   public long getFilePointer() {
     return bufferStart + bufferPosition;
   }
 
-  /** Sets current position in this file, where the next write will occur.
-   * @see #getFilePointer()
-   */
-  @Override
-  public void seek(long pos) throws IOException {
-    flush();
-    bufferStart = pos;
-  }
-
-  /** The number of bytes in the file. */
   @Override
   public abstract long length() throws IOException;
 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/ChecksumIndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/ChecksumIndexOutput.java?rev=1396020&r1=1396019&r2=1396020&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/ChecksumIndexOutput.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/ChecksumIndexOutput.java Tue Oct  9 13:12:35 2012
@@ -22,7 +22,7 @@ import java.util.zip.CRC32;
 import java.util.zip.Checksum;
 
 /** Writes bytes through to a primary IndexOutput, computing
- *  checksum.  Note that you cannot use seek().
+ *  checksum.
  *
  * @lucene.internal
  */
@@ -66,11 +66,6 @@ public class ChecksumIndexOutput extends
     return main.getFilePointer();
   }
 
-  @Override
-  public void seek(long pos) {
-    throw new UnsupportedOperationException();    
-  }
-
   /** writes the checksum */
   public void finishCommit() throws IOException {
     main.writeLong(getChecksum());

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/CompoundFileWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/CompoundFileWriter.java?rev=1396020&r1=1396019&r2=1396020&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/CompoundFileWriter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/CompoundFileWriter.java Tue Oct  9 13:12:35 2012
@@ -325,12 +325,6 @@ final class CompoundFileWriter implement
     }
 
     @Override
-    public void seek(long pos) throws IOException {
-      assert !closed;
-      delegate.seek(offset + pos);
-    }
-
-    @Override
     public long length() throws IOException {
       assert !closed;
       return delegate.length() - offset;

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java?rev=1396020&r1=1396019&r2=1396020&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java Tue Oct  9 13:12:35 2012
@@ -548,13 +548,6 @@ public abstract class FSDirectory extend
       }
     }
 
-    /** Random-access methods */
-    @Override
-    public void seek(long pos) throws IOException {
-      super.seek(pos);
-      file.seek(pos);
-    }
-
     @Override
     public long length() throws IOException {
       return file.length();

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/IndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/IndexOutput.java?rev=1396020&r1=1396019&r2=1396020&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/IndexOutput.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/IndexOutput.java Tue Oct  9 13:12:35 2012
@@ -39,14 +39,9 @@ public abstract class IndexOutput extend
 
   /** Returns the current position in this file, where the next write will
    * occur.
-   * @see #seek(long)
    */
   public abstract long getFilePointer();
 
-  /** Sets current position in this file, where the next write will occur.
-   * @see #getFilePointer()
-   */
-  public abstract void seek(long pos) throws IOException;
 
   /** The number of bytes in the file. */
   public abstract long length() throws IOException;

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java?rev=1396020&r1=1396019&r2=1396020&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java Tue Oct  9 13:12:35 2012
@@ -103,19 +103,6 @@ public class RAMOutputStream extends Ind
   }
 
   @Override
-  public void seek(long pos) throws IOException {
-    // set the file length in case we seek back
-    // and flush() has not been called yet
-    setFileLength();
-    if (pos < bufferStart || pos >= bufferStart + bufferLength) {
-      currentBufferIndex = (int) (pos / BUFFER_SIZE);
-      switchCurrentBuffer();
-    }
-
-    bufferPosition = (int) (pos % BUFFER_SIZE);
-  }
-
-  @Override
   public long length() {
     return file.length;
   }

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/util/SlowRAMDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/util/SlowRAMDirectory.java?rev=1396020&r1=1396019&r2=1396020&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/util/SlowRAMDirectory.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/util/SlowRAMDirectory.java Tue Oct  9 13:12:35 2012
@@ -154,7 +154,6 @@ public class SlowRAMDirectory extends RA
     @Override public void flush() throws IOException { io.flush(); }
     @Override public long getFilePointer() { return io.getFilePointer(); }
     @Override public long length() throws IOException { return io.length(); }
-    @Override public void seek(long pos) throws IOException { io.seek(pos); }
   }
   
 }

Modified: lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/store/NativeUnixDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/store/NativeUnixDirectory.java?rev=1396020&r1=1396019&r2=1396020&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/store/NativeUnixDirectory.java (original)
+++ lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/store/NativeUnixDirectory.java Tue Oct  9 13:12:35 2012
@@ -236,26 +236,6 @@ public class NativeUnixDirectory extends
       return filePos + bufferPos;
     }
 
-    // TODO: seek is fragile at best; it can only properly
-    // handle seek & then change bytes that fit entirely
-    // within one buffer
-    @Override
-    public void seek(long pos) throws IOException {
-      if (pos != getFilePointer()) {
-        dump();
-        final long alignedPos = pos & ALIGN_NOT_MASK;
-        filePos = alignedPos;
-        int n = (int) NativePosixUtil.pread(fos.getFD(), filePos, buffer);
-        if (n < bufferSize) {
-          buffer.limit(n);
-        }
-        //System.out.println("seek refill=" + n);
-        final int delta = (int) (pos - alignedPos);
-        buffer.position(delta);
-        bufferPos = delta;
-      }
-    }
-
     @Override
     public long length() {
       return fileLength + bufferPos;

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexOutputWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexOutputWrapper.java?rev=1396020&r1=1396019&r2=1396020&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexOutputWrapper.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexOutputWrapper.java Tue Oct  9 13:12:35 2012
@@ -140,11 +140,6 @@ public class MockIndexOutputWrapper exte
   }
 
   @Override
-  public void seek(long pos) throws IOException {
-    delegate.seek(pos);
-  }
-
-  @Override
   public long length() throws IOException {
     return delegate.length();
   }

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java?rev=1396020&r1=1396019&r2=1396020&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java Tue Oct  9 13:12:35 2012
@@ -29,7 +29,6 @@ import java.util.TimeZone;
 
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.appending.AppendingCodec;
 import org.apache.lucene.codecs.asserting.AssertingCodec;
 import org.apache.lucene.codecs.compressing.CompressingCodec;
 import org.apache.lucene.codecs.lucene40.Lucene40Codec;
@@ -162,8 +161,6 @@ final class TestRuleSetupAndRestoreClass
       };
     } else if ("SimpleText".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 9 && !shouldAvoidCodec("SimpleText"))) {
       codec = new SimpleTextCodec();
-    } else if ("Appending".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 8 && !shouldAvoidCodec("Appending"))) {
-      codec = new AppendingCodec();
     } else if ("Asserting".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 7 && !shouldAvoidCodec("Asserting"))) {
       codec = new AssertingCodec();
     } else if ("Compressing".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 6 && !shouldAvoidCodec("Compressing"))) {

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/ThrottledIndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/ThrottledIndexOutput.java?rev=1396020&r1=1396019&r2=1396020&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/ThrottledIndexOutput.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/ThrottledIndexOutput.java Tue Oct  9 13:12:35 2012
@@ -89,12 +89,6 @@ public class ThrottledIndexOutput extend
   }
 
   @Override
-  public void seek(long pos) throws IOException {
-    sleep(seekDelayMillis);
-    delegate.seek(pos);
-  }
-
-  @Override
   public long length() throws IOException {
     return delegate.length();
   }