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 2010/12/06 01:47:28 UTC

svn commit: r1042501 [5/13] - in /lucene/dev/branches/docvalues: ./ lucene/ lucene/contrib/ lucene/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/ lucene/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ lucene/c...

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=1042501&r1=1042500&r2=1042501&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 Dec  6 00:47:16 2010
@@ -22,6 +22,7 @@ import java.io.Serializable;
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.queryParser.QueryParser;
 import org.apache.lucene.util.AttributeSource;
@@ -268,14 +269,14 @@ public abstract class MultiTermQuery ext
    * provide attributes, the rewrite method uses to inform about e.g. maximum competitive boosts.
    * This is currently only used by {@link TopTermsRewrite}
    */
-  protected abstract TermsEnum getTermsEnum(IndexReader reader, AttributeSource atts) throws IOException;
+  protected abstract TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException;
 
   /** Convenience method, if no attributes are needed:
    * This simply passes empty attributes and is equal to:
-   * <code>getTermsEnum(reader, new AttributeSource())</code>
+   * <code>getTermsEnum(terms, new AttributeSource())</code>
    */
-  protected final TermsEnum getTermsEnum(IndexReader reader) throws IOException {
-    return getTermsEnum(reader, new AttributeSource());
+  protected final TermsEnum getTermsEnum(Terms terms) throws IOException {
+    return getTermsEnum(terms, new AttributeSource());
   }
 
   /**

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java Mon Dec  6 00:47:16 2010
@@ -118,7 +118,7 @@ public class MultiTermQueryWrapperFilter
       return DocIdSet.EMPTY_DOCIDSET;
     }
 
-    final TermsEnum termsEnum = query.getTermsEnum(reader);
+    final TermsEnum termsEnum = query.getTermsEnum(terms);
     assert termsEnum != null;
     if (termsEnum.next() != null) {
       // fill into a OpenBitSet

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=1042501&r1=1042500&r2=1042501&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 Dec  6 00:47:16 2010
@@ -25,7 +25,7 @@ import org.apache.lucene.analysis.Numeri
 import org.apache.lucene.document.NumericField; // for javadocs
 import org.apache.lucene.util.NumericUtils;
 import org.apache.lucene.util.ToStringUtils;
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.Terms;
 import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.index.TermsEnum;
@@ -302,11 +302,11 @@ public final class NumericRangeQuery<T e
   }
 
   @Override @SuppressWarnings("unchecked")
-  protected TermsEnum getTermsEnum(final IndexReader reader, AttributeSource atts) throws IOException {
+  protected TermsEnum getTermsEnum(final Terms terms, AttributeSource atts) throws IOException {
     // very strange: java.lang.Number itsself is not Comparable, but all subclasses used here are
     return (min != null && max != null && ((Comparable<T>) min).compareTo(max) > 0) ?
       TermsEnum.EMPTY :
-      new NumericRangeTermsEnum(reader);
+      new NumericRangeTermsEnum(terms.iterator());
   }
 
   /** Returns <code>true</code> if the lower endpoint is inclusive */
@@ -385,8 +385,8 @@ public final class NumericRangeQuery<T e
     private final LinkedList<BytesRef> rangeBounds = new LinkedList<BytesRef>();
     private final Comparator<BytesRef> termComp;
 
-    NumericRangeTermsEnum(final IndexReader reader) throws IOException {
-      super(reader, getField());
+    NumericRangeTermsEnum(final TermsEnum tenum) throws IOException {
+      super(tenum);
       switch (valSize) {
         case 64: {
           // lower

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/PrefixQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/PrefixQuery.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/PrefixQuery.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/PrefixQuery.java Mon Dec  6 00:47:16 2010
@@ -19,11 +19,9 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 
-import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.ToStringUtils;
 
@@ -46,15 +44,14 @@ public class PrefixQuery extends MultiTe
   public Term getPrefix() { return prefix; }
   
   @Override  
-  protected TermsEnum getTermsEnum(IndexReader reader, AttributeSource atts) throws IOException {
+  protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
+    TermsEnum tenum = terms.iterator();
+    
     if (prefix.bytes().length == 0) {
       // no prefix -- match all terms for this field:
-      // NOTE: for now, MultiTermQuery enums terms at the
-      // MultiReader level, so we must use MultiFields here:
-      final Terms terms = MultiFields.getTerms(reader, getField());
-      return (terms != null) ? terms.iterator() : TermsEnum.EMPTY;
+      return tenum;
     }
-    return new PrefixTermsEnum(reader, prefix);
+    return new PrefixTermsEnum(tenum, prefix);
   }
 
   /** Prints a user-readable version of this query. */

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/PrefixTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/PrefixTermsEnum.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/PrefixTermsEnum.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/PrefixTermsEnum.java Mon Dec  6 00:47:16 2010
@@ -19,8 +19,8 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 
-import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.util.BytesRef;
 
 /**
@@ -34,8 +34,8 @@ public class PrefixTermsEnum extends Fil
 
   private final BytesRef prefixRef;
 
-  public PrefixTermsEnum(IndexReader reader, Term prefix) throws IOException {
-    super(reader, prefix.field());
+  public PrefixTermsEnum(TermsEnum tenum, Term prefix) throws IOException {
+    super(tenum);
     setInitialSeekTerm(prefixRef = prefix.bytes());
   }
 

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=1042501&r1=1042500&r2=1042501&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 Dec  6 00:47:16 2010
@@ -18,15 +18,15 @@ package org.apache.lucene.search;
  */
 
 
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Collection;
+
 import org.apache.lucene.index.FieldInvertState;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.Explanation.IDFExplanation;
 import org.apache.lucene.util.SmallFloat;
 
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.Collection;
-
 
 /** 
  * Expert: Scoring API.
@@ -562,16 +562,6 @@ public abstract class Similarity impleme
       NORM_TABLE[i] = SmallFloat.byte315ToFloat((byte)i);
   }
 
-  /**
-   * Decodes a normalization factor stored in an index.
-   * @see #decodeNormValue(byte)
-   * @deprecated Use {@link #decodeNormValue} instead.
-   */
-  @Deprecated
-  public static float decodeNorm(byte b) {
-    return NORM_TABLE[b & 0xFF];  // & 0xFF maps negative bytes to positive above 127
-  }
-
   /** Decodes a normalization factor stored in an index.
    * @see #encodeNormValue(float)
    */
@@ -579,17 +569,6 @@ public abstract class Similarity impleme
     return NORM_TABLE[b & 0xFF];  // & 0xFF maps negative bytes to positive above 127
   }
 
-  /** Returns a table for decoding normalization bytes.
-   * @see #encodeNormValue(float)
-   * @see #decodeNormValue(byte)
-   * 
-   * @deprecated Use instance methods for encoding/decoding norm values to enable customization.
-   */
-  @Deprecated
-  public static float[] getNormDecoder() {
-    return NORM_TABLE;
-  }
-
   /**
    * Compute the normalization value for a field, given the accumulated
    * state of term processing for this field (see {@link FieldInvertState}).
@@ -670,20 +649,6 @@ public abstract class Similarity impleme
     return SmallFloat.floatToByte315(f);
   }
   
-  /**
-   * Static accessor kept for backwards compability reason, use encodeNormValue instead.
-   * @param f norm-value to encode
-   * @return byte representing the given float
-   * @deprecated Use {@link #encodeNormValue} instead.
-   * 
-   * @see #encodeNormValue(float)
-   */
-  @Deprecated
-  public static byte encodeNorm(float f) {
-    return SmallFloat.floatToByte315(f);
-  }
-
-
   /** Computes a score factor based on a term or phrase's frequency in a
    * document.  This value is multiplied by the {@link #idf(int, int)}
    * factor for each term in the query and these products are then summed to

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/SingleTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/SingleTermsEnum.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/SingleTermsEnum.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/SingleTermsEnum.java Mon Dec  6 00:47:16 2010
@@ -19,8 +19,8 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 
-import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.util.BytesRef;
 
 /**
@@ -39,8 +39,8 @@ public final class SingleTermsEnum exten
    * After calling the constructor the enumeration is already pointing to the term,
    * if it exists.
    */
-  public SingleTermsEnum(IndexReader reader, Term singleTerm) throws IOException {
-    super(reader, singleTerm.field());
+  public SingleTermsEnum(TermsEnum tenum, Term singleTerm) throws IOException {
+    super(tenum);
     singleRef = singleTerm.bytes();
     setInitialSeekTerm(singleRef);
   }

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=1042501&r1=1042500&r2=1042501&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 Dec  6 00:47:16 2010
@@ -22,13 +22,7 @@ import java.io.Serializable;
 import java.util.Comparator;
 import java.util.Locale;
 
-import org.apache.lucene.search.cache.ByteValuesCreator;
-import org.apache.lucene.search.cache.CachedArrayCreator;
-import org.apache.lucene.search.cache.DoubleValuesCreator;
-import org.apache.lucene.search.cache.FloatValuesCreator;
-import org.apache.lucene.search.cache.IntValuesCreator;
-import org.apache.lucene.search.cache.LongValuesCreator;
-import org.apache.lucene.search.cache.ShortValuesCreator;
+import org.apache.lucene.search.cache.*;
 import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.BytesRef;
 
@@ -147,7 +141,7 @@ implements Serializable {
    * @throws IllegalArgumentException if the parser fails to
    *  subclass an existing numeric parser, or field is null
    *  
-   *  @deprecated use EntryCreator version
+   *  @deprecated (4.0) use EntryCreator version
    */
   @Deprecated
   public SortField (String field, FieldCache.Parser parser) {
@@ -165,7 +159,7 @@ implements Serializable {
    * @throws IllegalArgumentException if the parser fails to
    *  subclass an existing numeric parser, or field is null
    *  
-   *  @deprecated use EntryCreator version
+   *  @deprecated (4.0) use EntryCreator version
    */
   @Deprecated
   public SortField (String field, FieldCache.Parser parser, boolean reverse) {
@@ -323,7 +317,7 @@ implements Serializable {
   /** Returns the instance of a {@link FieldCache} parser that fits to the given sort type.
    * May return <code>null</code> if no parser was specified. Sorting is using the default parser then.
    * @return An instance of a {@link FieldCache} parser, or <code>null</code>.
-   * @deprecated use getEntryCreator()
+   * @deprecated (4.0) use getEntryCreator()
    */
   @Deprecated
   public FieldCache.Parser getParser() {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TermCollectingRewrite.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TermCollectingRewrite.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TermCollectingRewrite.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TermCollectingRewrite.java Mon Dec  6 00:47:16 2010
@@ -58,7 +58,7 @@ abstract class TermCollectingRewrite<Q e
         continue;
       }
 
-      final TermsEnum termsEnum = query.getTermsEnum(r, collector.attributes);
+      final TermsEnum termsEnum = query.getTermsEnum(terms, collector.attributes);
       assert termsEnum != null;
 
       if (termsEnum == TermsEnum.EMPTY)

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TermRangeQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TermRangeQuery.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TermRangeQuery.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TermRangeQuery.java Mon Dec  6 00:47:16 2010
@@ -20,12 +20,10 @@ package org.apache.lucene.search;
 import java.io.IOException;
 import java.text.Collator;
 
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.MultiFields;
-import org.apache.lucene.util.ToStringUtils;
+import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.util.AttributeSource;
+import org.apache.lucene.util.ToStringUtils;
 
 /**
  * A Query that matches documents within an range of terms.
@@ -131,26 +129,20 @@ public class TermRangeQuery extends Mult
   public Collator getCollator() { return collator; }
   
   @Override
-  protected TermsEnum getTermsEnum(IndexReader reader, AttributeSource atts) throws IOException {
+  protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
     if (collator == null && lowerTerm != null && upperTerm != null && lowerTerm.compareTo(upperTerm) > 0) {
       return TermsEnum.EMPTY;
     }
+    
+    TermsEnum tenum = terms.iterator();
+    
     if ((lowerTerm == null || (collator == null && includeLower && "".equals(lowerTerm))) && upperTerm == null) {
-      // NOTE: for now, MultiTermQuery enums terms at the
-      // MultiReader level, so we must use MultiFields here:
-      final Terms terms = MultiFields.getTerms(reader, field);
-      return (terms != null) ? terms.iterator() : null;
+      return tenum;
     }
-    return new TermRangeTermsEnum(reader, field,
+    return new TermRangeTermsEnum(tenum,
         lowerTerm, upperTerm, includeLower, includeUpper, collator);
   }
 
-  /** @deprecated */
-  @Deprecated
-  public String field() {
-    return getField();
-  }
-
   /** Prints a user-readable version of this query. */
   @Override
   public String toString(String field) {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TermRangeTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TermRangeTermsEnum.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TermRangeTermsEnum.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TermRangeTermsEnum.java Mon Dec  6 00:47:16 2010
@@ -21,7 +21,7 @@ import java.io.IOException;
 import java.text.Collator;
 import java.util.Comparator;
 
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.util.BytesRef;
 
 /**
@@ -51,9 +51,8 @@ public class TermRangeTermsEnum extends 
    * (you can't select all but the first or last term without 
    * explicitly specifying the term to exclude.)
    * 
-   * @param reader
-   * @param field
-   *          An interned field that holds both lower and upper terms.
+   * @param tenum
+   *          TermsEnum to filter
    * @param lowerTermText
    *          The term text at the lower end of the range
    * @param upperTermText
@@ -69,9 +68,9 @@ public class TermRangeTermsEnum extends 
    * 
    * @throws IOException
    */
-  public TermRangeTermsEnum(IndexReader reader, String field, String lowerTermText, String upperTermText, 
+  public TermRangeTermsEnum(TermsEnum tenum, String lowerTermText, String upperTermText, 
     boolean includeLower, boolean includeUpper, Collator collator) throws IOException {
-    super(reader, field);
+    super(tenum);
     this.collator = collator;
     this.upperTermText = upperTermText;
     this.lowerTermText = lowerTermText;

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TopFieldCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TopFieldCollector.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TopFieldCollector.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TopFieldCollector.java Mon Dec  6 00:47:16 2010
@@ -913,6 +913,10 @@ public abstract class TopFieldCollector 
       throw new IllegalArgumentException("Sort must contain at least one field");
     }
     
+    if (numHits <= 0) {
+      throw new IllegalArgumentException("numHits must be > 0; please use TotalHitCountCollector if you just need the total hit count");
+    }
+
     FieldValueHitQueue queue = FieldValueHitQueue.create(sort.fields, numHits);
     if (queue.getComparators().length == 1) {
       if (docsScoredInOrder) {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TopScoreDocCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TopScoreDocCollector.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TopScoreDocCollector.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/TopScoreDocCollector.java Mon Dec  6 00:47:16 2010
@@ -109,6 +109,10 @@ public abstract class TopScoreDocCollect
    */
   public static TopScoreDocCollector create(int numHits, boolean docsScoredInOrder) {
     
+    if (numHits <= 0) {
+      throw new IllegalArgumentException("numHits must be > 0; please use TotalHitCountCollector if you just need the total hit count");
+    }
+
     if (docsScoredInOrder) {
       return new InOrderTopScoreDocCollector(numHits);
     } else {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/cache/DocTermsCreator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/cache/DocTermsCreator.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/cache/DocTermsCreator.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/cache/DocTermsCreator.java Mon Dec  6 00:47:16 2010
@@ -165,7 +165,7 @@ public class DocTermsCreator extends Ent
     @Override
     public BytesRef getTerm(int docID, BytesRef ret) {
       final long pointer = docToOffset.get(docID);
-      return bytes.fillUsingLengthPrefix(ret, pointer);
+      return bytes.fill(ret, pointer);
     }
   }
 }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/cache/DocTermsIndexCreator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/cache/DocTermsIndexCreator.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/cache/DocTermsIndexCreator.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/cache/DocTermsIndexCreator.java Mon Dec  6 00:47:16 2010
@@ -213,7 +213,7 @@ public class DocTermsIndexCreator extend
 
     @Override
     public BytesRef lookup(int ord, BytesRef ret) {
-      return bytes.fillUsingLengthPrefix(ret, termOrdToBytesOffset.get(ord));
+      return bytes.fill(ret, termOrdToBytesOffset.get(ord));
     }
 
     @Override
@@ -235,7 +235,7 @@ public class DocTermsIndexCreator extend
         currentBlockNumber = 0;
         blocks = bytes.getBlocks();
         blockEnds = bytes.getBlockEnds();
-        currentBlockNumber = bytes.fillUsingLengthPrefix2(term, termOrdToBytesOffset.get(0));
+        currentBlockNumber = bytes.fillAndGetIndex(term, termOrdToBytesOffset.get(0));
         end = blockEnds[currentBlockNumber];
       }
 
@@ -249,7 +249,7 @@ public class DocTermsIndexCreator extend
       public SeekStatus seek(long ord) throws IOException {
         assert(ord >= 0 && ord <= numOrd);
         // TODO: if gap is small, could iterate from current position?  Or let user decide that?
-        currentBlockNumber = bytes.fillUsingLengthPrefix2(term, termOrdToBytesOffset.get((int)ord));
+        currentBlockNumber = bytes.fillAndGetIndex(term, termOrdToBytesOffset.get((int)ord));
         end = blockEnds[currentBlockNumber];
         currentOrd = (int)ord;
         return SeekStatus.FOUND;

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/function/MultiValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/function/MultiValueSource.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/function/MultiValueSource.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/function/MultiValueSource.java Mon Dec  6 00:47:16 2010
@@ -19,9 +19,9 @@ package org.apache.lucene.search.functio
 
 import java.io.IOException;
 
-import org.apache.lucene.util.ReaderUtil;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.search.Explanation;
+import org.apache.lucene.util.ReaderUtil;
 
 /** This class wraps another ValueSource, but protects
  *  against accidental double RAM usage in FieldCache when
@@ -31,7 +31,7 @@ import org.apache.lucene.search.Explanat
  *  lookup, as it must resolve the incoming document to the
  *  right sub-reader using a binary search.</p>
  *
- *  @deprecated This class is temporary, to ease the
+ *  @deprecated (4.0) This class is temporary, to ease the
  *  migration to segment-based searching. Please change your
  *  code to not pass composite readers to these APIs. */
 

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/DataOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/DataOutput.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/DataOutput.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/DataOutput.java Mon Dec  6 00:47:16 2010
@@ -105,59 +105,6 @@ public abstract class DataOutput {
     writeBytes(utf8Result.bytes, 0, utf8Result.length);
   }
 
-  /** Writes a sub sequence of characters from s as the old
-   *  format (modified UTF-8 encoded bytes).
-   * @param s the source of the characters
-   * @param start the first character in the sequence
-   * @param length the number of characters in the sequence
-   * @deprecated -- please pre-convert to utf8 bytes
-   * instead or use {@link #writeString}
-   */
-  @Deprecated
-  public void writeChars(String s, int start, int length)
-       throws IOException {
-    final int end = start + length;
-    for (int i = start; i < end; i++) {
-      final int code = s.charAt(i);
-      if (code >= 0x01 && code <= 0x7F)
-        writeByte((byte)code);
-      else if (((code >= 0x80) && (code <= 0x7FF)) || code == 0) {
-        writeByte((byte)(0xC0 | (code >> 6)));
-        writeByte((byte)(0x80 | (code & 0x3F)));
-      } else {
-        writeByte((byte)(0xE0 | (code >>> 12)));
-        writeByte((byte)(0x80 | ((code >> 6) & 0x3F)));
-        writeByte((byte)(0x80 | (code & 0x3F)));
-      }
-    }
-  }
-
-  /** Writes a sub sequence of characters from char[] as
-   *  the old format (modified UTF-8 encoded bytes).
-   * @param s the source of the characters
-   * @param start the first character in the sequence
-   * @param length the number of characters in the sequence
-   * @deprecated -- please pre-convert to utf8 bytes instead or use {@link #writeString}
-   */
-  @Deprecated
-  public void writeChars(char[] s, int start, int length)
-    throws IOException {
-    final int end = start + length;
-    for (int i = start; i < end; i++) {
-      final int code = s[i];
-      if (code >= 0x01 && code <= 0x7F)
-        writeByte((byte)code);
-      else if (((code >= 0x80) && (code <= 0x7FF)) || code == 0) {
-        writeByte((byte)(0xC0 | (code >> 6)));
-        writeByte((byte)(0x80 | (code & 0x3F)));
-      } else {
-        writeByte((byte)(0xE0 | (code >>> 12)));
-        writeByte((byte)(0x80 | ((code >> 6) & 0x3F)));
-        writeByte((byte)(0x80 | (code & 0x3F)));
-      }
-    }
-  }
-
   private static int COPY_BUFFER_SIZE = 16384;
   private byte[] copyBuffer;
 

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/Directory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/Directory.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/Directory.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/Directory.java Mon Dec  6 00:47:16 2010
@@ -95,19 +95,6 @@ public abstract class Directory implemen
        throws IOException;
 
   /**
-   * Ensure that any writes to this file are moved to
-   * stable storage.  Lucene uses this to properly commit
-   * changes to the index, to prevent a machine/OS crash
-   * from corrupting the index.
-   * @deprecated use {@link #sync(Collection)} instead.
-   * For easy migration you can change your code to call
-   * sync(Collections.singleton(name))
-   */
-  @Deprecated
-  public void sync(String name) throws IOException { // TODO 4.0 kill me
-  }
-
-  /**
    * Ensure that any writes to these files are moved to
    * stable storage.  Lucene uses this to properly commit
    * changes to the index, to prevent a machine/OS crash
@@ -118,10 +105,7 @@ public abstract class Directory implemen
    * For other impls the operation can be a noop, for various
    * reasons.
    */
-  public void sync(Collection<String> names) throws IOException { // TODO 4.0 make me abstract
-    for (String name : names)
-      sync(name);
-  }
+  public abstract void sync(Collection<String> names) throws IOException;
 
   /** Returns a stream reading an existing file. */
   public abstract IndexInput openInput(String name)
@@ -233,41 +217,6 @@ public abstract class Directory implemen
   }
 
   /**
-   * Copy contents of a directory src to a directory dest. If a file in src
-   * already exists in dest then the one in dest will be blindly overwritten.
-   * <p>
-   * <b>NOTE:</b> the source directory cannot change while this method is
-   * running. Otherwise the results are undefined and you could easily hit a
-   * FileNotFoundException.
-   * <p>
-   * <b>NOTE:</b> this method only copies files that look like index files (ie,
-   * have extensions matching the known extensions of index files).
-   * 
-   * @param src source directory
-   * @param dest destination directory
-   * @param closeDirSrc if <code>true</code>, call {@link #close()} method on 
-   *        source directory
-   * @deprecated should be replaced with calls to
-   *             {@link #copy(Directory, String, String)} for every file that
-   *             needs copying. You can use the following code:
-   * 
-   * <pre>
-   * for (String file : src.listAll()) {
-   *   src.copy(dest, file, file);
-   * }
-   * </pre>
-   */
-  @Deprecated
-  public static void copy(Directory src, Directory dest, boolean closeDirSrc) throws IOException {
-    for (String file : src.listAll()) {
-      src.copy(dest, file, file);
-    }
-    if (closeDirSrc) {
-      src.close();
-    }
-  }
-
-  /**
    * @throws AlreadyClosedException if this Directory is closed
    */
   protected final void ensureOpen() throws AlreadyClosedException {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/FSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/FSDirectory.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/FSDirectory.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/FSDirectory.java Mon Dec  6 00:47:16 2010
@@ -321,12 +321,6 @@ public abstract class FSDirectory extend
     staleFiles.add(io.name);
   }
 
-  @Deprecated
-  @Override
-  public void sync(String name) throws IOException {
-    sync(Collections.singleton(name));
-  }
-
   @Override
   public void sync(Collection<String> names) throws IOException {
     ensureOpen();
@@ -383,12 +377,6 @@ public abstract class FSDirectory extend
     isOpen = false;
   }
 
-  /** @deprecated Use {@link #getDirectory} instead. */
-  @Deprecated
-  public File getFile() {
-    return getDirectory();
-  }
-
   /** @return the underlying filesystem directory */
   public File getDirectory() {
     ensureOpen();

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/FileSwitchDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/FileSwitchDirectory.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/FileSwitchDirectory.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/FileSwitchDirectory.java Mon Dec  6 00:47:16 2010
@@ -135,12 +135,6 @@ public class FileSwitchDirectory extends
     return getDirectory(name).createOutput(name);
   }
 
-  @Deprecated
-  @Override
-  public void sync(String name) throws IOException {
-    sync(Collections.singleton(name));
-  }
-
   @Override
   public void sync(Collection<String> names) throws IOException {
     List<String> primaryNames = new ArrayList<String>();

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/NoLockFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/NoLockFactory.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/NoLockFactory.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/NoLockFactory.java Mon Dec  6 00:47:16 2010
@@ -33,14 +33,7 @@ public class NoLockFactory extends LockF
   private static NoLock singletonLock = new NoLock();
   private static NoLockFactory singleton = new NoLockFactory();
   
-  /**
-   * @deprecated This constructor was not intended to be public and should not be used.
-   *  It will be made private in Lucene 4.0
-   * @see #getNoLockFactory()
-   */
-  // make private in 4.0!
-  @Deprecated
-  public NoLockFactory() {}
+  private NoLockFactory() {}
 
   public static NoLockFactory getNoLockFactory() {
     return singleton;

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=1042501&r1=1042500&r2=1042501&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 Dec  6 00:47:16 2010
@@ -20,8 +20,12 @@ package org.apache.lucene.store;
 import java.io.IOException;
 import java.io.FileNotFoundException;
 import java.io.Serializable;
-import java.util.HashMap;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.lucene.util.ThreadInterruptedException;
@@ -35,8 +39,8 @@ public class RAMDirectory extends Direct
 
   private static final long serialVersionUID = 1l;
 
-  HashMap<String,RAMFile> fileMap = new HashMap<String,RAMFile>();
-  final AtomicLong sizeInBytes = new AtomicLong();
+  protected final Map<String,RAMFile> fileMap = new ConcurrentHashMap<String,RAMFile>();
+  protected final AtomicLong sizeInBytes = new AtomicLong();
   
   // *****
   // Lock acquisition sequence:  RAMDirectory, then RAMFile
@@ -78,25 +82,21 @@ public class RAMDirectory extends Direct
   }
 
   @Override
-  public synchronized final String[] listAll() {
+  public final String[] listAll() {
     ensureOpen();
+    // NOTE: fileMap.keySet().toArray(new String[0]) is broken in non Sun JDKs,
+    // and the code below is resilient to map changes during the array population.
     Set<String> fileNames = fileMap.keySet();
-    String[] result = new String[fileNames.size()];
-    int i = 0;
-    for(final String fileName: fileNames) 
-      result[i++] = fileName;
-    return result;
+    List<String> names = new ArrayList<String>(fileNames.size());
+    for (String name : fileNames) names.add(name);
+    return names.toArray(new String[names.size()]);
   }
 
   /** Returns true iff the named file exists in this directory. */
   @Override
   public final boolean fileExists(String name) {
     ensureOpen();
-    RAMFile file;
-    synchronized (this) {
-      file = fileMap.get(name);
-    }
-    return file != null;
+    return fileMap.containsKey(name);
   }
 
   /** Returns the time the named file was last modified.
@@ -105,12 +105,10 @@ public class RAMDirectory extends Direct
   @Override
   public final long fileModified(String name) throws IOException {
     ensureOpen();
-    RAMFile file;
-    synchronized (this) {
-      file = fileMap.get(name);
-    }
-    if (file==null)
+    RAMFile file = fileMap.get(name);
+    if (file == null) {
       throw new FileNotFoundException(name);
+    }
     return file.getLastModified();
   }
 
@@ -120,12 +118,10 @@ public class RAMDirectory extends Direct
   @Override
   public void touchFile(String name) throws IOException {
     ensureOpen();
-    RAMFile file;
-    synchronized (this) {
-      file = fileMap.get(name);
-    }
-    if (file==null)
+    RAMFile file = fileMap.get(name);
+    if (file == null) {
       throw new FileNotFoundException(name);
+    }
     
     long ts2, ts1 = System.currentTimeMillis();
     do {
@@ -146,19 +142,18 @@ public class RAMDirectory extends Direct
   @Override
   public final long fileLength(String name) throws IOException {
     ensureOpen();
-    RAMFile file;
-    synchronized (this) {
-      file = fileMap.get(name);
-    }
-    if (file==null)
+    RAMFile file = fileMap.get(name);
+    if (file == null) {
       throw new FileNotFoundException(name);
+    }
     return file.getLength();
   }
   
-  /** Return total size in bytes of all files in this
-   * directory.  This is currently quantized to
-   * RAMOutputStream.BUFFER_SIZE. */
-  public synchronized final long sizeInBytes() {
+  /**
+   * Return total size in bytes of all files in this directory. This is
+   * currently quantized to RAMOutputStream.BUFFER_SIZE.
+   */
+  public final long sizeInBytes() {
     ensureOpen();
     return sizeInBytes.get();
   }
@@ -167,43 +162,52 @@ public class RAMDirectory extends Direct
    * @throws IOException if the file does not exist
    */
   @Override
-  public synchronized void deleteFile(String name) throws IOException {
+  public void deleteFile(String name) throws IOException {
     ensureOpen();
-    RAMFile file = fileMap.get(name);
-    if (file!=null) {
-        fileMap.remove(name);
-        file.directory = null;
-        sizeInBytes.addAndGet(-file.sizeInBytes);
-    } else
+    RAMFile file = fileMap.remove(name);
+    if (file != null) {
+      file.directory = null;
+      sizeInBytes.addAndGet(-file.sizeInBytes);
+    } else {
       throw new FileNotFoundException(name);
+    }
   }
 
   /** Creates a new, empty file in the directory with the given name. Returns a stream writing this file. */
   @Override
   public IndexOutput createOutput(String name) throws IOException {
     ensureOpen();
-    RAMFile file = new RAMFile(this);
-    synchronized (this) {
-      RAMFile existing = fileMap.get(name);
-      if (existing!=null) {
-        sizeInBytes.addAndGet(-existing.sizeInBytes);
-        existing.directory = null;
-      }
-      fileMap.put(name, file);
+    RAMFile file = newRAMFile();
+    RAMFile existing = fileMap.remove(name);
+    if (existing != null) {
+      sizeInBytes.addAndGet(-existing.sizeInBytes);
+      existing.directory = null;
     }
+    fileMap.put(name, file);
     return new RAMOutputStream(file);
   }
 
+  /**
+   * Returns a new {@link RAMFile} for storing data. This method can be
+   * overridden to return different {@link RAMFile} impls, that e.g. override
+   * {@link RAMFile#newBuffer(int)}.
+   */
+  protected RAMFile newRAMFile() {
+    return new RAMFile(this);
+  }
+
+  @Override
+  public void sync(Collection<String> names) throws IOException {
+  }
+
   /** Returns a stream reading an existing file. */
   @Override
   public IndexInput openInput(String name) throws IOException {
     ensureOpen();
-    RAMFile file;
-    synchronized (this) {
-      file = fileMap.get(name);
-    }
-    if (file == null)
+    RAMFile file = fileMap.get(name);
+    if (file == null) {
       throw new FileNotFoundException(name);
+    }
     return new RAMInputStream(file);
   }
 
@@ -211,6 +215,6 @@ public class RAMDirectory extends Direct
   @Override
   public void close() {
     isOpen = false;
-    fileMap = null;
+    fileMap.clear();
   }
 }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/IndexableBinaryStringTools.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/IndexableBinaryStringTools.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/IndexableBinaryStringTools.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/IndexableBinaryStringTools.java Mon Dec  6 00:47:16 2010
@@ -37,19 +37,6 @@ import java.nio.ByteBuffer;
  * problem, a char is appended, indicating the number of encoded bytes in the
  * final content char.
  * <p/>
- * Some methods in this class are defined over CharBuffers and ByteBuffers, but
- * these are deprecated in favor of methods that operate directly on byte[] and
- * char[] arrays.  Note that this class calls array() and arrayOffset()
- * on the CharBuffers and ByteBuffers it uses, so only wrapped arrays may be
- * used.  This class interprets the arrayOffset() and limit() values returned 
- * by its input buffers as beginning and end+1 positions on the wrapped array,
- * respectively; similarly, on the output buffer, arrayOffset() is the first
- * position written to, and limit() is set to one past the final output array
- * position.
- * <p/>
- * WARNING: This means that the deprecated Buffer-based methods 
- * only work correctly with buffers that have an offset of 0. For example, they
- * will not correctly interpret buffers returned by {@link ByteBuffer#slice}.  
  *
  * @lucene.experimental
  */
@@ -72,28 +59,6 @@ public final class IndexableBinaryString
   private IndexableBinaryStringTools() {}
 
   /**
-   * Returns the number of chars required to encode the given byte sequence.
-   * 
-   * @param original The byte sequence to be encoded. Must be backed by an
-   *        array.
-   * @return The number of chars required to encode the given byte sequence
-   * @throws IllegalArgumentException If the given ByteBuffer is not backed by
-   *         an array
-   * @deprecated Use {@link #getEncodedLength(byte[], int, int)} instead. This
-   *             method will be removed in Lucene 4.0
-   */
-  @Deprecated
-  public static int getEncodedLength(ByteBuffer original)
-    throws IllegalArgumentException {
-    if (original.hasArray()) {
-      return getEncodedLength(original.array(), original.arrayOffset(),
-          original.limit() - original.arrayOffset());
-    } else {
-      throw new IllegalArgumentException("original argument must have a backing array");
-    }
-  }
-  
-  /**
    * Returns the number of chars required to encode the given bytes.
    * 
    * @param inputArray byte sequence to be encoded
@@ -107,28 +72,6 @@ public final class IndexableBinaryString
     return (int)((8L * inputLength + 14L) / 15L) + 1;
   }
 
-
-  /**
-   * Returns the number of bytes required to decode the given char sequence.
-   * 
-   * @param encoded The char sequence to be decoded. Must be backed by an array.
-   * @return The number of bytes required to decode the given char sequence
-   * @throws IllegalArgumentException If the given CharBuffer is not backed by
-   *         an array
-   * @deprecated Use {@link #getDecodedLength(char[], int, int)} instead. This
-   *             method will be removed in Lucene 4.0
-   */
-  @Deprecated
-  public static int getDecodedLength(CharBuffer encoded) 
-    throws IllegalArgumentException {
-    if (encoded.hasArray()) {
-      return getDecodedLength(encoded.array(), encoded.arrayOffset(), 
-          encoded.limit() - encoded.arrayOffset());
-    } else {
-      throw new IllegalArgumentException("encoded argument must have a backing array");
-    }
-  }
-  
   /**
    * Returns the number of bytes required to decode the given char sequence.
    * 
@@ -150,36 +93,6 @@ public final class IndexableBinaryString
   }
 
   /**
-   * Encodes the input byte sequence into the output char sequence. Before
-   * calling this method, ensure that the output CharBuffer has sufficient
-   * capacity by calling {@link #getEncodedLength(java.nio.ByteBuffer)}.
-   * 
-   * @param input The byte sequence to encode
-   * @param output Where the char sequence encoding result will go. The limit is
-   *        set to one past the position of the final char.
-   * @throws IllegalArgumentException If either the input or the output buffer
-   *         is not backed by an array
-   * @deprecated Use {@link #encode(byte[], int, int, char[], int, int)}
-   *             instead. This method will be removed in Lucene 4.0
-   */
-  @Deprecated
-  public static void encode(ByteBuffer input, CharBuffer output) {
-    if (input.hasArray() && output.hasArray()) {
-      final int inputOffset = input.arrayOffset();
-      final int inputLength = input.limit() - inputOffset;
-      final int outputOffset = output.arrayOffset();
-      final int outputLength = getEncodedLength(input.array(), inputOffset,
-          inputLength);
-      output.limit(outputLength + outputOffset);
-      output.position(0);
-      encode(input.array(), inputOffset, inputLength, output.array(),
-          outputOffset, outputLength);
-    } else {
-      throw new IllegalArgumentException("Arguments must have backing arrays");
-    }
-  }
-  
-  /**
    * Encodes the input byte sequence into the output char sequence.  Before
    * calling this method, ensure that the output array has sufficient
    * capacity by calling {@link #getEncodedLength(byte[], int, int)}.
@@ -235,36 +148,6 @@ public final class IndexableBinaryString
 
   /**
    * Decodes the input char sequence into the output byte sequence. Before
-   * calling this method, ensure that the output ByteBuffer has sufficient
-   * capacity by calling {@link #getDecodedLength(java.nio.CharBuffer)}.
-   * 
-   * @param input The char sequence to decode
-   * @param output Where the byte sequence decoding result will go. The limit is
-   *        set to one past the position of the final char.
-   * @throws IllegalArgumentException If either the input or the output buffer
-   *         is not backed by an array
-   * @deprecated Use {@link #decode(char[], int, int, byte[], int, int)}
-   *             instead. This method will be removed in Lucene 4.0
-   */
-  @Deprecated
-  public static void decode(CharBuffer input, ByteBuffer output) {
-    if (input.hasArray() && output.hasArray()) {
-      final int inputOffset = input.arrayOffset();
-      final int inputLength = input.limit() - inputOffset;
-      final int outputOffset = output.arrayOffset();
-      final int outputLength = getDecodedLength(input.array(), inputOffset,
-          inputLength);
-      output.limit(outputLength + outputOffset);
-      output.position(0);
-      decode(input.array(), inputOffset, inputLength, output.array(),
-          outputOffset, outputLength);
-    } else {
-      throw new IllegalArgumentException("Arguments must have backing arrays");
-    }
-  }
-
-  /**
-   * Decodes the input char sequence into the output byte sequence. Before
    * calling this method, ensure that the output array has sufficient capacity
    * by calling {@link #getDecodedLength(char[], int, int)}.
    * 
@@ -330,46 +213,6 @@ public final class IndexableBinaryString
     }
   }
 
-  /**
-   * Decodes the given char sequence, which must have been encoded by
-   * {@link #encode(java.nio.ByteBuffer)} or
-   * {@link #encode(java.nio.ByteBuffer, java.nio.CharBuffer)}.
-   * 
-   * @param input The char sequence to decode
-   * @return A byte sequence containing the decoding result. The limit is set to
-   *         one past the position of the final char.
-   * @throws IllegalArgumentException If the input buffer is not backed by an
-   *         array
-   * @deprecated Use {@link #decode(char[], int, int, byte[], int, int)}
-   *             instead. This method will be removed in Lucene 4.0
-   */
-  @Deprecated
-  public static ByteBuffer decode(CharBuffer input) {
-    byte[] outputArray = new byte[getDecodedLength(input)];
-    ByteBuffer output = ByteBuffer.wrap(outputArray);
-    decode(input, output);
-    return output;
-  }
-
-  /**
-   * Encodes the input byte sequence.
-   * 
-   * @param input The byte sequence to encode
-   * @return A char sequence containing the encoding result. The limit is set to
-   *         one past the position of the final char.
-   * @throws IllegalArgumentException If the input buffer is not backed by an
-   *         array
-   * @deprecated Use {@link #encode(byte[], int, int, char[], int, int)}
-   *             instead. This method will be removed in Lucene 4.0
-   */
-  @Deprecated
-  public static CharBuffer encode(ByteBuffer input) {
-    char[] outputArray = new char[getEncodedLength(input)];
-    CharBuffer output = CharBuffer.wrap(outputArray);
-    encode(input, output);
-    return output;
-  }
-  
   static class CodingCase {
     int numBytes, initialShift, middleShift, finalShift, advanceBytes = 2;
     short middleMask, finalMask;

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/NumericUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/NumericUtils.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/NumericUtils.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/NumericUtils.java Mon Dec  6 00:47:16 2010
@@ -17,10 +17,10 @@ package org.apache.lucene.util;
  * limitations under the License.
  */
 
-import org.apache.lucene.analysis.NumericTokenStream; // for javadocs
-import org.apache.lucene.document.NumericField; // for javadocs
+import org.apache.lucene.analysis.NumericTokenStream;
+import org.apache.lucene.document.NumericField;
+import org.apache.lucene.search.NumericRangeFilter;
 import org.apache.lucene.search.NumericRangeQuery; // for javadocs
-import org.apache.lucene.search.NumericRangeFilter; // for javadocs
 
 // TODO: Remove the commented out methods before release!
 
@@ -130,32 +130,6 @@ public final class NumericUtils {
     return hash;
   }
 
-  /*
-   * Returns prefix coded bits after reducing the precision by <code>shift</code> bits.
-   * This is method is used by {@link LongRangeBuilder}.
-   * @param val the numeric value
-   * @param shift how many bits to strip from the right
-   * @deprecated This method is no longer needed!
-   *
-  @Deprecated
-  public static String longToPrefixCoded(final long val, final int shift) {
-    final BytesRef buffer = new BytesRef(BUF_SIZE_LONG);
-    longToPrefixCoded(val, shift, buffer);
-    return buffer.utf8ToString();
-  }*/
-
-  /*
-   * This is a convenience method, that returns prefix coded bits of a long without
-   * reducing the precision. It can be used to store the full precision value as a
-   * stored field in index.
-   * <p>To decode, use {@link #prefixCodedToLong}.
-   * @deprecated This method is no longer needed!
-   *
-  @Deprecated
-  public static String longToPrefixCoded(final long val) {
-    return longToPrefixCoded(val, 0);
-  }*/
-  
   /**
    * Returns prefix coded bits after reducing the precision by <code>shift</code> bits.
    * This is method is used by {@link NumericTokenStream}.
@@ -190,46 +164,6 @@ public final class NumericUtils {
     return hash;
   }
 
-  /*
-   * Returns prefix coded bits after reducing the precision by <code>shift</code> bits.
-   * This is method is used by {@link IntRangeBuilder}.
-   * @param val the numeric value
-   * @param shift how many bits to strip from the right
-   * @deprecated This method is no longer needed!
-   *
-  @Deprecated
-  public static String intToPrefixCoded(final int val, final int shift) {
-    final BytesRef buffer = new BytesRef(BUF_SIZE_INT);
-    intToPrefixCoded(val, shift, buffer);
-    return buffer.utf8ToString();
-  }*/
-
-  /*
-   * This is a convenience method, that returns prefix coded bits of an int without
-   * reducing the precision. It can be used to store the full precision value as a
-   * stored field in index.
-   * <p>To decode, use {@link #prefixCodedToInt}.
-   * @deprecated This method is no longer needed!
-   *
-  @Deprecated
-  public static String intToPrefixCoded(final int val) {
-    return intToPrefixCoded(val, 0);
-  }*/
-
-  /*
-   * Returns a long from prefixCoded characters.
-   * Rightmost bits will be zero for lower precision codes.
-   * This method can be used to decode e.g. a stored field.
-   * @throws NumberFormatException if the supplied string is
-   * not correctly prefix encoded.
-   * @see #longToPrefixCoded(long)
-   * @deprecated This method is no longer needed!
-   *
-  @Deprecated
-  public static long prefixCodedToLong(final String prefixCoded) {
-    return prefixCodedToLong(new BytesRef(prefixCoded));
-  }*/
-
   /**
    * Returns the shift value from a prefix encoded {@code long}.
    * @throws NumberFormatException if the supplied {@link BytesRef} is
@@ -278,21 +212,7 @@ public final class NumericUtils {
     return (sortableBits << getPrefixCodedLongShift(val)) ^ 0x8000000000000000L;
   }
 
-  /*
-   * Returns an int from prefixCoded characters.
-   * Rightmost bits will be zero for lower precision codes.
-   * This method can be used to decode a term's value.
-   * @throws NumberFormatException if the supplied string is
-   * not correctly prefix encoded.
-   * @see #intToPrefixCoded(int)
-   * @deprecated This method is no longer needed!
-   *
-  @Deprecated
-  public static int prefixCodedToInt(final String prefixCoded) {
-    return prefixCodedToInt(new BytesRef(prefixCoded));
-  }*/
-
-  /*
+  /**
    * Returns an int from prefixCoded bytes.
    * Rightmost bits will be zero for lower precision codes.
    * This method can be used to decode a term's value.
@@ -329,16 +249,6 @@ public final class NumericUtils {
     return f;
   }
 
-  /*
-   * Convenience method: this just returns:
-   *   longToPrefixCoded(doubleToSortableLong(val))
-   * @deprecated This method is no longer needed!
-   *
-  @Deprecated
-  public static String doubleToPrefixCoded(double val) {
-    return longToPrefixCoded(doubleToSortableLong(val));
-  }*/
-
   /**
    * Converts a sortable <code>long</code> back to a <code>double</code>.
    * @see #doubleToSortableLong
@@ -348,16 +258,6 @@ public final class NumericUtils {
     return Double.longBitsToDouble(val);
   }
 
-  /*
-   * Convenience method: this just returns:
-   *    sortableLongToDouble(prefixCodedToLong(val))
-   * @deprecated This method is no longer needed!
-   *
-  @Deprecated
-  public static double prefixCodedToDouble(String val) {
-    return sortableLongToDouble(prefixCodedToLong(val));
-  }*/
-
   /**
    * Converts a <code>float</code> value to a sortable signed <code>int</code>.
    * The value is converted by getting their IEEE 754 floating-point &quot;float format&quot;
@@ -371,16 +271,6 @@ public final class NumericUtils {
     return f;
   }
 
-  /*
-   * Convenience method: this just returns:
-   *   intToPrefixCoded(floatToSortableInt(val))
-   * @deprecated This method is no longer needed!
-   *
-  @Deprecated
-  public static String floatToPrefixCoded(float val) {
-    return intToPrefixCoded(floatToSortableInt(val));
-  }*/
-
   /**
    * Converts a sortable <code>int</code> back to a <code>float</code>.
    * @see #floatToSortableInt
@@ -390,16 +280,6 @@ public final class NumericUtils {
     return Float.intBitsToFloat(val);
   }
 
-  /*
-   * Convenience method: this just returns:
-   *    sortableIntToFloat(prefixCodedToInt(val))
-   * @deprecated This method is no longer needed!
-   *
-  @Deprecated
-  public static float prefixCodedToFloat(String val) {
-    return sortableIntToFloat(prefixCodedToInt(val));
-  }*/
-
   /**
    * Splits a long range recursively.
    * You may implement a builder that adds clauses to a

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/PagedBytes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/PagedBytes.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/PagedBytes.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/PagedBytes.java Mon Dec  6 00:47:16 2010
@@ -29,7 +29,8 @@ import java.io.IOException;
  *  using copy, and then retrieve slices (BytesRef) into it
  *  using fill.
  *
- * <p>@lucene.internal</p>*/
+ * @lucene.internal
+ **/
 public final class PagedBytes {
   private final List<byte[]> blocks = new ArrayList<byte[]>();
   private final List<Integer> blockEnd = new ArrayList<Integer>();
@@ -63,8 +64,16 @@ public final class PagedBytes {
       blockSize = pagedBytes.blockSize;
     }
 
-    /** Get a slice out of the byte array. */
-    public BytesRef fill(BytesRef b, long start, int length) {
+    /**
+     * Gets a slice out of {@link PagedBytes} starting at <i>start</i> with a
+     * given length. Iff the slice spans across a block border this method will
+     * allocate sufficient resources and copy the paged data.
+     * <p>
+     * Slices spanning more than one block are not supported.
+     * </p>
+     * @lucene.internal 
+     **/
+    public BytesRef fillSlice(BytesRef b, long start, int length) {
       assert length >= 0: "length=" + length;
       final int index = (int) (start >> blockBits);
       final int offset = (int) (start & blockMask);
@@ -91,8 +100,18 @@ public final class PagedBytes {
       return b;
     }
     
-    /** Reads length as 1 or 2 byte vInt prefix, starting @ start */
-    public BytesRef fillUsingLengthPrefix(BytesRef b, long start) {
+    /**
+     * Reads length as 1 or 2 byte vInt prefix, starting at <i>start</i>.
+     * <p>
+     * <b>Note:</b> this method does not support slices spanning across block
+     * borders.
+     * </p>
+     * 
+     * @return the given {@link BytesRef}
+     * 
+     * @lucene.internal
+     **/
+    public BytesRef fill(BytesRef b, long start) {
       final int index = (int) (start >> blockBits);
       final int offset = (int) (start & blockMask);
       final byte[] block = b.bytes = blocks[index];
@@ -108,8 +127,17 @@ public final class PagedBytes {
       return b;
     }
 
-    /** @lucene.internal  Reads length as 1 or 2 byte vInt prefix, starting @ start.  Returns the block number of the term. */
-    public int fillUsingLengthPrefix2(BytesRef b, long start) {
+    /**
+     * Reads length as 1 or 2 byte vInt prefix, starting at <i>start</i>. *
+     * <p>
+     * <b>Note:</b> this method does not support slices spanning across block
+     * borders.
+     * </p>
+     * 
+     * @return the internal block number of the slice.
+     * @lucene.internal
+     **/
+    public int fillAndGetIndex(BytesRef b, long start) {
       final int index = (int) (start >> blockBits);
       final int offset = (int) (start & blockMask);
       final byte[] block = b.bytes = blocks[index];
@@ -125,10 +153,21 @@ public final class PagedBytes {
       return index;
     }
 
-    /** @lucene.internal  Reads length as 1 or 2 byte vInt prefix, starting @ start. 
-     * Returns the start offset of the next part, suitable as start parameter on next call
-     * to sequentially read all BytesRefs. */
-    public long fillUsingLengthPrefix3(BytesRef b, long start) {
+    /**
+     * Reads length as 1 or 2 byte vInt prefix, starting at <i>start</i> and
+     * returns the start offset of the next part, suitable as start parameter on
+     * next call to sequentially read all {@link BytesRef}.
+     * 
+     * <p>
+     * <b>Note:</b> this method does not support slices spanning across block
+     * borders.
+     * </p>
+     * 
+     * @return the start offset of the next part, suitable as start parameter on
+     *         next call to sequentially read all {@link BytesRef}.
+     * @lucene.internal
+     **/
+    public long fillAndGetStart(BytesRef b, long start) {
       final int index = (int) (start >> blockBits);
       final int offset = (int) (start & blockMask);
       final byte[] block = b.bytes = blocks[index];

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/StringHelper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/StringHelper.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/StringHelper.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/StringHelper.java Mon Dec  6 00:47:16 2010
@@ -44,7 +44,7 @@ public abstract class StringHelper {
    * @param bytes2 The second byte[] to compare
    * @return The number of common elements.
    */
-  public static final int bytesDifference(byte[] bytes1, int len1, byte[] bytes2, int len2) {
+  public static int bytesDifference(byte[] bytes1, int len1, byte[] bytes2, int len2) {
     int len = len1 < len2 ? len1 : len2;
     for (int i = 0; i < len; i++)
       if (bytes1[i] != bytes2[i])
@@ -52,29 +52,6 @@ public abstract class StringHelper {
     return len;
   }
 
-  /**
-   * Compares two strings, character by character, and returns the
-   * first position where the two strings differ from one another.
-   *
-   * @param s1 The first string to compare
-   * @param s2 The second string to compare
-   * @return The first position where the two strings differ.
-   * 
-   * @deprecated This method cannot handle supplementary characters.
-   */
-  @Deprecated
-  public static final int stringDifference(String s1, String s2) {
-    int len1 = s1.length();
-    int len2 = s2.length();
-    int len = len1 < len2 ? len1 : len2;
-    for (int i = 0; i < len; i++) {
-      if (s1.charAt(i) != s2.charAt(i)) {
-	      return i;
-      }
-    }
-    return len;
-  }
-
   private StringHelper() {
   }
 }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/Version.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/Version.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/Version.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/Version.java Mon Dec  6 00:47:16 2010
@@ -28,29 +28,18 @@ package org.apache.lucene.util;
  * your indexing code to match, and re-index.
  */
 public enum Version {
-
-  /** Match settings and bugs in Lucene's 2.0 release. */
-  LUCENE_20,
-
-  /** Match settings and bugs in Lucene's 2.1 release. */
-  LUCENE_21,
-
-  /** Match settings and bugs in Lucene's 2.2 release. */
-  LUCENE_22,
-
-  /** Match settings and bugs in Lucene's 2.3 release. */
-  LUCENE_23,
-
-  /** Match settings and bugs in Lucene's 2.4 release. */
-  LUCENE_24,
-
-  /** Match settings and bugs in Lucene's 2.9 release. */
-  LUCENE_29,
-
-  /** Match settings and bugs in Lucene's 3.0 release. */
+  /**
+   * Match settings and bugs in Lucene's 3.0 release.
+   * @deprecated (4.0) Use latest
+   */
+  @Deprecated
   LUCENE_30,
 
-  /** Match settings and bugs in Lucene's 3.1 release. */
+  /**
+   * Match settings and bugs in Lucene's 3.1 release.
+   * @deprecated (4.0) Use latest
+   */
+  @Deprecated
   LUCENE_31,
 
   /** Match settings and bugs in Lucene's 4.0 release. 

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=1042501&r1=1042500&r2=1042501&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 Dec  6 00:47:16 2010
@@ -487,7 +487,7 @@ public class Automaton implements Serial
    * Returns a sorted array of transitions for each state (and sets state
    * numbers).
    */
-  Transition[][] getSortedTransitions() {
+  public Transition[][] getSortedTransitions() {
     final State[] states = getNumberedStates();
     Transition[][] transitions = new Transition[states.length][];
     for (State s : states) {

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=1042501&r1=1042500&r2=1042501&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 Dec  6 00:47:16 2010
@@ -131,8 +131,8 @@ public abstract class RunAutomaton imple
     this.maxInterval = maxInterval;
     a.determinize();
     points = a.getStartPoints();
-    initial = a.initial.number;
     final State[] states = a.getNumberedStates();
+    initial = a.initial.number;
     size = states.length;
     accept = new boolean[size];
     transitions = new int[size * points.length];

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/TestExternalCodecs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/TestExternalCodecs.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/TestExternalCodecs.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/TestExternalCodecs.java Mon Dec  6 00:47:16 2010
@@ -634,10 +634,12 @@ public class TestExternalCodecs extends 
     
     final int NUM_DOCS = 173;
     Directory dir = newDirectory();
-    IndexWriter w = new IndexWriter(dir,
-                                    newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(MockTokenizer.WHITESPACE, true, true)).setCodecProvider(provider));
-
-    w.setMergeFactor(3);
+    IndexWriter w = new IndexWriter(
+        dir,
+        newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(MockTokenizer.WHITESPACE, true, true)).
+            setCodecProvider(provider).
+            setMergePolicy(newLogMergePolicy(3))
+    );
     Document doc = new Document();
     // uses default codec:
     doc.add(newField("field1", "this field uses the standard codec as the test", Field.Store.NO, Field.Index.ANALYZED));

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/TestSearch.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/TestSearch.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/TestSearch.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/TestSearch.java Mon Dec  6 00:47:16 2010
@@ -114,29 +114,15 @@ public class TestSearch extends LuceneTe
         Query query = parser.parse(queries[j]);
         out.println("Query: " + query.toString("contents"));
 
-      //DateFilter filter =
-      //  new DateFilter("modified", Time(1997,0,1), Time(1998,0,1));
-      //DateFilter filter = DateFilter.Before("modified", Time(1997,00,01));
-      //System.out.println(filter);
-
         hits = searcher.search(query, null, 1000).scoreDocs;
 
         out.println(hits.length + " total results");
         for (int i = 0 ; i < hits.length && i < 10; i++) {
           Document d = searcher.doc(hits[i].doc);
-          out.println(i + " " + hits[i].score
-// 			   + " " + DateField.stringToDate(d.get("modified"))
-                             + " " + d.get("contents"));
+          out.println(i + " " + hits[i].score + " " + d.get("contents"));
         }
       }
       searcher.close();
       directory.close();
   }
-
-  static long Time(int year, int month, int day) {
-    GregorianCalendar calendar = new GregorianCalendar();
-    calendar.clear();
-    calendar.set(year, month, day);
-    return calendar.getTime().getTime();
-  }
 }

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/analysis/TestCharTokenizers.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/analysis/TestCharTokenizers.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/analysis/TestCharTokenizers.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/analysis/TestCharTokenizers.java Mon Dec  6 00:47:16 2010
@@ -18,10 +18,8 @@ package org.apache.lucene.analysis;
  */
 
 import java.io.IOException;
-import java.io.Reader;
 import java.io.StringReader;
 
-import org.apache.lucene.util.Version;
 
 /**
  * Testcase for {@link CharTokenizer} subclasses
@@ -92,96 +90,4 @@ public class TestCharTokenizers extends 
     MockTokenizer tokenizer = new MockTokenizer(new StringReader(builder.toString() + builder.toString()), MockTokenizer.SIMPLE, true);
     assertTokenStreamContents(tokenizer, new String[] {builder.toString().toLowerCase(), builder.toString().toLowerCase()});
   }
-
-  public void testIsTokenCharCharInSubclass() {
-    new TestingCharTokenizer(Version.LUCENE_30, new StringReader(""));
-    try {
-      new TestingCharTokenizer(TEST_VERSION_CURRENT, new StringReader(""));
-      fail("version 3.1 is not permitted if char based method is implemented");
-    } catch (IllegalArgumentException e) {
-      // expected
-    }
-  }
-
-  public void testNormalizeCharInSubclass() {
-    new TestingCharTokenizerNormalize(Version.LUCENE_30, new StringReader(""));
-    try {
-      new TestingCharTokenizerNormalize(TEST_VERSION_CURRENT,
-          new StringReader(""));
-      fail("version 3.1 is not permitted if char based method is implemented");
-    } catch (IllegalArgumentException e) {
-      // expected
-    }
-  }
-
-  public void testNormalizeAndIsTokenCharCharInSubclass() {
-    new TestingCharTokenizerNormalizeIsTokenChar(Version.LUCENE_30,
-        new StringReader(""));
-    try {
-      new TestingCharTokenizerNormalizeIsTokenChar(TEST_VERSION_CURRENT,
-          new StringReader(""));
-      fail("version 3.1 is not permitted if char based method is implemented");
-    } catch (IllegalArgumentException e) {
-      // expected
-    }
-  }
-
-  static final class TestingCharTokenizer extends CharTokenizer {
-    public TestingCharTokenizer(Version matchVersion, Reader input) {
-      super(matchVersion, input);
-    }
-
-    @Override
-    protected boolean isTokenChar(int c) {
-      return Character.isLetter(c);
-    }
-
-    @Deprecated @Override
-    protected boolean isTokenChar(char c) {
-      return Character.isLetter(c);
-    }
-  }
-
-  static final class TestingCharTokenizerNormalize extends CharTokenizer {
-    public TestingCharTokenizerNormalize(Version matchVersion, Reader input) {
-      super(matchVersion, input);
-    }
-
-    @Deprecated @Override
-    protected char normalize(char c) {
-      return c;
-    }
-
-    @Override
-    protected int normalize(int c) {
-      return c;
-    }
-  }
-
-  static final class TestingCharTokenizerNormalizeIsTokenChar extends CharTokenizer {
-    public TestingCharTokenizerNormalizeIsTokenChar(Version matchVersion,
-        Reader input) {
-      super(matchVersion, input);
-    }
-
-    @Deprecated @Override
-    protected char normalize(char c) {
-      return c;
-    }
-
-    @Override
-    protected int normalize(int c) {
-      return c;
-    }
-
-    @Override
-    protected boolean isTokenChar(int c) {
-      return Character.isLetter(c);
-    }
-
-    @Deprecated @Override
-    protected boolean isTokenChar(char c) {
-      return Character.isLetter(c);
-    }
-  }
 }

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/analysis/TestToken.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/analysis/TestToken.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/analysis/TestToken.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/analysis/TestToken.java Mon Dec  6 00:47:16 2010
@@ -176,20 +176,20 @@ public class TestToken extends LuceneTes
     char[] content = "hello".toCharArray();
     t.copyBuffer(content, 0, 5);
     char[] buf = t.buffer();
-    Token copy = (Token) TestSimpleAttributeImpls.assertCloneIsEqual(t);
+    Token copy = assertCloneIsEqual(t);
     assertEquals(t.toString(), copy.toString());
     assertNotSame(buf, copy.buffer());
 
     Payload pl = new Payload(new byte[]{1,2,3,4});
     t.setPayload(pl);
-    copy = (Token) TestSimpleAttributeImpls.assertCloneIsEqual(t);
+    copy = assertCloneIsEqual(t);
     assertEquals(pl, copy.getPayload());
     assertNotSame(pl, copy.getPayload());
   }
   
   public void testCopyTo() throws Exception {
     Token t = new Token();
-    Token copy = (Token) TestSimpleAttributeImpls.assertCopyIsEqual(t);
+    Token copy = assertCopyIsEqual(t);
     assertEquals("", t.toString());
     assertEquals("", copy.toString());
 
@@ -197,13 +197,13 @@ public class TestToken extends LuceneTes
     char[] content = "hello".toCharArray();
     t.copyBuffer(content, 0, 5);
     char[] buf = t.buffer();
-    copy = (Token) TestSimpleAttributeImpls.assertCopyIsEqual(t);
+    copy = assertCopyIsEqual(t);
     assertEquals(t.toString(), copy.toString());
     assertNotSame(buf, copy.buffer());
 
     Payload pl = new Payload(new byte[]{1,2,3,4});
     t.setPayload(pl);
-    copy = (Token) TestSimpleAttributeImpls.assertCopyIsEqual(t);
+    copy = assertCopyIsEqual(t);
     assertEquals(pl, copy.getPayload());
     assertNotSame(pl, copy.getPayload());
   }
@@ -240,4 +240,21 @@ public class TestToken extends LuceneTes
     assertTrue("TypeAttribute is not implemented by Token",
       ts.addAttribute(TypeAttribute.class) instanceof Token);
   }
+
+  public static <T extends AttributeImpl> T assertCloneIsEqual(T att) {
+    @SuppressWarnings("unchecked")
+    T clone = (T) att.clone();
+    assertEquals("Clone must be equal", att, clone);
+    assertEquals("Clone's hashcode must be equal", att.hashCode(), clone.hashCode());
+    return clone;
+  }
+
+  public static <T extends AttributeImpl> T assertCopyIsEqual(T att) throws Exception {
+    @SuppressWarnings("unchecked")
+    T copy = (T) att.getClass().newInstance();
+    att.copyTo(copy);
+    assertEquals("Copied instance must be equal", att, copy);
+    assertEquals("Copied instance's hashcode must be equal", att.hashCode(), copy.hashCode());
+    return copy;
+  }
 }

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/analysis/tokenattributes/TestCharTermAttributeImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/analysis/tokenattributes/TestCharTermAttributeImpl.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/analysis/tokenattributes/TestCharTermAttributeImpl.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/analysis/tokenattributes/TestCharTermAttributeImpl.java Mon Dec  6 00:47:16 2010
@@ -17,6 +17,7 @@ package org.apache.lucene.analysis.token
  * limitations under the License.
  */
 
+import org.apache.lucene.analysis.TestToken;
 import org.apache.lucene.util.LuceneTestCase;
 import java.nio.CharBuffer;
 import java.util.Formatter;
@@ -91,7 +92,7 @@ public class TestCharTermAttributeImpl e
     char[] content = "hello".toCharArray();
     t.copyBuffer(content, 0, 5);
     char[] buf = t.buffer();
-    CharTermAttributeImpl copy = (CharTermAttributeImpl) TestSimpleAttributeImpls.assertCloneIsEqual(t);
+    CharTermAttributeImpl copy = TestToken.assertCloneIsEqual(t);
     assertEquals(t.toString(), copy.toString());
     assertNotSame(buf, copy.buffer());
   }
@@ -113,7 +114,7 @@ public class TestCharTermAttributeImpl e
   
   public void testCopyTo() throws Exception {
     CharTermAttributeImpl t = new CharTermAttributeImpl();
-    CharTermAttributeImpl copy = (CharTermAttributeImpl) TestSimpleAttributeImpls.assertCopyIsEqual(t);
+    CharTermAttributeImpl copy = TestToken.assertCopyIsEqual(t);
     assertEquals("", t.toString());
     assertEquals("", copy.toString());
 
@@ -121,7 +122,7 @@ public class TestCharTermAttributeImpl e
     char[] content = "hello".toCharArray();
     t.copyBuffer(content, 0, 5);
     char[] buf = t.buffer();
-    copy = (CharTermAttributeImpl) TestSimpleAttributeImpls.assertCopyIsEqual(t);
+    copy = TestToken.assertCopyIsEqual(t);
     assertEquals(t.toString(), copy.toString());
     assertNotSame(buf, copy.buffer());
   }

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/document/TestBinaryDocument.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/document/TestBinaryDocument.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/document/TestBinaryDocument.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/document/TestBinaryDocument.java Mon Dec  6 00:47:16 2010
@@ -37,14 +37,6 @@ public class TestBinaryDocument extends 
     Fieldable binaryFldStored = new Field("binaryStored", binaryValStored.getBytes());
     Fieldable stringFldStored = new Field("stringStored", binaryValStored, Field.Store.YES, Field.Index.NO, Field.TermVector.NO);
 
-    try {
-      // binary fields with store off are not allowed
-      new Field("fail", binaryValStored.getBytes(), Field.Store.NO);
-      fail();
-    }
-    catch (IllegalArgumentException iae) {
-    }
-    
     Document doc = new Document();
     
     doc.add(binaryFldStored);

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/RandomIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/RandomIndexWriter.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/RandomIndexWriter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/RandomIndexWriter.java Mon Dec  6 00:47:16 2010
@@ -24,7 +24,6 @@ import java.util.Random;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.Version;
@@ -51,7 +50,10 @@ public class RandomIndexWriter implement
 
     public MockIndexWriter(Random r,Directory dir, IndexWriterConfig conf) throws IOException {
       super(dir, conf);
-      this.r = r;
+      // must make a private random since our methods are
+      // called from different threads; else test failures may
+      // not be reproducible from the original seed
+      this.r = new Random(r.nextInt());
     }
 
     @Override
@@ -84,7 +86,7 @@ public class RandomIndexWriter implement
     flushAt = _TestUtil.nextInt(r, 10, 1000);
     if (LuceneTestCase.VERBOSE) {
       System.out.println("RIW config=" + w.getConfig());
-      System.out.println("codec default=" + CodecProvider.getDefaultCodec());
+      System.out.println("codec default=" + w.getConfig().getCodecProvider().getDefaultFieldCodec());
     }
   } 
 

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/Test2BTerms.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/Test2BTerms.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/Test2BTerms.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/Test2BTerms.java Mon Dec  6 00:47:16 2010
@@ -123,7 +123,7 @@ public class Test2BTerms extends LuceneT
   @Ignore("Takes ~4 hours to run on a fast machine!!  And requires that you don't use PreFlex codec.")
   public void test2BTerms() throws IOException {
 
-    if ("PreFlex".equals(CodecProvider.getDefaultCodec())) {
+    if ("PreFlex".equals(CodecProvider.getDefault().getDefaultFieldCodec())) {
       throw new RuntimeException("thist test cannot run with PreFlex codec");
     }
 
@@ -132,13 +132,14 @@ public class Test2BTerms extends LuceneT
     int TERMS_PER_DOC = 1000000;
 
     Directory dir = FSDirectory.open(_TestUtil.getTempDir("2BTerms"));
-    IndexWriter w = new IndexWriter(dir,
-                                    new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer())
-                                                  .setMaxBufferedDocs(IndexWriterConfig.DISABLE_AUTO_FLUSH)
-                                                .setRAMBufferSizeMB(256.0).setMergeScheduler(new ConcurrentMergeScheduler()));
-    ((LogMergePolicy) w.getConfig().getMergePolicy()).setUseCompoundFile(false);
-    ((LogMergePolicy) w.getConfig().getMergePolicy()).setUseCompoundDocStore(false);
-    ((LogMergePolicy) w.getConfig().getMergePolicy()).setMergeFactor(10);
+    IndexWriter w = new IndexWriter(
+        dir,
+        new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer()).
+            setMaxBufferedDocs(IndexWriterConfig.DISABLE_AUTO_FLUSH).
+            setRAMBufferSizeMB(256.0).
+            setMergeScheduler(new ConcurrentMergeScheduler()).
+            setMergePolicy(newLogMergePolicy(false, 10))
+    );
 
     Document doc = new Document();
     Field field = new Field("field", new MyTokenStream(TERMS_PER_DOC));