You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by us...@apache.org on 2012/08/31 00:43:59 UTC

svn commit: r1379200 [3/11] - in /lucene/dev/branches/lucene3312: ./ dev-tools/ dev-tools/eclipse/ dev-tools/idea/.idea/libraries/ dev-tools/maven/ dev-tools/maven/lucene/core/ dev-tools/maven/lucene/test-framework/ dev-tools/scripts/ lucene/ lucene/an...

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PositionIncrementAttributeImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PositionIncrementAttributeImpl.java?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PositionIncrementAttributeImpl.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PositionIncrementAttributeImpl.java Thu Aug 30 22:43:41 2012
@@ -17,40 +17,15 @@ package org.apache.lucene.analysis.token
  * limitations under the License.
  */
 
-import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.util.AttributeImpl;
 
-/** Determines the position of this token
- * relative to the previous Token in a {@link TokenStream}, used in phrase
- * searching.
- *
- * <p>The default value is one.
- *
- * <p>Some common uses for this are:<ul>
- *
- * <li>Set it to zero to put multiple terms in the same position.  This is
- * useful if, e.g., a word has multiple stems.  Searches for phrases
- * including either stem will match.  In this case, all but the first stem's
- * increment should be set to zero: the increment of the first instance
- * should be one.  Repeating a token with an increment of zero can also be
- * used to boost the scores of matches on that token.
- *
- * <li>Set it to values greater than one to inhibit exact phrase matches.
- * If, for example, one does not want phrases to match across removed stop
- * words, then one could build a stop word filter that removes stop words and
- * also sets the increment to the number of stop words removed before each
- * non-stop word.  Then exact phrase queries will only match when the terms
- * occur with no intervening stop words.
- *
- * </ul>
- */
+/** Default implementation of {@link PositionIncrementAttribute}. */
 public class PositionIncrementAttributeImpl extends AttributeImpl implements PositionIncrementAttribute, Cloneable {
   private int positionIncrement = 1;
   
-  /** Set the position increment. The default value is one.
-   *
-   * @param positionIncrement the distance from the prior term
-   */
+  /** Initialize this attribute with position increment of 1 */
+  public PositionIncrementAttributeImpl() {}
+
   public void setPositionIncrement(int positionIncrement) {
     if (positionIncrement < 0) {
       throw new IllegalArgumentException
@@ -59,9 +34,6 @@ public class PositionIncrementAttributeI
     this.positionIncrement = positionIncrement;
   }
 
-  /** Returns the position increment of this Token.
-   * @see #setPositionIncrement
-   */
   public int getPositionIncrement() {
     return positionIncrement;
   }

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PositionLengthAttribute.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PositionLengthAttribute.java?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PositionLengthAttribute.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PositionLengthAttribute.java Thu Aug 30 22:43:41 2012
@@ -26,11 +26,20 @@ import org.apache.lucene.util.Attribute;
  *  produced by decompounding, word splitting/joining,
  *  synonym filtering, etc.
  *
- * <p>The default value is one. */
+ * <p>NOTE: this is optional, and most analyzers
+ *  don't change the default value (1). */
 
 public interface PositionLengthAttribute extends Attribute {
-  /** @param positionLength how many positions this token
-   *  spans. */
+  /**
+   * Set the position length of this Token.
+   * <p>
+   * The default value is one. 
+   * @param positionLength how many positions this token
+   *  spans. 
+   * @throws IllegalArgumentException if <code>positionLength</code> 
+   *         is zero or negative.
+   * @see #getPositionLength()
+   */
   public void setPositionLength(int positionLength);
 
   /** Returns the position length of this Token.

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PositionLengthAttributeImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PositionLengthAttributeImpl.java?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PositionLengthAttributeImpl.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PositionLengthAttributeImpl.java Thu Aug 30 22:43:41 2012
@@ -19,13 +19,13 @@ package org.apache.lucene.analysis.token
 
 import org.apache.lucene.util.AttributeImpl;
 
-/** See {@link PositionLengthAttribute}. */
+/** Default implementation of {@link PositionLengthAttribute}. */
 public class PositionLengthAttributeImpl extends AttributeImpl implements PositionLengthAttribute, Cloneable {
   private int positionLength = 1;
   
-  /** @param positionLength how many positions this token
-   *  spans.  NOTE: this is optional, and most analyzers
-   *  don't change the default value (1). */
+  /** Initializes this attribute with position length of 1. */
+  public PositionLengthAttributeImpl() {}
+  
   public void setPositionLength(int positionLength) {
     if (positionLength < 1) {
       throw new IllegalArgumentException
@@ -34,9 +34,6 @@ public class PositionLengthAttributeImpl
     this.positionLength = positionLength;
   }
 
-  /** Returns the position length of this Token.
-   * @see #setPositionLength    
-   */
   public int getPositionLength() {
     return positionLength;
   }

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/TermToBytesRefAttribute.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/TermToBytesRefAttribute.java?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/TermToBytesRefAttribute.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/TermToBytesRefAttribute.java Thu Aug 30 22:43:41 2012
@@ -56,7 +56,7 @@ public interface TermToBytesRefAttribute
    * Updates the bytes {@link #getBytesRef()} to contain this term's
    * final encoding, and returns its hashcode.
    * @return the hashcode as defined by {@link BytesRef#hashCode}:
-   * <pre>
+   * <pre class="prettyprint">
    *  int hash = 0;
    *  for (int i = termBytes.offset; i &lt; termBytes.offset+termBytes.length; i++) {
    *    hash = 31*hash + termBytes.bytes[i];

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/TypeAttribute.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/TypeAttribute.java?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/TypeAttribute.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/TypeAttribute.java Thu Aug 30 22:43:41 2012
@@ -27,10 +27,15 @@ public interface TypeAttribute extends A
   /** the default type */
   public static final String DEFAULT_TYPE = "word";
 
-  /** Returns this Token's lexical type.  Defaults to "word". */
+  /** 
+   * Returns this Token's lexical type.  Defaults to "word". 
+   * @see #setType(String)
+   */
   public String type();
 
-  /** Set the lexical type.
-      @see #type() */
+  /** 
+   * Set the lexical type.
+   * @see #type() 
+   */
   public void setType(String type);
 }

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/TypeAttributeImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/TypeAttributeImpl.java?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/TypeAttributeImpl.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/TypeAttributeImpl.java Thu Aug 30 22:43:41 2012
@@ -19,27 +19,24 @@ package org.apache.lucene.analysis.token
 
 import org.apache.lucene.util.AttributeImpl;
 
-/**
- * A Token's lexical type. The Default value is "word". 
- */
+/** Default implementation of {@link TypeAttribute}. */
 public class TypeAttributeImpl extends AttributeImpl implements TypeAttribute, Cloneable {
   private String type;
   
+  /** Initialize this attribute with {@link TypeAttribute#DEFAULT_TYPE} */
   public TypeAttributeImpl() {
     this(DEFAULT_TYPE); 
   }
   
+  /** Initialize this attribute with <code>type</code> */
   public TypeAttributeImpl(String type) {
     this.type = type;
   }
   
-  /** Returns this Token's lexical type.  Defaults to "word". */
   public String type() {
     return type;
   }
 
-  /** Set the lexical type.
-      @see #type() */
   public void setType(String type) {
     this.type = type;
   }

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/Codec.java?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/Codec.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/Codec.java Thu Aug 30 22:43:41 2012
@@ -41,6 +41,14 @@ public abstract class Codec implements N
 
   private final String name;
 
+  /**
+   * Creates a new codec.
+   * <p>
+   * The provided name will be written into the index segment: in order to
+   * for the segment to be read this class should be registered with Java's
+   * SPI mechanism (registered in META-INF/ of your jar file, etc).
+   * @param name must be all ascii alphanumeric, and less than 128 characters in length.
+   */
   public Codec(String name) {
     NamedSPILoader.checkServiceName(name);
     this.name = name;
@@ -118,6 +126,10 @@ public abstract class Codec implements N
     defaultCodec = codec;
   }
 
+  /**
+   * returns the codec's name. Subclasses can override to provide
+   * more detail (such as parameters).
+   */
   @Override
   public String toString() {
     return name;

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/PostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/PostingsFormat.java?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/PostingsFormat.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/PostingsFormat.java Thu Aug 30 22:43:41 2012
@@ -18,14 +18,24 @@ package org.apache.lucene.codecs;
  */
 
 import java.io.IOException;
+import java.util.ServiceLoader;
 import java.util.Set;
 
+import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat; // javadocs
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.util.NamedSPILoader;
 
 /** 
  * Encodes/decodes terms, postings, and proximity data.
+ * <p>
+ * Note, when extending this class, the name ({@link #getName}) may
+ * written into the index in certain configurations. In order for the segment 
+ * to be read, the name must resolve to your implementation via {@link #forName(String)}.
+ * This method uses Java's 
+ * {@link ServiceLoader Service Provider Interface} to resolve codec names.
+ * <p>
+ * @see ServiceLoader
  * @lucene.experimental */
 public abstract class PostingsFormat implements NamedSPILoader.NamedSPI {
 
@@ -38,11 +48,21 @@ public abstract class PostingsFormat imp
    */
   private final String name;
   
+  /**
+   * Creates a new postings format.
+   * <p>
+   * The provided name will be written into the index segment in some configurations
+   * (such as when using {@link PerFieldPostingsFormat}): in such configurations,
+   * for the segment to be read this class should be registered with Java's
+   * SPI mechanism (registered in META-INF/ of your jar file, etc).
+   * @param name must be all ascii alphanumeric, and less than 128 characters in length.
+   */
   protected PostingsFormat(String name) {
     NamedSPILoader.checkServiceName(name);
     this.name = name;
   }
 
+  /** Returns this posting format's name */
   @Override
   public final String getName() {
     return name;

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/bloom/BloomFilteringPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/bloom/BloomFilteringPostingsFormat.java?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/bloom/BloomFilteringPostingsFormat.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/bloom/BloomFilteringPostingsFormat.java Thu Aug 30 22:43:41 2012
@@ -1,485 +1,485 @@
-package org.apache.lucene.codecs.bloom;
-
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.codecs.FieldsConsumer;
-import org.apache.lucene.codecs.FieldsProducer;
-import org.apache.lucene.codecs.PostingsConsumer;
-import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.TermStats;
-import org.apache.lucene.codecs.TermsConsumer;
-import org.apache.lucene.index.DocsAndPositionsEnum;
-import org.apache.lucene.index.DocsEnum;
-import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.SegmentReadState;
-import org.apache.lucene.index.SegmentWriteState;
-import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.store.DataOutput;
-import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.FuzzySet;
-import org.apache.lucene.util.FuzzySet.ContainsResult;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.automaton.CompiledAutomaton;
-import org.apache.lucene.util.hash.MurmurHash2;
-
-/**
- * <p>
- * A {@link PostingsFormat} useful for low doc-frequency fields such as primary
- * keys. Bloom filters are maintained in a ".blm" file which offers "fast-fail"
- * for reads in segments known to have no record of the key. A choice of
- * delegate PostingsFormat is used to record all other Postings data.
- * </p>
- * <p>
- * A choice of {@link BloomFilterFactory} can be passed to tailor Bloom Filter
- * settings on a per-field basis. The default configuration is
- * {@link DefaultBloomFilterFactory} which allocates a ~8mb bitset and hashes
- * values using {@link MurmurHash2}. This should be suitable for most purposes.
- * </p>
- * <p>
- * The format of the blm file is as follows:
- * </p>
- * <ul>
- * <li>BloomFilter (.blm) --&gt; Header, DelegatePostingsFormatName,
- * NumFilteredFields, Filter<sup>NumFilteredFields</sup></li>
- * <li>Filter --&gt; FieldNumber, FuzzySet</li>
- * <li>FuzzySet --&gt;See {@link FuzzySet#serialize(DataOutput)}</li>
- * <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
- * <li>DelegatePostingsFormatName --&gt; {@link DataOutput#writeString(String)
- * String} The name of a ServiceProvider registered {@link PostingsFormat}</li>
- * <li>NumFilteredFields --&gt; {@link DataOutput#writeInt Uint32}</li>
- * <li>FieldNumber --&gt; {@link DataOutput#writeInt Uint32} The number of the
- * field in this segment</li>
- * </ul>
- * @lucene.experimental
- */
-public class BloomFilteringPostingsFormat extends PostingsFormat {
-  
-  public static final String BLOOM_CODEC_NAME = "BloomFilter";
-  public static final int BLOOM_CODEC_VERSION = 1;
-  
-  /** Extension of Bloom Filters file */
-  static final String BLOOM_EXTENSION = "blm";
-  
-  BloomFilterFactory bloomFilterFactory = new DefaultBloomFilterFactory();
-  private PostingsFormat delegatePostingsFormat;
-  
-  /**
-   * Creates Bloom filters for a selection of fields created in the index. This
-   * is recorded as a set of Bitsets held as a segment summary in an additional
-   * "blm" file. This PostingsFormat delegates to a choice of delegate
-   * PostingsFormat for encoding all other postings data.
-   * 
-   * @param delegatePostingsFormat
-   *          The PostingsFormat that records all the non-bloom filter data i.e.
-   *          postings info.
-   * @param bloomFilterFactory
-   *          The {@link BloomFilterFactory} responsible for sizing BloomFilters
-   *          appropriately
-   */
-  public BloomFilteringPostingsFormat(PostingsFormat delegatePostingsFormat,
-      BloomFilterFactory bloomFilterFactory) {
-    super(BLOOM_CODEC_NAME);
-    this.delegatePostingsFormat = delegatePostingsFormat;
-    this.bloomFilterFactory = bloomFilterFactory;
-  }
-  
-  /**
-   * Creates Bloom filters for a selection of fields created in the index. This
-   * is recorded as a set of Bitsets held as a segment summary in an additional
-   * "blm" file. This PostingsFormat delegates to a choice of delegate
-   * PostingsFormat for encoding all other postings data. This choice of
-   * constructor defaults to the {@link DefaultBloomFilterFactory} for
-   * configuring per-field BloomFilters.
-   * 
-   * @param delegatePostingsFormat
-   *          The PostingsFormat that records all the non-bloom filter data i.e.
-   *          postings info.
-   */
-  public BloomFilteringPostingsFormat(PostingsFormat delegatePostingsFormat) {
-    this(delegatePostingsFormat, new DefaultBloomFilterFactory());
-  }
-  
-  // Used only by core Lucene at read-time via Service Provider instantiation -
-  // do not use at Write-time in application code.
-  public BloomFilteringPostingsFormat() {
-    super(BLOOM_CODEC_NAME);
-  }
-  
-  public FieldsConsumer fieldsConsumer(SegmentWriteState state)
-      throws IOException {
-    if (delegatePostingsFormat == null) {
-      throw new UnsupportedOperationException("Error - " + getClass().getName()
-          + " has been constructed without a choice of PostingsFormat");
-    }
-    return new BloomFilteredFieldsConsumer(
-        delegatePostingsFormat.fieldsConsumer(state), state,
-        delegatePostingsFormat);
-  }
-  
-  public FieldsProducer fieldsProducer(SegmentReadState state)
-      throws IOException {
-    return new BloomFilteredFieldsProducer(state);
-  }
-  
-  public class BloomFilteredFieldsProducer extends FieldsProducer {
-    private FieldsProducer delegateFieldsProducer;
-    HashMap<String,FuzzySet> bloomsByFieldName = new HashMap<String,FuzzySet>();
-    
-    public BloomFilteredFieldsProducer(SegmentReadState state)
-        throws IOException {
-      
-      String bloomFileName = IndexFileNames.segmentFileName(
-          state.segmentInfo.name, state.segmentSuffix, BLOOM_EXTENSION);
-      IndexInput bloomIn = null;
-      try {
-        bloomIn = state.dir.openInput(bloomFileName, state.context);
-        CodecUtil.checkHeader(bloomIn, BLOOM_CODEC_NAME, BLOOM_CODEC_VERSION,
-            BLOOM_CODEC_VERSION);
-        // // Load the hash function used in the BloomFilter
-        // hashFunction = HashFunction.forName(bloomIn.readString());
-        // Load the delegate postings format
-        PostingsFormat delegatePostingsFormat = PostingsFormat.forName(bloomIn
-            .readString());
-        
-        this.delegateFieldsProducer = delegatePostingsFormat
-            .fieldsProducer(state);
-        int numBlooms = bloomIn.readInt();
-        for (int i = 0; i < numBlooms; i++) {
-          int fieldNum = bloomIn.readInt();
-          FuzzySet bloom = FuzzySet.deserialize(bloomIn);
-          FieldInfo fieldInfo = state.fieldInfos.fieldInfo(fieldNum);
-          bloomsByFieldName.put(fieldInfo.name, bloom);
-        }
-      } finally {
-        IOUtils.close(bloomIn);
-      }
-      
-    }
-    
-    public Iterator<String> iterator() {
-      return delegateFieldsProducer.iterator();
-    }
-    
-    public void close() throws IOException {
-      delegateFieldsProducer.close();
-    }
-    
-    public Terms terms(String field) throws IOException {
-      FuzzySet filter = bloomsByFieldName.get(field);
-      if (filter == null) {
-        return delegateFieldsProducer.terms(field);
-      } else {
-        Terms result = delegateFieldsProducer.terms(field);
-        if (result == null) {
-          return null;
-        }
-        return new BloomFilteredTerms(result, filter);
-      }
-    }
-    
-    public int size() {
-      return delegateFieldsProducer.size();
-    }
-    
-    class BloomFilteredTerms extends Terms {
-      private Terms delegateTerms;
-      private FuzzySet filter;
-      
-      public BloomFilteredTerms(Terms terms, FuzzySet filter) {
-        this.delegateTerms = terms;
-        this.filter = filter;
-      }
-      
-      @Override
-      public TermsEnum intersect(CompiledAutomaton compiled,
-          final BytesRef startTerm) throws IOException {
-        return delegateTerms.intersect(compiled, startTerm);
-      }
-      
-      @Override
-      public TermsEnum iterator(TermsEnum reuse) throws IOException {
-        TermsEnum result;
-        if ((reuse != null) && (reuse instanceof BloomFilteredTermsEnum)) {
-          // recycle the existing BloomFilteredTermsEnum by asking the delegate
-          // to recycle its contained TermsEnum
-          BloomFilteredTermsEnum bfte = (BloomFilteredTermsEnum) reuse;
-          if (bfte.filter == filter) {
-            bfte.delegateTermsEnum = delegateTerms
-                .iterator(bfte.delegateTermsEnum);
-            return bfte;
-          }
-        }
-        // We have been handed something we cannot reuse (either null, wrong
-        // class or wrong filter) so allocate a new object
-        result = new BloomFilteredTermsEnum(delegateTerms.iterator(reuse),
-            filter);
-        return result;
-      }
-      
-      @Override
-      public Comparator<BytesRef> getComparator() throws IOException {
-        return delegateTerms.getComparator();
-      }
-      
-      @Override
-      public long size() throws IOException {
-        return delegateTerms.size();
-      }
-      
-      @Override
-      public long getSumTotalTermFreq() throws IOException {
-        return delegateTerms.getSumTotalTermFreq();
-      }
-      
-      @Override
-      public long getSumDocFreq() throws IOException {
-        return delegateTerms.getSumDocFreq();
-      }
-      
-      @Override
-      public int getDocCount() throws IOException {
-        return delegateTerms.getDocCount();
-      }
-
-      @Override
-      public boolean hasOffsets() {
-        return delegateTerms.hasOffsets();
-      }
-
-      @Override
-      public boolean hasPositions() {
-        return delegateTerms.hasPositions();
-      }
-      
-      @Override
-      public boolean hasPayloads() {
-        return delegateTerms.hasPayloads();
-      }
-    }
-    
-    class BloomFilteredTermsEnum extends TermsEnum {
-      
-      TermsEnum delegateTermsEnum;
-      private FuzzySet filter;
-      
-      public BloomFilteredTermsEnum(TermsEnum iterator, FuzzySet filter) {
-        this.delegateTermsEnum = iterator;
-        this.filter = filter;
-      }
-      
-      @Override
-      public final BytesRef next() throws IOException {
-        return delegateTermsEnum.next();
-      }
-      
-      @Override
-      public final Comparator<BytesRef> getComparator() {
-        return delegateTermsEnum.getComparator();
-      }
-      
-      @Override
-      public final boolean seekExact(BytesRef text, boolean useCache)
-          throws IOException {
-        // The magical fail-fast speed up that is the entire point of all of
-        // this code - save a disk seek if there is a match on an in-memory
-        // structure
-        // that may occasionally give a false positive but guaranteed no false
-        // negatives
-        if (filter.contains(text) == ContainsResult.NO) {
-          return false;
-        }
-        return delegateTermsEnum.seekExact(text, useCache);
-      }
-      
-      @Override
-      public final SeekStatus seekCeil(BytesRef text, boolean useCache)
-          throws IOException {
-        return delegateTermsEnum.seekCeil(text, useCache);
-      }
-      
-      @Override
-      public final void seekExact(long ord) throws IOException {
-        delegateTermsEnum.seekExact(ord);
-      }
-      
-      @Override
-      public final BytesRef term() throws IOException {
-        return delegateTermsEnum.term();
-      }
-      
-      @Override
-      public final long ord() throws IOException {
-        return delegateTermsEnum.ord();
-      }
-      
-      @Override
-      public final int docFreq() throws IOException {
-        return delegateTermsEnum.docFreq();
-      }
-      
-      @Override
-      public final long totalTermFreq() throws IOException {
-        return delegateTermsEnum.totalTermFreq();
-      }
-      
-
-      @Override
-      public DocsAndPositionsEnum docsAndPositions(Bits liveDocs,
-          DocsAndPositionsEnum reuse, int flags) throws IOException {
-        return delegateTermsEnum.docsAndPositions(liveDocs, reuse, flags);
-      }
-
-      @Override
-      public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags)
-          throws IOException {
-        return delegateTermsEnum.docs(liveDocs, reuse, flags);
-      }
-      
-      
-    }
-    
-  }
-  
-  class BloomFilteredFieldsConsumer extends FieldsConsumer {
-    private FieldsConsumer delegateFieldsConsumer;
-    private Map<FieldInfo,FuzzySet> bloomFilters = new HashMap<FieldInfo,FuzzySet>();
-    private SegmentWriteState state;
-    
-    // private PostingsFormat delegatePostingsFormat;
-    
-    public BloomFilteredFieldsConsumer(FieldsConsumer fieldsConsumer,
-        SegmentWriteState state, PostingsFormat delegatePostingsFormat) {
-      this.delegateFieldsConsumer = fieldsConsumer;
-      // this.delegatePostingsFormat=delegatePostingsFormat;
-      this.state = state;
-    }
-    
-    @Override
-    public TermsConsumer addField(FieldInfo field) throws IOException {
-      FuzzySet bloomFilter = bloomFilterFactory.getSetForField(state,field);
-      if (bloomFilter != null) {
-        assert bloomFilters.containsKey(field) == false;
-        bloomFilters.put(field, bloomFilter);
-        return new WrappedTermsConsumer(delegateFieldsConsumer.addField(field),bloomFilter);
-      } else {
-        // No, use the unfiltered fieldsConsumer - we are not interested in
-        // recording any term Bitsets.
-        return delegateFieldsConsumer.addField(field);
-      }
-    }
-    
-    @Override
-    public void close() throws IOException {
-      delegateFieldsConsumer.close();
-      // Now we are done accumulating values for these fields
-      List<Entry<FieldInfo,FuzzySet>> nonSaturatedBlooms = new ArrayList<Map.Entry<FieldInfo,FuzzySet>>();
-      
-      for (Entry<FieldInfo,FuzzySet> entry : bloomFilters.entrySet()) {
-        FuzzySet bloomFilter = entry.getValue();
-        if(!bloomFilterFactory.isSaturated(bloomFilter,entry.getKey())){          
-          nonSaturatedBlooms.add(entry);
-        }
-      }
-      String bloomFileName = IndexFileNames.segmentFileName(
-          state.segmentInfo.name, state.segmentSuffix, BLOOM_EXTENSION);
-      IndexOutput bloomOutput = null;
-      try {
-        bloomOutput = state.directory
-            .createOutput(bloomFileName, state.context);
-        CodecUtil.writeHeader(bloomOutput, BLOOM_CODEC_NAME,
-            BLOOM_CODEC_VERSION);
-        // remember the name of the postings format we will delegate to
-        bloomOutput.writeString(delegatePostingsFormat.getName());
-        
-        // First field in the output file is the number of fields+blooms saved
-        bloomOutput.writeInt(nonSaturatedBlooms.size());
-        for (Entry<FieldInfo,FuzzySet> entry : nonSaturatedBlooms) {
-          FieldInfo fieldInfo = entry.getKey();
-          FuzzySet bloomFilter = entry.getValue();
-          bloomOutput.writeInt(fieldInfo.number);
-          saveAppropriatelySizedBloomFilter(bloomOutput, bloomFilter, fieldInfo);
-        }
-      } finally {
-        IOUtils.close(bloomOutput);
-      }
-      //We are done with large bitsets so no need to keep them hanging around
-      bloomFilters.clear(); 
-    }
-    
-    private void saveAppropriatelySizedBloomFilter(IndexOutput bloomOutput,
-        FuzzySet bloomFilter, FieldInfo fieldInfo) throws IOException {
-      
-      FuzzySet rightSizedSet = bloomFilterFactory.downsize(fieldInfo,
-          bloomFilter);
-      if (rightSizedSet == null) {
-        rightSizedSet = bloomFilter;
-      }
-      rightSizedSet.serialize(bloomOutput);
-    }
-    
-  }
-  
-  class WrappedTermsConsumer extends TermsConsumer {
-    private TermsConsumer delegateTermsConsumer;
-    private FuzzySet bloomFilter;
-    
-    public WrappedTermsConsumer(TermsConsumer termsConsumer,FuzzySet bloomFilter) {
-      this.delegateTermsConsumer = termsConsumer;
-      this.bloomFilter = bloomFilter;
-    }
-    
-    public PostingsConsumer startTerm(BytesRef text) throws IOException {
-      return delegateTermsConsumer.startTerm(text);
-    }
-    
-    public void finishTerm(BytesRef text, TermStats stats) throws IOException {
-      
-      // Record this term in our BloomFilter
-      if (stats.docFreq > 0) {
-        bloomFilter.addValue(text);
-      }
-      delegateTermsConsumer.finishTerm(text, stats);
-    }
-    
-    public void finish(long sumTotalTermFreq, long sumDocFreq, int docCount)
-        throws IOException {
-      delegateTermsConsumer.finish(sumTotalTermFreq, sumDocFreq, docCount);
-    }
-    
-    public Comparator<BytesRef> getComparator() throws IOException {
-      return delegateTermsConsumer.getComparator();
-    }
-    
-  }
-  
-}
+package org.apache.lucene.codecs.bloom;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.FieldsConsumer;
+import org.apache.lucene.codecs.FieldsProducer;
+import org.apache.lucene.codecs.PostingsConsumer;
+import org.apache.lucene.codecs.PostingsFormat;
+import org.apache.lucene.codecs.TermStats;
+import org.apache.lucene.codecs.TermsConsumer;
+import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.FuzzySet;
+import org.apache.lucene.util.FuzzySet.ContainsResult;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.automaton.CompiledAutomaton;
+import org.apache.lucene.util.hash.MurmurHash2;
+
+/**
+ * <p>
+ * A {@link PostingsFormat} useful for low doc-frequency fields such as primary
+ * keys. Bloom filters are maintained in a ".blm" file which offers "fast-fail"
+ * for reads in segments known to have no record of the key. A choice of
+ * delegate PostingsFormat is used to record all other Postings data.
+ * </p>
+ * <p>
+ * A choice of {@link BloomFilterFactory} can be passed to tailor Bloom Filter
+ * settings on a per-field basis. The default configuration is
+ * {@link DefaultBloomFilterFactory} which allocates a ~8mb bitset and hashes
+ * values using {@link MurmurHash2}. This should be suitable for most purposes.
+ * </p>
+ * <p>
+ * The format of the blm file is as follows:
+ * </p>
+ * <ul>
+ * <li>BloomFilter (.blm) --&gt; Header, DelegatePostingsFormatName,
+ * NumFilteredFields, Filter<sup>NumFilteredFields</sup></li>
+ * <li>Filter --&gt; FieldNumber, FuzzySet</li>
+ * <li>FuzzySet --&gt;See {@link FuzzySet#serialize(DataOutput)}</li>
+ * <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
+ * <li>DelegatePostingsFormatName --&gt; {@link DataOutput#writeString(String)
+ * String} The name of a ServiceProvider registered {@link PostingsFormat}</li>
+ * <li>NumFilteredFields --&gt; {@link DataOutput#writeInt Uint32}</li>
+ * <li>FieldNumber --&gt; {@link DataOutput#writeInt Uint32} The number of the
+ * field in this segment</li>
+ * </ul>
+ * @lucene.experimental
+ */
+public class BloomFilteringPostingsFormat extends PostingsFormat {
+  
+  public static final String BLOOM_CODEC_NAME = "BloomFilter";
+  public static final int BLOOM_CODEC_VERSION = 1;
+  
+  /** Extension of Bloom Filters file */
+  static final String BLOOM_EXTENSION = "blm";
+  
+  BloomFilterFactory bloomFilterFactory = new DefaultBloomFilterFactory();
+  private PostingsFormat delegatePostingsFormat;
+  
+  /**
+   * Creates Bloom filters for a selection of fields created in the index. This
+   * is recorded as a set of Bitsets held as a segment summary in an additional
+   * "blm" file. This PostingsFormat delegates to a choice of delegate
+   * PostingsFormat for encoding all other postings data.
+   * 
+   * @param delegatePostingsFormat
+   *          The PostingsFormat that records all the non-bloom filter data i.e.
+   *          postings info.
+   * @param bloomFilterFactory
+   *          The {@link BloomFilterFactory} responsible for sizing BloomFilters
+   *          appropriately
+   */
+  public BloomFilteringPostingsFormat(PostingsFormat delegatePostingsFormat,
+      BloomFilterFactory bloomFilterFactory) {
+    super(BLOOM_CODEC_NAME);
+    this.delegatePostingsFormat = delegatePostingsFormat;
+    this.bloomFilterFactory = bloomFilterFactory;
+  }
+  
+  /**
+   * Creates Bloom filters for a selection of fields created in the index. This
+   * is recorded as a set of Bitsets held as a segment summary in an additional
+   * "blm" file. This PostingsFormat delegates to a choice of delegate
+   * PostingsFormat for encoding all other postings data. This choice of
+   * constructor defaults to the {@link DefaultBloomFilterFactory} for
+   * configuring per-field BloomFilters.
+   * 
+   * @param delegatePostingsFormat
+   *          The PostingsFormat that records all the non-bloom filter data i.e.
+   *          postings info.
+   */
+  public BloomFilteringPostingsFormat(PostingsFormat delegatePostingsFormat) {
+    this(delegatePostingsFormat, new DefaultBloomFilterFactory());
+  }
+  
+  // Used only by core Lucene at read-time via Service Provider instantiation -
+  // do not use at Write-time in application code.
+  public BloomFilteringPostingsFormat() {
+    super(BLOOM_CODEC_NAME);
+  }
+  
+  public FieldsConsumer fieldsConsumer(SegmentWriteState state)
+      throws IOException {
+    if (delegatePostingsFormat == null) {
+      throw new UnsupportedOperationException("Error - " + getClass().getName()
+          + " has been constructed without a choice of PostingsFormat");
+    }
+    return new BloomFilteredFieldsConsumer(
+        delegatePostingsFormat.fieldsConsumer(state), state,
+        delegatePostingsFormat);
+  }
+  
+  public FieldsProducer fieldsProducer(SegmentReadState state)
+      throws IOException {
+    return new BloomFilteredFieldsProducer(state);
+  }
+  
+  public class BloomFilteredFieldsProducer extends FieldsProducer {
+    private FieldsProducer delegateFieldsProducer;
+    HashMap<String,FuzzySet> bloomsByFieldName = new HashMap<String,FuzzySet>();
+    
+    public BloomFilteredFieldsProducer(SegmentReadState state)
+        throws IOException {
+      
+      String bloomFileName = IndexFileNames.segmentFileName(
+          state.segmentInfo.name, state.segmentSuffix, BLOOM_EXTENSION);
+      IndexInput bloomIn = null;
+      try {
+        bloomIn = state.dir.openInput(bloomFileName, state.context);
+        CodecUtil.checkHeader(bloomIn, BLOOM_CODEC_NAME, BLOOM_CODEC_VERSION,
+            BLOOM_CODEC_VERSION);
+        // // Load the hash function used in the BloomFilter
+        // hashFunction = HashFunction.forName(bloomIn.readString());
+        // Load the delegate postings format
+        PostingsFormat delegatePostingsFormat = PostingsFormat.forName(bloomIn
+            .readString());
+        
+        this.delegateFieldsProducer = delegatePostingsFormat
+            .fieldsProducer(state);
+        int numBlooms = bloomIn.readInt();
+        for (int i = 0; i < numBlooms; i++) {
+          int fieldNum = bloomIn.readInt();
+          FuzzySet bloom = FuzzySet.deserialize(bloomIn);
+          FieldInfo fieldInfo = state.fieldInfos.fieldInfo(fieldNum);
+          bloomsByFieldName.put(fieldInfo.name, bloom);
+        }
+      } finally {
+        IOUtils.close(bloomIn);
+      }
+      
+    }
+    
+    public Iterator<String> iterator() {
+      return delegateFieldsProducer.iterator();
+    }
+    
+    public void close() throws IOException {
+      delegateFieldsProducer.close();
+    }
+    
+    public Terms terms(String field) throws IOException {
+      FuzzySet filter = bloomsByFieldName.get(field);
+      if (filter == null) {
+        return delegateFieldsProducer.terms(field);
+      } else {
+        Terms result = delegateFieldsProducer.terms(field);
+        if (result == null) {
+          return null;
+        }
+        return new BloomFilteredTerms(result, filter);
+      }
+    }
+    
+    public int size() {
+      return delegateFieldsProducer.size();
+    }
+    
+    class BloomFilteredTerms extends Terms {
+      private Terms delegateTerms;
+      private FuzzySet filter;
+      
+      public BloomFilteredTerms(Terms terms, FuzzySet filter) {
+        this.delegateTerms = terms;
+        this.filter = filter;
+      }
+      
+      @Override
+      public TermsEnum intersect(CompiledAutomaton compiled,
+          final BytesRef startTerm) throws IOException {
+        return delegateTerms.intersect(compiled, startTerm);
+      }
+      
+      @Override
+      public TermsEnum iterator(TermsEnum reuse) throws IOException {
+        TermsEnum result;
+        if ((reuse != null) && (reuse instanceof BloomFilteredTermsEnum)) {
+          // recycle the existing BloomFilteredTermsEnum by asking the delegate
+          // to recycle its contained TermsEnum
+          BloomFilteredTermsEnum bfte = (BloomFilteredTermsEnum) reuse;
+          if (bfte.filter == filter) {
+            bfte.delegateTermsEnum = delegateTerms
+                .iterator(bfte.delegateTermsEnum);
+            return bfte;
+          }
+        }
+        // We have been handed something we cannot reuse (either null, wrong
+        // class or wrong filter) so allocate a new object
+        result = new BloomFilteredTermsEnum(delegateTerms.iterator(reuse),
+            filter);
+        return result;
+      }
+      
+      @Override
+      public Comparator<BytesRef> getComparator() throws IOException {
+        return delegateTerms.getComparator();
+      }
+      
+      @Override
+      public long size() throws IOException {
+        return delegateTerms.size();
+      }
+      
+      @Override
+      public long getSumTotalTermFreq() throws IOException {
+        return delegateTerms.getSumTotalTermFreq();
+      }
+      
+      @Override
+      public long getSumDocFreq() throws IOException {
+        return delegateTerms.getSumDocFreq();
+      }
+      
+      @Override
+      public int getDocCount() throws IOException {
+        return delegateTerms.getDocCount();
+      }
+
+      @Override
+      public boolean hasOffsets() {
+        return delegateTerms.hasOffsets();
+      }
+
+      @Override
+      public boolean hasPositions() {
+        return delegateTerms.hasPositions();
+      }
+      
+      @Override
+      public boolean hasPayloads() {
+        return delegateTerms.hasPayloads();
+      }
+    }
+    
+    class BloomFilteredTermsEnum extends TermsEnum {
+      
+      TermsEnum delegateTermsEnum;
+      private FuzzySet filter;
+      
+      public BloomFilteredTermsEnum(TermsEnum iterator, FuzzySet filter) {
+        this.delegateTermsEnum = iterator;
+        this.filter = filter;
+      }
+      
+      @Override
+      public final BytesRef next() throws IOException {
+        return delegateTermsEnum.next();
+      }
+      
+      @Override
+      public final Comparator<BytesRef> getComparator() {
+        return delegateTermsEnum.getComparator();
+      }
+      
+      @Override
+      public final boolean seekExact(BytesRef text, boolean useCache)
+          throws IOException {
+        // The magical fail-fast speed up that is the entire point of all of
+        // this code - save a disk seek if there is a match on an in-memory
+        // structure
+        // that may occasionally give a false positive but guaranteed no false
+        // negatives
+        if (filter.contains(text) == ContainsResult.NO) {
+          return false;
+        }
+        return delegateTermsEnum.seekExact(text, useCache);
+      }
+      
+      @Override
+      public final SeekStatus seekCeil(BytesRef text, boolean useCache)
+          throws IOException {
+        return delegateTermsEnum.seekCeil(text, useCache);
+      }
+      
+      @Override
+      public final void seekExact(long ord) throws IOException {
+        delegateTermsEnum.seekExact(ord);
+      }
+      
+      @Override
+      public final BytesRef term() throws IOException {
+        return delegateTermsEnum.term();
+      }
+      
+      @Override
+      public final long ord() throws IOException {
+        return delegateTermsEnum.ord();
+      }
+      
+      @Override
+      public final int docFreq() throws IOException {
+        return delegateTermsEnum.docFreq();
+      }
+      
+      @Override
+      public final long totalTermFreq() throws IOException {
+        return delegateTermsEnum.totalTermFreq();
+      }
+      
+
+      @Override
+      public DocsAndPositionsEnum docsAndPositions(Bits liveDocs,
+          DocsAndPositionsEnum reuse, int flags) throws IOException {
+        return delegateTermsEnum.docsAndPositions(liveDocs, reuse, flags);
+      }
+
+      @Override
+      public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags)
+          throws IOException {
+        return delegateTermsEnum.docs(liveDocs, reuse, flags);
+      }
+      
+      
+    }
+    
+  }
+  
+  class BloomFilteredFieldsConsumer extends FieldsConsumer {
+    private FieldsConsumer delegateFieldsConsumer;
+    private Map<FieldInfo,FuzzySet> bloomFilters = new HashMap<FieldInfo,FuzzySet>();
+    private SegmentWriteState state;
+    
+    // private PostingsFormat delegatePostingsFormat;
+    
+    public BloomFilteredFieldsConsumer(FieldsConsumer fieldsConsumer,
+        SegmentWriteState state, PostingsFormat delegatePostingsFormat) {
+      this.delegateFieldsConsumer = fieldsConsumer;
+      // this.delegatePostingsFormat=delegatePostingsFormat;
+      this.state = state;
+    }
+    
+    @Override
+    public TermsConsumer addField(FieldInfo field) throws IOException {
+      FuzzySet bloomFilter = bloomFilterFactory.getSetForField(state,field);
+      if (bloomFilter != null) {
+        assert bloomFilters.containsKey(field) == false;
+        bloomFilters.put(field, bloomFilter);
+        return new WrappedTermsConsumer(delegateFieldsConsumer.addField(field),bloomFilter);
+      } else {
+        // No, use the unfiltered fieldsConsumer - we are not interested in
+        // recording any term Bitsets.
+        return delegateFieldsConsumer.addField(field);
+      }
+    }
+    
+    @Override
+    public void close() throws IOException {
+      delegateFieldsConsumer.close();
+      // Now we are done accumulating values for these fields
+      List<Entry<FieldInfo,FuzzySet>> nonSaturatedBlooms = new ArrayList<Map.Entry<FieldInfo,FuzzySet>>();
+      
+      for (Entry<FieldInfo,FuzzySet> entry : bloomFilters.entrySet()) {
+        FuzzySet bloomFilter = entry.getValue();
+        if(!bloomFilterFactory.isSaturated(bloomFilter,entry.getKey())){          
+          nonSaturatedBlooms.add(entry);
+        }
+      }
+      String bloomFileName = IndexFileNames.segmentFileName(
+          state.segmentInfo.name, state.segmentSuffix, BLOOM_EXTENSION);
+      IndexOutput bloomOutput = null;
+      try {
+        bloomOutput = state.directory
+            .createOutput(bloomFileName, state.context);
+        CodecUtil.writeHeader(bloomOutput, BLOOM_CODEC_NAME,
+            BLOOM_CODEC_VERSION);
+        // remember the name of the postings format we will delegate to
+        bloomOutput.writeString(delegatePostingsFormat.getName());
+        
+        // First field in the output file is the number of fields+blooms saved
+        bloomOutput.writeInt(nonSaturatedBlooms.size());
+        for (Entry<FieldInfo,FuzzySet> entry : nonSaturatedBlooms) {
+          FieldInfo fieldInfo = entry.getKey();
+          FuzzySet bloomFilter = entry.getValue();
+          bloomOutput.writeInt(fieldInfo.number);
+          saveAppropriatelySizedBloomFilter(bloomOutput, bloomFilter, fieldInfo);
+        }
+      } finally {
+        IOUtils.close(bloomOutput);
+      }
+      //We are done with large bitsets so no need to keep them hanging around
+      bloomFilters.clear(); 
+    }
+    
+    private void saveAppropriatelySizedBloomFilter(IndexOutput bloomOutput,
+        FuzzySet bloomFilter, FieldInfo fieldInfo) throws IOException {
+      
+      FuzzySet rightSizedSet = bloomFilterFactory.downsize(fieldInfo,
+          bloomFilter);
+      if (rightSizedSet == null) {
+        rightSizedSet = bloomFilter;
+      }
+      rightSizedSet.serialize(bloomOutput);
+    }
+    
+  }
+  
+  class WrappedTermsConsumer extends TermsConsumer {
+    private TermsConsumer delegateTermsConsumer;
+    private FuzzySet bloomFilter;
+    
+    public WrappedTermsConsumer(TermsConsumer termsConsumer,FuzzySet bloomFilter) {
+      this.delegateTermsConsumer = termsConsumer;
+      this.bloomFilter = bloomFilter;
+    }
+    
+    public PostingsConsumer startTerm(BytesRef text) throws IOException {
+      return delegateTermsConsumer.startTerm(text);
+    }
+    
+    public void finishTerm(BytesRef text, TermStats stats) throws IOException {
+      
+      // Record this term in our BloomFilter
+      if (stats.docFreq > 0) {
+        bloomFilter.addValue(text);
+      }
+      delegateTermsConsumer.finishTerm(text, stats);
+    }
+    
+    public void finish(long sumTotalTermFreq, long sumDocFreq, int docCount)
+        throws IOException {
+      delegateTermsConsumer.finish(sumTotalTermFreq, sumDocFreq, docCount);
+    }
+    
+    public Comparator<BytesRef> getComparator() throws IOException {
+      return delegateTermsConsumer.getComparator();
+    }
+    
+  }
+  
+}

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/bloom/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/bloom/package.html?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/bloom/package.html (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/bloom/package.html Thu Aug 30 22:43:41 2012
@@ -1,25 +1,25 @@
-<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
-<!--
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements.  See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License.  You may obtain a copy of the License at
-
-     http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
--->
-<html>
-<head>
-   <meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
-</head>
-<body>
-Codec PostingsFormat for fast access to low-frequency terms such as primary key fields.
-</body>
+<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements.  See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+<html>
+<head>
+   <meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
+</head>
+<body>
+Codec PostingsFormat for fast access to low-frequency terms such as primary key fields.
+</body>
 </html>
\ No newline at end of file

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/ByteDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/ByteDocValuesField.java?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/ByteDocValuesField.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/ByteDocValuesField.java Thu Aug 30 22:43:41 2012
@@ -38,12 +38,21 @@ import org.apache.lucene.index.DocValues
 
 public class ByteDocValuesField extends StoredField {
 
+  /**
+   * Type for 8-bit byte DocValues.
+   */
   public static final FieldType TYPE = new FieldType();
   static {
     TYPE.setDocValueType(DocValues.Type.FIXED_INTS_8);
     TYPE.freeze();
   }
 
+  /** 
+   * Creates a new DocValues field with the specified 8-bit byte value 
+   * @param name field name
+   * @param value 8-bit byte value
+   * @throws IllegalArgumentException if the field name is null.
+   */
   public ByteDocValuesField(String name, byte value) {
     super(name, TYPE);
     fieldsData = Byte.valueOf(value);

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/CompressionTools.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/CompressionTools.java?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/CompressionTools.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/CompressionTools.java Thu Aug 30 22:43:41 2012
@@ -92,10 +92,14 @@ public class CompressionTools {
     return compress(result.bytes, 0, result.length, compressionLevel);
   }
 
+  /** Decompress the byte array previously returned by
+   *  compress (referenced by the provided BytesRef) */
   public static byte[] decompress(BytesRef bytes) throws DataFormatException {
     return decompress(bytes.bytes, bytes.offset, bytes.length);
   }
 
+  /** Decompress the byte array previously returned by
+   *  compress */
   public static byte[] decompress(byte[] value) throws DataFormatException {
     return decompress(value, 0, value.length);
   }
@@ -130,6 +134,8 @@ public class CompressionTools {
     return decompressString(value, 0, value.length);
   }
 
+  /** Decompress the byte array previously returned by
+   *  compressString back into a String */
   public static String decompressString(byte[] value, int offset, int length) throws DataFormatException {
     final byte[] bytes = decompress(value, offset, length);
     CharsRef result = new CharsRef(bytes.length);
@@ -137,6 +143,8 @@ public class CompressionTools {
     return new String(result.chars, 0, result.length);
   }
 
+  /** Decompress the byte array (referenced by the provided BytesRef) 
+   *  previously returned by compressString back into a String */
   public static String decompressString(BytesRef bytes) throws DataFormatException {
     return decompressString(bytes.bytes, bytes.offset, bytes.length);
   }

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/DateTools.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/DateTools.java?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/DateTools.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/DateTools.java Thu Aug 30 22:43:41 2012
@@ -185,7 +185,20 @@ public class DateTools {
   /** Specifies the time granularity. */
   public static enum Resolution {
     
-    YEAR(4), MONTH(6), DAY(8), HOUR(10), MINUTE(12), SECOND(14), MILLISECOND(17);
+    /** Limit a date's resolution to year granularity. */
+    YEAR(4), 
+    /** Limit a date's resolution to month granularity. */
+    MONTH(6), 
+    /** Limit a date's resolution to day granularity. */
+    DAY(8), 
+    /** Limit a date's resolution to hour granularity. */
+    HOUR(10), 
+    /** Limit a date's resolution to minute granularity. */
+    MINUTE(12), 
+    /** Limit a date's resolution to second granularity. */
+    SECOND(14), 
+    /** Limit a date's resolution to millisecond granularity. */
+    MILLISECOND(17);
 
     final int formatLen;
     final SimpleDateFormat format;//should be cloned before use, since it's not threadsafe

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/DerefBytesDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/DerefBytesDocValuesField.java?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/DerefBytesDocValuesField.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/DerefBytesDocValuesField.java Thu Aug 30 22:43:41 2012
@@ -44,23 +44,49 @@ import org.apache.lucene.util.BytesRef;
 public class DerefBytesDocValuesField extends StoredField {
 
   // TODO: ideally indexer figures out var vs fixed on its own!?
+  /**
+   * Type for indirect bytes DocValues: all with the same length
+   */
   public static final FieldType TYPE_FIXED_LEN = new FieldType();
   static {
     TYPE_FIXED_LEN.setDocValueType(DocValues.Type.BYTES_FIXED_DEREF);
     TYPE_FIXED_LEN.freeze();
   }
 
+  /**
+   * Type for indirect bytes DocValues: can have variable lengths
+   */
   public static final FieldType TYPE_VAR_LEN = new FieldType();
   static {
     TYPE_VAR_LEN.setDocValueType(DocValues.Type.BYTES_VAR_DEREF);
     TYPE_VAR_LEN.freeze();
   }
 
+  /**
+   * Create a new variable-length indirect DocValues field.
+   * <p>
+   * This calls 
+   * {@link DerefBytesDocValuesField#DerefBytesDocValuesField(String, BytesRef, boolean)
+   *  DerefBytesDocValuesField(name, bytes, false}, meaning by default
+   * it allows for values of different lengths. If your values are all 
+   * the same length, use that constructor instead.
+   * @param name field name
+   * @param bytes binary content
+   * @throws IllegalArgumentException if the field name is null
+   */
   public DerefBytesDocValuesField(String name, BytesRef bytes) {
     super(name, TYPE_VAR_LEN);
     fieldsData = bytes;
   }
 
+  /**
+   * Create a new fixed or variable length indirect DocValues field.
+   * <p>
+   * @param name field name
+   * @param bytes binary content
+   * @param isFixedLength true if all values have the same length.
+   * @throws IllegalArgumentException if the field name is null
+   */
   public DerefBytesDocValuesField(String name, BytesRef bytes, boolean isFixedLength) {
     super(name, isFixedLength ? TYPE_FIXED_LEN : TYPE_VAR_LEN);
     fieldsData = bytes;

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/DocumentStoredFieldVisitor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/DocumentStoredFieldVisitor.java?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/DocumentStoredFieldVisitor.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/DocumentStoredFieldVisitor.java Thu Aug 30 22:43:41 2012
@@ -97,6 +97,10 @@ public class DocumentStoredFieldVisitor 
     return fieldsToAdd == null || fieldsToAdd.contains(fieldInfo.name) ? Status.YES : Status.NO;
   }
 
+  /**
+   * Retrieve the visited document.
+   * @return StoredDocument populated with stored fields.
+   */
   public StoredDocument getDocument() {
     return doc;
   }

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/DoubleDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/DoubleDocValuesField.java?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/DoubleDocValuesField.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/DoubleDocValuesField.java Thu Aug 30 22:43:41 2012
@@ -38,12 +38,21 @@ import org.apache.lucene.index.DocValues
 
 public class DoubleDocValuesField extends StoredField {
 
+  /**
+   * Type for 64-bit double DocValues.
+   */
   public static final FieldType TYPE = new FieldType();
   static {
     TYPE.setDocValueType(DocValues.Type.FLOAT_64);
     TYPE.freeze();
   }
 
+  /** 
+   * Creates a new DocValues field with the specified 64-bit double value 
+   * @param name field name
+   * @param value 64-bit double value
+   * @throws IllegalArgumentException if the field name is null
+   */
   public DoubleDocValuesField(String name, double value) {
     super(name, TYPE);
     fieldsData = Double.valueOf(value);

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/DoubleField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/DoubleField.java?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/DoubleField.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/DoubleField.java Thu Aug 30 22:43:41 2012
@@ -114,6 +114,10 @@ import org.apache.lucene.util.NumericUti
 
 public final class DoubleField extends Field {
   
+  /** 
+   * Type for a DoubleField that is not stored:
+   * normalization factors, frequencies, and positions are omitted.
+   */
   public static final FieldType TYPE_NOT_STORED = new FieldType();
   static {
     TYPE_NOT_STORED.setIndexed(true);
@@ -124,6 +128,10 @@ public final class DoubleField extends F
     TYPE_NOT_STORED.freeze();
   }
 
+  /** 
+   * Type for a stored DoubleField:
+   * normalization factors, frequencies, and positions are omitted.
+   */
   public static final FieldType TYPE_STORED = new FieldType();
   static {
     TYPE_STORED.setIndexed(true);
@@ -137,14 +145,26 @@ public final class DoubleField extends F
 
   /** Creates a stored or un-stored DoubleField with the provided value
    *  and default <code>precisionStep</code> {@link
-   *  NumericUtils#PRECISION_STEP_DEFAULT} (4). */
+   *  NumericUtils#PRECISION_STEP_DEFAULT} (4). 
+   *  @param name field name
+   *  @param value 64-bit double value
+   *  @param stored Store.YES if the content should also be stored
+   *  @throws IllegalArgumentException if the field name is null. 
+   */
   public DoubleField(String name, double value, Store stored) {
     super(name, stored == Store.YES ? TYPE_STORED : TYPE_NOT_STORED);
     fieldsData = Double.valueOf(value);
   }
   
   /** Expert: allows you to customize the {@link
-   *  FieldType}. */
+   *  FieldType}. 
+   *  @param name field name
+   *  @param value 64-bit double value
+   *  @param type customized field type: must have {@link FieldType#numericType()}
+   *         of {@link FieldType.NumericType#DOUBLE}.
+   *  @throws IllegalArgumentException if the field name or type is null, or
+   *          if the field type does not have a DOUBLE numericType()
+   */
   public DoubleField(String name, double value, FieldType type) {
     super(name, type);
     if (type.numericType() != FieldType.NumericType.DOUBLE) {

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/Field.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/Field.java?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/Field.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/Field.java Thu Aug 30 22:43:41 2012
@@ -61,23 +61,42 @@ import org.apache.lucene.index.FieldInve
  */
 public class Field implements IndexableField, StorableField {
 
+  /**
+   * Field's type
+   */
   protected final FieldType type;
+
+  /**
+   * Field's name
+   */
   protected final String name;
 
-  // Field's value:
+  /** Field's value */
   protected Object fieldsData;
 
-  // Pre-analyzed tokenStream for indexed fields; this is
-  // separate from fieldsData because you are allowed to
-  // have both; eg maybe field has a String value but you
-  // customize how it's tokenized:
+  /** Pre-analyzed tokenStream for indexed fields; this is
+   * separate from fieldsData because you are allowed to
+   * have both; eg maybe field has a String value but you
+   * customize how it's tokenized */
   protected TokenStream tokenStream;
 
   private transient TokenStream internalTokenStream;
   private transient ReusableStringReader internalReader;
 
+  /**
+   * Field's boost
+   * @see #boost()
+   */
   protected float boost = 1.0f;
 
+  /**
+   * Expert: creates a field with no initial value.
+   * Intended only for custom Field subclasses.
+   * @param name field name
+   * @param type field type
+   * @throws IllegalArgumentException if either the name or type
+   *         is null.
+   */
   protected Field(String name, FieldType type) {
     if (name == null) {
       throw new IllegalArgumentException("name cannot be null");
@@ -91,6 +110,13 @@ public class Field implements IndexableF
 
   /**
    * Create field with Reader value.
+   * @param name field name
+   * @param reader reader value
+   * @param type field type
+   * @throws IllegalArgumentException if either the name or type
+   *         is null, or if the field's type is stored(), or
+   *         if tokenized() is false.
+   * @throws NullPointerException if the reader is null
    */
   public Field(String name, Reader reader, FieldType type) {
     if (name == null) {
@@ -116,6 +142,13 @@ public class Field implements IndexableF
 
   /**
    * Create field with TokenStream value.
+   * @param name field name
+   * @param tokenStream TokenStream value
+   * @param type field type
+   * @throws IllegalArgumentException if either the name or type
+   *         is null, or if the field's type is stored(), or
+   *         if tokenized() is false, or if indexed() is false.
+   * @throws NullPointerException if the tokenStream is null
    */
   public Field(String name, TokenStream tokenStream, FieldType type) {
     if (name == null) {
@@ -139,6 +172,15 @@ public class Field implements IndexableF
   
   /**
    * Create field with binary value.
+   * 
+   * <p>NOTE: the provided byte[] is not copied so be sure
+   * not to change it until you're done with this field.
+   * @param name field name
+   * @param value byte array pointing to binary content (not copied)
+   * @param type field type
+   * @throws IllegalArgumentException if the field name is null,
+   *         or the field's type is indexed()
+   * @throws NullPointerException if the type is null
    */
   public Field(String name, byte[] value, FieldType type) {
     this(name, value, 0, value.length, type);
@@ -146,6 +188,17 @@ public class Field implements IndexableF
 
   /**
    * Create field with binary value.
+   * 
+   * <p>NOTE: the provided byte[] is not copied so be sure
+   * not to change it until you're done with this field.
+   * @param name field name
+   * @param value byte array pointing to binary content (not copied)
+   * @param offset starting position of the byte array
+   * @param length valid length of the byte array
+   * @param type field type
+   * @throws IllegalArgumentException if the field name is null,
+   *         or the field's type is indexed()
+   * @throws NullPointerException if the type is null
    */
   public Field(String name, byte[] value, int offset, int length, FieldType type) {
     this(name, new BytesRef(value, offset, length), type);
@@ -156,6 +209,12 @@ public class Field implements IndexableF
    *
    * <p>NOTE: the provided BytesRef is not copied so be sure
    * not to change it until you're done with this field.
+   * @param name field name
+   * @param bytes BytesRef pointing to binary content (not copied)
+   * @param type field type
+   * @throws IllegalArgumentException if the field name is null,
+   *         or the field's type is indexed()
+   * @throws NullPointerException if the type is null
    */
   public Field(String name, BytesRef bytes, FieldType type) {
     if (name == null) {
@@ -173,6 +232,13 @@ public class Field implements IndexableF
 
   /**
    * Create field with String value.
+   * @param name field name
+   * @param value string value
+   * @param type field type
+   * @throws IllegalArgumentException if either the name or value
+   *         is null, or if the field's type is neither indexed() nor stored(), 
+   *         or if indexed() is false but storeTermVectors() is true.
+   * @throws NullPointerException if the type is null
    */
   public Field(String name, String value, FieldType type) {
     if (name == null) {
@@ -214,7 +280,7 @@ public class Field implements IndexableF
   }
   
   /**
-   * The TokesStream for this field to be used when indexing, or null. If null,
+   * The TokenStream for this field to be used when indexing, or null. If null,
    * the Reader value or String value is analyzed to produce the indexed tokens.
    */
   public TokenStream tokenStreamValue() {
@@ -280,6 +346,10 @@ public class Field implements IndexableF
     fieldsData = value;
   }
 
+  /**
+   * Expert: change the value of this field. See 
+   * {@link #setStringValue(String)}.
+   */
   public void setByteValue(byte value) {
     if (!(fieldsData instanceof Byte)) {
       throw new IllegalArgumentException("cannot change value type from " + fieldsData.getClass().getSimpleName() + " to Byte");
@@ -287,6 +357,10 @@ public class Field implements IndexableF
     fieldsData = Byte.valueOf(value);
   }
 
+  /**
+   * Expert: change the value of this field. See 
+   * {@link #setStringValue(String)}.
+   */
   public void setShortValue(short value) {
     if (!(fieldsData instanceof Short)) {
       throw new IllegalArgumentException("cannot change value type from " + fieldsData.getClass().getSimpleName() + " to Short");
@@ -294,6 +368,10 @@ public class Field implements IndexableF
     fieldsData = Short.valueOf(value);
   }
 
+  /**
+   * Expert: change the value of this field. See 
+   * {@link #setStringValue(String)}.
+   */
   public void setIntValue(int value) {
     if (!(fieldsData instanceof Integer)) {
       throw new IllegalArgumentException("cannot change value type from " + fieldsData.getClass().getSimpleName() + " to Integer");
@@ -301,6 +379,10 @@ public class Field implements IndexableF
     fieldsData = Integer.valueOf(value);
   }
 
+  /**
+   * Expert: change the value of this field. See 
+   * {@link #setStringValue(String)}.
+   */
   public void setLongValue(long value) {
     if (!(fieldsData instanceof Long)) {
       throw new IllegalArgumentException("cannot change value type from " + fieldsData.getClass().getSimpleName() + " to Long");
@@ -308,6 +390,10 @@ public class Field implements IndexableF
     fieldsData = Long.valueOf(value);
   }
 
+  /**
+   * Expert: change the value of this field. See 
+   * {@link #setStringValue(String)}.
+   */
   public void setFloatValue(float value) {
     if (!(fieldsData instanceof Float)) {
       throw new IllegalArgumentException("cannot change value type from " + fieldsData.getClass().getSimpleName() + " to Float");
@@ -315,6 +401,10 @@ public class Field implements IndexableF
     fieldsData = Float.valueOf(value);
   }
 
+  /**
+   * Expert: change the value of this field. See 
+   * {@link #setStringValue(String)}.
+   */
   public void setDoubleValue(double value) {
     if (!(fieldsData instanceof Double)) {
       throw new IllegalArgumentException("cannot change value type from " + fieldsData.getClass().getSimpleName() + " to Double");
@@ -341,23 +431,21 @@ public class Field implements IndexableF
     return name;
   }
   
+  /** 
+   * {@inheritDoc}
+   * <p>
+   * The default value is <code>1.0f</code> (no boost).
+   * @see #setBoost(float)
+   */
   public float boost() {
     return boost;
   }
 
-  /** Sets the boost factor hits on this field.  This value will be
-   * multiplied into the score of all hits on this this field of this
-   * document.
-   *
-   * <p>The boost is used to compute the norm factor for the field.  By
-   * default, in the {@link org.apache.lucene.search.similarities.Similarity#computeNorm(FieldInvertState, Norm)} method, 
-   * the boost value is multiplied by the length normalization factor and then
-   * rounded by {@link org.apache.lucene.search.similarities.DefaultSimilarity#encodeNormValue(float)} before it is stored in the
-   * index.  One should attempt to ensure that this product does not overflow
-   * the range of that encoding.
-   *
-   * @see org.apache.lucene.search.similarities.Similarity#computeNorm(FieldInvertState, Norm)
-   * @see org.apache.lucene.search.similarities.DefaultSimilarity#encodeNormValue(float)
+  /** 
+   * Sets the boost factor on this field.
+   * @throws IllegalArgumentException if this field is not indexed, 
+   *         or if it omits norms. 
+   * @see #boost()
    */
   public void setBoost(float boost) {
     if (boost != 1.0f) {
@@ -406,9 +494,6 @@ public class Field implements IndexableF
     return type;
   }
 
-  /**
-   * {@inheritDoc}
-   */
   public TokenStream tokenStream(Analyzer analyzer) throws IOException {
     if (!fieldType().indexed()) {
       return null;

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/FieldType.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/FieldType.java?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/FieldType.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/FieldType.java Thu Aug 30 22:43:41 2012
@@ -17,6 +17,7 @@ package org.apache.lucene.document;
  * limitations under the License.
  */
 
+import org.apache.lucene.analysis.Analyzer; // javadocs
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.index.IndexableFieldType;
@@ -31,7 +32,16 @@ public class FieldType implements Indexa
   /** Data type of the numeric value
    * @since 3.2
    */
-  public static enum NumericType {INT, LONG, FLOAT, DOUBLE}
+  public static enum NumericType {
+    /** 32-bit integer numeric type */
+    INT, 
+    /** 64-bit long numeric type */
+    LONG, 
+    /** 32-bit float numeric type */
+    FLOAT, 
+    /** 64-bit double numeric type */
+    DOUBLE
+  }
 
   private boolean indexed;
   private boolean stored;
@@ -47,6 +57,9 @@ public class FieldType implements Indexa
   private int numericPrecisionStep = NumericUtils.PRECISION_STEP_DEFAULT;
   private DocValues.Type docValueType;
 
+  /**
+   * Create a new mutable FieldType with all of the properties from <code>ref</code>
+   */
   public FieldType(FieldType ref) {
     this.indexed = ref.indexed();
     this.stored = ref.stored();
@@ -62,6 +75,9 @@ public class FieldType implements Indexa
     // Do not copy frozen!
   }
   
+  /**
+   * Create a new FieldType with default properties.
+   */
   public FieldType() {
   }
 
@@ -80,100 +96,241 @@ public class FieldType implements Indexa
     this.frozen = true;
   }
   
+  /**
+   * {@inheritDoc}
+   * <p>
+   * The default is <code>false</code>.
+   * @see #setIndexed(boolean)
+   */
   public boolean indexed() {
     return this.indexed;
   }
   
+  /**
+   * Set to <code>true</code> to index (invert) this field.
+   * @param value true if this field should be indexed.
+   * @throws IllegalStateException if this FieldType is frozen against
+   *         future modifications.
+   * @see #indexed()
+   */
   public void setIndexed(boolean value) {
     checkIfFrozen();
     this.indexed = value;
   }
 
+  /**
+   * {@inheritDoc}
+   * <p>
+   * The default is <code>false</code>.
+   * @see #setStored(boolean)
+   */
   public boolean stored() {
     return this.stored;
   }
   
+  /**
+   * Set to <code>true</code> to store this field.
+   * @param value true if this field should be stored.
+   * @throws IllegalStateException if this FieldType is frozen against
+   *         future modifications.
+   * @see #stored()
+   */
   public void setStored(boolean value) {
     checkIfFrozen();
     this.stored = value;
   }
 
+  /**
+   * {@inheritDoc}
+   * <p>
+   * The default is <code>true</code>.
+   * @see #setTokenized(boolean)
+   */
   public boolean tokenized() {
     return this.tokenized;
   }
   
+  /**
+   * Set to <code>true</code> to tokenize this field's contents via the 
+   * configured {@link Analyzer}.
+   * @param value true if this field should be tokenized.
+   * @throws IllegalStateException if this FieldType is frozen against
+   *         future modifications.
+   * @see #tokenized()
+   */
   public void setTokenized(boolean value) {
     checkIfFrozen();
     this.tokenized = value;
   }
 
+  /**
+   * {@inheritDoc}
+   * <p>
+   * The default is <code>false</code>. 
+   * @see #setStoreTermVectors(boolean)
+   */
   public boolean storeTermVectors() {
     return this.storeTermVectors;
   }
   
+  /**
+   * Set to <code>true</code> if this field's indexed form should be also stored 
+   * into term vectors.
+   * @param value true if this field should store term vectors.
+   * @throws IllegalStateException if this FieldType is frozen against
+   *         future modifications.
+   * @see #storeTermVectors()
+   */
   public void setStoreTermVectors(boolean value) {
     checkIfFrozen();
     this.storeTermVectors = value;
   }
 
+  /**
+   * {@inheritDoc}
+   * <p>
+   * The default is <code>false</code>.
+   * @see #setStoreTermVectorOffsets(boolean)
+   */
   public boolean storeTermVectorOffsets() {
     return this.storeTermVectorOffsets;
   }
   
+  /**
+   * Set to <code>true</code> to also store token character offsets into the term
+   * vector for this field.
+   * @param value true if this field should store term vector offsets.
+   * @throws IllegalStateException if this FieldType is frozen against
+   *         future modifications.
+   * @see #storeTermVectorOffsets()
+   */
   public void setStoreTermVectorOffsets(boolean value) {
     checkIfFrozen();
     this.storeTermVectorOffsets = value;
   }
 
+  /**
+   * {@inheritDoc}
+   * <p>
+   * The default is <code>false</code>.
+   * @see #setStoreTermVectorPositions(boolean)
+   */
   public boolean storeTermVectorPositions() {
     return this.storeTermVectorPositions;
   }
   
+  /**
+   * Set to <code>true</code> to also store token positions into the term
+   * vector for this field.
+   * @param value true if this field should store term vector positions.
+   * @throws IllegalStateException if this FieldType is frozen against
+   *         future modifications.
+   * @see #storeTermVectorPositions()
+   */
   public void setStoreTermVectorPositions(boolean value) {
     checkIfFrozen();
     this.storeTermVectorPositions = value;
   }
   
+  /**
+   * {@inheritDoc}
+   * <p>
+   * The default is <code>false</code>.
+   * @see #setStoreTermVectorPayloads(boolean) 
+   */
   public boolean storeTermVectorPayloads() {
     return this.storeTermVectorPayloads;
   }
   
+  /**
+   * Set to <code>true</code> to also store token payloads into the term
+   * vector for this field.
+   * @param value true if this field should store term vector payloads.
+   * @throws IllegalStateException if this FieldType is frozen against
+   *         future modifications.
+   * @see #storeTermVectorPayloads()
+   */
   public void setStoreTermVectorPayloads(boolean value) {
     checkIfFrozen();
     this.storeTermVectorPayloads = value;
   }
   
+  /**
+   * {@inheritDoc}
+   * <p>
+   * The default is <code>false</code>.
+   * @see #setOmitNorms(boolean)
+   */
   public boolean omitNorms() {
     return this.omitNorms;
   }
   
+  /**
+   * Set to <code>true</code> to omit normalization values for the field.
+   * @param value true if this field should omit norms.
+   * @throws IllegalStateException if this FieldType is frozen against
+   *         future modifications.
+   * @see #omitNorms()
+   */
   public void setOmitNorms(boolean value) {
     checkIfFrozen();
     this.omitNorms = value;
   }
 
+  /**
+   * {@inheritDoc}
+   * <p>
+   * The default is {@link IndexOptions#DOCS_AND_FREQS_AND_POSITIONS}.
+   * @see #setIndexOptions(FieldInfo.IndexOptions)
+   */
   public IndexOptions indexOptions() {
     return this.indexOptions;
   }
   
+  /**
+   * Sets the indexing options for the field:
+   * @param value indexing options
+   * @throws IllegalStateException if this FieldType is frozen against
+   *         future modifications.
+   * @see #indexOptions()
+   */
   public void setIndexOptions(IndexOptions value) {
     checkIfFrozen();
     this.indexOptions = value;
   }
 
+  /**
+   * Specifies the field's numeric type.
+   * @param type numeric type, or null if the field has no numeric type.
+   * @throws IllegalStateException if this FieldType is frozen against
+   *         future modifications.
+   * @see #numericType()
+   */
   public void setNumericType(NumericType type) {
     checkIfFrozen();
     numericType = type;
   }
 
-  /** NumericDataType; if
-   *  non-null then the field's value will be indexed
-   *  numerically so that {@link NumericRangeQuery} can be
-   *  used at search time. */
+  /** 
+   * NumericType: if non-null then the field's value will be indexed
+   * numerically so that {@link NumericRangeQuery} can be used at 
+   * search time. 
+   * <p>
+   * The default is <code>null</code> (no numeric type) 
+   * @see #setNumericType(NumericType)
+   */
   public NumericType numericType() {
     return numericType;
   }
 
+  /**
+   * Sets the numeric precision step for the field.
+   * @param precisionStep numeric precision step for the field
+   * @throws IllegalArgumentException if precisionStep is less than 1. 
+   * @throws IllegalStateException if this FieldType is frozen against
+   *         future modifications.
+   * @see #numericPrecisionStep()
+   */
   public void setNumericPrecisionStep(int precisionStep) {
     checkIfFrozen();
     if (precisionStep < 1) {
@@ -182,7 +339,14 @@ public class FieldType implements Indexa
     this.numericPrecisionStep = precisionStep;
   }
 
-  /** Precision step for numeric field. */
+  /** 
+   * Precision step for numeric field. 
+   * <p>
+   * This has no effect if {@link #numericType()} returns null.
+   * <p>
+   * The default is {@link NumericUtils#PRECISION_STEP_DEFAULT}
+   * @see #setNumericPrecisionStep(int)
+   */
   public int numericPrecisionStep() {
     return numericPrecisionStep;
   }
@@ -239,11 +403,24 @@ public class FieldType implements Indexa
   
   /* from StorableFieldType */
   
+  /**
+   * {@inheritDoc}
+   * <p>
+   * The default is <code>null</code> (no docValues) 
+   * @see #setDocValueType(DocValues.Type)
+   */
   @Override
   public DocValues.Type docValueType() {
     return docValueType;
   }
 
+  /**
+   * Set's the field's DocValues.Type
+   * @param type DocValues type, or null if no DocValues should be stored.
+   * @throws IllegalStateException if this FieldType is frozen against
+   *         future modifications.
+   * @see #docValueType()
+   */
   public void setDocValueType(DocValues.Type type) {
     checkIfFrozen();
     docValueType = type;

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/FloatDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/FloatDocValuesField.java?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/FloatDocValuesField.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/FloatDocValuesField.java Thu Aug 30 22:43:41 2012
@@ -37,12 +37,21 @@ import org.apache.lucene.index.DocValues
 
 public class FloatDocValuesField extends StoredField {
 
+  /**
+   * Type for 32-bit float DocValues.
+   */
   public static final FieldType TYPE = new FieldType();
   static {
     TYPE.setDocValueType(DocValues.Type.FLOAT_32);
     TYPE.freeze();
   }
 
+  /** 
+   * Creates a new DocValues field with the specified 32-bit float value 
+   * @param name field name
+   * @param value 32-bit float value
+   * @throws IllegalArgumentException if the field name is null
+   */
   public FloatDocValuesField(String name, float value) {
     super(name, TYPE);
     fieldsData = Float.valueOf(value);