You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by si...@apache.org on 2011/01/02 19:49:24 UTC

svn commit: r1054457 - in /lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index: codecs/docvalues/ values/

Author: simonw
Date: Sun Jan  2 18:49:23 2011
New Revision: 1054457

URL: http://svn.apache.org/viewvc?rev=1054457&view=rev
Log:
added javadoc & cleaned up some API

Modified:
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesCodec.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesConsumer.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesProducerBase.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Bytes.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/DocValues.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/DocValuesEnum.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Floats.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Writer.java

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesCodec.java?rev=1054457&r1=1054456&r2=1054457&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesCodec.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesCodec.java Sun Jan  2 18:49:23 2011
@@ -104,11 +104,11 @@ public class DocValuesCodec extends Code
       if(info == null) {
         info = new DocValuesCodecInfo();
       }
-      final DocValuesConsumer consumer = DocValuesConsumer.create(info.docValuesId(state.segmentName, state.codecId, ""
+      final DocValuesConsumer consumer = Writer.create(field.getDocValues(), info.docValuesId(state.segmentName, state.codecId, ""
           + field.number),
       // TODO can we have a compound file per segment and codec for
           // docvalues?
-          state.directory, field, comparator, state.bytesUsed);
+          state.directory, comparator, state.bytesUsed);
       info.add(field.number);
       return consumer;
     }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesConsumer.java?rev=1054457&r1=1054456&r2=1054457&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesConsumer.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesConsumer.java Sun Jan  2 18:49:23 2011
@@ -18,43 +18,89 @@ package org.apache.lucene.index.codecs.d
  */
 import java.io.IOException;
 import java.util.Collection;
-import java.util.Comparator;
 import java.util.concurrent.atomic.AtomicLong;
 
-import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.values.DocValues;
 import org.apache.lucene.index.values.PerDocFieldValues;
 import org.apache.lucene.index.values.Writer;
-import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.BytesRef;
 
 /**
+ * Abstract API that consumes {@link PerDocFieldValues}.
+ * {@link DocValuesConsumer} are always associated with a specific field and
+ * segments. Concrete implementations of this API write the given
+ * {@link PerDocFieldValues} into a implementation specific format depending on
+ * the fields meta-data.
+ * 
  * @lucene.experimental
  */
-// TODO this might need to go in the codec package since is a direct relative to
-// TermsConsumer
 public abstract class DocValuesConsumer {
-  
-  protected AtomicLong bytesUsed = new AtomicLong(0);
-  
+  // TODO this might need to go in the codec package since is a direct relative
+  // to TermsConsumer
+  protected final AtomicLong bytesUsed;
+
+  /**
+   * Creates a new {@link DocValuesConsumer}.
+   * 
+   * @param bytesUsed
+   *          bytes-usage tracking reference used by implementation to track
+   *          internally allocated memory. All tracked bytes must be released
+   *          once {@link #finish(int)} has been called.
+   */
   protected DocValuesConsumer(AtomicLong bytesUsed) {
-    this.bytesUsed = bytesUsed;
-  }
-
-  public final long bytesUsed() {
-    return this.bytesUsed.get();
+    this.bytesUsed = bytesUsed == null ? new AtomicLong(0) : bytesUsed;
   }
 
-  public abstract void add(int docID, PerDocFieldValues docValues) throws IOException;
+  /**
+   * Adds the given {@link PerDocFieldValues} instance to this
+   * {@link DocValuesConsumer}
+   * 
+   * @param docID
+   *          the document ID to add the value for. The docID must always
+   *          increase or be <tt>0</tt> if it is the first call to this method.
+   * @param docValues
+   *          the values to add
+   * @throws IOException
+   *           if an {@link IOException} occurs
+   */
+  public abstract void add(int docID, PerDocFieldValues docValues)
+      throws IOException;
 
+  /**
+   * Called when the consumer of this API is doc with adding
+   * {@link PerDocFieldValues} to this {@link DocValuesConsumer}
+   * 
+   * @param docCount
+   *          the total number of documents in this {@link DocValuesConsumer}.
+   *          Must be greater than or equal the last given docID to
+   *          {@link #add(int, PerDocFieldValues)}.
+   * @throws IOException
+   */
   public abstract void finish(int docCount) throws IOException;
 
+  /**
+   * Gathers files associated with this {@link DocValuesConsumer}
+   * 
+   * @param files
+   *          the of files to add the consumers files to.
+   */
   public abstract void files(Collection<String> files) throws IOException;
 
+  /**
+   * Merges the given {@link org.apache.lucene.index.codecs.MergeState} into
+   * this {@link DocValuesConsumer}.
+   * 
+   * @param mergeState
+   *          the state to merge
+   * @param values
+   *          the docValues to merge in
+   * @throws IOException
+   *           if an {@link IOException} occurs
+   */
   public void merge(org.apache.lucene.index.codecs.MergeState mergeState,
       DocValues values) throws IOException {
+    assert mergeState != null;
     // TODO we need some kind of compatibility notation for values such
     // that two slightly different segments can be merged eg. fixed vs.
     // variable byte len or float32 vs. float64
@@ -74,20 +120,40 @@ public abstract class DocValuesConsumer 
       }
       docBase += reader.numDocs();
     }
-    if (merged)
+    if (merged) {
       finish(mergeState.mergedDocCount);
+    }
   }
 
+  /**
+   * Merges the given {@link MergeState} into this {@link DocValuesConsumer}.
+   * {@link MergeState#docBase} must always be increasing. Merging segments out
+   * of order is not supported.
+   * 
+   * @param mergeState
+   *          the {@link MergeState} to merge
+   * @throws IOException
+   *           if an {@link IOException} occurs
+   */
   protected abstract void merge(MergeState mergeState) throws IOException;
 
-  /*
-   * specialized auxiliary MergeState is necessary since we don't want to
-   * exploit internals up to the codec ones
+  /**
+   * Specialized auxiliary MergeState is necessary since we don't want to
+   * exploit internals up to the codecs consumer. An instance of this class is
+   * created for each merged low level {@link IndexReader} we are merging to
+   * support low level bulk copies.
    */
   public static class MergeState {
+    /**
+     * the source reader for this MergeState - merged values should be read from
+     * this instance
+     */
     public final DocValues reader;
+    /** the absolute docBase for this MergeState within the resulting segment */
     public final int docBase;
+    /** the number of documents in this MergeState */
     public final int docCount;
+    /** the deleted bits for this MergeState */
     public final Bits bits;
 
     public MergeState(DocValues reader, int docBase, int docCount, Bits bits) {
@@ -98,10 +164,4 @@ public abstract class DocValuesConsumer 
       this.bits = bits;
     }
   }
-
-  public static DocValuesConsumer create(String id,
-      Directory directory, FieldInfo field, Comparator<BytesRef> comp, AtomicLong bytesUsed)
-      throws IOException {
-    return Writer.create(field.getDocValues(), id, directory, comp, bytesUsed);
-  }
 }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesProducerBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesProducerBase.java?rev=1054457&r1=1054456&r2=1054457&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesProducerBase.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesProducerBase.java Sun Jan  2 18:49:23 2011
@@ -1,4 +1,5 @@
 package org.apache.lucene.index.codecs.docvalues;
+
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -32,18 +33,41 @@ import org.apache.lucene.store.Directory
 import org.apache.lucene.util.IntsRef;
 
 /**
+ * Abstract base class for FieldsProducer implementations supporting
+ * {@link DocValues}.
+ * 
  * @lucene.experimental
  */
-public abstract class DocValuesProducerBase extends FieldsProducer{
-  
+public abstract class DocValuesProducerBase extends FieldsProducer {
+
   protected final TreeMap<String, DocValues> docValues = new TreeMap<String, DocValues>();
   private final DocValuesCodecInfo info = new DocValuesCodecInfo();
 
-  protected DocValuesProducerBase(SegmentInfo si, Directory dir, FieldInfos fieldInfo, String codecId) throws IOException {
+  /**
+   * Creates a new {@link DocValuesProducerBase} instance and loads all
+   * {@link DocValues} instances for this segment and codec.
+   * 
+   * @param si
+   *          the segment info to load the {@link DocValues} for.
+   * @param dir
+   *          the directory to load the {@link DocValues} from.
+   * @param fieldInfo
+   *          the {@link FieldInfos}
+   * @param codecId
+   *          the codec ID
+   * @throws IOException
+   *           if an {@link IOException} occurs
+   */
+  protected DocValuesProducerBase(SegmentInfo si, Directory dir,
+      FieldInfos fieldInfo, String codecId) throws IOException {
     info.read(dir, si, codecId);
     load(fieldInfo, si.name, si.docCount, dir, codecId);
   }
 
+  /**
+   * Returns a {@link DocValues} instance for the given field name or
+   * <code>null</code> if this field has no {@link DocValues}.
+   */
   @Override
   public DocValues docValues(String field) throws IOException {
     return docValues.get(field);
@@ -58,15 +82,35 @@ public abstract class DocValuesProducerB
       final FieldInfo fieldInfo = fieldInfos.fieldInfo(fieldNumber);
       assert fieldInfo.hasDocValues();
       final String field = fieldInfo.name;
-      //TODO can we have a compound file  per segment and codec for docvalues?
-      final String id = info.docValuesId( segment, codecId, fieldNumber+"");
-      docValues.put(field, loadDocValues(docCount, dir, id, fieldInfo.getDocValues()));
+      // TODO can we have a compound file per segment and codec for docvalues?
+      final String id = info.docValuesId(segment, codecId, fieldNumber + "");
+      docValues.put(field, loadDocValues(docCount, dir, id, fieldInfo
+          .getDocValues()));
     }
   }
 
+  /**
+   * Loads a {@link DocValues} instance depending on the given {@link Type}.
+   * Codecs that use different implementations for a certain {@link Type} 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 DocValues} from
+   * @param id
+   *          the unique file ID within the segment
+   * @param type
+   *          the type to load
+   * @return a {@link DocValues} instance for the given type
+   * @throws IOException
+   *           if an {@link IOException} occurs
+   * @throws IllegalArgumentException
+   *           if the given {@link Type} is not supported
+   */
   protected DocValues loadDocValues(int docCount, Directory dir, String id,
-      Type v) throws IOException {
-    switch (v) {
+      Type type) throws IOException {
+    switch (type) {
     case PACKED_INTS:
       return Ints.getValues(dir, id, false);
     case SIMPLE_FLOAT_4BYTE:
@@ -86,15 +130,23 @@ public abstract class DocValuesProducerB
     case BYTES_VAR_SORTED:
       return Bytes.getValues(dir, id, Bytes.Mode.SORTED, false, docCount);
     default:
-      throw new IllegalStateException("unrecognized index values mode " + v);
+      throw new IllegalStateException("unrecognized index values mode " + type);
     }
   }
 
   @Override
   public void close() throws IOException {
     Collection<DocValues> values = docValues.values();
+    IOException ex = null;
     for (DocValues docValues : values) {
-      docValues.close();
+      try {
+        docValues.close();
+      } catch (IOException e) {
+        ex = e;
+      }
+    }
+    if (ex != null) {
+      throw ex;
     }
   }
 }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Bytes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Bytes.java?rev=1054457&r1=1054456&r2=1054457&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Bytes.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Bytes.java Sun Jan  2 18:49:23 2011
@@ -38,17 +38,19 @@ import org.apache.lucene.util.CodecUtil;
 import org.apache.lucene.util.PagedBytes;
 
 /**
- * Provides concrete Writer/Reader impls for byte[] value per document. There
- * are 6 package-private impls of this, for all combinations of
- * STRAIGHT/DEREF/SORTED X fixed/not fixed.
+ * Provides concrete Writer/Reader implementations for <tt>byte[]</tt> value per
+ * document. There are 6 package-private default implementations of this, for
+ * all combinations of {@link Mode#DEREF}/{@link Mode#STRAIGHT}/
+ * {@link Mode#SORTED} x fixed-length/variable-length.
  * 
  * <p>
- * NOTE: The total amount of byte[] data stored (across a single segment) cannot
- * exceed 2GB.
+ * NOTE: Currently the total amount of byte[] data stored (across a single
+ * segment) cannot exceed 2GB.
  * </p>
  * <p>
  * NOTE: Each byte[] must be <= 32768 bytes in length
  * </p>
+ * 
  * @lucene.experimental
  */
 public final class Bytes {
@@ -56,18 +58,56 @@ public final class Bytes {
   private Bytes() {  /* don't instantiate! */ }
 
   /**
-   *  
-   *
+   * Defines the {@link Writer}s store mode. The writer will either store the
+   * bytes sequentially ({@link #STRAIGHT}, dereferenced ({@link #DEREF}) or
+   * sorted ({@link #SORTED})
+   * 
    */
   public static enum Mode {
-    STRAIGHT, DEREF, SORTED
+    /**
+     * Mode for sequentially stored bytes
+     */
+    STRAIGHT,
+    /**
+     * Mode for dereferenced stored bytes
+     */
+    DEREF,
+    /**
+     * Mode for sorted stored bytes
+     */
+    SORTED
   };
 
-  // TODO -- i shouldn't have to specify fixed? can
-  // track itself & do the write thing at write time?
+  /**
+   * Creates a new <tt>byte[]</tt> {@link Writer} instances for the given
+   * directory.
+   * 
+   * @param dir
+   *          the directory to write the values to
+   * @param id
+   *          the id used to create a unique file name. Usually composed out of
+   *          the segment name and a unique id per segment.
+   * @param mode
+   *          the writers store mode
+   * @param comp
+   *          a {@link BytesRef} comparator - only used with {@link Mode#SORTED}
+   * @param fixedSize
+   *          <code>true</code> if all bytes subsequently passed to the
+   *          {@link Writer} will have the same length
+   * @param bytesUsed
+   *          an {@link AtomicLong} instance to track the used bytes within the
+   *          {@link Writer}. A call to {@link Writer#finish(int)} will release
+   *          all internally used resources and frees the memeory tracking
+   *          reference.
+   * @return a new {@link Writer} instance
+   * @throws IOException
+   *           if the files for the writer can not be created.
+   */
   public static Writer getWriter(Directory dir, String id, Mode mode,
-      Comparator<BytesRef> comp, boolean fixedSize, AtomicLong bytesUsed) throws IOException {
-
+      Comparator<BytesRef> comp, boolean fixedSize, AtomicLong bytesUsed)
+      throws IOException {
+    // TODO -- i shouldn't have to specify fixed? can
+    // track itself & do the write thing at write time?
     if (comp == null) {
       comp = BytesRef.getUTF8SortedAsUnicodeComparator();
     }
@@ -93,22 +133,35 @@ public final class Bytes {
     throw new IllegalArgumentException("");
   }
 
-  // TODO -- I can peek @ header to determing fixed/mode?
+  /**
+   * Creates a new {@link DocValues} instance that provides either memory
+   * resident or iterative access to a per-document stored <tt>byte[]</tt>
+   * value. The returned {@link DocValues} instance will be initialized without
+   * consuming a significant amount of memory.
+   * 
+   * @param dir
+   *          the directory to load the {@link DocValues} from.
+   * @param id
+   *          the file ID in the {@link Directory} to load the values from.
+   * @param mode
+   *          the mode used to store the values
+   * @param fixedSize
+   *          <code>true</code> iff the values are stored with fixed-size,
+   *          otherwise <code>false</code>
+   * @param maxDoc
+   *          the number of document values stored for the given ID
+   * @return an initialized {@link DocValues} instance.
+   * @throws IOException
+   *           if an {@link IOException} occurs
+   */
   public static DocValues getValues(Directory dir, String id, Mode mode,
       boolean fixedSize, int maxDoc) throws IOException {
+    // TODO -- I can peek @ header to determing fixed/mode?
     if (fixedSize) {
       if (mode == Mode.STRAIGHT) {
-        try {
-          return new FixedStraightBytesImpl.Reader(dir, id, maxDoc);
-        } catch (IOException e) {
-          throw e;
-        }
+        return new FixedStraightBytesImpl.Reader(dir, id, maxDoc);
       } else if (mode == Mode.DEREF) {
-        try {
-          return new FixedDerefBytesImpl.Reader(dir, id, maxDoc);
-        } catch (IOException e) {
-          throw e;
-        }
+        return new FixedDerefBytesImpl.Reader(dir, id, maxDoc);
       } else if (mode == Mode.SORTED) {
         return new FixedSortedBytesImpl.Reader(dir, id, maxDoc);
       }
@@ -122,7 +175,7 @@ public final class Bytes {
       }
     }
 
-    throw new IllegalArgumentException("");
+    throw new IllegalArgumentException("Illegal Mode: " + mode);
   }
 
   static abstract class BytesBaseSource extends Source {
@@ -146,21 +199,23 @@ public final class Bytes {
     }
 
     public void close() throws IOException {
-      data.close();
       try {
-        if (datIn != null)
-          datIn.close();
+        data.close(); // close data
       } finally {
-        if (idxIn != null) // if straight - no index needed
-          idxIn.close();
+        try {
+          if (datIn != null)
+            datIn.close();
+        } finally {
+          if (idxIn != null) // if straight - no index needed
+            idxIn.close();
+        }
       }
     }
-    
-    protected abstract int maxDoc();
 
-    public long ramBytesUsed() {
-      return 0; // TODO
-    }
+    /**
+     * Returns one greater than the largest possible document number.
+     */
+    protected abstract int maxDoc();
 
     @Override
     public DocValuesEnum getEnum(AttributeSource attrSource) throws IOException {
@@ -214,10 +269,13 @@ public final class Bytes {
     }
 
     public void close() throws IOException {
-      if (datIn != null)
-        datIn.close();
-      if (idxIn != null) // if straight
-        idxIn.close();
+      try {
+        if (datIn != null)
+          datIn.close();
+      } finally {
+        if (idxIn != null) // if straight
+          idxIn.close();
+      }
     }
 
     protected abstract int maxDoc();
@@ -318,12 +376,18 @@ public final class Bytes {
 
     @Override
     public synchronized void finish(int docCount) throws IOException {
-      if (datOut != null)
-        datOut.close();
-      if (idxOut != null)
-        idxOut.close();
-      if (pool != null)
-        pool.reset();
+      try {
+        if (datOut != null)
+          datOut.close();
+      } finally {
+        try {
+          if (idxOut != null)
+            idxOut.close();
+        } finally {
+          if (pool != null)
+            pool.reset();
+        }
+      }
     }
 
     @Override
@@ -331,7 +395,6 @@ public final class Bytes {
       add(docID, bytesRef);
     }
 
-
     @Override
     public void add(int docID, PerDocFieldValues docValues) throws IOException {
       final BytesRef ref;
@@ -339,7 +402,7 @@ public final class Bytes {
         add(docID, ref);
       }
     }
-    
+
     @Override
     protected void setNextEnum(DocValuesEnum valuesEnum) {
       bytesRef = valuesEnum.bytes();
@@ -391,19 +454,25 @@ public final class Bytes {
       return (IndexInput) datIn.clone();
     }
 
-    protected final IndexInput cloneIndex() { // TODO assert here for null
-      // rather than return null
-      return idxIn == null ? null : (IndexInput) idxIn.clone();
+    protected final IndexInput cloneIndex() {
+      assert idxIn != null;
+      return (IndexInput) idxIn.clone();
     }
 
     @Override
     public void close() throws IOException {
-      super.close();
-      if (datIn != null) {
-        datIn.close();
-      }
-      if (idxIn != null) {
-        idxIn.close();
+      try {
+        super.close();
+      } finally {
+        try {
+          if (datIn != null) {
+            datIn.close();
+          }
+        } finally {
+          if (idxIn != null) {
+            idxIn.close();
+          }
+        }
       }
     }
   }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/DocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/DocValues.java?rev=1054457&r1=1054456&r2=1054457&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/DocValues.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/DocValues.java Sun Jan  2 18:49:23 2011
@@ -26,9 +26,8 @@ import org.apache.lucene.index.IndexRead
 import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.BytesRef;
 
-
 /**
- * 
+ * TODO
  * @see FieldsEnum#docValues()
  * @see Fields#docValues(String)
  * @lucene.experimental
@@ -157,52 +156,117 @@ public abstract class DocValues implemen
    * implementations provide random access semantics similar to array lookups
    * and typically are entirely memory resident.
    * <p>
-   * {@link Source} defines 3 {@link Type} //TODO finish this 
+   * {@link Source} defines 3 {@link Type} //TODO finish this
    */
   public static abstract class Source {
     protected final MissingValue missingValue = new MissingValue();
 
+    /**
+     * Returns a <tt>long</tt> for the given document id or throws an
+     * {@link UnsupportedOperationException} if this source doesn't support
+     * <tt>long</tt> values.
+     * 
+     * @throws UnsupportedOperationException
+     *           if this source doesn't support <tt>long</tt> values.
+     * @see MissingValue
+     * @see #getMissing()
+     */
     public long getInt(int docID) {
       throw new UnsupportedOperationException("ints are not supported");
     }
 
+    /**
+     * Returns a <tt>double</tt> for the given document id or throws an
+     * {@link UnsupportedOperationException} if this source doesn't support
+     * <tt>double</tt> values.
+     * 
+     * @throws UnsupportedOperationException
+     *           if this source doesn't support <tt>double</tt> values.
+     * @see MissingValue
+     * @see #getMissing()
+     */
     public double getFloat(int docID) {
       throw new UnsupportedOperationException("floats are not supported");
     }
 
+    /**
+     * Returns a {@link BytesRef} for the given document id or throws an
+     * {@link UnsupportedOperationException} if this source doesn't support
+     * <tt>byte[]</tt> values.
+     * 
+     * @throws UnsupportedOperationException
+     *           if this source doesn't support <tt>byte[]</tt> values.
+     * @see MissingValue
+     * @see #getMissing()
+     */
     public BytesRef getBytes(int docID, BytesRef ref) {
       throw new UnsupportedOperationException("bytes are not supported");
     }
 
     /**
-     * Returns number of unique values. Some impls may throw
+     * Returns number of unique values. Some implementations may throw
      * UnsupportedOperationException.
      */
     public int getValueCount() {
       throw new UnsupportedOperationException();
     }
 
+    /**
+     * Returns a {@link DocValuesEnum} for this source.
+     */
     public DocValuesEnum getEnum() throws IOException {
       return getEnum(null);
     }
 
+    /**
+     * Returns a {@link MissingValue} instance for this {@link Source}.
+     * Depending on the type of this {@link Source} consumers of the API should
+     * check if the value returned from on of the getter methods represents a
+     * value for a missing document or rather a value for a document no value
+     * was specified during indexing.
+     */
     public MissingValue getMissing() {
       return missingValue;
     }
 
+    /**
+     * Returns the {@link Type} of this source.
+     * 
+     * @return the {@link Type} of this source.
+     */
     public abstract Type type();
 
+    /**
+     * Returns a {@link DocValuesEnum} for this source which uses the given
+     * {@link AttributeSource}.
+     */
     public abstract DocValuesEnum getEnum(AttributeSource attrSource)
         throws IOException;
-
   }
 
-  abstract static class SourceEnum extends DocValuesEnum {
+  /**
+   * {@link DocValuesEnum} utility for {@link Source} implemenations.
+   * 
+   */
+  public abstract static class SourceEnum extends DocValuesEnum {
     protected final Source source;
     protected final int numDocs;
     protected int pos = -1;
 
-    SourceEnum(AttributeSource attrs, Type type, Source source, int numDocs) {
+    /**
+     * Creates a new {@link SourceEnum}
+     * 
+     * @param attrs
+     *          the {@link AttributeSource} for this enum
+     * @param type
+     *          the enums {@link Type}
+     * @param source
+     *          the source this enum operates on
+     * @param numDocs
+     *          the number of documents within the source
+     */
+    protected SourceEnum(AttributeSource attrs, Type type, Source source,
+        int numDocs) {
       super(attrs, type);
       this.source = source;
       this.numDocs = numDocs;
@@ -225,6 +289,12 @@ public abstract class DocValues implemen
     }
   }
 
+  /**
+   * A sorted variant of {@link Source} for <tt>byte[]</tt> values per document.
+   * <p>
+   * Note: {@link DocValuesEnum} obtained from a {@link SortedSource} will
+   * enumerate values in document order and not in sorted order.
+   */
   public static abstract class SortedSource extends Source {
 
     @Override
@@ -243,27 +313,56 @@ public abstract class DocValues implemen
     public abstract BytesRef getByOrd(int ord, BytesRef bytesRef);
 
     public static class LookupResult {
+      /** <code>true</code> iff the values was found */
       public boolean found;
+      /**
+       * the ordinal of the value if found or the ordinal of the value if it
+       * would be present in the source
+       */
       public int ord;
     }
 
     /**
-     * Finds the largest ord whose value is <= the requested value. If
-     * {@link LookupResult#found} is true, then ord is an exact match. The
-     * returned {@link LookupResult} may be reused across calls.
+     * Finds the largest ord whose value is less or equal to the requested
+     * value. If {@link LookupResult#found} is true, then ord is an exact match.
+     * The returned {@link LookupResult} may be reused across calls.
      */
     public final LookupResult getByValue(BytesRef value) {
       return getByValue(value, new BytesRef());
     }
 
+    /**
+     * Performs a lookup by value.
+     * 
+     * @param value
+     *          the value to look up
+     * @param tmpRef
+     *          a temporary {@link BytesRef} instance used to compare internal
+     *          values to the given value. Must not be <code>null</code>
+     * @return the {@link LookupResult}
+     */
     public abstract LookupResult getByValue(BytesRef value, BytesRef tmpRef);
   }
 
+  /**
+   * {@link MissingValue} is used by {@link Source} implementations to define an
+   * Implementation dependent value for documents that had no value assigned
+   * during indexing. Its purpose is similar to a default value but since the a
+   * missing value across {@link Type} and its implementations can be highly
+   * dynamic the actual values are not constant but defined per {@link Source}
+   * through the {@link MissingValue} struct. The actual value used to indicate
+   * a missing value can even changed within the same field from one segment to
+   * another. Certain {@link Ints} implementations for instance use a value
+   * outside of value set as the missing value.
+   */
   public final static class MissingValue {
     public long longValue;
     public double doubleValue;
     public BytesRef bytesValue;
 
+    /**
+     * Copies the values from the given {@link MissingValue}.
+     */
     public final void copy(MissingValue values) {
       longValue = values.longValue;
       doubleValue = values.doubleValue;

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/DocValuesEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/DocValuesEnum.java?rev=1054457&r1=1054456&r2=1054457&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/DocValuesEnum.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/DocValuesEnum.java Sun Jan  2 18:49:23 2011
@@ -19,7 +19,6 @@ package org.apache.lucene.index.values;
 import java.io.IOException;
 
 import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.util.Attribute;
 import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.FloatsRef;
@@ -30,7 +29,7 @@ import org.apache.lucene.util.LongsRef;
  */
 public abstract class DocValuesEnum extends DocIdSetIterator {
   private AttributeSource source;
-  private Type enumType;
+  private final Type enumType;
   protected BytesRef bytesRef;
   protected FloatsRef floatsRef;
   protected LongsRef intsRef;
@@ -84,23 +83,12 @@ public abstract class DocValuesEnum exte
   }
 
   public AttributeSource attributes() {
-    if (source == null)
+    if (source == null) {
       source = new AttributeSource();
+    }
     return source;
   }
 
-  public <T extends Attribute> T addAttribute(Class<T> attr) {
-    return attributes().addAttribute(attr);
-  }
-
-  public <T extends Attribute> T getAttribute(Class<T> attr) {
-    return attributes().getAttribute(attr);
-  }
-
-  public <T extends Attribute> boolean hasAttribute(Class<T> attr) {
-    return attributes().hasAttribute(attr);
-  }
-
   public abstract void close() throws IOException;
 
   public static DocValuesEnum emptyEnum(Type type) {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java?rev=1054457&r1=1054456&r2=1054457&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java Sun Jan  2 18:49:23 2011
@@ -133,7 +133,7 @@ class FixedStraightBytesImpl {
 
     @Override
     public Source load() throws IOException {
-      return new Source(cloneData(), cloneIndex(), size, maxDoc);
+      return new Source(cloneData(), size, maxDoc);
     }
 
     @Override
@@ -145,9 +145,9 @@ class FixedStraightBytesImpl {
       private final int size;
       private final int maxDoc;
 
-      public Source(IndexInput datIn, IndexInput idxIn, int size, int maxDoc)
+      public Source(IndexInput datIn, int size, int maxDoc)
           throws IOException {
-        super(datIn, idxIn, new PagedBytes(PAGED_BYTES_BITS), size * maxDoc);
+        super(datIn, null, new PagedBytes(PAGED_BYTES_BITS), size * maxDoc);
         this.size = size;
         this.missingValue.bytesValue = new BytesRef(size);
         this.maxDoc = maxDoc;

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Floats.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Floats.java?rev=1054457&r1=1054456&r2=1054457&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Floats.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Floats.java Sun Jan  2 18:49:23 2011
@@ -24,6 +24,7 @@ import java.util.Collection;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.values.DocValues.Source;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
@@ -32,13 +33,16 @@ import org.apache.lucene.util.CodecUtil;
 import org.apache.lucene.util.FloatsRef;
 
 /**
- * Exposes writer/reader for floating point values. You can specify 4 (java
- * float) or 8 (java double) byte precision.
+ * Exposes {@link Writer} and reader ({@link Source}) for 32 bit and 64 bit
+ * floating point values.
+ * <p>
+ * Current implementations store either 4 byte or 8 byte floating points with
+ * full precision without any compression.
  * 
  * @lucene.experimental
  */
-// TODO - add bulk copy where possible
 public class Floats {
+  // TODO - add bulk copy where possible
   private static final String CODEC_NAME = "SimpleFloats";
   static final int VERSION_START = 0;
   static final int VERSION_CURRENT = VERSION_START;
@@ -47,6 +51,7 @@ public class Floats {
   private static final long LONG_DEFAULT = Double
       .doubleToRawLongBits(Double.NEGATIVE_INFINITY);
 
+  
   public static Writer getWriter(Directory dir, String id, int precisionBytes,
       AtomicLong bytesUsed) throws IOException {
     if (precisionBytes != 4 && precisionBytes != 8) {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java?rev=1054457&r1=1054456&r2=1054457&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java Sun Jan  2 18:49:23 2011
@@ -101,7 +101,6 @@ class VarSortedBytesImpl {
       final int count = hash.size();
       try {
         final int[] sortedEntries = hash.sort(comp);
-
         // first dump bytes data, recording index & offset as
         // we go
         long offset = 0;

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Writer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Writer.java?rev=1054457&r1=1054456&r2=1054457&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Writer.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Writer.java Sun Jan  2 18:49:23 2011
@@ -26,47 +26,126 @@ import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 
 /**
+ * Abstract API for per-document stored primitive values of type <tt>byte[]</tt>
+ * , <tt>long</tt> or <tt>double</tt>. The API accepts a single value for each
+ * document. The underlying storage mechanism, file formats, data-structures and
+ * representations depend on the actual implementation.
+ * <p>
+ * Document IDs passed to this API must always be increasing unless stated
+ * otherwise.
+ * </p>
+ * 
  * @lucene.experimental
  */
 public abstract class Writer extends DocValuesConsumer {
 
+  /**
+   * Creates a new {@link Writer}.
+   * 
+   * @param bytesUsed
+   *          bytes-usage tracking reference used by implementation to track
+   *          internally allocated memory. All tracked bytes must be released
+   *          once {@link #finish(int)} has been called.
+   */
   protected Writer(AtomicLong bytesUsed) {
     super(bytesUsed);
   }
 
+  /**
+   * Filename extension for index files
+   */
   public static final String INDEX_EXTENSION = "idx";
+  
+  /**
+   * Filename extension for data files.
+   */
   public static final String DATA_EXTENSION = "dat";
 
-  /** Records the specfied value for the docID */
+  /**
+   * Records the specified <tt>long</tt> value for the docID or throws an
+   * {@link UnsupportedOperationException} if this {@link Writer} doesn't record
+   * <tt>long</tt> values.
+   * 
+   * @throws UnsupportedOperationException
+   *           if this writer doesn't record <tt>long</tt> values
+   */
   public void add(int docID, long value) throws IOException {
     throw new UnsupportedOperationException();
   }
 
-  /** Records the specfied value for the docID */
+  /**
+   * Records the specified <tt>double</tt> value for the docID or throws an
+   * {@link UnsupportedOperationException} if this {@link Writer} doesn't record
+   * <tt>double</tt> values.
+   * 
+   * @throws UnsupportedOperationException
+   *           if this writer doesn't record <tt>double</tt> values
+   */
   public void add(int docID, double value) throws IOException {
     throw new UnsupportedOperationException();
   }
 
-  /** Records the specfied value for the docID */
+  /**
+   * Records the specified {@link BytesRef} value for the docID or throws an
+   * {@link UnsupportedOperationException} if this {@link Writer} doesn't record
+   * {@link BytesRef} values.
+   * 
+   * @throws UnsupportedOperationException
+   *           if this writer doesn't record {@link BytesRef} values
+   */
   public void add(int docID, BytesRef value) throws IOException {
     throw new UnsupportedOperationException();
   }
 
-  /** Records the specfied value for the docID */
+  /**
+   * Records a value from the given document id. The methods implementation
+   * obtains the value for the document id from the last {@link DocValuesEnum}
+   * set to {@link #setNextEnum(DocValuesEnum)}.
+   * <p>
+   * This method is used during merging to provide implementation agnostic
+   * default merge implementation.
+   * </p>
+   * <p>
+   * The given document id must be the same document id returned from
+   * {@link DocValuesEnum#docID()} when this method is called. All documents IDs
+   * between the given ID and the previously given ID or <tt>0</tt> if the
+   * method is call the first time are filled with default values depending on
+   * the {@link Writer} implementation. The given document ID must always be
+   * greater than the previous ID or <tt>0</tt> if called the first time.
+   */
   protected abstract void add(int docID) throws IOException;
-  
+
+  /**
+   * Sets the next {@link DocValuesEnum} to consume values from on calls to
+   * {@link #add(int)}
+   * 
+   * @param valuesEnum
+   *          the next {@link DocValuesEnum}, this must not be null
+   */
   protected abstract void setNextEnum(DocValuesEnum valuesEnum);
 
-  /** Finish writing, close any files */
+  /**
+   * Finish writing and close any files and resources used by this Writer.
+   * 
+   * @param docCount
+   *          the total number of documents for this writer. This must be
+   *          greater that or equal to the largest document id passed to one of
+   *          the add methods after the {@link Writer} was created.
+   */
   public abstract void finish(int docCount) throws IOException;
 
-  // enables bulk copies in subclasses per MergeState
   @Override
   protected void merge(MergeState state) throws IOException {
+    // This enables bulk copies in subclasses per MergeState, subclasses can
+    // simply override this and decide if they want to merge
+    // segments using this generic implementation or if a bulk merge is possible
+    // / feasible.
     final DocValuesEnum valEnum = state.reader.getEnum();
     assert valEnum != null;
     try {
-      setNextEnum(valEnum);
+      setNextEnum(valEnum); // set the current enum we are working on - the
+      // impl. will get the correct reference for the type
+      // it supports
       int docID = state.docBase;
       final Bits bits = state.bits;
       final int docCount = state.docCount;
@@ -91,9 +170,33 @@ public abstract class Writer extends Doc
     }
   }
 
-  public static Writer create(Type v, String id, Directory directory,
+  /**
+   * Factory method to create a {@link Writer} instance for a given type. This
+   * method returns default implementations for each of the different types
+   * defined in the {@link Type} enumeration.
+   * 
+   * @param type
+   *          the {@link Type} to create the {@link Writer} for
+   * @param id
+   *          the file name id used to create files within the writer.
+   * @param directory
+   *          the {@link Directory} to create the files from.
+   * @param comp
+   *          a {@link BytesRef} comparator used for {@link Bytes} variants. If
+   *          <code>null</code>
+   *          {@link BytesRef#getUTF8SortedAsUnicodeComparator()} is used as the
+   *          default.
+   * @param bytesUsed
+   *          a byte-usage tracking reference
+   * @return a new {@link Writer} instance for the given {@link Type}
+   * @throws IOException
+   */
+  public static Writer create(Type type, String id, Directory directory,
       Comparator<BytesRef> comp, AtomicLong bytesUsed) throws IOException {
-    switch (v) {
+    if (comp == null) {
+      comp = BytesRef.getUTF8SortedAsUnicodeComparator();
+    }
+    switch (type) {
     case PACKED_INTS:
       return Ints.getWriter(directory, id, true, bytesUsed);
     case SIMPLE_FLOAT_4BYTE:
@@ -101,19 +204,25 @@ public abstract class Writer extends Doc
     case SIMPLE_FLOAT_8BYTE:
       return Floats.getWriter(directory, id, 8, bytesUsed);
     case BYTES_FIXED_STRAIGHT:
-      return Bytes.getWriter(directory, id, Bytes.Mode.STRAIGHT, comp, true, bytesUsed);
+      return Bytes.getWriter(directory, id, Bytes.Mode.STRAIGHT, comp, true,
+          bytesUsed);
     case BYTES_FIXED_DEREF:
-      return Bytes.getWriter(directory, id, Bytes.Mode.DEREF, comp, true, bytesUsed);
+      return Bytes.getWriter(directory, id, Bytes.Mode.DEREF, comp, true,
+          bytesUsed);
     case BYTES_FIXED_SORTED:
-      return Bytes.getWriter(directory, id, Bytes.Mode.SORTED, comp, true, bytesUsed);
+      return Bytes.getWriter(directory, id, Bytes.Mode.SORTED, comp, true,
+          bytesUsed);
     case BYTES_VAR_STRAIGHT:
-      return Bytes.getWriter(directory, id, Bytes.Mode.STRAIGHT, comp, false, bytesUsed);
+      return Bytes.getWriter(directory, id, Bytes.Mode.STRAIGHT, comp, false,
+          bytesUsed);
     case BYTES_VAR_DEREF:
-      return Bytes.getWriter(directory, id, Bytes.Mode.DEREF, comp, false, bytesUsed);
+      return Bytes.getWriter(directory, id, Bytes.Mode.DEREF, comp, false,
+          bytesUsed);
     case BYTES_VAR_SORTED:
-      return Bytes.getWriter(directory, id, Bytes.Mode.SORTED, comp, false, bytesUsed);
+      return Bytes.getWriter(directory, id, Bytes.Mode.SORTED, comp, false,
+          bytesUsed);
     default:
-      throw new IllegalArgumentException("Unknown Values: " + v);
+      throw new IllegalArgumentException("Unknown Values: " + type);
     }
   }
 }