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 2011/10/05 17:59:35 UTC

svn commit: r1179284 - in /lucene/dev/trunk/lucene/src/java/org/apache/lucene/index: ./ codecs/

Author: rmuir
Date: Wed Oct  5 15:59:34 2011
New Revision: 1179284

URL: http://svn.apache.org/viewvc?rev=1179284&view=rev
Log:
LUCENE-2621: add hooks to codecprovider (for now) for stored fields

Added:
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultFieldsReader.java   (with props)
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultFieldsWriter.java   (with props)
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/FieldsReader.java   (with props)
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/FieldsWriter.java   (with props)
Removed:
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldsReader.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldsWriter.java
Modified:
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentCoreReaders.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReader.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java?rev=1179284&r1=1179283&r2=1179284&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java Wed Oct  5 15:59:34 2011
@@ -26,6 +26,7 @@ import java.text.NumberFormat;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.index.DocumentsWriterDeleteQueue.DeleteSlice;
+import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.search.similarities.SimilarityProvider;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FlushInfo;
@@ -151,6 +152,7 @@ public class DocumentsWriterPerThread {
   }
   private final static boolean INFO_VERBOSE = false;
   final DocumentsWriter parent;
+  final CodecProvider codecProvider;
   final IndexWriter writer;
   final Directory directory;
   final DocState docState;
@@ -181,6 +183,7 @@ public class DocumentsWriterPerThread {
     this.fieldInfos = fieldInfos;
     this.writer = parent.indexWriter;
     this.infoStream = parent.infoStream;
+    this.codecProvider = this.writer.codecs;
     this.docState = new DocState(this);
     this.docState.similarityProvider = parent.indexWriter.getConfig()
         .getSimilarityProvider();

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentCoreReaders.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentCoreReaders.java?rev=1179284&r1=1179283&r2=1179284&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentCoreReaders.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentCoreReaders.java Wed Oct  5 15:59:34 2011
@@ -22,6 +22,7 @@ import java.util.concurrent.atomic.Atomi
 
 import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.index.codecs.FieldsProducer;
+import org.apache.lucene.index.codecs.FieldsReader;
 import org.apache.lucene.index.codecs.PerDocValues;
 import org.apache.lucene.store.CompoundFileDirectory;
 import org.apache.lucene.store.Directory;
@@ -164,7 +165,7 @@ final class SegmentCoreReaders {
       }
       
       final String storesSegment = si.getDocStoreSegment();
-      fieldsReaderOrig = new FieldsReader(storeDir, storesSegment, fieldInfos, context,
+      fieldsReaderOrig = si.getSegmentCodecs().provider.fieldsReader(storeDir, storesSegment, fieldInfos, context,
           si.getDocStoreOffset(), si.docCount);
       
       // Verify two sources of "maxDoc" agree:

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java?rev=1179284&r1=1179283&r2=1179284&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java Wed Oct  5 15:59:34 2011
@@ -29,6 +29,8 @@ import org.apache.lucene.index.IndexRead
 import org.apache.lucene.index.MergePolicy.MergeAbortedException;
 import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.index.codecs.FieldsConsumer;
+import org.apache.lucene.index.codecs.FieldsReader;
+import org.apache.lucene.index.codecs.FieldsWriter;
 import org.apache.lucene.index.codecs.MergeState;
 import org.apache.lucene.index.codecs.PerDocConsumer;
 import org.apache.lucene.index.codecs.PerDocValues;
@@ -257,7 +259,7 @@ final class SegmentMerger {
     int docCount = 0;
 
     setMatchingSegmentReaders();
-    final FieldsWriter fieldsWriter = new FieldsWriter(directory, segment, context);
+    final FieldsWriter fieldsWriter = codecInfo.provider.fieldsWriter(directory, segment, context);
     try {
       int idx = 0;
       for (MergeState.IndexReaderAndLiveDocs reader : readers) {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReader.java?rev=1179284&r1=1179283&r2=1179284&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReader.java Wed Oct  5 15:59:34 2011
@@ -30,6 +30,7 @@ import java.util.concurrent.atomic.Atomi
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.codecs.FieldsReader;
 import org.apache.lucene.index.codecs.PerDocValues;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.BitVector;
@@ -76,7 +77,7 @@ public class SegmentReader extends Index
   private class FieldsReaderLocal extends CloseableThreadLocal<FieldsReader> {
     @Override
     protected FieldsReader initialValue() {
-      return (FieldsReader) core.getFieldsReaderOrig().clone();
+      return core.getFieldsReaderOrig().clone();
     }
   }
 

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java?rev=1179284&r1=1179283&r2=1179284&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java Wed Oct  5 15:59:34 2011
@@ -19,6 +19,8 @@ package org.apache.lucene.index;
 
 import java.io.IOException;
 
+import org.apache.lucene.index.codecs.CodecProvider;
+import org.apache.lucene.index.codecs.FieldsWriter;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.RamUsageEstimator;
@@ -33,10 +35,12 @@ final class StoredFieldsWriter {
   int freeCount;
 
   final DocumentsWriterPerThread.DocState docState;
+  final CodecProvider codecProvider;
 
   public StoredFieldsWriter(DocumentsWriterPerThread docWriter) {
     this.docWriter = docWriter;
     this.docState = docWriter.docState;
+    this.codecProvider = docWriter.codecProvider;
   }
 
   private int numStoredFields;
@@ -77,7 +81,7 @@ final class StoredFieldsWriter {
 
   private synchronized void initFieldsWriter(IOContext context) throws IOException {
     if (fieldsWriter == null) {
-      fieldsWriter = new FieldsWriter(docWriter.directory, docWriter.getSegment(), context);
+      fieldsWriter = codecProvider.fieldsWriter(docWriter.directory, docWriter.getSegment(), context);
       lastDocID = 0;
     }
   }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java?rev=1179284&r1=1179283&r2=1179284&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java Wed Oct  5 15:59:34 2011
@@ -17,12 +17,17 @@ package org.apache.lucene.index.codecs;
  * limitations under the License.
  */
 
+import java.io.IOException;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+
 /** Holds a set of codecs, keyed by name.  You subclass
  *  this, instantiate it, and register your codecs, then
  *  pass this instance to IndexReader/IndexWriter (via
@@ -105,6 +110,16 @@ public class CodecProvider {
   public SegmentInfosReader getSegmentInfosReader() {
     return infosReader;
   }
+  
+  /** expert */
+  public FieldsReader fieldsReader(Directory directory, String segment, FieldInfos fn, IOContext context, int docStoreOffset, int size) throws IOException {
+    return new DefaultFieldsReader(directory, segment, fn, context, docStoreOffset, size);
+  }
+
+  /** expert */
+  public FieldsWriter fieldsWriter(Directory directory, String segment, IOContext context) throws IOException {
+    return new DefaultFieldsWriter(directory, segment, context);
+  }
 
   static private CodecProvider defaultCodecs = new CoreCodecProvider();
 

Added: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultFieldsReader.java?rev=1179284&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultFieldsReader.java (added)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultFieldsReader.java Wed Oct  5 15:59:34 2011
@@ -0,0 +1,303 @@
+package org.apache.lucene.index.codecs;
+
+/**
+ * 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.index.CorruptIndexException;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.FieldReaderException;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.IndexFormatTooNewException;
+import org.apache.lucene.index.IndexFormatTooOldException;
+import org.apache.lucene.index.StoredFieldVisitor;
+import org.apache.lucene.store.AlreadyClosedException;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.CloseableThreadLocal;
+import org.apache.lucene.util.IOUtils;
+
+import java.io.Closeable;
+
+/**
+ * Class responsible for access to stored document fields.
+ * <p/>
+ * It uses &lt;segment&gt;.fdt and &lt;segment&gt;.fdx; files.
+ * 
+ * @lucene.internal
+ */
+public final class DefaultFieldsReader extends FieldsReader implements Cloneable, Closeable {
+  private final static int FORMAT_SIZE = 4;
+
+  private final FieldInfos fieldInfos;
+  private CloseableThreadLocal<IndexInput> fieldsStreamTL = new CloseableThreadLocal<IndexInput>();
+  
+  // The main fieldStream, used only for cloning.
+  private final IndexInput cloneableFieldsStream;
+
+  // This is a clone of cloneableFieldsStream used for reading documents.
+  // It should not be cloned outside of a synchronized context.
+  private final IndexInput fieldsStream;
+
+  private final IndexInput cloneableIndexStream;
+  private final IndexInput indexStream;
+  private int numTotalDocs;
+  private int size;
+  private boolean closed;
+  private final int format;
+
+  // The docID offset where our docs begin in the index
+  // file.  This will be 0 if we have our own private file.
+  private int docStoreOffset;
+
+  private boolean isOriginal = false;
+
+  /** Returns a cloned FieldsReader that shares open
+   *  IndexInputs with the original one.  It is the caller's
+   *  job not to close the original FieldsReader until all
+   *  clones are called (eg, currently SegmentReader manages
+   *  this logic). */
+  @Override
+  public DefaultFieldsReader clone() {
+    ensureOpen();
+    return new DefaultFieldsReader(fieldInfos, numTotalDocs, size, format, docStoreOffset, cloneableFieldsStream, cloneableIndexStream);
+  }
+
+  /** Verifies that the code version which wrote the segment is supported. */
+  public static void checkCodeVersion(Directory dir, String segment) throws IOException {
+    final String indexStreamFN = IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_INDEX_EXTENSION);
+    IndexInput idxStream = dir.openInput(indexStreamFN, IOContext.DEFAULT);
+    
+    try {
+      int format = idxStream.readInt();
+      if (format < DefaultFieldsWriter.FORMAT_MINIMUM)
+        throw new IndexFormatTooOldException(indexStreamFN, format, DefaultFieldsWriter.FORMAT_MINIMUM, DefaultFieldsWriter.FORMAT_CURRENT);
+      if (format > DefaultFieldsWriter.FORMAT_CURRENT)
+        throw new IndexFormatTooNewException(indexStreamFN, format, DefaultFieldsWriter.FORMAT_MINIMUM, DefaultFieldsWriter.FORMAT_CURRENT);
+    } finally {
+      idxStream.close();
+    }
+  
+  }
+  
+  // Used only by clone
+  private DefaultFieldsReader(FieldInfos fieldInfos, int numTotalDocs, int size, int format, int docStoreOffset,
+                       IndexInput cloneableFieldsStream, IndexInput cloneableIndexStream) {
+    this.fieldInfos = fieldInfos;
+    this.numTotalDocs = numTotalDocs;
+    this.size = size;
+    this.format = format;
+    this.docStoreOffset = docStoreOffset;
+    this.cloneableFieldsStream = cloneableFieldsStream;
+    this.cloneableIndexStream = cloneableIndexStream;
+    fieldsStream = (IndexInput) cloneableFieldsStream.clone();
+    indexStream = (IndexInput) cloneableIndexStream.clone();
+  }
+  
+  public DefaultFieldsReader(Directory d, String segment, FieldInfos fn) throws IOException {
+    this(d, segment, fn, IOContext.DEFAULT, -1, 0);
+  }
+
+  public DefaultFieldsReader(Directory d, String segment, FieldInfos fn, IOContext context, int docStoreOffset, int size) throws IOException {
+    boolean success = false;
+    isOriginal = true;
+    try {
+      fieldInfos = fn;
+
+      cloneableFieldsStream = d.openInput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_EXTENSION), context);
+      final String indexStreamFN = IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_INDEX_EXTENSION);
+      cloneableIndexStream = d.openInput(indexStreamFN, context);
+      
+      format = cloneableIndexStream.readInt();
+
+      if (format < DefaultFieldsWriter.FORMAT_MINIMUM)
+        throw new IndexFormatTooOldException(indexStreamFN, format, DefaultFieldsWriter.FORMAT_MINIMUM, DefaultFieldsWriter.FORMAT_CURRENT);
+      if (format > DefaultFieldsWriter.FORMAT_CURRENT)
+        throw new IndexFormatTooNewException(indexStreamFN, format, DefaultFieldsWriter.FORMAT_MINIMUM, DefaultFieldsWriter.FORMAT_CURRENT);
+
+      fieldsStream = (IndexInput) cloneableFieldsStream.clone();
+
+      final long indexSize = cloneableIndexStream.length() - FORMAT_SIZE;
+      
+      if (docStoreOffset != -1) {
+        // We read only a slice out of this shared fields file
+        this.docStoreOffset = docStoreOffset;
+        this.size = size;
+
+        // Verify the file is long enough to hold all of our
+        // docs
+        assert ((int) (indexSize / 8)) >= size + this.docStoreOffset: "indexSize=" + indexSize + " size=" + size + " docStoreOffset=" + docStoreOffset;
+      } else {
+        this.docStoreOffset = 0;
+        this.size = (int) (indexSize >> 3);
+      }
+
+      indexStream = (IndexInput) cloneableIndexStream.clone();
+      numTotalDocs = (int) (indexSize >> 3);
+      success = true;
+    } finally {
+      // With lock-less commits, it's entirely possible (and
+      // fine) to hit a FileNotFound exception above. In
+      // this case, we want to explicitly close any subset
+      // of things that were opened so that we don't have to
+      // wait for a GC to do so.
+      if (!success) {
+        close();
+      }
+    }
+  }
+
+  /**
+   * @throws AlreadyClosedException if this FieldsReader is closed
+   */
+  private void ensureOpen() throws AlreadyClosedException {
+    if (closed) {
+      throw new AlreadyClosedException("this FieldsReader is closed");
+    }
+  }
+
+  /**
+   * Closes the underlying {@link org.apache.lucene.store.IndexInput} streams, including any ones associated with a
+   * lazy implementation of a Field.  This means that the Fields values will not be accessible.
+   *
+   * @throws IOException
+   */
+  public final void close() throws IOException {
+    if (!closed) {
+      if (isOriginal) {
+        IOUtils.close(fieldsStream, indexStream, fieldsStreamTL, cloneableFieldsStream, cloneableIndexStream);
+      } else {
+        IOUtils.close(fieldsStream, indexStream, fieldsStreamTL);
+      }
+      closed = true;
+    }
+  }
+
+  public final int size() {
+    return size;
+  }
+
+  private void seekIndex(int docID) throws IOException {
+    indexStream.seek(FORMAT_SIZE + (docID + docStoreOffset) * 8L);
+  }
+
+  public final void visitDocument(int n, StoredFieldVisitor visitor) throws CorruptIndexException, IOException {
+    seekIndex(n);
+    fieldsStream.seek(indexStream.readLong());
+
+    final int numFields = fieldsStream.readVInt();
+    for (int fieldIDX = 0; fieldIDX < numFields; fieldIDX++) {
+      int fieldNumber = fieldsStream.readVInt();
+      FieldInfo fieldInfo = fieldInfos.fieldInfo(fieldNumber);
+      
+      int bits = fieldsStream.readByte() & 0xFF;
+      assert bits <= (DefaultFieldsWriter.FIELD_IS_NUMERIC_MASK | DefaultFieldsWriter.FIELD_IS_BINARY): "bits=" + Integer.toHexString(bits);
+
+      final boolean binary = (bits & DefaultFieldsWriter.FIELD_IS_BINARY) != 0;
+      final int numeric = bits & DefaultFieldsWriter.FIELD_IS_NUMERIC_MASK;
+
+      final boolean doStop;
+      if (binary) {
+        final int numBytes = fieldsStream.readVInt();
+        doStop = visitor.binaryField(fieldInfo, fieldsStream, numBytes);
+      } else if (numeric != 0) {
+        switch(numeric) {
+        case DefaultFieldsWriter.FIELD_IS_NUMERIC_INT:
+          doStop = visitor.intField(fieldInfo, fieldsStream.readInt());
+          break;
+        case DefaultFieldsWriter.FIELD_IS_NUMERIC_LONG:
+          doStop = visitor.longField(fieldInfo, fieldsStream.readLong());
+          break;
+        case DefaultFieldsWriter.FIELD_IS_NUMERIC_FLOAT:
+          doStop = visitor.floatField(fieldInfo, Float.intBitsToFloat(fieldsStream.readInt()));
+          break;
+        case DefaultFieldsWriter.FIELD_IS_NUMERIC_DOUBLE:
+          doStop = visitor.doubleField(fieldInfo, Double.longBitsToDouble(fieldsStream.readLong()));
+          break;
+        default:
+          throw new FieldReaderException("Invalid numeric type: " + Integer.toHexString(numeric));
+        }
+      } else {
+        // Text:
+        final int numUTF8Bytes = fieldsStream.readVInt();
+        doStop = visitor.stringField(fieldInfo, fieldsStream, numUTF8Bytes);
+      }
+
+      if (doStop) {
+        return;
+      }
+    }
+  }
+
+  /** Returns the length in bytes of each raw document in a
+   *  contiguous range of length numDocs starting with
+   *  startDocID.  Returns the IndexInput (the fieldStream),
+   *  already seeked to the starting point for startDocID.*/
+  public final IndexInput rawDocs(int[] lengths, int startDocID, int numDocs) throws IOException {
+    seekIndex(startDocID);
+    long startOffset = indexStream.readLong();
+    long lastOffset = startOffset;
+    int count = 0;
+    while (count < numDocs) {
+      final long offset;
+      final int docID = docStoreOffset + startDocID + count + 1;
+      assert docID <= numTotalDocs;
+      if (docID < numTotalDocs) 
+        offset = indexStream.readLong();
+      else
+        offset = fieldsStream.length();
+      lengths[count++] = (int) (offset-lastOffset);
+      lastOffset = offset;
+    }
+
+    fieldsStream.seek(startOffset);
+
+    return fieldsStream;
+  }
+
+  /**
+   * Skip the field.  We still have to read some of the information about the field, but can skip past the actual content.
+   * This will have the most payoff on large fields.
+   */
+  private void skipField(int numeric) throws IOException {
+    final int numBytes;
+    switch(numeric) {
+      case 0:
+        numBytes = fieldsStream.readVInt();
+        break;
+      case DefaultFieldsWriter.FIELD_IS_NUMERIC_INT:
+      case DefaultFieldsWriter.FIELD_IS_NUMERIC_FLOAT:
+        numBytes = 4;
+        break;
+      case DefaultFieldsWriter.FIELD_IS_NUMERIC_LONG:
+      case DefaultFieldsWriter.FIELD_IS_NUMERIC_DOUBLE:
+        numBytes = 8;
+        break;
+      default:
+        throw new FieldReaderException("Invalid numeric type: " + Integer.toHexString(numeric));
+    }
+    
+    skipFieldBytes(numBytes);
+  }
+  
+  private void skipFieldBytes(int toRead) throws IOException {
+    fieldsStream.seek(fieldsStream.getFilePointer() + toRead);
+  }
+}

Added: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultFieldsWriter.java?rev=1179284&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultFieldsWriter.java (added)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultFieldsWriter.java Wed Oct  5 15:59:34 2011
@@ -0,0 +1,236 @@
+package org.apache.lucene.index.codecs;
+
+/**
+ * Copyright 2004 The Apache Software Foundation
+ *
+ * Licensed 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.index.FieldInfos;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+
+/** @lucene.experimental */
+public final class DefaultFieldsWriter extends FieldsWriter {
+  // NOTE: bit 0 is free here!  You can steal it!
+  static final int FIELD_IS_BINARY = 1 << 1;
+
+  // the old bit 1 << 2 was compressed, is now left out
+
+  private static final int _NUMERIC_BIT_SHIFT = 3;
+  static final int FIELD_IS_NUMERIC_MASK = 0x07 << _NUMERIC_BIT_SHIFT;
+
+  static final int FIELD_IS_NUMERIC_INT = 1 << _NUMERIC_BIT_SHIFT;
+  static final int FIELD_IS_NUMERIC_LONG = 2 << _NUMERIC_BIT_SHIFT;
+  static final int FIELD_IS_NUMERIC_FLOAT = 3 << _NUMERIC_BIT_SHIFT;
+  static final int FIELD_IS_NUMERIC_DOUBLE = 4 << _NUMERIC_BIT_SHIFT;
+  // currently unused: static final int FIELD_IS_NUMERIC_SHORT = 5 << _NUMERIC_BIT_SHIFT;
+  // currently unused: static final int FIELD_IS_NUMERIC_BYTE = 6 << _NUMERIC_BIT_SHIFT;
+
+  // the next possible bits are: 1 << 6; 1 << 7
+  
+  // Lucene 3.0: Removal of compressed fields
+  static final int FORMAT_LUCENE_3_0_NO_COMPRESSED_FIELDS = 2;
+
+  // Lucene 3.2: NumericFields are stored in binary format
+  static final int FORMAT_LUCENE_3_2_NUMERIC_FIELDS = 3;
+
+  // NOTE: if you introduce a new format, make it 1 higher
+  // than the current one, and always change this if you
+  // switch to a new format!
+  static final int FORMAT_CURRENT = FORMAT_LUCENE_3_2_NUMERIC_FIELDS;
+
+  // when removing support for old versions, leave the last supported version here
+  static final int FORMAT_MINIMUM = FORMAT_LUCENE_3_0_NO_COMPRESSED_FIELDS;
+
+  // If null - we were supplied with streams, if notnull - we manage them ourselves
+  private Directory directory;
+  private String segment;
+  private IndexOutput fieldsStream;
+  private IndexOutput indexStream;
+
+  DefaultFieldsWriter(Directory directory, String segment, IOContext context) throws IOException {
+    this.directory = directory;
+    this.segment = segment;
+
+    boolean success = false;
+    try {
+      fieldsStream = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_EXTENSION), context);
+      indexStream = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_INDEX_EXTENSION), context);
+
+      fieldsStream.writeInt(FORMAT_CURRENT);
+      indexStream.writeInt(FORMAT_CURRENT);
+
+      success = true;
+    } finally {
+      if (!success) {
+        abort();
+      }
+    }
+  }
+
+  DefaultFieldsWriter(IndexOutput fdx, IndexOutput fdt) {
+    directory = null;
+    segment = null;
+    fieldsStream = fdt;
+    indexStream = fdx;
+  }
+
+  void setFieldsStream(IndexOutput stream) {
+    this.fieldsStream = stream;
+  }
+
+  // Writes the contents of buffer into the fields stream
+  // and adds a new entry for this document into the index
+  // stream.  This assumes the buffer was already written
+  // in the correct fields format.
+  public void startDocument(int numStoredFields) throws IOException {
+    indexStream.writeLong(fieldsStream.getFilePointer());
+    fieldsStream.writeVInt(numStoredFields);
+  }
+
+  public void skipDocument() throws IOException {
+    indexStream.writeLong(fieldsStream.getFilePointer());
+    fieldsStream.writeVInt(0);
+  }
+
+  public void close() throws IOException {
+    if (directory != null) {
+      try {
+        IOUtils.close(fieldsStream, indexStream);
+      } finally {
+        fieldsStream = indexStream = null;
+      }
+    }
+  }
+
+  public void abort() {
+    if (directory != null) {
+      try {
+        close();
+      } catch (IOException ignored) {
+      }
+      try {
+        directory.deleteFile(IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_EXTENSION));
+      } catch (IOException ignored) {
+      }
+      try {
+        directory.deleteFile(IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_INDEX_EXTENSION));
+      } catch (IOException ignored) {
+      }
+    }
+  }
+
+  public final void writeField(int fieldNumber, IndexableField field) throws IOException {
+    fieldsStream.writeVInt(fieldNumber);
+    int bits = 0;
+    final BytesRef bytes;
+    final String string;
+    // TODO: maybe a field should serialize itself?
+    // this way we don't bake into indexer all these
+    // specific encodings for different fields?  and apps
+    // can customize...
+    if (field.numeric()) {
+      switch (field.numericDataType()) {
+        case INT:
+          bits |= FIELD_IS_NUMERIC_INT; break;
+        case LONG:
+          bits |= FIELD_IS_NUMERIC_LONG; break;
+        case FLOAT:
+          bits |= FIELD_IS_NUMERIC_FLOAT; break;
+        case DOUBLE:
+          bits |= FIELD_IS_NUMERIC_DOUBLE; break;
+        default:
+          assert false : "Should never get here";
+      }
+      string = null;
+      bytes = null;
+    } else {
+      bytes = field.binaryValue();
+      if (bytes != null) {
+        bits |= FIELD_IS_BINARY;
+        string = null;
+      } else {
+        string = field.stringValue();
+      }
+    }
+
+    fieldsStream.writeByte((byte) bits);
+
+    if (bytes != null) {
+      fieldsStream.writeVInt(bytes.length);
+      fieldsStream.writeBytes(bytes.bytes, bytes.offset, bytes.length);
+    } else if (string != null) {
+      fieldsStream.writeString(field.stringValue());
+    } else {
+      final Number n = field.numericValue();
+      if (n == null) {
+        throw new IllegalArgumentException("field " + field.name() + " is stored but does not have binaryValue, stringValue nor numericValue");
+      }
+      switch (field.numericDataType()) {
+        case INT:
+          fieldsStream.writeInt(n.intValue()); break;
+        case LONG:
+          fieldsStream.writeLong(n.longValue()); break;
+        case FLOAT:
+          fieldsStream.writeInt(Float.floatToIntBits(n.floatValue())); break;
+        case DOUBLE:
+          fieldsStream.writeLong(Double.doubleToLongBits(n.doubleValue())); break;
+        default:
+          assert false : "Should never get here";
+      }
+    }
+  }
+
+  /** Bulk write a contiguous series of documents.  The
+   *  lengths array is the length (in bytes) of each raw
+   *  document.  The stream IndexInput is the
+   *  fieldsStream from which we should bulk-copy all
+   *  bytes. */
+  public final void addRawDocuments(IndexInput stream, int[] lengths, int numDocs) throws IOException {
+    long position = fieldsStream.getFilePointer();
+    long start = position;
+    for(int i=0;i<numDocs;i++) {
+      indexStream.writeLong(position);
+      position += lengths[i];
+    }
+    fieldsStream.copyBytes(stream, position-start);
+    assert fieldsStream.getFilePointer() == position;
+  }
+
+  public final void addDocument(Iterable<? extends IndexableField> doc, FieldInfos fieldInfos) throws IOException {
+    indexStream.writeLong(fieldsStream.getFilePointer());
+
+    int storedCount = 0;
+    for (IndexableField field : doc) {
+      if (field.fieldType().stored()) {
+        storedCount++;
+      }
+    }
+    fieldsStream.writeVInt(storedCount);
+
+    for (IndexableField field : doc) {
+      if (field.fieldType().stored()) {
+        writeField(fieldInfos.fieldNumber(field.name()), field);
+      }
+    }
+  }
+}

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java?rev=1179284&r1=1179283&r2=1179284&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java Wed Oct  5 15:59:34 2011
@@ -20,7 +20,6 @@ package org.apache.lucene.index.codecs;
 import java.io.IOException;
 
 import org.apache.lucene.index.CorruptIndexException;
-import org.apache.lucene.index.FieldsReader;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.IndexFormatTooOldException;
 import org.apache.lucene.index.IndexFormatTooNewException;
@@ -79,7 +78,7 @@ public class DefaultSegmentInfosReader e
           }
 
           try {
-            FieldsReader.checkCodeVersion(dir, si.getDocStoreSegment());
+            DefaultFieldsReader.checkCodeVersion(dir, si.getDocStoreSegment());
           } finally {
             // If we opened the directory, close it
             if (dir != directory) dir.close();

Added: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/FieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/FieldsReader.java?rev=1179284&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/FieldsReader.java (added)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/FieldsReader.java Wed Oct  5 15:59:34 2011
@@ -0,0 +1,39 @@
+package org.apache.lucene.index.codecs;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.StoredFieldVisitor;
+import org.apache.lucene.store.IndexInput;
+
+/**
+ * Copyright 2004 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+public abstract class FieldsReader implements Cloneable, Closeable {
+  
+  public abstract void visitDocument(int n, StoredFieldVisitor visitor) throws CorruptIndexException, IOException;
+  
+  /** Returns the length in bytes of each raw document in a
+   *  contiguous range of length numDocs starting with
+   *  startDocID.  Returns the IndexInput (the fieldStream),
+   *  already seeked to the starting point for startDocID.*/
+  public abstract IndexInput rawDocs(int[] lengths, int startDocID, int numDocs) throws IOException;
+
+  public abstract int size();
+
+  public abstract FieldsReader clone();
+}

Added: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/FieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/FieldsWriter.java?rev=1179284&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/FieldsWriter.java (added)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/FieldsWriter.java Wed Oct  5 15:59:34 2011
@@ -0,0 +1,44 @@
+package org.apache.lucene.index.codecs;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.store.IndexInput;
+
+/**
+ * Copyright 2004 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+public abstract class FieldsWriter implements Closeable {
+
+  public abstract void addDocument(Iterable<? extends IndexableField> doc, FieldInfos fieldInfos) throws IOException;
+  
+  /** Bulk write a contiguous series of documents.  The
+   *  lengths array is the length (in bytes) of each raw
+   *  document.  The stream IndexInput is the
+   *  fieldsStream from which we should bulk-copy all
+   *  bytes. */
+  public abstract void addRawDocuments(IndexInput stream, int[] lengths, int numDocs) throws IOException;
+  
+  public abstract void startDocument(int numStoredFields) throws IOException;
+  
+  public abstract void skipDocument() throws IOException;
+  
+  public abstract void writeField(int fieldNumber, IndexableField field) throws IOException;
+
+  public abstract void abort();
+}