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/01 05:05:07 UTC

svn commit: r1177888 [3/16] - in /lucene/dev/branches/lucene2621: ./ dev-tools/eclipse/ dev-tools/idea/lucene/contrib/ dev-tools/maven/ lucene/ lucene/contrib/ lucene/contrib/demo/src/java/org/apache/lucene/demo/ lucene/contrib/demo/src/java/org/apache...

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/document/Field.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/document/Field.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/document/Field.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/document/Field.java Sat Oct  1 03:04:53 2011
@@ -60,7 +60,7 @@ public class Field implements IndexableF
     this.type = type;
   }
   
-  public Field(String name, IndexableFieldType type, Reader reader) {
+  public Field(String name, Reader reader, IndexableFieldType type) {
     if (name == null) {
       throw new NullPointerException("name cannot be null");
     }
@@ -76,7 +76,7 @@ public class Field implements IndexableF
     this.type = type;
   }
   
-  public Field(String name, IndexableFieldType type, TokenStream tokenStream) {
+  public Field(String name, TokenStream tokenStream, IndexableFieldType type) {
     if (name == null) {
       throw new NullPointerException("name cannot be null");
     }
@@ -93,15 +93,15 @@ public class Field implements IndexableF
     this.type = type;
   }
   
-  public Field(String name, IndexableFieldType type, byte[] value) {
-    this(name, type, value, 0, value.length);
+  public Field(String name, byte[] value, IndexableFieldType type) {
+    this(name, value, 0, value.length, type);
   }
 
-  public Field(String name, IndexableFieldType type, byte[] value, int offset, int length) {
-    this(name, type, new BytesRef(value, offset, length));
+  public Field(String name, byte[] value, int offset, int length, IndexableFieldType type) {
+    this(name, new BytesRef(value, offset, length), type);
   }
 
-  public Field(String name, IndexableFieldType type, BytesRef bytes) {
+  public Field(String name, BytesRef bytes, IndexableFieldType type) {
     if (type.indexed() && !type.tokenized()) {
       throw new IllegalArgumentException("Non-tokenized fields must use String values");
     }
@@ -111,7 +111,7 @@ public class Field implements IndexableF
     this.name = name;
   }
   
-  public Field(String name, IndexableFieldType type, String value) {
+  public Field(String name, String value, IndexableFieldType type) {
     if (name == null) {
       throw new IllegalArgumentException("name cannot be null");
     }
@@ -350,9 +350,9 @@ public class Field implements IndexableF
     if (tokenStream != null) {
       return tokenStream;
     } else if (readerValue() != null) {
-      return analyzer.reusableTokenStream(name(), readerValue());
+      return analyzer.tokenStream(name(), readerValue());
     } else if (stringValue() != null) {
-      return analyzer.reusableTokenStream(name(), new StringReader(stringValue()));
+      return analyzer.tokenStream(name(), new StringReader(stringValue()));
     }
 
     throw new IllegalArgumentException("Field must have either TokenStream, String or Reader value");

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/document/StringField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/document/StringField.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/document/StringField.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/document/StringField.java Sat Oct  1 03:04:53 2011
@@ -54,7 +54,7 @@ public final class StringField extends F
   
   /** Creates a new un-stored StringField */
   public StringField(String name, String value) {
-    super(name, TYPE_UNSTORED, value);
+    super(name, value, TYPE_UNSTORED);
   }
   
   @Override

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/document/TextField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/document/TextField.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/document/TextField.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/document/TextField.java Sat Oct  1 03:04:53 2011
@@ -50,16 +50,16 @@ public final class TextField extends Fie
 
   /** Creates a new un-stored TextField */
   public TextField(String name, Reader reader) {
-    super(name, TextField.TYPE_UNSTORED, reader);
+    super(name, reader, TextField.TYPE_UNSTORED);
   }
 
   /** Creates a new un-stored TextField */
   public TextField(String name, String value) {
-    super(name, TextField.TYPE_UNSTORED, value);
+    super(name, value, TextField.TYPE_UNSTORED);
   }
   
   /** Creates a new un-stored TextField */
   public TextField(String name, TokenStream stream) {
-    super(name, TextField.TYPE_UNSTORED, stream);
+    super(name, stream, TextField.TYPE_UNSTORED);
   }
 }

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java Sat Oct  1 03:04:53 2011
@@ -83,9 +83,9 @@ final class DocFieldProcessor extends Do
     // FieldInfo.storePayload.
     final String fileName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.FIELD_INFOS_EXTENSION);
     state.fieldInfos.write(state.directory, fileName);
-    for (DocValuesConsumer consumers : docValues.values()) {
-      consumers.finish(state.numDocs);
-    };
+    for (DocValuesConsumerAndDocID consumers : docValues.values()) {
+      consumers.docValuesConsumer.finish(state.numDocs);
+    }
     // close perDocConsumer during flush to ensure all files are flushed due to PerCodec CFS
     IOUtils.close(perDocConsumers.values());
   }
@@ -297,14 +297,28 @@ final class DocFieldProcessor extends Do
     }
   }
 
-  final private Map<String, DocValuesConsumer> docValues = new HashMap<String, DocValuesConsumer>();
+  private static class DocValuesConsumerAndDocID {
+    public int docID;
+    final DocValuesConsumer docValuesConsumer;
+
+    public DocValuesConsumerAndDocID(DocValuesConsumer docValuesConsumer) {
+      this.docValuesConsumer = docValuesConsumer;
+    }
+  }
+
+  final private Map<String, DocValuesConsumerAndDocID> docValues = new HashMap<String, DocValuesConsumerAndDocID>();
   final private Map<Integer, PerDocConsumer> perDocConsumers = new HashMap<Integer, PerDocConsumer>();
 
   DocValuesConsumer docValuesConsumer(DocState docState, FieldInfo fieldInfo) 
       throws IOException {
-    DocValuesConsumer docValuesConsumer = docValues.get(fieldInfo.name);
-    if (docValuesConsumer != null) {
-      return docValuesConsumer;
+    DocValuesConsumerAndDocID docValuesConsumerAndDocID = docValues.get(fieldInfo.name);
+    if (docValuesConsumerAndDocID != null) {
+      if (docState.docID == docValuesConsumerAndDocID.docID) {
+        throw new IllegalArgumentException("IndexDocValuesField \"" + fieldInfo.name + "\" appears more than once in this document (only one value is allowed, per field)");
+      }
+      assert docValuesConsumerAndDocID.docID < docState.docID;
+      docValuesConsumerAndDocID.docID = docState.docID;
+      return docValuesConsumerAndDocID.docValuesConsumer;
     }
     PerDocConsumer perDocConsumer = perDocConsumers.get(fieldInfo.getCodecId());
     if (perDocConsumer == null) {
@@ -316,6 +330,7 @@ final class DocFieldProcessor extends Do
       perDocConsumers.put(Integer.valueOf(fieldInfo.getCodecId()), perDocConsumer);
     }
     boolean success = false;
+    DocValuesConsumer docValuesConsumer = null;
     try {
       docValuesConsumer = perDocConsumer.addValuesField(fieldInfo);
       fieldInfo.commitDocValues();
@@ -325,7 +340,10 @@ final class DocFieldProcessor extends Do
         fieldInfo.revertUncommitted();
       }
     }
-    docValues.put(fieldInfo.name, docValuesConsumer);
+
+    docValuesConsumerAndDocID = new DocValuesConsumerAndDocID(docValuesConsumer);
+    docValuesConsumerAndDocID.docID = docState.docID;
+    docValues.put(fieldInfo.name, docValuesConsumerAndDocID);
     return docValuesConsumer;
   }
 }

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexReader.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexReader.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexReader.java Sat Oct  1 03:04:53 2011
@@ -27,6 +27,7 @@ import java.util.Map;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.lucene.document.Document;
+import org.apache.lucene.document.DocumentStoredFieldVisitor;
 import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.index.codecs.PerDocValues;
@@ -199,11 +200,45 @@ public abstract class IndexReader implem
    * references.
    *
    * @see #decRef
+   * @see #tryIncRef
    */
   public void incRef() {
     ensureOpen();
     refCount.incrementAndGet();
   }
+  
+  /**
+   * Expert: increments the refCount of this IndexReader
+   * instance only if the IndexReader has not been closed yet
+   * and returns <code>true</code> iff the refCount was
+   * successfully incremented, otherwise <code>false</code>.
+   * If this method returns <code>false</code> the reader is either
+   * already closed or is currently been closed. Either way this
+   * reader instance shouldn't be used by an application unless
+   * <code>true</code> is returned.
+   * <p>
+   * RefCounts are used to determine when a
+   * reader can be closed safely, i.e. as soon as there are
+   * no more references.  Be sure to always call a
+   * corresponding {@link #decRef}, in a finally clause;
+   * otherwise the reader may never be closed.  Note that
+   * {@link #close} simply calls decRef(), which means that
+   * the IndexReader will not really be closed until {@link
+   * #decRef} has been called for all outstanding
+   * references.
+   *
+   * @see #decRef
+   * @see #incRef
+   */
+  public boolean tryIncRef() {
+    int count;
+    while ((count = refCount.get()) > 0) {
+      if(refCount.compareAndSet(count, count+1)) {
+        return true;
+      }
+    }
+    return false;
+  }
 
   /** {@inheritDoc} */
   @Override

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexWriter.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexWriter.java Sat Oct  1 03:04:53 2011
@@ -1631,7 +1631,7 @@ public class IndexWriter implements Clos
    * default merge policy, but individual merge policies may implement
    * optimize in different ways.
    *
-   * <p> Optimize is a fairly costly operation, so you
+   * <p> Optimize is a very costly operation, so you
    * should only do it if your search performance really
    * requires it.  Many search applications do fine never
    * calling optimize. </p>

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/PersistentSnapshotDeletionPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/PersistentSnapshotDeletionPolicy.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/PersistentSnapshotDeletionPolicy.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/PersistentSnapshotDeletionPolicy.java Sat Oct  1 03:04:53 2011
@@ -188,12 +188,12 @@ public class PersistentSnapshotDeletionP
     Document d = new Document();
     FieldType ft = new FieldType();
     ft.setStored(true);
-    d.add(new Field(SNAPSHOTS_ID, ft, ""));
+    d.add(new Field(SNAPSHOTS_ID, "", ft));
     for (Entry<String, String> e : super.getSnapshots().entrySet()) {
-      d.add(new Field(e.getKey(), ft, e.getValue()));
+      d.add(new Field(e.getKey(), e.getValue(), ft));
     }
     if (id != null) {
-      d.add(new Field(id, ft, segment));
+      d.add(new Field(id, segment, ft));
     }
     writer.addDocument(d);
     writer.commit();

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/StoredFieldVisitor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/StoredFieldVisitor.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/StoredFieldVisitor.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/StoredFieldVisitor.java Sat Oct  1 03:04:53 2011
@@ -20,6 +20,7 @@ package org.apache.lucene.index;
 import java.io.IOException;
 
 import org.apache.lucene.document.Document;
+import org.apache.lucene.document.DocumentStoredFieldVisitor;
 import org.apache.lucene.store.IndexInput;
 
 /**

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/Codec.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/Codec.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/Codec.java Sat Oct  1 03:04:53 2011
@@ -18,7 +18,6 @@ package org.apache.lucene.index.codecs;
  */
 
 import java.io.IOException;
-import java.util.Comparator;
 import java.util.Set;
 
 import org.apache.lucene.index.PerDocWriteState;
@@ -26,7 +25,6 @@ import org.apache.lucene.index.SegmentIn
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.BytesRef;
 
 /** @lucene.experimental */
 public abstract class Codec {
@@ -34,22 +32,9 @@ public abstract class Codec {
   /** Unique name that's used to retrieve this codec when
    *  reading the index */
   public final String name;
-  protected final boolean dvUseCompoundFile;
-  protected final Comparator<BytesRef> docValuesSortComparator;
   
   protected Codec(String name) {
-    this(name, true);
-  }
-  
-  protected Codec(String name, boolean docValuesUseCompoundFile) {
-    this(name, docValuesUseCompoundFile, BytesRef.getUTF8SortedAsUnicodeComparator());
-  }
-
-  protected Codec(String name, boolean docValuesUseCompoundFile,
-      Comparator<BytesRef> docValuesSortComparator) {
     this.name = name;
-    this.dvUseCompoundFile = docValuesUseCompoundFile;
-    this.docValuesSortComparator = docValuesSortComparator;
   }
 
   /** Writes a new segment */
@@ -87,38 +72,6 @@ public abstract class Codec {
 
   /** Records all file extensions this codec uses */
   public abstract void getExtensions(Set<String> extensions);
-
-  /**
-   * Returns <code>true</code> iff compound file should be used for
-   * IndexDocValues, otherwise <code>false</code>. The default is
-   * <code>true</code>.
-   * <p>
-   * NOTE: To change the default value you need to subclass a {@link Codec} with
-   * a distinct name since this value is final and should not be changed to
-   * prevent the risk of a index corruption. This setting is private to a
-   * {@link Codec}. If you intend to change this value on an existing
-   * {@link Codec} re-indexing is required.
-   * 
-   * @return <code>true</code> iff compound file should be used for
-   *         IndexDocValues, otherwise <code>false</code>.
-   */
-  public boolean getDocValuesUseCFS() {
-    return dvUseCompoundFile;
-  }
-
-  /**
-   * Returns the {@link BytesRef} comparator for sorted IndexDocValue variants.
-   * The default is {@link BytesRef#getUTF8SortedAsUnicodeComparator()}.
-   * <p>
-   * NOTE: To change the default value you need to subclass a {@link Codec} with
-   * a distinct name since this value is final and should not be changed to
-   * prevent the risk of a index corruption. This setting is private to a
-   * {@link Codec}. If you intend to change this value on an existing
-   * {@link Codec} re-indexing is required.
-   */
-  public Comparator<BytesRef> getDocValuesSortComparator() {
-    return docValuesSortComparator;
-  }
   
   @Override
   public String toString() {

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesConsumer.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesConsumer.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesConsumer.java Sat Oct  1 03:04:53 2011
@@ -18,7 +18,6 @@ package org.apache.lucene.index.codecs;
  */
 
 import java.io.IOException;
-import java.util.Comparator;
 import java.util.Set;
 
 import org.apache.lucene.index.FieldInfo;
@@ -26,113 +25,51 @@ import org.apache.lucene.index.FieldInfo
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.PerDocWriteState;
 import org.apache.lucene.index.SegmentInfo;
-import org.apache.lucene.index.values.Writer;
+import org.apache.lucene.index.codecs.DocValuesWriterBase;
 import org.apache.lucene.store.CompoundFileDirectory;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.IOContext;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.Counter;
 
 /**
- * 
+ * Default PerDocConsumer implementation that uses compound file.
  * @lucene.experimental
  */
-public class DefaultDocValuesConsumer extends PerDocConsumer {
-  private final String segmentName;
-  private final int codecId;
+public class DefaultDocValuesConsumer extends DocValuesWriterBase {
   private final Directory directory;
-  private final Counter bytesUsed;
-  private final Comparator<BytesRef> comparator;
-  private boolean useCompoundFile;
-  private final IOContext context;
   
-  public DefaultDocValuesConsumer(PerDocWriteState state, Comparator<BytesRef> comparator, boolean useCompoundFile) throws IOException {
-    this.segmentName = state.segmentName;
-    this.codecId = state.codecId;
-    this.bytesUsed = state.bytesUsed;
-    this.context = state.context;
+  public DefaultDocValuesConsumer(PerDocWriteState state) throws IOException {
+    super(state);
     //TODO maybe we should enable a global CFS that all codecs can pull on demand to further reduce the number of files?
-    this.directory = useCompoundFile ? new CompoundFileDirectory(state.directory,
-        IndexFileNames.segmentFileName(segmentName, codecId,
-            IndexFileNames.COMPOUND_FILE_EXTENSION), context, true) : state.directory;
-    this.comparator = comparator;
-    this.useCompoundFile = useCompoundFile;
+    this.directory = new CompoundFileDirectory(state.directory,
+        IndexFileNames.segmentFileName(state.segmentName, state.codecId,
+            IndexFileNames.COMPOUND_FILE_EXTENSION), state.context, true);
   }
-
-  public void close() throws IOException {
-    if (useCompoundFile) {
-      this.directory.close();
-    }
+  
+  @Override
+  protected Directory getDirectory() {
+    return directory;
   }
 
   @Override
-  public DocValuesConsumer addValuesField(FieldInfo field) throws IOException {
-    return Writer.create(field.getDocValues(),
-        docValuesId(segmentName, codecId, field.number),
-        directory, comparator, bytesUsed, context);
+  public void close() throws IOException {
+    this.directory.close();
   }
-  
+
   @SuppressWarnings("fallthrough")
-  public static void files(Directory dir, SegmentInfo segmentInfo, int codecId,
-      Set<String> files, boolean useCompoundFile) throws IOException {
+  public static void files(Directory dir, SegmentInfo segmentInfo, int codecId, Set<String> files) throws IOException {
     FieldInfos fieldInfos = segmentInfo.getFieldInfos();
     for (FieldInfo fieldInfo : fieldInfos) {
       if (fieldInfo.getCodecId() == codecId && fieldInfo.hasDocValues()) {
-        String filename = docValuesId(segmentInfo.name, codecId,
-            fieldInfo.number);
-        if (useCompoundFile) {
-          files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecId, IndexFileNames.COMPOUND_FILE_EXTENSION));
-          files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecId, IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION));
-          assert dir.fileExists(IndexFileNames.segmentFileName(segmentInfo.name, codecId, IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION)); 
-          assert dir.fileExists(IndexFileNames.segmentFileName(segmentInfo.name, codecId, IndexFileNames.COMPOUND_FILE_EXTENSION)); 
-          return;
-        } else {
-          switch (fieldInfo.getDocValues()) {
-          case BYTES_FIXED_DEREF:
-          case BYTES_VAR_DEREF:
-          case BYTES_VAR_SORTED:
-          case BYTES_FIXED_SORTED:
-          case BYTES_VAR_STRAIGHT:
-            files.add(IndexFileNames.segmentFileName(filename, "",
-                Writer.INDEX_EXTENSION));
-            assert dir.fileExists(IndexFileNames.segmentFileName(filename, "",
-                Writer.INDEX_EXTENSION));
-            // until here all types use an index
-          case BYTES_FIXED_STRAIGHT:
-          case FLOAT_32:
-          case FLOAT_64:
-          case VAR_INTS:
-          case FIXED_INTS_16:
-          case FIXED_INTS_32:
-          case FIXED_INTS_64:
-          case FIXED_INTS_8:
-            files.add(IndexFileNames.segmentFileName(filename, "",
-                Writer.DATA_EXTENSION));
-            assert dir.fileExists(IndexFileNames.segmentFileName(filename, "",
-                Writer.DATA_EXTENSION));
-            break;
-        
-          default:
-            assert false;
-          }
-        }
+        files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecId, IndexFileNames.COMPOUND_FILE_EXTENSION));
+        files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecId, IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION));
+        assert dir.fileExists(IndexFileNames.segmentFileName(segmentInfo.name, codecId, IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION)); 
+        assert dir.fileExists(IndexFileNames.segmentFileName(segmentInfo.name, codecId, IndexFileNames.COMPOUND_FILE_EXTENSION)); 
+        return;
       }
     }
   }
   
-
-  static String docValuesId(String segmentsName, int codecID, int fieldId) {
-    return segmentsName + "_" + codecID + "-" + fieldId;
-  }
-  
-  public static void getDocValuesExtensions(Set<String> extensions, boolean useCompoundFile) {
-    if (useCompoundFile) {
-      extensions.add(IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION);
-      extensions.add(IndexFileNames.COMPOUND_FILE_EXTENSION);
-    } else {
-      extensions.add(Writer.DATA_EXTENSION);
-      extensions.add(Writer.INDEX_EXTENSION);
-    }
+  public static void getExtensions(Set<String> extensions) {
+    extensions.add(IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION);
+    extensions.add(IndexFileNames.COMPOUND_FILE_EXTENSION);
   }
-
 }

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesProducer.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesProducer.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesProducer.java Sat Oct  1 03:04:53 2011
@@ -16,184 +16,50 @@ package org.apache.lucene.index.codecs;
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 import java.io.Closeable;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Comparator;
+import java.util.Map;
 import java.util.TreeMap;
 
-import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.SegmentInfo;
-import org.apache.lucene.index.values.Bytes;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.codecs.DocValuesReaderBase;
 import org.apache.lucene.index.values.IndexDocValues;
-import org.apache.lucene.index.values.Floats;
-import org.apache.lucene.index.values.Ints;
-import org.apache.lucene.index.values.ValueType;
 import org.apache.lucene.store.CompoundFileDirectory;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.IOContext;
-import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 
 /**
- * Abstract base class for FieldsProducer implementations supporting
- * {@link IndexDocValues}.
- * 
+ * Default PerDocValues implementation that uses compound file.
  * @lucene.experimental
  */
-public class DefaultDocValuesProducer extends PerDocValues {
-
+public class DefaultDocValuesProducer extends DocValuesReaderBase {
   protected final TreeMap<String, IndexDocValues> docValues;
-  private final boolean useCompoundFile;
-  private final Closeable cfs;
-  private final Comparator<BytesRef> sortComparator;
+  private final Directory cfs;
 
   /**
-   * 
    * Creates a new {@link DefaultDocValuesProducer} instance and loads all
    * {@link IndexDocValues} instances for this segment and codec.
-   * 
-   * @param si
-   *          the segment info to load the {@link IndexDocValues} for.
-   * @param dir
-   *          the directory to load the {@link IndexDocValues} from.
-   * @param fieldInfo
-   *          the {@link FieldInfos}
-   * @param codecId
-   *          the codec ID
-   * @param useCompoundFile
-   *          if <code>true</code> this producer opens a compound file to read
-   *          IndexDocValues fields, otherwise each field defines its own set of
-   *          files.
-   * @param sortComparator
-   *          defines the sort order for sorted IndexDocValues variants
-   * @throws IOException
-   *           if an {@link IOException} occurs
    */
-  public DefaultDocValuesProducer(SegmentInfo si, Directory dir, 
-      FieldInfos fieldInfo, int codecId, boolean useCompoundFile, Comparator<BytesRef> sortComparator, IOContext context) throws IOException {
-    this.useCompoundFile = useCompoundFile;
-    this.sortComparator = sortComparator;
-    final Directory directory;
-    if (useCompoundFile) {
-      cfs = directory = new CompoundFileDirectory(dir, IndexFileNames.segmentFileName(si.name, codecId, IndexFileNames.COMPOUND_FILE_EXTENSION), context, false);
-    } else {
-      cfs = null;
-      directory = dir;
-    }
-    docValues = load(fieldInfo, si.name, si.docCount, directory, codecId, context);
-  }
-
-  /**
-   * Returns a {@link IndexDocValues} instance for the given field name or
-   * <code>null</code> if this field has no {@link IndexDocValues}.
-   */
-  @Override
-  public IndexDocValues docValues(String field) throws IOException {
-    return docValues.get(field);
-  }
-
-  // Only opens files... doesn't actually load any values
-  protected TreeMap<String, IndexDocValues> load(FieldInfos fieldInfos,
-      String segment, int docCount, Directory dir, int codecId, IOContext context)
-      throws IOException {
-    TreeMap<String, IndexDocValues> values = new TreeMap<String, IndexDocValues>();
-    boolean success = false;
-    try {
-
-      for (FieldInfo fieldInfo : fieldInfos) {
-        if (codecId == fieldInfo.getCodecId() && fieldInfo.hasDocValues()) {
-          final String field = fieldInfo.name;
-          // TODO can we have a compound file per segment and codec for
-          // docvalues?
-          final String id = DefaultDocValuesConsumer.docValuesId(segment,
-              codecId, fieldInfo.number);
-          values.put(field,
-              loadDocValues(docCount, dir, id, fieldInfo.getDocValues(), sortComparator, context));
-        }
-      }
-      success = true;
-    } finally {
-      if (!success) {
-        // if we fail we must close all opened resources if there are any
-        closeInternal(values.values());
-      }
-    }
-    return values;
+  public DefaultDocValuesProducer(SegmentReadState state) throws IOException {
+    cfs = new CompoundFileDirectory(state.dir, 
+        IndexFileNames.segmentFileName(state.segmentInfo.name, state.codecId, IndexFileNames.COMPOUND_FILE_EXTENSION), 
+        state.context, false);
+    docValues = load(state.fieldInfos, state.segmentInfo.name, state.segmentInfo.docCount, cfs, state.codecId, state.context);
   }
   
-
-  /**
-   * Loads a {@link IndexDocValues} instance depending on the given {@link ValueType}.
-   * Codecs that use different implementations for a certain {@link ValueType} can
-   * simply override this method and return their custom implementations.
-   * 
-   * @param docCount
-   *          number of documents in the segment
-   * @param dir
-   *          the {@link Directory} to load the {@link IndexDocValues} from
-   * @param id
-   *          the unique file ID within the segment
-   * @param type
-   *          the type to load
-   * @param sortComparator byte comparator used by sorted variants
-   * @return a {@link IndexDocValues} instance for the given type
-   * @throws IOException
-   *           if an {@link IOException} occurs
-   * @throws IllegalArgumentException
-   *           if the given {@link ValueType} is not supported
-   */
-  protected IndexDocValues loadDocValues(int docCount, Directory dir, String id,
-      ValueType type, Comparator<BytesRef> sortComparator, IOContext context) throws IOException {
-    switch (type) {
-    case FIXED_INTS_16:
-    case FIXED_INTS_32:
-    case FIXED_INTS_64:
-    case FIXED_INTS_8:
-    case VAR_INTS:
-      return Ints.getValues(dir, id, docCount, context);
-    case FLOAT_32:
-      return Floats.getValues(dir, id, docCount, context);
-    case FLOAT_64:
-      return Floats.getValues(dir, id, docCount, context);
-    case BYTES_FIXED_STRAIGHT:
-      return Bytes.getValues(dir, id, Bytes.Mode.STRAIGHT, true, docCount, sortComparator, context);
-    case BYTES_FIXED_DEREF:
-      return Bytes.getValues(dir, id, Bytes.Mode.DEREF, true, docCount, sortComparator, context);
-    case BYTES_FIXED_SORTED:
-      return Bytes.getValues(dir, id, Bytes.Mode.SORTED, true, docCount, sortComparator, context);
-    case BYTES_VAR_STRAIGHT:
-      return Bytes.getValues(dir, id, Bytes.Mode.STRAIGHT, false, docCount, sortComparator, context);
-    case BYTES_VAR_DEREF:
-      return Bytes.getValues(dir, id, Bytes.Mode.DEREF, false, docCount, sortComparator, context);
-    case BYTES_VAR_SORTED:
-      return Bytes.getValues(dir, id, Bytes.Mode.SORTED, false, docCount, sortComparator, context);
-    default:
-      throw new IllegalStateException("unrecognized index values mode " + type);
-    }
-  }
-
-  public void close() throws IOException {
-    closeInternal(docValues.values());
-  }
-
-  private void closeInternal(Collection<? extends Closeable> closeables) throws IOException {
-    final Collection<? extends Closeable> toClose;
-    if (useCompoundFile) {
-      final ArrayList<Closeable> list = new ArrayList<Closeable>(closeables);
-      list.add(cfs);
-      toClose = list; 
-    } else {
-      toClose = closeables;
-    } 
-    IOUtils.close(toClose);
+  @Override
+  protected Map<String,IndexDocValues> docValues() {
+    return docValues;
   }
 
   @Override
-  public Collection<String> fields() {
-    return docValues.keySet();
+  protected void closeInternal(Collection<? extends Closeable> closeables) throws IOException {
+    final ArrayList<Closeable> list = new ArrayList<Closeable>(closeables);
+    list.add(cfs);
+    IOUtils.close(list);
   }
 }

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/PerDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/PerDocValues.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/PerDocValues.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/PerDocValues.java Sat Oct  1 03:04:53 2011
@@ -51,5 +51,4 @@ public abstract class PerDocValues imple
    * Returns all fields this {@link PerDocValues} contains values for.
    */
   public abstract Collection<String> fields();
-  
 }

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/memory/MemoryCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/memory/MemoryCodec.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/memory/MemoryCodec.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/memory/MemoryCodec.java Sat Oct  1 03:04:53 2011
@@ -787,22 +787,22 @@ public class MemoryCodec extends Codec {
   @Override
   public void files(Directory dir, SegmentInfo segmentInfo, int id, Set<String> files) throws IOException {
     files.add(IndexFileNames.segmentFileName(segmentInfo.name, id, EXTENSION));
-    DefaultDocValuesConsumer.files(dir, segmentInfo, id, files, getDocValuesUseCFS());
+    DefaultDocValuesConsumer.files(dir, segmentInfo, id, files);
   }
 
   @Override
   public void getExtensions(Set<String> extensions) {
     extensions.add(EXTENSION);
-    DefaultDocValuesConsumer.getDocValuesExtensions(extensions, getDocValuesUseCFS());
+    DefaultDocValuesConsumer.getExtensions(extensions);
   }
 
   @Override
   public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
-    return new DefaultDocValuesConsumer(state, getDocValuesSortComparator(), getDocValuesUseCFS());
+    return new DefaultDocValuesConsumer(state);
   }
 
   @Override
   public PerDocValues docsProducer(SegmentReadState state) throws IOException {
-    return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, getDocValuesUseCFS(), getDocValuesSortComparator(), IOContext.READONCE);
+    return new DefaultDocValuesProducer(state);
   }
 }

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java Sat Oct  1 03:04:53 2011
@@ -134,22 +134,22 @@ public class PulsingCodec extends Codec 
   public void files(Directory dir, SegmentInfo segmentInfo, int codecID, Set<String> files) throws IOException {
     StandardPostingsReader.files(dir, segmentInfo, codecID, files);
     BlockTreeTermsReader.files(dir, segmentInfo, codecID, files);
-    DefaultDocValuesConsumer.files(dir, segmentInfo, codecID, files, getDocValuesUseCFS());
+    DefaultDocValuesConsumer.files(dir, segmentInfo, codecID, files);
   }
 
   @Override
   public void getExtensions(Set<String> extensions) {
     StandardCodec.getStandardExtensions(extensions);
-    DefaultDocValuesConsumer.getDocValuesExtensions(extensions, getDocValuesUseCFS());
+    DefaultDocValuesConsumer.getExtensions(extensions);
   }
 
   @Override
   public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
-    return new DefaultDocValuesConsumer(state, getDocValuesSortComparator(), getDocValuesUseCFS());
+    return new DefaultDocValuesConsumer(state);
   }
 
   @Override
   public PerDocValues docsProducer(SegmentReadState state) throws IOException {
-    return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, getDocValuesUseCFS(), getDocValuesSortComparator(), state.context);
+    return new DefaultDocValuesProducer(state);
   }
 }

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java Sat Oct  1 03:04:53 2011
@@ -69,23 +69,23 @@ public class SimpleTextCodec extends Cod
   @Override
   public void files(Directory dir, SegmentInfo segmentInfo, int id, Set<String> files) throws IOException {
     files.add(getPostingsFileName(segmentInfo.name, id));
-    DefaultDocValuesConsumer.files(dir, segmentInfo, id, files, getDocValuesUseCFS());
+    DefaultDocValuesConsumer.files(dir, segmentInfo, id, files);
   }
 
   @Override
   public void getExtensions(Set<String> extensions) {
     extensions.add(POSTINGS_EXTENSION);
-    DefaultDocValuesConsumer.getDocValuesExtensions(extensions, getDocValuesUseCFS());
+    DefaultDocValuesConsumer.getExtensions(extensions);
   }
   
   // TODO: would be great if these used a plain text impl
   @Override
   public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
-    return new DefaultDocValuesConsumer(state, getDocValuesSortComparator(), getDocValuesUseCFS());
+    return new DefaultDocValuesConsumer(state);
   }
 
   @Override
   public PerDocValues docsProducer(SegmentReadState state) throws IOException {
-    return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, getDocValuesUseCFS(), getDocValuesSortComparator(), state.context);
+    return new DefaultDocValuesProducer(state);
   }
 }

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java Sat Oct  1 03:04:53 2011
@@ -110,19 +110,19 @@ public class StandardCodec extends Codec
   public void files(Directory dir, SegmentInfo segmentInfo, int codecID, Set<String> files) throws IOException {
     StandardPostingsReader.files(dir, segmentInfo, codecID, files);
     BlockTreeTermsReader.files(dir, segmentInfo, codecID, files);
-    DefaultDocValuesConsumer.files(dir, segmentInfo, codecID, files, getDocValuesUseCFS());
+    DefaultDocValuesConsumer.files(dir, segmentInfo, codecID, files);
   }
 
   @Override
   public void getExtensions(Set<String> extensions) {
     getStandardExtensions(extensions);
-    DefaultDocValuesConsumer.getDocValuesExtensions(extensions, getDocValuesUseCFS());
   }
 
   public static void getStandardExtensions(Set<String> extensions) {
     extensions.add(FREQ_EXTENSION);
     extensions.add(PROX_EXTENSION);
     BlockTreeTermsReader.getExtensions(extensions);
+    DefaultDocValuesConsumer.getExtensions(extensions);
   }
 
   @Override
@@ -132,11 +132,11 @@ public class StandardCodec extends Codec
 
   @Override
   public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
-    return new DefaultDocValuesConsumer(state, getDocValuesSortComparator(), getDocValuesUseCFS());
+    return new DefaultDocValuesConsumer(state);
   }
 
   @Override
   public PerDocValues docsProducer(SegmentReadState state) throws IOException {
-    return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, getDocValuesUseCFS(), getDocValuesSortComparator(), state.context);
+    return new DefaultDocValuesProducer(state);
   }
 }

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/Bytes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/Bytes.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/Bytes.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/Bytes.java Sat Oct  1 03:04:53 2011
@@ -27,16 +27,25 @@ import org.apache.lucene.index.IndexFile
 import org.apache.lucene.index.values.IndexDocValues.SortedSource;
 import org.apache.lucene.index.values.IndexDocValues.Source;
 import org.apache.lucene.index.values.IndexDocValues.SourceEnum;
+import org.apache.lucene.store.DataOutput;
 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.ArrayUtil;
 import org.apache.lucene.util.AttributeSource;
+import org.apache.lucene.util.ByteBlockPool;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefHash;
 import org.apache.lucene.util.CodecUtil;
 import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.PagedBytes;
+import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.ByteBlockPool.Allocator;
+import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
+import org.apache.lucene.util.BytesRefHash.TrackingDirectBytesStartArray;
+import org.apache.lucene.util.packed.PackedInts;
 
 /**
  * Provides concrete Writer/Reader implementations for <tt>byte[]</tt> value per
@@ -185,16 +194,18 @@ public final class Bytes {
   }
 
   // TODO open up this API?
-  static abstract class BytesBaseSource extends Source {
+  static abstract class BytesSourceBase extends Source {
     private final PagedBytes pagedBytes;
+    private final ValueType type;
     protected final IndexInput datIn;
     protected final IndexInput idxIn;
     protected final static int PAGED_BYTES_BITS = 15;
     protected final PagedBytes.Reader data;
     protected final long totalLengthInBytes;
+    
 
-    protected BytesBaseSource(IndexInput datIn, IndexInput idxIn,
-        PagedBytes pagedBytes, long bytesToRead) throws IOException {
+    protected BytesSourceBase(IndexInput datIn, IndexInput idxIn,
+        PagedBytes pagedBytes, long bytesToRead, ValueType type) throws IOException {
       assert bytesToRead <= datIn.length() : " file size is less than the expected size diff: "
           + (bytesToRead - datIn.length()) + " pos: " + datIn.getFilePointer();
       this.datIn = datIn;
@@ -203,6 +214,7 @@ public final class Bytes {
       this.pagedBytes.copy(datIn, bytesToRead);
       data = pagedBytes.freeze(true);
       this.idxIn = idxIn;
+      this.type = type;
     }
 
     public void close() throws IOException {
@@ -220,6 +232,17 @@ public final class Bytes {
         }
       }
     }
+    
+    @Override
+    public ValueType type() {
+      return type;
+    }
+    
+
+    @Override
+    public int getValueCount() {
+      throw new UnsupportedOperationException();
+    }
 
     /**
      * Returns one greater than the largest possible document number.
@@ -245,18 +268,46 @@ public final class Bytes {
     }
 
   }
+  
+  static abstract class DerefBytesSourceBase extends BytesSourceBase {
+    protected final PackedInts.Reader addresses;
+    public DerefBytesSourceBase(IndexInput datIn, IndexInput idxIn,  long bytesToRead, ValueType type) throws IOException {
+      super(datIn, idxIn, new PagedBytes(PAGED_BYTES_BITS), bytesToRead, type);
+      addresses = PackedInts.getReader(idxIn);
+    }
+    
+    @Override
+    public int getValueCount() {
+      return addresses.size();
+    }
+    
+    @Override
+    protected int maxDoc() {
+      return addresses.size();
+    }
+
+  }
 
-  static abstract class BytesBaseSortedSource extends SortedSource {
+  static abstract class BytesSortedSourceBase extends SortedSource {
+    private final PagedBytes pagedBytes;
+    private final Comparator<BytesRef> comp;
+    protected final PackedInts.Reader docToOrdIndex;
+    private final ValueType type;
+    
     protected final IndexInput datIn;
     protected final IndexInput idxIn;
     protected final BytesRef defaultValue = new BytesRef();
     protected final static int PAGED_BYTES_BITS = 15;
-    private final PagedBytes pagedBytes;
     protected final PagedBytes.Reader data;
-    private final Comparator<BytesRef> comp;
+    
 
-    protected BytesBaseSortedSource(IndexInput datIn, IndexInput idxIn,
-        Comparator<BytesRef> comp, PagedBytes pagedBytes, long bytesToRead)
+    protected BytesSortedSourceBase(IndexInput datIn, IndexInput idxIn,
+        Comparator<BytesRef> comp, long bytesToRead, ValueType type) throws IOException {
+      this(datIn, idxIn, comp, new PagedBytes(PAGED_BYTES_BITS), bytesToRead, type);
+    }
+    
+    protected BytesSortedSourceBase(IndexInput datIn, IndexInput idxIn,
+        Comparator<BytesRef> comp, PagedBytes pagedBytes, long bytesToRead,ValueType type)
         throws IOException {
       assert bytesToRead <= datIn.length() : " file size is less than the expected size diff: "
           + (bytesToRead - datIn.length()) + " pos: " + datIn.getFilePointer();
@@ -267,8 +318,15 @@ public final class Bytes {
       this.idxIn = idxIn;
       this.comp = comp == null ? BytesRef.getUTF8SortedAsUnicodeComparator()
           : comp;
+      docToOrdIndex = PackedInts.getReader(idxIn);
+      this.type = type;
 
     }
+    
+    @Override
+    public int ord(int docID) {
+      return (int) docToOrdIndex.get(docID) -1;
+    }
 
     @Override
     public BytesRef getByOrd(int ord, BytesRef bytesRef) {
@@ -277,22 +335,15 @@ public final class Bytes {
     }
 
     protected void closeIndexInput() throws IOException {
-      try {
-        if (datIn != null) {
-          datIn.close();
-        }
-      } finally {
-        if (idxIn != null) {// if straight
-          idxIn.close();
-        }
-      }
+      IOUtils.close(datIn, idxIn);
     }
-
+    
     /**
      * Returns the largest doc id + 1 in this doc values source
      */
-    protected abstract int maxDoc();
-
+    public int maxDoc() {
+      return docToOrdIndex.size();
+    }
     /**
      * Copies the value for the given ord to the given {@link BytesRef} and
      * returns it.
@@ -336,6 +387,11 @@ public final class Bytes {
         }
       };
     }
+    
+    @Override
+    public ValueType type() {
+      return type;
+    }
   }
 
   // TODO: open up this API?!
@@ -359,7 +415,7 @@ public final class Bytes {
       this.context = context;
     }
     
-    protected IndexOutput getDataOut() throws IOException {
+    protected IndexOutput getOrCreateDataOut() throws IOException {
       if (datOut == null) {
         boolean success = false;
         try {
@@ -375,8 +431,16 @@ public final class Bytes {
       }
       return datOut;
     }
+    
+    protected IndexOutput getIndexOut() {
+      return idxOut;
+    }
+    
+    protected IndexOutput getDataOut() {
+      return datOut;
+    }
 
-    protected IndexOutput getIndexOut() throws IOException {
+    protected IndexOutput getOrCreateIndexOut() throws IOException {
       boolean success = false;
       try {
         if (idxOut == null) {
@@ -503,5 +567,223 @@ public final class Bytes {
       }
     }
   }
+  
+  static abstract class DerefBytesWriterBase extends BytesWriterBase {
+    protected int size = -1;
+    protected int[] docToEntry;
+    protected final BytesRefHash hash;
+    
+    protected DerefBytesWriterBase(Directory dir, String id, String codecName,
+        int codecVersion, Counter bytesUsed, IOContext context)
+        throws IOException {
+      this(dir, id, codecName, codecVersion, new DirectTrackingAllocator(
+          ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed), bytesUsed, context);
+    }
+
+    protected DerefBytesWriterBase(Directory dir, String id, String codecName, int codecVersion, Allocator allocator,
+        Counter bytesUsed, IOContext context) throws IOException {
+      super(dir, id, codecName, codecVersion, bytesUsed, context);
+      hash = new BytesRefHash(new ByteBlockPool(allocator),
+          BytesRefHash.DEFAULT_CAPACITY, new TrackingDirectBytesStartArray(
+              BytesRefHash.DEFAULT_CAPACITY, bytesUsed));
+      docToEntry = new int[1];
+      bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT);
+    }
+    
+    protected static int writePrefixLength(DataOutput datOut, BytesRef bytes)
+        throws IOException {
+      if (bytes.length < 128) {
+        datOut.writeByte((byte) bytes.length);
+        return 1;
+      } else {
+        datOut.writeByte((byte) (0x80 | (bytes.length >> 8)));
+        datOut.writeByte((byte) (bytes.length & 0xff));
+        return 2;
+      }
+    }
+
+    @Override
+    public void add(int docID, BytesRef bytes) throws IOException {
+      if (bytes.length == 0) { // default value - skip it
+        return;
+      }
+      checkSize(bytes);
+      int ord = hash.add(bytes);
+      if (ord < 0) {
+        ord = (-ord) - 1;
+      }
+      if (docID >= docToEntry.length) {
+        final int size = docToEntry.length;
+        docToEntry = ArrayUtil.grow(docToEntry, 1 + docID);
+        bytesUsed.addAndGet((docToEntry.length - size)
+            * RamUsageEstimator.NUM_BYTES_INT);
+      }
+      docToEntry[docID] = 1 + ord;
+    }
+    
+    protected void checkSize(BytesRef bytes) {
+      if (size == -1) {
+        size = bytes.length;
+      } else if (bytes.length != size) {
+        throw new IllegalArgumentException("expected bytes size=" + size
+            + " but got " + bytes.length);
+      }
+    }
+    
+    // Important that we get docCount, in case there were
+    // some last docs that we didn't see
+    @Override
+    public void finish(int docCount) throws IOException {
+      boolean success = false;
+      try {
+        finishInternal(docCount);
+        success = true;
+      } finally {
+        releaseResources();
+        if (success) {
+          IOUtils.close(getIndexOut(), getDataOut());
+        } else {
+          IOUtils.closeWhileHandlingException(getIndexOut(), getDataOut());
+        }
+        
+      }
+    }
+    
+    protected abstract void finishInternal(int docCount) throws IOException;
+    
+    protected void releaseResources() {
+      hash.close();
+      bytesUsed
+      .addAndGet((-docToEntry.length) * RamUsageEstimator.NUM_BYTES_INT);
+      docToEntry = null;
+    }
+    
+    protected void writeIndex(IndexOutput idxOut, int docCount,
+        long maxValue, int[] toEntry) throws IOException {
+      writeIndex(idxOut, docCount, maxValue, (int[])null, toEntry);
+    }
+    
+    protected void writeIndex(IndexOutput idxOut, int docCount,
+        long maxValue, int[] addresses, int[] toEntry) throws IOException {
+      final PackedInts.Writer w = PackedInts.getWriter(idxOut, docCount,
+          PackedInts.bitsRequired(maxValue));
+      final int limit = docCount > docToEntry.length ? docToEntry.length
+          : docCount;
+      assert toEntry.length >= limit -1;
+      if (addresses != null) {
+        for (int i = 0; i < limit; i++) {
+          assert addresses[toEntry[i]] >= 0;
+          w.add(addresses[toEntry[i]]);
+        }
+      } else {
+        for (int i = 0; i < limit; i++) {
+          assert toEntry[i] >= 0;
+          w.add(toEntry[i]);
+        }
+      }
+      for (int i = limit; i < docCount; i++) {
+        w.add(0);
+      }
+      w.finish();
+    }
+    
+    protected void writeIndex(IndexOutput idxOut, int docCount,
+        long maxValue, long[] addresses, int[] toEntry) throws IOException {
+      final PackedInts.Writer w = PackedInts.getWriter(idxOut, docCount,
+          PackedInts.bitsRequired(maxValue));
+      final int limit = docCount > docToEntry.length ? docToEntry.length
+          : docCount;
+      assert toEntry.length >= limit -1;
+      if (addresses != null) {
+        for (int i = 0; i < limit; i++) {
+          assert addresses[toEntry[i]] >= 0;
+          w.add(addresses[toEntry[i]]);
+        }
+      } else {
+        for (int i = 0; i < limit; i++) {
+          assert toEntry[i] >= 0;
+          w.add(toEntry[i]);
+        }
+      }
+      for (int i = limit; i < docCount; i++) {
+        w.add(0);
+      }
+      w.finish();
+    }
+    
+  }
+  
+  abstract static class DerefBytesEnumBase extends ValuesEnum {
+    private final PackedInts.ReaderIterator idx;
+    private final int valueCount;
+    private int pos = -1;
+    protected final IndexInput datIn;
+    protected final long fp;
+    protected final int size;
+
+    protected DerefBytesEnumBase(AttributeSource source, IndexInput datIn,
+        IndexInput idxIn, int size, ValueType enumType) throws IOException {
+      super(source, enumType);
+      this.datIn = datIn;
+      this.size = size;
+      idx = PackedInts.getReaderIterator(idxIn);
+      fp = datIn.getFilePointer();
+      if (size > 0) {
+        bytesRef.grow(this.size);
+        bytesRef.length = this.size;
+      }
+      bytesRef.offset = 0;
+      valueCount = idx.size();
+    }
+
+    protected void copyFrom(ValuesEnum valuesEnum) {
+      bytesRef = valuesEnum.bytesRef;
+      if (bytesRef.bytes.length < size) {
+        bytesRef.grow(size);
+      }
+      bytesRef.length = size;
+      bytesRef.offset = 0;
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      if (target < valueCount) {
+        long address;
+        while ((address = idx.advance(target)) == 0) {
+          if (++target >= valueCount) {
+            return pos = NO_MORE_DOCS;
+          }
+        }
+        pos = idx.ord();
+        fill(address, bytesRef);
+        return pos;
+      }
+      return pos = NO_MORE_DOCS;
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      if (pos >= valueCount) {
+        return pos = NO_MORE_DOCS;
+      }
+      return advance(pos + 1);
+    }
+
+    public void close() throws IOException {
+      try {
+        datIn.close();
+      } finally {
+        idx.close();
+      }
+    }
+
+    protected abstract void fill(long address, BytesRef ref) throws IOException;
+
+    @Override
+    public int docID() {
+      return pos;
+    }
+
+  }
 
 }
\ No newline at end of file

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java Sat Oct  1 03:04:53 2011
@@ -19,26 +19,17 @@ package org.apache.lucene.index.values;
 
 import java.io.IOException;
 
-import org.apache.lucene.index.values.Bytes.BytesBaseSource;
 import org.apache.lucene.index.values.Bytes.BytesReaderBase;
-import org.apache.lucene.index.values.Bytes.BytesWriterBase;
+import org.apache.lucene.index.values.Bytes.DerefBytesSourceBase;
+import org.apache.lucene.index.values.Bytes.DerefBytesEnumBase;
+import org.apache.lucene.index.values.Bytes.DerefBytesWriterBase;
 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.ArrayUtil;
 import org.apache.lucene.util.AttributeSource;
-import org.apache.lucene.util.ByteBlockPool;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefHash;
 import org.apache.lucene.util.Counter;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.PagedBytes;
-import org.apache.lucene.util.RamUsageEstimator;
-import org.apache.lucene.util.ByteBlockPool.Allocator;
-import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
-import org.apache.lucene.util.BytesRefHash.TrackingDirectBytesStartArray;
-import org.apache.lucene.util.packed.PackedInts;
 
 // Stores fixed-length byte[] by deref, ie when two docs
 // have the same value, they store only 1 byte[]
@@ -51,135 +42,55 @@ class FixedDerefBytesImpl {
   static final int VERSION_START = 0;
   static final int VERSION_CURRENT = VERSION_START;
 
-  static class Writer extends BytesWriterBase {
-    private int size = -1;
-    private int[] docToID;
-    private final BytesRefHash hash;
+  public static class Writer extends DerefBytesWriterBase {
     public Writer(Directory dir, String id, Counter bytesUsed, IOContext context)
         throws IOException {
-      this(dir, id, new DirectTrackingAllocator(ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed),
-          bytesUsed, context);
-    }
-
-    public Writer(Directory dir, String id, Allocator allocator,
-        Counter bytesUsed, IOContext context) throws IOException {
       super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context);
-      hash = new BytesRefHash(new ByteBlockPool(allocator),
-          BytesRefHash.DEFAULT_CAPACITY, new TrackingDirectBytesStartArray(
-              BytesRefHash.DEFAULT_CAPACITY, bytesUsed));
-      docToID = new int[1];
-      bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT);
-    }
-
-    @Override
-    public void add(int docID, BytesRef bytes) throws IOException {
-      if (bytes.length == 0) // default value - skip it
-        return;
-      if (size == -1) {
-        size = bytes.length;
-      } else if (bytes.length != size) {
-        throw new IllegalArgumentException("expected bytes size=" + size
-            + " but got " + bytes.length);
-      }
-      int ord = hash.add(bytes);
-      if (ord < 0) {
-        ord = (-ord) - 1;
-      }
-      if (docID >= docToID.length) {
-        final int size = docToID.length;
-        docToID = ArrayUtil.grow(docToID, 1 + docID);
-        bytesUsed.addAndGet((docToID.length - size)
-            * RamUsageEstimator.NUM_BYTES_INT);
-      }
-      docToID[docID] = 1 + ord;
     }
 
-    // Important that we get docCount, in case there were
-    // some last docs that we didn't see
     @Override
-    public void finish(int docCount) throws IOException {
-      boolean success = false;
+    protected void finishInternal(int docCount) throws IOException {
       final int numValues = hash.size();
-      final IndexOutput datOut = getDataOut();
-      try {
-        datOut.writeInt(size);
-        if (size != -1) {
-          final BytesRef bytesRef = new BytesRef(size);
-          for (int i = 0; i < numValues; i++) {
-            hash.get(i, bytesRef);
-            datOut.writeBytes(bytesRef.bytes, bytesRef.offset, bytesRef.length);
-          }
-        }
-        success = true;
-      } finally {
-        if (success) {
-          IOUtils.close(datOut);
-        } else {
-          IOUtils.closeWhileHandlingException(datOut);
+      final IndexOutput datOut = getOrCreateDataOut();
+      datOut.writeInt(size);
+      if (size != -1) {
+        final BytesRef bytesRef = new BytesRef(size);
+        for (int i = 0; i < numValues; i++) {
+          hash.get(i, bytesRef);
+          datOut.writeBytes(bytesRef.bytes, bytesRef.offset, bytesRef.length);
         }
-        hash.close();
-      }
-      success = false;
-      final IndexOutput idxOut = getIndexOut();
-      try {
-        final int count = 1 + numValues;
-        idxOut.writeInt(count - 1);
-        // write index
-        final PackedInts.Writer w = PackedInts.getWriter(idxOut, docCount,
-            PackedInts.bitsRequired(count - 1));
-        final int limit = docCount > docToID.length ? docToID.length : docCount;
-        for (int i = 0; i < limit; i++) {
-          w.add(docToID[i]);
-        }
-        // fill up remaining doc with zeros
-        for (int i = limit; i < docCount; i++) {
-          w.add(0);
-        }
-        w.finish();
-        success = true;
-      } finally {
-        if (success) {
-          IOUtils.close(idxOut);
-        } else {
-          IOUtils.closeWhileHandlingException(idxOut);
-        }
-        bytesUsed
-            .addAndGet((-docToID.length) * RamUsageEstimator.NUM_BYTES_INT);
-        docToID = null;
       }
+      final IndexOutput idxOut = getOrCreateIndexOut();
+      idxOut.writeInt(numValues);
+      writeIndex(idxOut, docCount, numValues, docToEntry);
     }
   }
 
   public static class Reader extends BytesReaderBase {
     private final int size;
-
+    private final int numValuesStored;
     Reader(Directory dir, String id, int maxDoc, IOContext context) throws IOException {
       super(dir, id, CODEC_NAME, VERSION_START, true, context);
       size = datIn.readInt();
+      numValuesStored = idxIn.readInt();
     }
 
     @Override
     public Source load() throws IOException {
-      final IndexInput index = cloneIndex();
-      return new Source(cloneData(), index, size, index.readInt());
+      return new Source(cloneData(), cloneIndex(), size, numValuesStored);
     }
 
-    private static class Source extends BytesBaseSource {
-      private final PackedInts.Reader index;
+    private static final class Source extends DerefBytesSourceBase {
       private final int size;
-      private final int numValues;
 
-      protected Source(IndexInput datIn, IndexInput idxIn, int size,
-          int numValues) throws IOException {
-        super(datIn, idxIn, new PagedBytes(PAGED_BYTES_BITS), size * numValues);
+      protected Source(IndexInput datIn, IndexInput idxIn, int size, long numValues) throws IOException {
+        super(datIn, idxIn, size * numValues, ValueType.BYTES_FIXED_DEREF);
         this.size = size;
-        this.numValues = numValues;
-        index = PackedInts.getReader(idxIn);
       }
 
       @Override
       public BytesRef getBytes(int docID, BytesRef bytesRef) {
-        final int id = (int) index.get(docID);
+        final int id = (int) addresses.get(docID);
         if (id == 0) {
           bytesRef.length = 0;
           return bytesRef;
@@ -187,95 +98,18 @@ class FixedDerefBytesImpl {
         return data.fillSlice(bytesRef, ((id - 1) * size), size);
       }
 
-      @Override
-      public int getValueCount() {
-        return numValues;
-      }
-
-      @Override
-      public ValueType type() {
-        return ValueType.BYTES_FIXED_DEREF;
-      }
-
-      @Override
-      protected int maxDoc() {
-        return index.size();
-      }
     }
 
     @Override
     public ValuesEnum getEnum(AttributeSource source) throws IOException {
-      return new DerefBytesEnum(source, cloneData(), cloneIndex(), size);
+        return new DerefBytesEnum(source, cloneData(), cloneIndex(), size);
     }
 
-    static class DerefBytesEnum extends ValuesEnum {
-      protected final IndexInput datIn;
-      private final PackedInts.ReaderIterator idx;
-      protected final long fp;
-      private final int size;
-      private final int valueCount;
-      private int pos = -1;
+    final static class DerefBytesEnum extends DerefBytesEnumBase {
 
       public DerefBytesEnum(AttributeSource source, IndexInput datIn,
           IndexInput idxIn, int size) throws IOException {
-        this(source, datIn, idxIn, size, ValueType.BYTES_FIXED_DEREF);
-      }
-
-      protected DerefBytesEnum(AttributeSource source, IndexInput datIn,
-          IndexInput idxIn, int size, ValueType enumType) throws IOException {
-        super(source, enumType);
-        this.datIn = datIn;
-        this.size = size;
-        idxIn.readInt();// read valueCount
-        idx = PackedInts.getReaderIterator(idxIn);
-        fp = datIn.getFilePointer();
-        if (size > 0) {
-          bytesRef.grow(this.size);
-          bytesRef.length = this.size;
-        }
-        bytesRef.offset = 0;
-        valueCount = idx.size();
-      }
-
-      protected void copyFrom(ValuesEnum valuesEnum) {
-        bytesRef = valuesEnum.bytesRef;
-        if (bytesRef.bytes.length < size) {
-          bytesRef.grow(size);
-        }
-        bytesRef.length = size;
-        bytesRef.offset = 0;
-      }
-
-      @Override
-      public int advance(int target) throws IOException {
-        if (target < valueCount) {
-          long address;
-          while ((address = idx.advance(target)) == 0) {
-            if (++target >= valueCount) {
-              return pos = NO_MORE_DOCS;
-            }
-          }
-          pos = idx.ord();
-          fill(address, bytesRef);
-          return pos;
-        }
-        return pos = NO_MORE_DOCS;
-      }
-
-      @Override
-      public int nextDoc() throws IOException {
-        if (pos >= valueCount) {
-          return pos = NO_MORE_DOCS;
-        }
-        return advance(pos + 1);
-      }
-
-      public void close() throws IOException {
-        try {
-          datIn.close();
-        } finally {
-          idx.close();
-        }
+        super(source, datIn, idxIn, size, ValueType.BYTES_FIXED_DEREF);
       }
 
       protected void fill(long address, BytesRef ref) throws IOException {
@@ -284,12 +118,6 @@ class FixedDerefBytesImpl {
         ref.length = size;
         ref.offset = 0;
       }
-
-      @Override
-      public int docID() {
-        return pos;
-      }
-
     }
 
     @Override

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java Sat Oct  1 03:04:53 2011
@@ -20,28 +20,17 @@ package org.apache.lucene.index.values;
 import java.io.IOException;
 import java.util.Comparator;
 
-import org.apache.lucene.index.values.Bytes.BytesBaseSortedSource;
+import org.apache.lucene.index.values.Bytes.BytesSortedSourceBase;
 import org.apache.lucene.index.values.Bytes.BytesReaderBase;
-import org.apache.lucene.index.values.Bytes.BytesWriterBase;
+import org.apache.lucene.index.values.Bytes.DerefBytesWriterBase;
 import org.apache.lucene.index.values.FixedDerefBytesImpl.Reader.DerefBytesEnum;
 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.ArrayUtil;
 import org.apache.lucene.util.AttributeSource;
-import org.apache.lucene.util.ByteBlockPool;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefHash;
-import org.apache.lucene.util.CodecUtil;
 import org.apache.lucene.util.Counter;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.PagedBytes;
-import org.apache.lucene.util.RamUsageEstimator;
-import org.apache.lucene.util.ByteBlockPool.Allocator;
-import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
-import org.apache.lucene.util.BytesRefHash.TrackingDirectBytesStartArray;
-import org.apache.lucene.util.packed.PackedInts;
 
 // Stores fixed-length byte[] by deref, ie when two docs
 // have the same value, they store only 1 byte[]
@@ -55,132 +44,49 @@ class FixedSortedBytesImpl {
   static final int VERSION_START = 0;
   static final int VERSION_CURRENT = VERSION_START;
 
-  static class Writer extends BytesWriterBase {
-    private int size = -1;
-    private int[] docToEntry;
+  static class Writer extends DerefBytesWriterBase {
     private final Comparator<BytesRef> comp;
-    private final BytesRefHash hash;
 
     public Writer(Directory dir, String id, Comparator<BytesRef> comp,
         Counter bytesUsed, IOContext context) throws IOException {
-      this(dir, id, comp, new DirectTrackingAllocator(ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed),
-          bytesUsed, context);
-    }
-
-    public Writer(Directory dir, String id, Comparator<BytesRef> comp,
-        Allocator allocator, Counter bytesUsed, IOContext context) throws IOException {
       super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context);
-      ByteBlockPool pool = new ByteBlockPool(allocator);
-      hash = new BytesRefHash(pool, BytesRefHash.DEFAULT_CAPACITY,
-          new TrackingDirectBytesStartArray(BytesRefHash.DEFAULT_CAPACITY,
-              bytesUsed));
-      docToEntry = new int[1];
-      bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT);
       this.comp = comp;
     }
 
-    @Override
-    public void add(int docID, BytesRef bytes) throws IOException {
-      if (bytes.length == 0)
-        return; // default - skip it
-      if (size == -1) {
-        size = bytes.length;
-      } else if (bytes.length != size) {
-        throw new IllegalArgumentException("expected bytes size=" + size
-            + " but got " + bytes.length);
-      }
-      if (docID >= docToEntry.length) {
-        final int[] newArray = new int[ArrayUtil.oversize(1 + docID,
-            RamUsageEstimator.NUM_BYTES_INT)];
-        System.arraycopy(docToEntry, 0, newArray, 0, docToEntry.length);
-        bytesUsed.addAndGet((newArray.length - docToEntry.length)
-            * RamUsageEstimator.NUM_BYTES_INT);
-        docToEntry = newArray;
-      }
-      int e = hash.add(bytes);
-      docToEntry[docID] = 1 + (e < 0 ? (-e) - 1 : e);
-    }
-
     // Important that we get docCount, in case there were
     // some last docs that we didn't see
     @Override
-    public void finish(int docCount) throws IOException {
-      final IndexOutput datOut = getDataOut();
-      boolean success = false;
+    public void finishInternal(int docCount) throws IOException {
+      final IndexOutput datOut = getOrCreateDataOut();
       final int count = hash.size();
-      final int[] address = new int[count];
-
-      try {
-        datOut.writeInt(size);
-        if (size != -1) {
-          final int[] sortedEntries = hash.sort(comp);
-          // first dump bytes data, recording address as we go
-          final BytesRef bytesRef = new BytesRef(size);
-          for (int i = 0; i < count; i++) {
-            final int e = sortedEntries[i];
-            final BytesRef bytes = hash.get(e, bytesRef);
-            assert bytes.length == size;
-            datOut.writeBytes(bytes.bytes, bytes.offset, bytes.length);
-            address[e] = 1 + i;
-          }
-        }
-        success = true;
-      } finally {
-        if (success) {
-          IOUtils.close(datOut);
-        } else {
-          IOUtils.closeWhileHandlingException(datOut);
-        }
-        hash.close();
-      }
-      final IndexOutput idxOut = getIndexOut();
-      success = false;
-      try {
-        idxOut.writeInt(count);
-        // next write index
-        final PackedInts.Writer w = PackedInts.getWriter(idxOut, docCount,
-            PackedInts.bitsRequired(count));
-        final int limit;
-        if (docCount > docToEntry.length) {
-          limit = docToEntry.length;
-        } else {
-          limit = docCount;
-        }
-        for (int i = 0; i < limit; i++) {
-          final int e = docToEntry[i];
-          if (e == 0) {
-            // null is encoded as zero
-            w.add(0);
-          } else {
-            assert e > 0 && e <= count : "index must  0 > && <= " + count
-                + " was: " + e;
-            w.add(address[e - 1]);
-          }
-        }
-
-        for (int i = limit; i < docCount; i++) {
-          w.add(0);
-        }
-        w.finish();
-      } finally {
-        if (success) {
-          IOUtils.close(idxOut);
-        } else {
-          IOUtils.closeWhileHandlingException(idxOut);
-        }
-        bytesUsed.addAndGet((-docToEntry.length)
-            * RamUsageEstimator.NUM_BYTES_INT);
-        docToEntry = null;
-      }
+      final int[] address = new int[count+1]; // addr 0 is default values
+      datOut.writeInt(size);
+      if (size != -1) {
+        final int[] sortedEntries = hash.sort(comp);
+        // first dump bytes data, recording address as we go
+        final BytesRef bytesRef = new BytesRef(size);
+        for (int i = 0; i < count; i++) {
+          final int e = sortedEntries[i];
+          final BytesRef bytes = hash.get(e, bytesRef);
+          assert bytes.length == size;
+          datOut.writeBytes(bytes.bytes, bytes.offset, bytes.length);
+          address[e + 1] = 1 + i;
+        }
+      }
+      final IndexOutput idxOut = getOrCreateIndexOut();
+      idxOut.writeInt(count);
+      writeIndex(idxOut, docCount, count, address, docToEntry);
     }
   }
 
   public static class Reader extends BytesReaderBase {
     private final int size;
+    private final int numValuesStored;
 
     public Reader(Directory dir, String id, int maxDoc, IOContext context) throws IOException {
       super(dir, id, CODEC_NAME, VERSION_START, true, context);
       size = datIn.readInt();
+      numValuesStored = idxIn.readInt();
     }
 
     @Override
@@ -192,58 +98,35 @@ class FixedSortedBytesImpl {
     @Override
     public SortedSource loadSorted(Comparator<BytesRef> comp)
         throws IOException {
-      final IndexInput idxInput = cloneIndex();
-      final IndexInput datInput = cloneData();
-      datInput.seek(CodecUtil.headerLength(CODEC_NAME) + 4);
-      idxInput.seek(CodecUtil.headerLength(CODEC_NAME));
-      return new Source(datInput, idxInput, size, idxInput.readInt(), comp);
+      return new Source(cloneData(), cloneIndex(), size, numValuesStored, comp);
     }
 
-    private static class Source extends BytesBaseSortedSource {
-
-      private final PackedInts.Reader index;
-      private final int numValue;
+    private static class Source extends BytesSortedSourceBase {
+      private final int valueCount;
       private final int size;
 
       public Source(IndexInput datIn, IndexInput idxIn, int size,
           int numValues, Comparator<BytesRef> comp) throws IOException {
-        super(datIn, idxIn, comp, new PagedBytes(PAGED_BYTES_BITS), size
-            * numValues);
+        super(datIn, idxIn, comp, size * numValues, ValueType.BYTES_FIXED_SORTED);
         this.size = size;
-        this.numValue = numValues;
-        index = PackedInts.getReader(idxIn);
+        this.valueCount = numValues;
         closeIndexInput();
       }
 
       @Override
-      public int ord(int docID) {
-        return (int) index.get(docID) -1;
-      }
-
-      @Override
       public int getByValue(BytesRef bytes, BytesRef tmpRef) {
-        return binarySearch(bytes, tmpRef, 0, numValue - 1);
+        return binarySearch(bytes, tmpRef, 0, valueCount - 1);
       }
 
       @Override
       public int getValueCount() {
-        return numValue;
+        return valueCount;
       }
 
       @Override
       protected BytesRef deref(int ord, BytesRef bytesRef) {
         return data.fillSlice(bytesRef, (ord * size), size);
       }
-
-      @Override
-      public ValueType type() {
-        return ValueType.BYTES_FIXED_SORTED;
-      }
-
-      @Override
-      protected int maxDoc() {
-        return index.size();
-      }
     }
 
     @Override

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java Sat Oct  1 03:04:53 2011
@@ -21,7 +21,7 @@ import static org.apache.lucene.util.Byt
 
 import java.io.IOException;
 
-import org.apache.lucene.index.values.Bytes.BytesBaseSource;
+import org.apache.lucene.index.values.Bytes.BytesSourceBase;
 import org.apache.lucene.index.values.Bytes.BytesReaderBase;
 import org.apache.lucene.index.values.Bytes.BytesWriterBase;
 import org.apache.lucene.store.Directory;
@@ -46,26 +46,24 @@ class FixedStraightBytesImpl {
   static final String CODEC_NAME = "FixedStraightBytes";
   static final int VERSION_START = 0;
   static final int VERSION_CURRENT = VERSION_START;
-
-  static class Writer extends BytesWriterBase {
-    private int size = -1;
+  
+  static abstract class FixedBytesWriterBase extends BytesWriterBase {
+    protected int lastDocID = -1;
     // start at -1 if the first added value is > 0
-    private int lastDocID = -1;
+    protected int size = -1;
+    private final int byteBlockSize = BYTE_BLOCK_SIZE;
     private final ByteBlockPool pool;
-    private boolean merge;
-    private final int byteBlockSize;
-    private IndexOutput datOut;
 
-    public Writer(Directory dir, String id, Counter bytesUsed, IOContext context) throws IOException {
-      super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context);
+    protected FixedBytesWriterBase(Directory dir, String id, String codecName,
+        int version, Counter bytesUsed, IOContext context) throws IOException {
+      super(dir, id, codecName, version, bytesUsed, context);
       pool = new ByteBlockPool(new DirectTrackingAllocator(bytesUsed));
-      byteBlockSize = BYTE_BLOCK_SIZE;
     }
-
+    
     @Override
     public void add(int docID, BytesRef bytes) throws IOException {
       assert lastDocID < docID;
-      assert !merge;
+
       if (size == -1) {
         if (bytes.length > BYTE_BLOCK_SIZE) {
           throw new IllegalArgumentException("bytes arrays > " + Short.MAX_VALUE + " are not supported");
@@ -84,7 +82,6 @@ class FixedStraightBytesImpl {
     }
     
     private final void advancePool(int docID) {
-      assert !merge;
       long numBytes = (docID - (lastDocID+1))*size;
       while(numBytes > 0) {
         if (numBytes + pool.byteUpto < byteBlockSize) {
@@ -97,14 +94,50 @@ class FixedStraightBytesImpl {
       }
       assert numBytes == 0;
     }
+    
+    protected void set(BytesRef ref, int docId) {
+      assert BYTE_BLOCK_SIZE % size == 0 : "BYTE_BLOCK_SIZE ("+ BYTE_BLOCK_SIZE + ") must be a multiple of the size: " + size;
+      ref.offset = docId*size;
+      ref.length = size;
+      pool.deref(ref);
+    }
+    
+    protected void resetPool() {
+      pool.dropBuffersAndReset();
+    }
+    
+    protected void writeData(IndexOutput out) throws IOException {
+      pool.writePool(out);
+    }
+    
+    protected void writeZeros(int num, IndexOutput out) throws IOException {
+      final byte[] zeros = new byte[size];
+      for (int i = 0; i < num; i++) {
+        out.writeBytes(zeros, zeros.length);
+      }
+    }
+  }
+
+  static class Writer extends FixedBytesWriterBase {
+    private boolean merge;
+    private IndexOutput datOut;
+    
+    public Writer(Directory dir, String id, Counter bytesUsed, IOContext context) throws IOException {
+      super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context);
+    }
+
+    public Writer(Directory dir, String id, String codecName, int version, Counter bytesUsed, IOContext context) throws IOException {
+      super(dir, id, codecName, version, bytesUsed, context);
+    }
+
 
     @Override
     protected void merge(MergeState state) throws IOException {
       merge = true;
-      datOut = getDataOut();
+      datOut = getOrCreateDataOut();
       boolean success = false;
       try {
-        if (state.liveDocs == null && state.reader instanceof Reader) {
+        if (state.liveDocs == null && state.reader instanceof Reader ) {
           Reader reader = (Reader) state.reader;
           final int maxDocs = reader.maxDoc;
           if (maxDocs == 0) {
@@ -113,7 +146,10 @@ class FixedStraightBytesImpl {
           if (size == -1) {
             size = reader.size;
             datOut.writeInt(size);
-          }
+          } else if (size != reader.size) {
+            throw new IllegalArgumentException("expected bytes size=" + size
+                + " but got " + reader.size);
+           }
           if (lastDocID+1 < state.docBase) {
             fill(datOut, state.docBase);
             lastDocID = state.docBase-1;
@@ -137,7 +173,7 @@ class FixedStraightBytesImpl {
         }
       }
     }
-
+    
     @Override
     protected void mergeDoc(int docID) throws IOException {
       assert lastDocID < docID;
@@ -158,11 +194,7 @@ class FixedStraightBytesImpl {
     // Fills up to but not including this docID
     private void fill(IndexOutput datOut, int docID) throws IOException {
       assert size >= 0;
-      final long numBytes = (docID - (lastDocID+1))*size;
-      final byte zero = 0;
-      for (long i = 0; i < numBytes; i++) {
-        datOut.writeByte(zero);
-      }
+      writeZeros((docID - (lastDocID+1)), datOut);
     }
 
     @Override
@@ -172,12 +204,12 @@ class FixedStraightBytesImpl {
         if (!merge) {
           // indexing path - no disk IO until here
           assert datOut == null;
-          datOut = getDataOut();
+          datOut = getOrCreateDataOut();
           if (size == -1) {
             datOut.writeInt(0);
           } else {
             datOut.writeInt(size);
-            pool.writePool(datOut);
+            writeData(datOut);
           }
           if (lastDocID + 1 < docCount) {
             fill(datOut, docCount);
@@ -193,7 +225,7 @@ class FixedStraightBytesImpl {
         }
         success = true;
       } finally {
-        pool.dropBuffersAndReset();
+        resetPool();
         if (success) {
           IOUtils.close(datOut);
         } else {
@@ -201,14 +233,19 @@ class FixedStraightBytesImpl {
         }
       }
     }
+  
   }
   
   public static class Reader extends BytesReaderBase {
-    private final int size;
-    private final int maxDoc;
-
+    protected final int size;
+    protected final int maxDoc;
+    
     Reader(Directory dir, String id, int maxDoc, IOContext context) throws IOException {
-      super(dir, id, CODEC_NAME, VERSION_START, false, context);
+      this(dir, id, CODEC_NAME, VERSION_CURRENT, maxDoc, context);
+    }
+
+    protected Reader(Directory dir, String id, String codec, int version, int maxDoc, IOContext context) throws IOException {
+      super(dir, id, codec, version, false, context);
       size = datIn.readInt();
       this.maxDoc = maxDoc;
     }
@@ -271,13 +308,13 @@ class FixedStraightBytesImpl {
 
     }
 
-    private static class StraightBytesSource extends BytesBaseSource {
+    private final static class StraightBytesSource extends BytesSourceBase {
       private final int size;
       private final int maxDoc;
 
       public StraightBytesSource(IndexInput datIn, int size, int maxDoc)
           throws IOException {
-        super(datIn, null, new PagedBytes(PAGED_BYTES_BITS), size * maxDoc);
+        super(datIn, null, new PagedBytes(PAGED_BYTES_BITS), size * maxDoc, ValueType.BYTES_FIXED_STRAIGHT);
         this.size = size;
         this.maxDoc = maxDoc;
       }
@@ -293,11 +330,6 @@ class FixedStraightBytesImpl {
       }
 
       @Override
-      public ValueType type() {
-        return ValueType.BYTES_FIXED_STRAIGHT;
-      }
-
-      @Override
       protected int maxDoc() {
         return maxDoc;
       }
@@ -308,66 +340,68 @@ class FixedStraightBytesImpl {
       return new FixedStraightBytesEnum(source, cloneData(), size, maxDoc);
     }
 
-    private static final class FixedStraightBytesEnum extends ValuesEnum {
-      private final IndexInput datIn;
-      private final int size;
-      private final int maxDoc;
-      private int pos = -1;
-      private final long fp;
+   
 
-      public FixedStraightBytesEnum(AttributeSource source, IndexInput datIn,
-          int size, int maxDoc) throws IOException {
-        super(source, ValueType.BYTES_FIXED_STRAIGHT);
-        this.datIn = datIn;
-        this.size = size;
-        this.maxDoc = maxDoc;
-        bytesRef.grow(size);
-        bytesRef.length = size;
-        bytesRef.offset = 0;
-        fp = datIn.getFilePointer();
-      }
+    @Override
+    public ValueType type() {
+      return ValueType.BYTES_FIXED_STRAIGHT;
+    }
+  }
+  
+  static class FixedStraightBytesEnum extends ValuesEnum {
+    private final IndexInput datIn;
+    private final int size;
+    private final int maxDoc;
+    private int pos = -1;
+    private final long fp;
 
-      protected void copyFrom(ValuesEnum valuesEnum) {
-        bytesRef = valuesEnum.bytesRef;
-        if (bytesRef.bytes.length < size) {
-          bytesRef.grow(size);
-        }
-        bytesRef.length = size;
-        bytesRef.offset = 0;
-      }
+    public FixedStraightBytesEnum(AttributeSource source, IndexInput datIn,
+        int size, int maxDoc) throws IOException {
+      super(source, ValueType.BYTES_FIXED_STRAIGHT);
+      this.datIn = datIn;
+      this.size = size;
+      this.maxDoc = maxDoc;
+      bytesRef.grow(size);
+      bytesRef.length = size;
+      bytesRef.offset = 0;
+      fp = datIn.getFilePointer();
+    }
 
-      public void close() throws IOException {
-        datIn.close();
+    protected void copyFrom(ValuesEnum valuesEnum) {
+      super.copyFrom(valuesEnum);
+      if (bytesRef.bytes.length < size) {
+        bytesRef.grow(size);
       }
+      bytesRef.length = size;
+      bytesRef.offset = 0;
+    }
 
-      @Override
-      public int advance(int target) throws IOException {
-        if (target >= maxDoc || size == 0) {
-          return pos = NO_MORE_DOCS;
-        }
-        if ((target - 1) != pos) // pos inc == 1
-          datIn.seek(fp + target * size);
-        datIn.readBytes(bytesRef.bytes, 0, size);
-        return pos = target;
-      }
+    public void close() throws IOException {
+      datIn.close();
+    }
 
-      @Override
-      public int docID() {
-        return pos;
-      }
+    @Override
+    public int advance(int target) throws IOException {
+      if (target >= maxDoc || size == 0) {
+        return pos = NO_MORE_DOCS;
+      }
+      if ((target - 1) != pos) // pos inc == 1
+        datIn.seek(fp + target * size);
+      datIn.readBytes(bytesRef.bytes, 0, size);
+      return pos = target;
+    }
 
-      @Override
-      public int nextDoc() throws IOException {
-        if (pos >= maxDoc) {
-          return pos = NO_MORE_DOCS;
-        }
-        return advance(pos + 1);
-      }
+    @Override
+    public int docID() {
+      return pos;
     }
 
     @Override
-    public ValueType type() {
-      return ValueType.BYTES_FIXED_STRAIGHT;
+    public int nextDoc() throws IOException {
+      if (pos >= maxDoc) {
+        return pos = NO_MORE_DOCS;
+      }
+      return advance(pos + 1);
     }
   }
 }