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/02/21 15:13:40 UTC

svn commit: r1072973 [6/11] - in /lucene/dev/branches/docvalues: ./ dev-tools/eclipse/ dev-tools/idea/.idea/ dev-tools/idea/lucene/contrib/ant/ dev-tools/idea/lucene/contrib/demo/ dev-tools/idea/lucene/contrib/highlighter/ dev-tools/idea/lucene/contrib...

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java Mon Feb 21 14:13:28 2011
@@ -47,6 +47,7 @@ public class StandardPostingsReader exte
 
   int skipInterval;
   int maxSkipLevels;
+  int skipMinimum;
 
   //private String segment;
 
@@ -86,6 +87,7 @@ public class StandardPostingsReader exte
 
     skipInterval = termsIn.readInt();
     maxSkipLevels = termsIn.readInt();
+    skipMinimum = termsIn.readInt();
   }
 
   // Must keep final because we do non-standard clone
@@ -179,7 +181,7 @@ public class StandardPostingsReader exte
     //System.out.println("  freqFP=" + termState.freqOffset);
     assert termState.freqOffset < freqIn.length();
 
-    if (termState.docFreq >= skipInterval) {
+    if (termState.docFreq >= skipMinimum) {
       termState.skipOffset = termState.bytesReader.readVInt();
       //System.out.println("  skipOffset=" + termState.skipOffset + " vs freqIn.length=" + freqIn.length());
       assert termState.freqOffset + termState.skipOffset < freqIn.length();
@@ -378,7 +380,7 @@ public class StandardPostingsReader exte
     @Override
     public int advance(int target) throws IOException {
 
-      if ((target - skipInterval) >= doc && limit >= skipInterval) {
+      if ((target - skipInterval) >= doc && limit >= skipMinimum) {
 
         // There are enough docs in the posting to have
         // skip data, and it isn't too close.
@@ -528,7 +530,7 @@ public class StandardPostingsReader exte
 
       //System.out.println("StandardR.D&PE advance target=" + target);
 
-      if ((target - skipInterval) >= doc && limit >= skipInterval) {
+      if ((target - skipInterval) >= doc && limit >= skipMinimum) {
 
         // There are enough docs in the posting to have
         // skip data, and it isn't too close
@@ -725,7 +727,7 @@ public class StandardPostingsReader exte
 
       //System.out.println("StandardR.D&PE advance seg=" + segment + " target=" + target + " this=" + this);
 
-      if ((target - skipInterval) >= doc && limit >= skipInterval) {
+      if ((target - skipInterval) >= doc && limit >= skipMinimum) {
 
         // There are enough docs in the posting to have
         // skip data, and it isn't too close

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java Mon Feb 21 14:13:28 2011
@@ -23,6 +23,7 @@ package org.apache.lucene.index.codecs.s
 import java.io.IOException;
 
 import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentWriteState;
@@ -44,8 +45,22 @@ public final class StandardPostingsWrite
   final IndexOutput freqOut;
   final IndexOutput proxOut;
   final DefaultSkipListWriter skipListWriter;
-  final int skipInterval;
-  final int maxSkipLevels;
+  /** Expert: The fraction of TermDocs entries stored in skip tables,
+   * used to accelerate {@link DocsEnum#advance(int)}.  Larger values result in
+   * smaller indexes, greater acceleration, but fewer accelerable cases, while
+   * smaller values result in bigger indexes, less acceleration and more
+   * accelerable cases. More detailed experiments would be useful here. */
+  final int skipInterval = 16;
+  
+  /**
+   * Expert: minimum docFreq to write any skip data at all
+   */
+  final int skipMinimum = skipInterval;
+
+  /** Expert: The maximum number of skip levels. Smaller values result in 
+   * slightly smaller indexes, but slower skipping in big posting lists.
+   */
+  final int maxSkipLevels = 10;
   final int totalNumDocs;
   IndexOutput termsOut;
 
@@ -84,14 +99,11 @@ public final class StandardPostingsWrite
 
     totalNumDocs = state.numDocs;
 
-    skipListWriter = new DefaultSkipListWriter(state.skipInterval,
-                                               state.maxSkipLevels,
+    skipListWriter = new DefaultSkipListWriter(skipInterval,
+                                               maxSkipLevels,
                                                state.numDocs,
                                                freqOut,
                                                proxOut);
-     
-    skipInterval = state.skipInterval;
-    maxSkipLevels = state.maxSkipLevels;
   }
 
   @Override
@@ -100,6 +112,7 @@ public final class StandardPostingsWrite
     CodecUtil.writeHeader(termsOut, CODEC, VERSION_CURRENT);
     termsOut.writeInt(skipInterval);                // write skipInterval
     termsOut.writeInt(maxSkipLevels);               // write maxSkipLevels
+    termsOut.writeInt(skipMinimum);                 // write skipMinimum
   }
 
   @Override
@@ -218,7 +231,7 @@ public final class StandardPostingsWrite
     }
     lastFreqStart = freqStart;
 
-    if (df >= skipInterval) {
+    if (df >= skipMinimum) {
       bytesWriter.writeVInt((int) (skipListWriter.writeSkip(freqOut)-freqStart));
     }
 

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/messages/Message.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/messages/Message.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/messages/Message.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/messages/Message.java Mon Feb 21 14:13:28 2011
@@ -17,14 +17,13 @@ package org.apache.lucene.messages;
  * limitations under the License.
  */
 
-import java.io.Serializable;
 import java.util.Locale;
 
 /**
  * Message Interface for a lazy loading.
  * For Native Language Support (NLS), system of software internationalization.
  */
-public interface Message extends Serializable {
+public interface Message {
 
   public String getKey();
 

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/messages/MessageImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/messages/MessageImpl.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/messages/MessageImpl.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/messages/MessageImpl.java Mon Feb 21 14:13:28 2011
@@ -25,8 +25,6 @@ import java.util.Locale;
  */
 public class MessageImpl implements Message {
 
-  private static final long serialVersionUID = -3077643314630884523L;
-
   private String key;
 
   private Object[] arguments = new Object[0];

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/queryParser/QueryParserBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/queryParser/QueryParserBase.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/queryParser/QueryParserBase.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/queryParser/QueryParserBase.java Mon Feb 21 14:13:28 2011
@@ -467,7 +467,14 @@ public abstract class QueryParserBase {
   /**
    * @exception org.apache.lucene.queryParser.ParseException throw in overridden method to disallow
    */
-  protected Query getFieldQuery(String field, String queryText, boolean quoted)  throws ParseException {
+  protected Query getFieldQuery(String field, String queryText, boolean quoted) throws ParseException {
+    return newFieldQuery(analyzer, field, queryText, quoted);
+  }
+  
+  /**
+   * @exception org.apache.lucene.queryParser.ParseException throw in overridden method to disallow
+   */
+  protected Query newFieldQuery(Analyzer analyzer, String field, String queryText, boolean quoted)  throws ParseException {
     // Use the analyzer to get all the tokens, and then build a TermQuery,
     // PhraseQuery, or nothing based on the term count
 

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/AutomatonQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/AutomatonQuery.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/AutomatonQuery.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/AutomatonQuery.java Mon Feb 21 14:13:28 2011
@@ -18,7 +18,6 @@ package org.apache.lucene.search;
  */
 
 import java.io.IOException;
-import java.io.Serializable;
 
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.Terms;
@@ -61,7 +60,7 @@ public class AutomatonQuery extends Mult
    * in the ctor the query computes one of these, the actual
    * implementation depends upon the automaton's structure.
    */
-  private abstract class TermsEnumFactory implements Serializable {
+  private abstract class TermsEnumFactory {
     protected abstract TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException;
   }
   

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/BooleanClause.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/BooleanClause.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/BooleanClause.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/BooleanClause.java Mon Feb 21 14:13:28 2011
@@ -18,7 +18,7 @@ package org.apache.lucene.search;
  */
 
 /** A clause in a BooleanQuery. */
-public class BooleanClause implements java.io.Serializable {
+public class BooleanClause {
   
   /** Specifies how clauses are to occur in matching documents. */
   public static enum Occur {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/CachingWrapperFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/CachingWrapperFilter.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/CachingWrapperFilter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/CachingWrapperFilter.java Mon Feb 21 14:13:28 2011
@@ -17,7 +17,6 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
-import java.io.Serializable;
 import java.io.IOException;
 import java.util.Map;
 import java.util.WeakHashMap;
@@ -68,7 +67,7 @@ public class CachingWrapperFilter extend
 
   protected final FilterCache<DocIdSet> cache;
 
-  static abstract class FilterCache<T> implements Serializable {
+  static abstract class FilterCache<T> {
 
     /**
      * A transient Filter cache (package private because of test)

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/DefaultSimilarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/DefaultSimilarity.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/DefaultSimilarity.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/DefaultSimilarity.java Mon Feb 21 14:13:28 2011
@@ -31,7 +31,7 @@ public class DefaultSimilarity extends S
    *
    *  @lucene.experimental */
   @Override
-  public float computeNorm(String field, FieldInvertState state) {
+  public float computeNorm(FieldInvertState state) {
     final int numTerms;
     if (discountOverlaps)
       numTerms = state.getLength() - state.getNumOverlap();

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Explanation.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Explanation.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Explanation.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Explanation.java Mon Feb 21 14:13:28 2011
@@ -17,11 +17,10 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
-import java.io.Serializable;
 import java.util.ArrayList;
 
 /** Expert: Describes the score computation for document and query. */
-public class Explanation implements java.io.Serializable {
+public class Explanation {
   private float value;                            // the value of this node
   private String description;                     // what it represents
   private ArrayList<Explanation> details;                      // sub-explanations
@@ -135,7 +134,7 @@ public class Explanation implements java
    * before storing any large or un-serializable fields.
    *
    */
-  public static abstract class IDFExplanation implements Serializable {
+  public static abstract class IDFExplanation {
     /**
      * @return the idf factor
      */

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldCache.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldCache.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldCache.java Mon Feb 21 14:13:28 2011
@@ -29,7 +29,6 @@ import org.apache.lucene.analysis.Numeri
 import org.apache.lucene.util.packed.PackedInts;
 
 import java.io.IOException;
-import java.io.Serializable;
 import java.io.PrintStream;
 
 import java.text.DecimalFormat;
@@ -61,7 +60,7 @@ public interface FieldCache {
    * is used to specify a custom parser to {@link
    * SortField#SortField(String, FieldCache.Parser)}.
    */
-  public interface Parser extends Serializable {
+  public interface Parser {
   }
 
   /** Interface to parse bytes from document fields.

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldComparatorSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldComparatorSource.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldComparatorSource.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldComparatorSource.java Mon Feb 21 14:13:28 2011
@@ -18,7 +18,6 @@ package org.apache.lucene.search;
  */
 
 import java.io.IOException;
-import java.io.Serializable;
 
 /**
  * Provides a {@link FieldComparator} for custom field sorting.
@@ -26,7 +25,7 @@ import java.io.Serializable;
  * @lucene.experimental
  *
  */
-public abstract class FieldComparatorSource implements Serializable {
+public abstract class FieldComparatorSource {
 
   /**
    * Creates a comparator for the field in the given index.

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Filter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Filter.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Filter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Filter.java Mon Feb 21 14:13:28 2011
@@ -27,7 +27,7 @@ import org.apache.lucene.util.DocIdBitSe
  *  Abstract base class for restricting which documents may
  *  be returned during searching.
  */
-public abstract class Filter implements java.io.Serializable {
+public abstract class Filter {
   
   /**
    * Creates a {@link DocIdSet} enumerating the documents that should be

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java Mon Feb 21 14:13:28 2011
@@ -325,22 +325,26 @@ public final class FuzzyTermsEnum extend
     
     /** finds the smallest Lev(n) DFA that accepts the term. */
     @Override
-    protected AcceptStatus accept(BytesRef term) {
-      if (term.equals(termRef)) { // ed = 0
-        boostAtt.setBoost(1.0F);
-        return AcceptStatus.YES_AND_SEEK;
-      }
-      
-      int codePointCount = -1;
+    protected AcceptStatus accept(BytesRef term) {    
+      int ed = matchers.length - 1;
       
-      // TODO: benchmark doing this backwards
-      for (int i = 1; i < matchers.length; i++)
-        if (matchers[i].run(term.bytes, term.offset, term.length)) {
-          // this sucks, we convert just to score based on length.
-          if (codePointCount == -1) {
-            codePointCount = UnicodeUtil.codePointCount(term);
+      if (matches(term, ed)) { // we match the outer dfa
+        // now compute exact edit distance
+        while (ed > 0) {
+          if (matches(term, ed - 1)) {
+            ed--;
+          } else {
+            break;
           }
-          final float similarity = 1.0f - ((float) i / (float) 
+        }
+        
+        // scale to a boost and return (if similarity > minSimilarity)
+        if (ed == 0) { // exact match
+          boostAtt.setBoost(1.0F);
+          return AcceptStatus.YES_AND_SEEK;
+        } else {
+          final int codePointCount = UnicodeUtil.codePointCount(term);
+          final float similarity = 1.0f - ((float) ed / (float) 
               (Math.min(codePointCount, termLength)));
           if (similarity > minSimilarity) {
             boostAtt.setBoost((similarity - minSimilarity) * scale_factor);
@@ -349,8 +353,14 @@ public final class FuzzyTermsEnum extend
             return AcceptStatus.NO_AND_SEEK;
           }
         }
-      
-      return AcceptStatus.NO_AND_SEEK;
+      } else {
+        return AcceptStatus.NO_AND_SEEK;
+      }
+    }
+    
+    /** returns true if term is within k edits of the query term */
+    final boolean matches(BytesRef term, int k) {
+      return k == 0 ? term.equals(termRef) : matchers[k].run(term.bytes, term.offset, term.length);
     }
     
     /** defers to superclass, except can start at an arbitrary location */

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/MultiTermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/MultiTermQuery.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/MultiTermQuery.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/MultiTermQuery.java Mon Feb 21 14:13:28 2011
@@ -18,7 +18,6 @@ package org.apache.lucene.search;
  */
 
 import java.io.IOException;
-import java.io.Serializable;
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
@@ -67,7 +66,7 @@ public abstract class MultiTermQuery ext
   transient int numberOfTerms = 0;
 
   /** Abstract class that defines how the query is rewritten. */
-  public static abstract class RewriteMethod implements Serializable {
+  public static abstract class RewriteMethod {
     public abstract Query rewrite(IndexReader reader, MultiTermQuery query) throws IOException;
   }
 
@@ -90,11 +89,6 @@ public abstract class MultiTermQuery ext
       result.setBoost(query.getBoost());
       return result;
     }
-
-    // Make sure we are still a singleton even after deserializing
-    protected Object readResolve() {
-      return CONSTANT_SCORE_FILTER_REWRITE;
-    }
   };
 
   /** A rewrite method that first translates each term into
@@ -240,11 +234,6 @@ public abstract class MultiTermQuery ext
     public void setDocCountPercent(double percent) {
       throw new UnsupportedOperationException("Please create a private instance");
     }
-
-    // Make sure we are still a singleton even after deserializing
-    protected Object readResolve() {
-      return CONSTANT_SCORE_AUTO_REWRITE_DEFAULT;
-    }
   };
 
   /**

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/NumericRangeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/NumericRangeFilter.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/NumericRangeFilter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/NumericRangeFilter.java Mon Feb 21 14:13:28 2011
@@ -179,4 +179,7 @@ public final class NumericRangeFilter<T 
   /** Returns the upper value of this range filter */
   public T getMax() { return query.getMax(); }
   
+  /** Returns the precision step. */
+  public int getPrecisionStep() { return query.getPrecisionStep(); }
+  
 }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/NumericRangeQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/NumericRangeQuery.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/NumericRangeQuery.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/NumericRangeQuery.java Mon Feb 21 14:13:28 2011
@@ -319,6 +319,9 @@ public final class NumericRangeQuery<T e
   /** Returns the upper value of this range query */
   public T getMax() { return max; }
   
+  /** Returns the precision step. */
+  public int getPrecisionStep() { return precisionStep; }
+  
   @Override
   public String toString(final String field) {
     final StringBuilder sb = new StringBuilder();

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Query.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Query.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Query.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Query.java Mon Feb 21 14:13:28 2011
@@ -44,7 +44,7 @@ import org.apache.lucene.index.Term;
     <li>{@link org.apache.lucene.queryParser.QueryParser QueryParser}
     </ul>
 */
-public abstract class Query implements java.io.Serializable, Cloneable {
+public abstract class Query implements Cloneable {
   private float boost = 1.0f;                     // query boost factor
 
   /** Sets the boost for this query clause to <code>b</code>.  Documents

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ScoreDoc.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ScoreDoc.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ScoreDoc.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ScoreDoc.java Mon Feb 21 14:13:28 2011
@@ -19,7 +19,7 @@ package org.apache.lucene.search;
 
 /** Expert: Returned by low-level search implementations.
  * @see TopDocs */
-public class ScoreDoc implements java.io.Serializable {
+public class ScoreDoc {
   /** Expert: The score of this document for the query. */
   public float score;
 

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Similarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Similarity.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Similarity.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Similarity.java Mon Feb 21 14:13:28 2011
@@ -19,7 +19,6 @@ package org.apache.lucene.search;
 
 
 import java.io.IOException;
-import java.io.Serializable;
 import java.util.Collection;
 
 import org.apache.lucene.index.FieldInvertState;
@@ -525,7 +524,7 @@ import org.apache.lucene.util.SmallFloat
  * @see org.apache.lucene.index.IndexWriterConfig#setSimilarityProvider(SimilarityProvider)
  * @see IndexSearcher#setSimilarityProvider(SimilarityProvider)
  */
-public abstract class Similarity implements Serializable {
+public abstract class Similarity {
   
   public static final int NO_DOC_ID_PROVIDED = -1;
 
@@ -562,49 +561,13 @@ public abstract class Similarity impleme
    * Thus they have limited precision, and documents
    * must be re-indexed if this method is altered.
    *
-   * <p>For backward compatibility this method by default calls
-   * {@link #lengthNorm(String, int)} passing
-   * {@link FieldInvertState#getLength()} as the second argument, and
-   * then multiplies this value by {@link FieldInvertState#getBoost()}.</p>
-   * 
    * @lucene.experimental
    * 
-   * @param field field name
    * @param state current processing state for this field
    * @return the calculated float norm
    */
-  public abstract float computeNorm(String field, FieldInvertState state);
+  public abstract float computeNorm(FieldInvertState state);
   
-  /** Computes the normalization value for a field given the total number of
-   * terms contained in a field.  These values, together with field boosts, are
-   * stored in an index and multipled into scores for hits on each field by the
-   * search code.
-   *
-   * <p>Matches in longer fields are less precise, so implementations of this
-   * method usually return smaller values when <code>numTokens</code> is large,
-   * and larger values when <code>numTokens</code> is small.
-   * 
-   * <p>Note that the return values are computed under 
-   * {@link org.apache.lucene.index.IndexWriter#addDocument(org.apache.lucene.document.Document)} 
-   * and then stored using
-   * {@link #encodeNormValue(float)}.  
-   * Thus they have limited precision, and documents
-   * must be re-indexed if this method is altered.
-   *
-   * @param fieldName the name of the field
-   * @param numTokens the total number of tokens contained in fields named
-   * <i>fieldName</i> of <i>doc</i>.
-   * @return a normalization factor for hits on this field of this document
-   *
-   * @see org.apache.lucene.document.Field#setBoost(float)
-   *
-   * @deprecated Please override computeNorm instead
-   */
-  @Deprecated
-  public final float lengthNorm(String fieldName, int numTokens) {
-    throw new UnsupportedOperationException("please use computeNorm instead");
-  }
-
   /** Encodes a normalization factor for storage in an index.
    *
    * <p>The encoding uses a three-bit mantissa, a five-bit exponent, and
@@ -782,7 +745,6 @@ public abstract class Similarity impleme
    * The default implementation returns 1.
    *
    * @param docId The docId currently being scored.  If this value is {@link #NO_DOC_ID_PROVIDED}, then it should be assumed that the PayloadQuery implementation does not provide document information
-   * @param fieldName The fieldName of the term this payload belongs to
    * @param start The start position of the payload
    * @param end The end position of the payload
    * @param payload The payload byte array to be scored
@@ -792,7 +754,7 @@ public abstract class Similarity impleme
    *
    */
   // TODO: maybe switch this API to BytesRef?
-  public float scorePayload(int docId, String fieldName, int start, int end, byte [] payload, int offset, int length)
+  public float scorePayload(int docId, int start, int end, byte [] payload, int offset, int length)
   {
     return 1;
   }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Sort.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Sort.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Sort.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Sort.java Mon Feb 21 14:13:28 2011
@@ -17,7 +17,6 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
-import java.io.Serializable;
 import java.util.Arrays;
 
 
@@ -97,8 +96,7 @@ import java.util.Arrays;
  *
  * @since   lucene 1.4
  */
-public class Sort
-implements Serializable {
+public class Sort {
 
   /**
    * Represents sorting by computed relevance. Using this sort criteria returns

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/SortField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/SortField.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/SortField.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/SortField.java Mon Feb 21 14:13:28 2011
@@ -18,7 +18,6 @@ package org.apache.lucene.search;
  */
 
 import java.io.IOException;
-import java.io.Serializable;
 import java.util.Comparator;
 import java.util.Locale;
 
@@ -39,8 +38,7 @@ import org.apache.lucene.util.BytesRef;
  * @since   lucene 1.4
  * @see Sort
  */
-public class SortField
-implements Serializable {
+public class SortField {
 
   /** Sort by document score (relevance).  Sort values are Float and higher
    * values are at the front. */
@@ -436,13 +434,6 @@ implements Serializable {
     return hash;
   }
 
-  // field must be interned after reading from stream
-  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
-    in.defaultReadObject();
-    if (field != null)
-      field = StringHelper.intern(field);
-  }
-
   private boolean useIndexValues;
 
   public void setUseIndexValues(boolean b) {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TopDocs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TopDocs.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TopDocs.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TopDocs.java Mon Feb 21 14:13:28 2011
@@ -20,7 +20,7 @@ package org.apache.lucene.search;
 /** Represents hits returned by {@link
  * IndexSearcher#search(Query,Filter,int)} and {@link
  * IndexSearcher#search(Query,int)}. */
-public class TopDocs implements java.io.Serializable {
+public class TopDocs {
   /** The total number of hits for the query.
   */
   public int totalHits;

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Weight.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Weight.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Weight.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/Weight.java Mon Feb 21 14:13:28 2011
@@ -18,7 +18,6 @@ package org.apache.lucene.search;
  */
 
 import java.io.IOException;
-import java.io.Serializable;
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
@@ -54,7 +53,7 @@ import org.apache.lucene.index.IndexRead
  * 
  * @since 2.9
  */
-public abstract class Weight implements Serializable {
+public abstract class Weight {
 
   /**
    * An explanation of the score computation for the named document.

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/cache/EntryCreator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/cache/EntryCreator.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/cache/EntryCreator.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/cache/EntryCreator.java Mon Feb 21 14:13:28 2011
@@ -18,7 +18,6 @@ package org.apache.lucene.search.cache;
  */
 
 import java.io.IOException;
-import java.io.Serializable;
 
 import org.apache.lucene.index.IndexReader;
 
@@ -27,7 +26,7 @@ import org.apache.lucene.index.IndexRead
  * 
  * @lucene.experimental
  */
-public abstract class EntryCreator<T> implements Serializable
+public abstract class EntryCreator<T>
 {
   public abstract T create( IndexReader reader ) throws IOException;
   public abstract T validate( T entry, IndexReader reader ) throws IOException;

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/function/ValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/function/ValueSource.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/function/ValueSource.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/function/ValueSource.java Mon Feb 21 14:13:28 2011
@@ -23,7 +23,6 @@ import org.apache.lucene.index.IndexRead
 import org.apache.lucene.search.function.DocValues;
 
 import java.io.IOException;
-import java.io.Serializable;
 
 /**
  * Expert: source of values for basic function queries.
@@ -37,7 +36,7 @@ import java.io.Serializable;
  *
  *
  */
-public abstract class ValueSource implements Serializable {
+public abstract class ValueSource {
 
   /**
    * Return the DocValues used by the function query.

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/payloads/PayloadFunction.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/payloads/PayloadFunction.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/payloads/PayloadFunction.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/payloads/PayloadFunction.java Mon Feb 21 14:13:28 2011
@@ -16,7 +16,6 @@ package org.apache.lucene.search.payload
  * limitations under the License.
  */
 
-import java.io.Serializable;
 import org.apache.lucene.search.Explanation;
 
 /**
@@ -29,7 +28,7 @@ import org.apache.lucene.search.Explanat
  *               change
  * 
  **/
-public abstract class PayloadFunction implements Serializable {
+public abstract class PayloadFunction {
 
   /**
    * Calculate the score up to this point for this doc and field

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java Mon Feb 21 14:13:28 2011
@@ -192,7 +192,7 @@ public class PayloadNearQuery extends Sp
     protected void processPayloads(Collection<byte[]> payLoads, int start, int end) {
       for (final byte[] thePayload : payLoads) {
         payloadScore = function.currentScore(doc, fieldName, start, end,
-            payloadsSeen, payloadScore, similarity.scorePayload(doc, fieldName,
+            payloadsSeen, payloadScore, similarity.scorePayload(doc,
                 spans.start(), spans.end(), thePayload, 0, thePayload.length));
         ++payloadsSeen;
       }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java Mon Feb 21 14:13:28 2011
@@ -41,7 +41,7 @@ import java.io.IOException;
  * {@link org.apache.lucene.index.Term} occurs.
  * <p>
  * In order to take advantage of this, you must override
- * {@link org.apache.lucene.search.Similarity#scorePayload(int, String, int, int, byte[],int,int)}
+ * {@link org.apache.lucene.search.Similarity#scorePayload(int, int, int, byte[],int,int)}
  * which returns 1 by default.
  * <p>
  * Payload scores are aggregated using a pluggable {@link PayloadFunction}.
@@ -119,14 +119,14 @@ public class PayloadTermQuery extends Sp
           if (payload != null) {
             payloadScore = function.currentScore(doc, term.field(),
                                                  spans.start(), spans.end(), payloadsSeen, payloadScore,
-                                                 similarity.scorePayload(doc, term.field(), spans.start(),
+                                                 similarity.scorePayload(doc, spans.start(),
                                                                          spans.end(), payload.bytes,
                                                                          payload.offset,
                                                                          payload.length));
           } else {
             payloadScore = function.currentScore(doc, term.field(),
                                                  spans.start(), spans.end(), payloadsSeen, payloadScore,
-                                                 similarity.scorePayload(doc, term.field(), spans.start(),
+                                                 similarity.scorePayload(doc, spans.start(),
                                                                          spans.end(), null,
                                                                          0,
                                                                          0));

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/RAMDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/RAMDirectory.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/RAMDirectory.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/RAMDirectory.java Mon Feb 21 14:13:28 2011
@@ -19,7 +19,6 @@ package org.apache.lucene.store;
 
 import java.io.IOException;
 import java.io.FileNotFoundException;
-import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
@@ -35,10 +34,7 @@ import org.apache.lucene.util.ThreadInte
  * implementation is by default the {@link SingleInstanceLockFactory}
  * but can be changed with {@link #setLockFactory}.
  */
-public class RAMDirectory extends Directory implements Serializable {
-
-  private static final long serialVersionUID = 1l;
-
+public class RAMDirectory extends Directory {
   protected final Map<String,RAMFile> fileMap = new ConcurrentHashMap<String,RAMFile>();
   protected final AtomicLong sizeInBytes = new AtomicLong();
   

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/RAMFile.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/RAMFile.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/RAMFile.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/RAMFile.java Mon Feb 21 14:13:28 2011
@@ -18,13 +18,9 @@ package org.apache.lucene.store;
  */
 
 import java.util.ArrayList;
-import java.io.Serializable;
 
 /** @lucene.internal */
-public class RAMFile implements Serializable {
-
-  private static final long serialVersionUID = 1l;
-
+public class RAMFile {
   protected ArrayList<byte[]> buffers = new ArrayList<byte[]>();
   long length;
   RAMDirectory directory;

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/AttributeImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/AttributeImpl.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/AttributeImpl.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/AttributeImpl.java Mon Feb 21 14:13:28 2011
@@ -17,7 +17,6 @@ package org.apache.lucene.util;
  * limitations under the License.
  */
 
-import java.io.Serializable;
 import java.lang.reflect.Field;
 import java.lang.reflect.Modifier;
 import java.lang.ref.WeakReference;
@@ -30,7 +29,7 @@ import java.util.LinkedList;
  * Attributes are used to add data in a dynamic, yet type-safe way to a source
  * of usually streamed objects, e. g. a {@link org.apache.lucene.analysis.TokenStream}.
  */
-public abstract class AttributeImpl implements Cloneable, Serializable, Attribute {  
+public abstract class AttributeImpl implements Cloneable, Attribute {  
   /**
    * Clears the values in this AttributeImpl and resets it to its 
    * default value. If this implementation implements more than one Attribute interface

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/BytesRef.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/BytesRef.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/BytesRef.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/BytesRef.java Mon Feb 21 14:13:28 2011
@@ -22,14 +22,13 @@ import java.io.Serializable;
 import java.io.UnsupportedEncodingException;
 import java.io.ObjectInput;
 import java.io.ObjectOutput;
-import java.io.Externalizable;
 import java.io.IOException;
 
 /** Represents byte[], as a slice (offset + length) into an
  *  existing byte[].
  *
  *  @lucene.experimental */
-public final class BytesRef implements Comparable<BytesRef>, Externalizable {
+public final class BytesRef implements Comparable<BytesRef> {
 
   static final int HASH_PRIME = 31;
   public static final byte[] EMPTY_BYTES = new byte[0]; 
@@ -367,25 +366,4 @@ public final class BytesRef implements C
       return a.length - b.length;
     }
   }
-
-  public void writeExternal(ObjectOutput out)
-    throws IOException
-  {
-    out.writeInt(length);
-    if (length > 0) {
-      out.write(bytes, offset, length);
-    }
-  }
-
-  public void readExternal( ObjectInput in ) throws
-      IOException, ClassNotFoundException {
-    length = in.readInt();
-    offset = 0;
-    if (length > 0) {
-      bytes = new byte[length];
-      in.read(bytes, 0, length);
-    } else {
-      bytes = EMPTY_BYTES;
-    }
-  }
 }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/MapBackedSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/MapBackedSet.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/MapBackedSet.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/MapBackedSet.java Mon Feb 21 14:13:28 2011
@@ -17,7 +17,6 @@ package org.apache.lucene.util;
  * limitations under the License.
  */
 
-import java.io.Serializable;
 import java.util.AbstractSet;
 import java.util.Iterator;
 import java.util.Map;
@@ -28,10 +27,7 @@ import java.util.Map;
  * 
  * @lucene.internal
  */
-public final class MapBackedSet<E> extends AbstractSet<E> implements Serializable {
-
-  private static final long serialVersionUID = -6761513279741915432L;
-
+public final class MapBackedSet<E> extends AbstractSet<E> {
   private final Map<E, Boolean> map;
 
   /**

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/OpenBitSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/OpenBitSet.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/OpenBitSet.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/OpenBitSet.java Mon Feb 21 14:13:28 2011
@@ -18,7 +18,6 @@
 package org.apache.lucene.util;
 
 import java.util.Arrays;
-import java.io.Serializable;
 
 import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.DocIdSetIterator;
@@ -75,7 +74,7 @@ Test system: AMD Opteron, 64 bit linux, 
 </table>
  */
 
-public class OpenBitSet extends DocIdSet implements Bits, Cloneable, Serializable {
+public class OpenBitSet extends DocIdSet implements Bits, Cloneable {
   protected long[] bits;
   protected int wlen;   // number of words (elements) used in the array
 

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/PriorityQueue.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/PriorityQueue.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/PriorityQueue.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/PriorityQueue.java Mon Feb 21 14:13:28 2011
@@ -32,7 +32,9 @@ public abstract class PriorityQueue<T> {
   protected T[] heap;
 
   /** Determines the ordering of objects in this priority queue.  Subclasses
-    must define this one method. */
+   *  must define this one method.
+   *  @return <code>true</code> iff parameter <tt>a</tt> is less than parameter <tt>b</tt>.
+   */
   protected abstract boolean lessThan(T a, T b);
 
   /**

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/Automaton.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/Automaton.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/Automaton.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/Automaton.java Mon Feb 21 14:13:28 2011
@@ -29,7 +29,6 @@
 
 package org.apache.lucene.util.automaton;
 
-import java.io.Serializable;
 import java.util.Arrays;
 import java.util.BitSet;
 import java.util.Collection;
@@ -75,7 +74,7 @@ import org.apache.lucene.util.RamUsageEs
  * </p>
  * @lucene.experimental
  */
-public class Automaton implements Serializable, Cloneable {
+public class Automaton implements Cloneable {
   
   /**
    * Minimize using Hopcroft's O(n log n) algorithm. This is regarded as one of

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/RunAutomaton.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/RunAutomaton.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/RunAutomaton.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/RunAutomaton.java Mon Feb 21 14:13:28 2011
@@ -29,14 +29,12 @@
 
 package org.apache.lucene.util.automaton;
 
-import java.io.Serializable;
-
 /**
  * Finite-state automaton with fast run operation.
  * 
  * @lucene.experimental
  */
-public abstract class RunAutomaton implements Serializable {
+public abstract class RunAutomaton {
   final int maxInterval;
   final int size;
   final boolean[] accept;

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/State.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/State.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/State.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/State.java Mon Feb 21 14:13:28 2011
@@ -31,7 +31,6 @@ package org.apache.lucene.util.automaton
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.RamUsageEstimator;
 
-import java.io.Serializable;
 import java.util.Collection;
 import java.util.Comparator;
 import java.util.Iterator;
@@ -41,7 +40,7 @@ import java.util.Iterator;
  * 
  * @lucene.experimental
  */
-public class State implements Serializable, Comparable<State> {
+public class State implements Comparable<State> {
   
   boolean accept;
   public Transition[] transitionsArray;

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/Transition.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/Transition.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/Transition.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/Transition.java Mon Feb 21 14:13:28 2011
@@ -29,7 +29,6 @@
 
 package org.apache.lucene.util.automaton;
 
-import java.io.Serializable;
 import java.util.Comparator;
 
 /**
@@ -40,7 +39,7 @@ import java.util.Comparator;
  * 
  * @lucene.experimental
  */
-public class Transition implements Serializable, Cloneable {
+public class Transition implements Cloneable {
   
   /*
    * CLASS INVARIANT: min<=max

Modified: lucene/dev/branches/docvalues/lucene/src/java/overview.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/overview.html?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/overview.html (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/overview.html Mon Feb 21 14:13:28 2011
@@ -159,27 +159,19 @@ method.</li>
 Some simple examples of code which does this are:
 <ul>
 <li>
-&nbsp;<a href="http://svn.apache.org/repos/asf/lucene/dev/trunk/lucene/src/demo/org/apache/lucene/demo/FileDocument.java">FileDocument.java</a> contains
-code to create a Document for a file.</li>
-
-<li>
-&nbsp;<a href="http://svn.apache.org/repos/asf/lucene/dev/trunk/lucene/src/demo/org/apache/lucene/demo/IndexFiles.java">IndexFiles.java</a> creates an
+&nbsp;<a href="http://svn.apache.org/repos/asf/lucene/dev/trunk/lucene/contrib/demo/src/java/org/apache/lucene/demo/IndexFiles.java">IndexFiles.java</a> creates an
 index for all the files contained in a directory.</li>
 
 <li>
-&nbsp;<a href="http://svn.apache.org/repos/asf/lucene/dev/trunk/lucene/src/demo/org/apache/lucene/demo/DeleteFiles.java">DeleteFiles.java</a> deletes some
-of these files from the index.</li>
-
-<li>
-&nbsp;<a href="http://svn.apache.org/repos/asf/lucene/dev/trunk/lucene/src/demo/org/apache/lucene/demo/SearchFiles.java">SearchFiles.java</a> prompts for
+&nbsp;<a href="http://svn.apache.org/repos/asf/lucene/dev/trunk/lucene/demo/src/java/org/apache/lucene/demo/SearchFiles.java">SearchFiles.java</a> prompts for
 queries and searches an index.</li>
 </ul>
 To demonstrate these, try something like:
-<blockquote><tt>> <b>java -cp lucene.jar:lucene-demo.jar org.apache.lucene.demo.IndexFiles rec.food.recipes/soups</b></tt>
+<blockquote><tt>> <b>java -cp lucene.jar:lucene-demo.jar:lucene-analyzers-common.jar org.apache.lucene.demo.IndexFiles rec.food.recipes/soups</b></tt>
 <br><tt>adding rec.food.recipes/soups/abalone-chowder</tt>
 <br><tt>&nbsp; </tt>[ ... ]
 
-<p><tt>> <b>java -cp lucene.jar:lucene-demo.jar org.apache.lucene.demo.SearchFiles</b></tt>
+<p><tt>> <b>java -cp lucene.jar:lucene-demo.jar:lucene-analyzers-common.jar org.apache.lucene.demo.SearchFiles</b></tt>
 <br><tt>Query: <b>chowder</b></tt>
 <br><tt>Searching for: chowder</tt>
 <br><tt>34 total matching documents</tt>
@@ -195,16 +187,5 @@ and the word "manhattan" ... ]
 <br>&nbsp;&nbsp;&nbsp; [ Note: "+" and "-" are canonical, but "AND", "OR"
 and "NOT" may be used. ]</blockquote>
 
-The <a href="http://svn.apache.org/repos/asf/lucene/dev/trunk/lucene/src/demo/org/apache/lucene/demo/IndexHTML.java">IndexHTML</a> demo is more sophisticated.&nbsp;
-It incrementally maintains an index of HTML files, adding new files as
-they appear, deleting old files as they disappear and re-indexing files
-as they change.
-<blockquote><tt>> <b>java -cp lucene.jar:lucene-demo.jar org.apache.lucene.demo.IndexHTML -create java/jdk1.1.6/docs/relnotes</b></tt>
-<br><tt>adding java/jdk1.1.6/docs/relnotes/SMICopyright.html</tt>
-<br><tt>&nbsp; </tt>[ ... create an index containing all the relnotes ]
-<p><tt>> <b>rm java/jdk1.1.6/docs/relnotes/smicopyright.html</b></tt>
-<p><tt>> <b>java -cp lucene.jar:lucene-demo.jar org.apache.lucene.demo.IndexHTML java/jdk1.1.6/docs/relnotes</b></tt>
-<br><tt>deleting java/jdk1.1.6/docs/relnotes/SMICopyright.html</tt></blockquote>
-
 </body>
 </html>

Modified: lucene/dev/branches/docvalues/lucene/src/site/src/documentation/content/xdocs/demo.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/site/src/documentation/content/xdocs/demo.xml?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/site/src/documentation/content/xdocs/demo.xml (original)
+++ lucene/dev/branches/docvalues/lucene/src/site/src/documentation/content/xdocs/demo.xml Mon Feb 21 14:13:28 2011
@@ -18,10 +18,10 @@ It walks you through some basic installa
 </section>
 
 
-<section id="About the Demos"><title>About the Demos</title>
+<section id="About the Demo"><title>About the Demo</title>
 <p>
-The Lucene command-line demo code consists of two applications that demonstrate various
-functionalities of Lucene and how one should go about adding Lucene to their applications.
+The Lucene command-line demo code consists of an application that demonstrates various
+functionalities of Lucene and how you can add Lucene to your applications.
 </p>
 </section>
 
@@ -30,14 +30,23 @@ functionalities of Lucene and how one sh
 First, you should <a href="http://www.apache.org/dyn/closer.cgi/lucene/java/">download</a> the
 latest Lucene distribution and then extract it to a working directory.  Alternatively, you can <a
 href="http://wiki.apache.org/lucene-java/SourceRepository">check out the sources from
-Subversion</a>, and then run <code>ant war-demo</code> to generate the JARs and WARs.
+Subversion</a>, and then run <code>ant</code> in the <code>lucene/contrib/demo/</code>
+directory to generate the JARs.
 </p>
 <p>
-You should see the Lucene JAR file in the directory you created when you extracted the archive.  It
-should be named something like <code>lucene-core-{version}.jar</code>.  You should also see a file
-called <code>lucene-demos-{version}.jar</code>.  If you checked out the sources from Subversion then
-the JARs are located under the <code>build</code> subdirectory (after running <code>ant</code>
-successfully).  Put both of these files in your Java CLASSPATH.
+You need three JARs: the Lucene JAR, the common analysis JAR, and the Lucene demo JAR.  You should
+see the Lucene JAR file in the directory you created when you extracted the archive -- it
+should be named something like <code>lucene-core-{version}.jar</code>.  You should also see files
+called <code>lucene-analysis-common-{version}.jar</code> and <code>lucene-demos-{version}.jar</code>.
+If you checked out the sources from Subversion then the JARs are located at:
+</p>
+<ul>
+  <li><code>lucene/build/lucene-core-4.0-SNAPSHOT.jar</code></li>
+  <li><code>modules/analysis/build/common/lucene-analyzers-common-4.0-SNAPSHOT.jar</code></li>
+  <li><code>lucene/build/contrib/demo/lucene-demo-4.0-SNAPSHOT.jar</code></li>
+</ul>
+<p>
+Put all three of these files in your Java CLASSPATH.
 </p>
 </section>
 
@@ -47,7 +56,7 @@ Once you've gotten this far you're proba
 you've set your CLASSPATH correctly, just type:
 
 <pre>
-    java org.apache.lucene.demo.IndexFiles {full-path-to-lucene}/src
+    java org.apache.lucene.demo.IndexFiles -docs {path-to-lucene}/src
 </pre>
 
 This will produce a subdirectory called <code>index</code> which will contain an index of all of the

Modified: lucene/dev/branches/docvalues/lucene/src/site/src/documentation/content/xdocs/demo2.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/site/src/documentation/content/xdocs/demo2.xml?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/site/src/documentation/content/xdocs/demo2.xml (original)
+++ lucene/dev/branches/docvalues/lucene/src/site/src/documentation/content/xdocs/demo2.xml Mon Feb 21 14:13:28 2011
@@ -23,9 +23,9 @@ how to use Lucene in their applications.
 
 <p>
 Relative to the directory created when you extracted Lucene or retrieved it from Subversion, you
-should see a directory called <code>src</code> which in turn contains a directory called
-<code>demo</code>.  This is the root for all of the Lucene demos.  Under this directory is
-<code>org/apache/lucene/demo</code>.  This is where all the Java sources for the demos live.
+should see a directory called <code>lucene/contrib/demo/</code>.  This is the root for the Lucene
+demo.  Under this directory is <code>src/java/org/apache/lucene/demo/</code>.  This is where all
+the Java sources for the demo live.
 </p>
 
 <p>
@@ -39,62 +39,80 @@ Bring it up in <code>vi</code> or your e
 
 <p>
 As we discussed in the previous walk-through, the <a
-href="api/demo/org/apache/lucene/demo/IndexFiles.html">IndexFiles</a> class creates a Lucene
+href="api/contrib-demo/org/apache/lucene/demo/IndexFiles.html">IndexFiles</a> class creates a Lucene
 Index. Let's take a look at how it does this.
 </p>
 
 <p>
-The first substantial thing the <code>main</code> function does is instantiate <a
-href="api/core/org/apache/lucene/index/IndexWriter.html">IndexWriter</a>.  It passes the string
-"<code>index</code>" and a new instance of a class called <a
-href="api/core/org/apache/lucene/analysis/standard/StandardAnalyzer.html">StandardAnalyzer</a>. 
-The "<code>index</code>" string is the name of the filesystem directory where all index information
-should be stored.  Because we're not passing a full path, this will be created as a subdirectory of
-the current working directory (if it does not already exist). On some platforms, it may be created
-in other directories (such as the user's home directory).
+The <code>main()</code> method parses the command-line parameters, then in preparation for
+instantiating <a href="api/core/org/apache/lucene/index/IndexWriter.html">IndexWriter</a>, opens a 
+<a href="api/core/org/apache/lucene/store/Directory.html">Directory</a> and instantiates
+<a href="api/module-analysis-common/org/apache/lucene/analysis/standard/StandardAnalyzer.html"
+>StandardAnalyzer</a> and
+<a href="api/core/org/apache/lucene/index/IndexWriterConfig.html">IndexWriterConfig</a>.
 </p>
 
 <p>
-The <a href="api/core/org/apache/lucene/index/IndexWriter.html">IndexWriter</a> is the main
-class responsible for creating indices.  To use it you must instantiate it with a path that it can
-write the index into.  If this path does not exist it will first create it.  Otherwise it will
-refresh the index at that path.  You can also create an index using one of the subclasses of <a
-href="api/core/org/apache/lucene/store/Directory.html">Directory</a>.  In any case, you must also pass an
-instance of <a
-href="api/core/org/apache/lucene/analysis/Analyzer.html">org.apache.lucene.analysis.Analyzer</a>.
+The value of the <code>-index</code> command-line parameter is the name of the filesystem directory
+where all index information should be stored.  If <code>IndexFiles</code> is invoked with a 
+relative path given in the <code>-index</code> command-line parameter, or if the <code>-index</code>
+command-line parameter is not given, causing the default relative index path "<code>index</code>"
+to be used, the index path will be created as a subdirectory of the current working directory
+(if it does not already exist).  On some platforms, the index path may be created in a different
+directory (such as the user's home directory).
 </p>
 
 <p>
-The particular <a href="api/core/org/apache/lucene/analysis/Analyzer.html">Analyzer</a> we
-are using, <a
-href="api/core/org/apache/lucene/analysis/standard/StandardAnalyzer.html">StandardAnalyzer</a>, is
-little more than a standard Java Tokenizer, converting all strings to lowercase and filtering out
-stop words and characters from the index.  By stop words and characters I mean common language
-words such as articles (a, an, the, etc.) and other strings that may have less value for searching
-(e.g. <b>'s</b>) .  It should be noted that there are different rules for every language, and you
-should use the proper analyzer for each.  Lucene currently provides Analyzers for a number of
-different languages (see the <code>*Analyzer.java</code> sources under <a
-href="http://svn.apache.org/repos/asf/lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/">modules/analysis/common/src/java/org/apache/lucene/analysis</a>).
+The <code>-docs</code> command-line parameter value is the location of the directory containing
+files to be indexed.
+</p>
+  
+<p>
+The <code>-update</code> command-line parameter tells <code>IndexFiles</code> not to delete the
+index if it already exists.  When <code>-update</code> is not given, <code>IndexFiles</code> will
+first wipe the slate clean before indexing any documents.
 </p>
 
 <p>
-Looking further down in the file, you should see the <code>indexDocs()</code> code.  This recursive
-function simply crawls the directories and uses <a
-href="api/demo/org/apache/lucene/demo/FileDocument.html">FileDocument</a> to create <a
-href="api/core/org/apache/lucene/document/Document.html">Document</a> objects.  The <a
-href="api/core/org/apache/lucene/document/Document.html">Document</a> is simply a data object to
-represent the content in the file as well as its creation time and location.  These instances are
-added to the <code>indexWriter</code>.  Take a look inside <a
-href="api/demo/org/apache/lucene/demo/FileDocument.html">FileDocument</a>.  It's not particularly
-complicated.  It just adds fields to the <a
-href="api/core/org/apache/lucene/document/Document.html">Document</a>.
+Lucene <a href="api/core/org/apache/lucene/store/Directory.html">Directory</a>s are used by the
+<code>IndexWriter</code> to store information in the index.  In addition to the 
+<a href="api/core/org/apache/lucen/store/FSDirectory.html">FSDirectory</a> implementation we are using,
+there are several other <code>Directory</code> subclasses that can write to RAM, to databases, etc.
+</p>
+  
+<p>
+Lucene <a href="api/core/org/apache/lucene/analysis/Analyzer.html">Analyzer</a>s are processing pipelines
+that break up text into indexed tokens, a.k.a. terms, and optionally perform other operations on these
+tokens, e.g. downcasing, synonym insertion, filtering out unwanted tokens, etc.  The <code>Analyzer</code>
+we are using is <code>StandardAnalyzer</code>, which creates tokens using the Word Break rules from the
+Unicode Text Segmentation algorithm specified in <a href="http://unicode.org/reports/tr29/">Unicode
+Standard Annex #29</a>; converts tokens to lowercase; and then filters out stopwords.  Stopwords are
+common language words such as articles (a, an, the, etc.) and other tokens that may have less value for
+searching.  It should be noted that there are different rules for every language, and you should use the
+proper analyzer for each.  Lucene currently provides Analyzers for a number of different languages (see
+the <code>*Analyzer.java</code> sources under 
+<a href="http://svn.apache.org/repos/asf/lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/"
+>modules/analysis/common/src/java/org/apache/lucene/analysis</a>).
 </p>
 
 <p>
-As you can see there isn't much to creating an index.  The devil is in the details.  You may also
-wish to examine the other samples in this directory, particularly the <a
-href="api/demo/org/apache/lucene/demo/IndexHTML.html">IndexHTML</a> class.  It is a bit more
-complex but builds upon this example.
+The <code>IndexWriterConfig</code> instance holds all configuration for <code>IndexWriter</code>.  For
+example, we set the <code>OpenMode</code> to use here based on the value of the <code>-update</code>
+command-line parameter.
+</p>
+  
+<p>
+Looking further down in the file, after <code>IndexWriter</code> is instantiated, you should see the
+<code>indexDocs()</code> code.  This recursive function crawls the directories and creates
+<a href="api/core/org/apache/lucene/document/Document.html">Document</a> objects.  The 
+<code>Document</code> is simply a data object to represent the text content from the file as well as
+its creation time and location.  These instances are added to the <code>IndexWriter</code>.  If
+the <code>-update</code> command-line parameter is given, the <code>IndexWriter</code> 
+<code>OpenMode</code> will be set to <code>OpenMode.CREATE_OR_APPEND</code>, and rather than
+adding documents to the index, the <code>IndexWriter</code> will <strong>update</strong> them
+in the index by attempting to find an already-indexed document with the same identifier (in our
+case, the file path serves as the identifier); deleting it from the index if it exists; and then
+adding the new document to the index.
 </p>
 
 </section>
@@ -102,43 +120,30 @@ complex but builds upon this example.
 <section id="Searching Files"><title>Searching Files</title>
 
 <p>
-The <a href="api/demo/org/apache/lucene/demo/SearchFiles.html">SearchFiles</a> class is
-quite simple.  It primarily collaborates with an <a
-href="api/core/org/apache/lucene/search/IndexSearcher.html">IndexSearcher</a>, <a
-href="api/core/org/apache/lucene/analysis/standard/StandardAnalyzer.html">StandardAnalyzer</a>
-(which is used in the <a
-href="api/core/org/apache/lucene/demo/IndexFiles.html">IndexFiles</a> class as well) and a
-<a href="api/core/org/apache/lucene/queryParser/QueryParser.html">QueryParser</a>.  The
+The <a href="api/contrib-demo/org/apache/lucene/demo/SearchFiles.html">SearchFiles</a> class is
+quite simple.  It primarily collaborates with an 
+<a href="api/core/org/apache/lucene/search/IndexSearcher.html">IndexSearcher</a>, 
+<a href="api/modules-analysis-common/org/apache/lucene/analysis/standard/StandardAnalyzer.html"
+>StandardAnalyzer</a> (which is used in the
+<a href="api/contrib-demo/org/apache/lucene/demo/IndexFiles.html">IndexFiles</a> class as well)
+and a <a href="api/core/org/apache/lucene/queryParser/QueryParser.html">QueryParser</a>.  The
 query parser is constructed with an analyzer used to interpret your query text in the same way the
-documents are interpreted: finding the end of words and removing useless words like 'a', 'an' and
-'the'.  The <a href="api/core/org/apache/lucene/search/Query.html">Query</a> object contains
-the results from the <a
-href="api/core/org/apache/lucene/queryParser/QueryParser.html">QueryParser</a> which is passed to
-the searcher.  Note that it's also possible to programmatically construct a rich <a
-href="api/core/org/apache/lucene/search/Query.html">Query</a> object without using the query
+documents are interpreted: finding word boundaries, downcasing, and removing useless words like
+'a', 'an' and 'the'.  The <a href="api/core/org/apache/lucene/search/Query.html">Query</a>
+object contains the results from the
+<a href="api/core/org/apache/lucene/queryParser/QueryParser.html">QueryParser</a> which is passed
+to the searcher.  Note that it's also possible to programmatically construct a rich 
+<a href="api/core/org/apache/lucene/search/Query.html">Query</a> object without using the query
 parser.  The query parser just enables decoding the <a href="queryparsersyntax.html">Lucene query
-syntax</a> into the corresponding <a
-href="api/core/org/apache/lucene/search/Query.html">Query</a> object. Search can be executed in 
-two different ways: 
-<ul>
-<li>Streaming: A <a href="api/core/org/apache/lucene/search/Collector.html">Collector</a> subclass
-simply prints out the document ID and score for each matching document.</li>
-<li>Paging: Using a <a href="api/core/org/apache/lucene/search/TopScoreDocCollector.html">TopScoreDocCollector</a>
- the search results are printed in pages, sorted by score (i. e. relevance).</li>
-</ul>  
+syntax</a> into the corresponding <a href="api/core/org/apache/lucene/search/Query.html">Query</a>
+object.
 </p>
 
-</section>
-
-<section id="The Web example..."><title>The Web example...</title>
-
 <p>
-<a href="demo3.html">read on&gt;&gt;&gt;</a>
+<code>SearchFiles</code> uses the <code>IndexSearcher.search(query,n)</code> method that returns
+<a href="api/core/org/apache/lucene/search/TopDocs.html">TopDocs</a> with max <code>n</code> hits.
+The results are printed in pages, sorted by score (i.e. relevance).
 </p>
-
 </section>
-
 </body>
 </document>
-
-

Modified: lucene/dev/branches/docvalues/lucene/src/site/src/documentation/content/xdocs/gettingstarted.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/site/src/documentation/content/xdocs/gettingstarted.xml?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/site/src/documentation/content/xdocs/gettingstarted.xml (original)
+++ lucene/dev/branches/docvalues/lucene/src/site/src/documentation/content/xdocs/gettingstarted.xml Mon Feb 21 14:13:28 2011
@@ -14,7 +14,7 @@
     <title>Getting Started</title>
 <p>
 This document is intended as a "getting started" guide.  It has three audiences: first-time users
-looking to install Apache Lucene in their application or web server; developers looking to modify or base
+looking to install Apache Lucene in their application; developers looking to modify or base
 the applications they develop on Lucene; and developers looking to become involved in and contribute
 to the development of Lucene.  This document is written in tutorial and walk-through format.  The
 goal is to help you "get started".  It does not go into great depth on some of the conceptual or
@@ -33,20 +33,6 @@ may wish to skip sections.
 	<li><a href="demo2.html">About the sources and implementation for the command-line Lucene
 	demo</a>.  This section walks through the implementation details (sources) of the
 	command-line Lucene demo.  This section is intended for developers.</li> <p/>
-
-	<li><a href="demo3.html">About installing and configuring the demo template web
-	application</a>.  While this walk-through assumes Tomcat as your container of choice,
-	there is no reason you can't (provided you have the requisite knowledge) adapt the
-	instructions to your container.  This section is intended for those responsible for the
-	development or deployment of Lucene-based web applications.</li> <p/>
-
-	<li><a href="demo4.html">About the sources used to construct the demo template web
-	application</a>.  Please note the template application is designed to highlight features of
-	Lucene and is <b>not</b> an example of best practices.  (One would hopefully use MVC
-	architecture such as provided by Jakarta Struts and taglibs, but showing you how to do that
-	would be WAY beyond the scope of this guide.)  This section is intended for developers and
-	those wishing to customize the demo template web application to their needs.  </li>
-
 </ul>
 </section>
 

Modified: lucene/dev/branches/docvalues/lucene/src/site/src/documentation/content/xdocs/lucene-contrib/index.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/site/src/documentation/content/xdocs/lucene-contrib/index.xml?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/site/src/documentation/content/xdocs/lucene-contrib/index.xml (original)
+++ lucene/dev/branches/docvalues/lucene/src/site/src/documentation/content/xdocs/lucene-contrib/index.xml Mon Feb 21 14:13:28 2011
@@ -59,6 +59,12 @@
                 <p>The benchmark contribution contains tools for benchmarking Lucene using standard, freely available corpora.</p>
                 <p>See <a href="../api/contrib-benchmark/index.html">benchmark javadoc</a></p>
             </section>            
+        
+            <section id="demo"><title>demo</title>
+                <p>The demo contrib contains the Lucene demo: IndexFiles and SearchFiles, described under
+                   <a href="../gettingstarted.html">Getting Started</a>.</p>
+                <p>See <a href="../api/contrib-demo/index.html">demo javadoc</a></p>
+            </section>
             
             <section id="db"><title>db</title>
                 <p>Provides integration with Berkley DB.</p>

Modified: lucene/dev/branches/docvalues/lucene/src/site/src/documentation/content/xdocs/site.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/site/src/documentation/content/xdocs/site.xml?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/site/src/documentation/content/xdocs/site.xml (original)
+++ lucene/dev/branches/docvalues/lucene/src/site/src/documentation/content/xdocs/site.xml Mon Feb 21 14:13:28 2011
@@ -50,12 +50,12 @@ See http://forrest.apache.org/docs/linki
       <javadoc label="Javadocs">
          <javadoc-all label="All" href="ext:javadocs-all"/>
 		 <javadoc-core label="Core" href="ext:javadocs-core"/>
-		 <javadoc-demo label="Demo" href="ext:javadocs-demo"/>
 		 <javadoc-contrib label="Contrib">
 		    <javadoc-contrib-ant label="Ant" href="ext:javadocs-contrib-ant"/>
 		    <javadoc-contrib-bdb label="Bdb" href="ext:javadocs-contrib-bdb"/>
 		    <javadoc-contrib-bdb-je label="Bdb-je" href="ext:javadocs-contrib-bdb-je"/>
 		    <javadoc-contrib-benchmark label="Benchmark" href="ext:javadocs-contrib-benchmark"/>
+        <javadoc-contrib-demo label="Demo" href="ext:javadocs-contrib-demo"/>
 		    <javadoc-contrib-highlighter label="Highlighter" href="ext:javadocs-contrib-highlighter"/>
 		    <javadoc-contrib-instantiated label="Instantiated" href="ext:javadocs-contrib-instantiated"/>
 		    <javadoc-contrib-lucli label="Lucli" href="ext:javadocs-contrib-lucli"/>
@@ -98,11 +98,11 @@ See http://forrest.apache.org/docs/linki
     <changes-contrib href="changes/Contrib-Changes.html" />
     <javadocs-all href="api/all/index.html"/>
 	<javadocs-core href="api/core/index.html"/>
-	<javadocs-demo href="api/demo/index.html"/>
 	<javadocs-contrib-ant href="api/contrib-ant/index.html"/>
 	<javadocs-contrib-bdb href="api/contrib-bdb/index.html"/>
 	<javadocs-contrib-bdb-je href="api/contrib-bdb-je/index.html"/>
 	<javadocs-contrib-benchmark href="api/contrib-benchmark/index.html"/>
+  <javadocs-contrib-demo href="api/contrib-demo/index.html"/>
 	<javadocs-contrib-highlighter href="api/contrib-highlighter/index.html"/>
 	<javadocs-contrib-instantiated href="api/contrib-instantiated/index.html"/>
 	<javadocs-contrib-lucli href="api/contrib-lucli/index.html"/>

Modified: lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/analysis/MockAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/analysis/MockAnalyzer.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/analysis/MockAnalyzer.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/analysis/MockAnalyzer.java Mon Feb 21 14:13:28 2011
@@ -36,6 +36,10 @@ public final class MockAnalyzer extends 
   private final boolean payload;
   private int positionIncrementGap;
 
+  /**
+   * Calls {@link #MockAnalyzer(CharacterRunAutomaton, boolean, CharacterRunAutomaton, boolean, boolean) 
+   * MockAnalyzer(runAutomaton, lowerCase, filter, enablePositionIncrements, true}).
+   */
   public MockAnalyzer(CharacterRunAutomaton runAutomaton, boolean lowerCase, CharacterRunAutomaton filter, boolean enablePositionIncrements) {
     this(runAutomaton, lowerCase, filter, enablePositionIncrements, true);    
   }
@@ -47,7 +51,7 @@ public final class MockAnalyzer extends 
    * @param lowerCase true if the tokenizer should lowercase terms
    * @param filter DFA describing how terms should be filtered (set of stopwords, etc)
    * @param enablePositionIncrements true if position increments should reflect filtered terms.
-   * @param payload if payloads should be added
+   * @param payload if payloads should be added containing the positions (for testing)
    */
   public MockAnalyzer(CharacterRunAutomaton runAutomaton, boolean lowerCase, CharacterRunAutomaton filter, boolean enablePositionIncrements, boolean payload) {
     this.runAutomaton = runAutomaton;
@@ -58,21 +62,26 @@ public final class MockAnalyzer extends 
   }
 
   /**
-   * Creates a new MockAnalyzer, with no filtering.
-   * 
-   * @param runAutomaton DFA describing how tokenization should happen (e.g. [a-zA-Z]+)
-   * @param lowerCase true if the tokenizer should lowercase terms
+   * Calls {@link #MockAnalyzer(CharacterRunAutomaton, boolean, CharacterRunAutomaton, boolean, boolean) 
+   * MockAnalyzer(runAutomaton, lowerCase, MockTokenFilter.EMPTY_STOPSET, false, true}).
    */
   public MockAnalyzer(CharacterRunAutomaton runAutomaton, boolean lowerCase) {
     this(runAutomaton, lowerCase, MockTokenFilter.EMPTY_STOPSET, false, true);
   }
 
+  /**
+   * Calls {@link #MockAnalyzer(CharacterRunAutomaton, boolean, CharacterRunAutomaton, boolean, boolean) 
+   * MockAnalyzer(runAutomaton, lowerCase, MockTokenFilter.EMPTY_STOPSET, false, payload}).
+   */
   public MockAnalyzer(CharacterRunAutomaton runAutomaton, boolean lowerCase, boolean payload) {
     this(runAutomaton, lowerCase, MockTokenFilter.EMPTY_STOPSET, false, payload);
   }
   
   /** 
-   * Create a Whitespace-lowercasing analyzer with no stopwords removal 
+   * Create a Whitespace-lowercasing analyzer with no stopwords removal.
+   * <p>
+   * Calls {@link #MockAnalyzer(CharacterRunAutomaton, boolean, CharacterRunAutomaton, boolean, boolean) 
+   * MockAnalyzer(MockTokenizer.WHITESPACE, true, MockTokenFilter.EMPTY_STOPSET, false, true}).
    */
   public MockAnalyzer() {
     this(MockTokenizer.WHITESPACE, true);

Modified: lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/analysis/MockTokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/analysis/MockTokenFilter.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/analysis/MockTokenFilter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/analysis/MockTokenFilter.java Mon Feb 21 14:13:28 2011
@@ -60,6 +60,13 @@ public final class MockTokenFilter exten
   private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
   private final PositionIncrementAttribute posIncrAtt = addAttribute(PositionIncrementAttribute.class);
   
+  /**
+   * Create a new MockTokenFilter.
+   * 
+   * @param input TokenStream to filter
+   * @param filter DFA representing the terms that should be removed.
+   * @param enablePositionIncrements true if the removal should accumulate position increments.
+   */
   public MockTokenFilter(TokenStream input, CharacterRunAutomaton filter, boolean enablePositionIncrements) {
     super(input);
     this.filter = filter;

Modified: lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/analysis/MockTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/analysis/MockTokenizer.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/analysis/MockTokenizer.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/analysis/MockTokenizer.java Mon Feb 21 14:13:28 2011
@@ -20,7 +20,7 @@ package org.apache.lucene.analysis;
 import java.io.IOException;
 import java.io.Reader;
 
-import org.apache.lucene.util.Version;
+import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.automaton.CharacterRunAutomaton;
 import org.apache.lucene.util.automaton.RegExp;
 
@@ -46,14 +46,14 @@ public class MockTokenizer extends CharT
   private int state;
 
   public MockTokenizer(AttributeFactory factory, Reader input, CharacterRunAutomaton runAutomaton, boolean lowerCase) {
-    super(Version.LUCENE_CURRENT, factory, input);
+    super(LuceneTestCase.TEST_VERSION_CURRENT, factory, input);
     this.runAutomaton = runAutomaton;
     this.lowerCase = lowerCase;
     this.state = runAutomaton.getInitialState();
   }
 
   public MockTokenizer(Reader input, CharacterRunAutomaton runAutomaton, boolean lowerCase) {
-    super(Version.LUCENE_CURRENT, input);
+    super(LuceneTestCase.TEST_VERSION_CURRENT, input);
     this.runAutomaton = runAutomaton;
     this.lowerCase = lowerCase;
     this.state = runAutomaton.getInitialState();

Modified: lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java?rev=1072973&r1=1072972&r2=1072973&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java Mon Feb 21 14:13:28 2011
@@ -91,6 +91,10 @@ public class RandomIndexWriter implement
     }
   } 
 
+  /**
+   * Adds a Document.
+   * @see IndexWriter#addDocument(Document)
+   */
   public void addDocument(Document doc) throws IOException {
     w.addDocument(doc);
     if (docCount++ == flushAt) {
@@ -102,6 +106,21 @@ public class RandomIndexWriter implement
     }
   }
   
+  /**
+   * Updates a document.
+   * @see IndexWriter#updateDocument(Term, Document)
+   */
+  public void updateDocument(Term t, Document doc) throws IOException {
+    w.updateDocument(t, doc);
+    if (docCount++ == flushAt) {
+      if (LuceneTestCase.VERBOSE) {
+        System.out.println("RIW.updateDocument: now doing a commit");
+      }
+      w.commit();
+      flushAt += _TestUtil.nextInt(r, 10, 1000);
+    }
+  }
+  
   public void addIndexes(Directory... dirs) throws CorruptIndexException, IOException {
     w.addIndexes(dirs);
   }
@@ -127,17 +146,21 @@ public class RandomIndexWriter implement
   }
 
   public IndexReader getReader() throws IOException {
+    return getReader(true);
+  }
+
+  public IndexReader getReader(boolean applyDeletions) throws IOException {
     getReaderCalled = true;
     if (r.nextInt(4) == 2)
       w.optimize();
     // If we are writing with PreFlexRW, force a full
     // IndexReader.open so terms are sorted in codepoint
     // order during searching:
-    if (!w.codecs.getDefaultFieldCodec().equals("PreFlex") && r.nextBoolean()) {
+    if (!applyDeletions || !w.codecs.getDefaultFieldCodec().equals("PreFlex") && r.nextBoolean()) {
       if (LuceneTestCase.VERBOSE) {
         System.out.println("RIW.getReader: use NRT reader");
       }
-      return w.getReader();
+      return w.getReader(applyDeletions);
     } else {
       if (LuceneTestCase.VERBOSE) {
         System.out.println("RIW.getReader: open new reader");
@@ -147,6 +170,10 @@ public class RandomIndexWriter implement
     }
   }
 
+  /**
+   * Close this writer.
+   * @see IndexWriter#close()
+   */
   public void close() throws IOException {
     // if someone isn't using getReader() API, we want to be sure to
     // maybeOptimize since presumably they might open a reader on the dir.
@@ -156,6 +183,13 @@ public class RandomIndexWriter implement
     w.close();
   }
 
+  /**
+   * Forces an optimize.
+   * <p>
+   * NOTE: this should be avoided in tests unless absolutely necessary,
+   * as it will result in less test coverage.
+   * @see IndexWriter#optimize()
+   */
   public void optimize() throws IOException {
     w.optimize();
   }