You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@lucene.apache.org by tj...@apache.org on 2004/05/20 01:05:27 UTC

cvs commit: jakarta-lucene/src/test/org/apache/lucene/search SampleComparable.java TestSort.java

tjones      2004/05/19 16:05:27

  Modified:    src/java/org/apache/lucene/search
                        FieldDocSortedHitQueue.java
                        FieldSortedHitQueue.java IndexSearcher.java
                        ScoreDocComparator.java SortComparatorSource.java
                        SortField.java
               src/test/org/apache/lucene/search SampleComparable.java
                        TestSort.java
  Added:       src/java/org/apache/lucene/search FieldCache.java
                        FieldCacheImpl.java SortComparator.java
  Removed:     src/java/org/apache/lucene/search FloatSortedHitQueue.java
                        IntegerSortedHitQueue.java
                        MultiFieldSortedHitQueue.java
                        ScoreDocLookupComparator.java
                        StringSortedHitQueue.java
  Log:
  expose term cache as a public object
  
  Revision  Changes    Path
  1.4       +7 -3      jakarta-lucene/src/java/org/apache/lucene/search/FieldDocSortedHitQueue.java
  
  Index: FieldDocSortedHitQueue.java
  ===================================================================
  RCS file: /home/cvs/jakarta-lucene/src/java/org/apache/lucene/search/FieldDocSortedHitQueue.java,v
  retrieving revision 1.3
  retrieving revision 1.4
  diff -u -r1.3 -r1.4
  --- FieldDocSortedHitQueue.java	22 Apr 2004 22:23:14 -0000	1.3
  +++ FieldDocSortedHitQueue.java	19 May 2004 23:05:27 -0000	1.4
  @@ -101,7 +101,9 @@
   					case SortField.STRING:
   						String s1 = (String) docA.fields[i];
   						String s2 = (String) docB.fields[i];
  -						c = s2.compareTo(s1);
  +						if (s2 == null) c = -1;      // could be null if there are
  +						else if (s1 == null) c = 1;  // no terms in the given field
  +						else c = s2.compareTo(s1);
   						break;
   					case SortField.FLOAT:
   						float f1 = ((Float)docA.fields[i]).floatValue();
  @@ -139,7 +141,9 @@
   					case SortField.STRING:
   						String s1 = (String) docA.fields[i];
   						String s2 = (String) docB.fields[i];
  -						c = s1.compareTo(s2);
  +						if (s1 == null) c = -1;      // could be null if there are
  +						else if (s2 == null) c = 1;  // no terms in the given field
  +						else c = s1.compareTo(s2);
   						break;
   					case SortField.FLOAT:
   						float f1 = ((Float)docA.fields[i]).floatValue();
  
  
  
  1.9       +258 -287  jakarta-lucene/src/java/org/apache/lucene/search/FieldSortedHitQueue.java
  
  Index: FieldSortedHitQueue.java
  ===================================================================
  RCS file: /home/cvs/jakarta-lucene/src/java/org/apache/lucene/search/FieldSortedHitQueue.java,v
  retrieving revision 1.8
  retrieving revision 1.9
  diff -u -r1.8 -r1.9
  --- FieldSortedHitQueue.java	7 May 2004 15:18:09 -0000	1.8
  +++ FieldSortedHitQueue.java	19 May 2004 23:05:27 -0000	1.9
  @@ -17,311 +17,282 @@
    */
   
   import org.apache.lucene.index.IndexReader;
  -import org.apache.lucene.index.Term;
  -import org.apache.lucene.index.TermEnum;
   import org.apache.lucene.util.PriorityQueue;
   
   import java.io.IOException;
  -import java.util.Hashtable;
  +import java.util.WeakHashMap;
  +import java.util.Map;
   
   /**
  - * Expert: Base class for collecting results from a search and sorting
  - * them by terms in a given field in each document.
  - *
  - * <p>When one of these objects is created, a TermEnumerator is
  - * created to fetch all the terms in the index for the given field.
  - * The value of each term is assumed to represent a
  - * sort position.  Each document is assumed to contain one of the
  - * terms, indicating where in the sort it belongs.
  - *
  - * <p><h3>Memory Usage</h3>
  - *
  - * <p>A static cache is maintained.  This cache contains an integer
  - * or float array of length <code>IndexReader.maxDoc()</code> for each field
  - * name for which a sort is performed.  In other words, the size of the
  - * cache in bytes is:
  - *
  - * <p><code>4 * IndexReader.maxDoc() * (# of different fields actually used to sort)</code>
  - *
  - * <p>For String fields, the cache is larger: in addition to the
  - * above array, the value of every term in the field is kept in memory.
  - * If there are many unique terms in the field, this could 
  - * be quite large.
  - *
  - * <p>Note that the size of the cache is not affected by how many
  - * fields are in the index and <i>might</i> be used to sort - only by
  - * the ones actually used to sort a result set.
  - *
  - * <p>The cache is cleared each time a new <code>IndexReader</code> is
  - * passed in, or if the value returned by <code>maxDoc()</code>
  - * changes for the current IndexReader.  This class is not set up to
  - * be able to efficiently sort hits from more than one index
  - * simultaneously.
  + * Expert: A hit queue for sorting by hits by terms in more than one field.
  + * Uses <code>FieldCache.DEFAULT</code> for maintaining internal term lookup tables.
    *
    * <p>Created: Dec 8, 2003 12:56:03 PM
    *
    * @author  Tim Jones (Nacimiento Software)
    * @since   lucene 1.4
    * @version $Id$
  + * @see Searchable#search(Query,Filter,int,Sort)
  + * @see FieldCache
    */
  -abstract class FieldSortedHitQueue
  +class FieldSortedHitQueue
   extends PriorityQueue {
   
  -    /**
  -     * Keeps track of the IndexReader which the cache
  -     * applies to.  If it changes, the cache is cleared.
  -     * We only store the hashcode so as not to mess up
  -     * garbage collection by having a reference to an
  -     * IndexReader.
  -     */
  -    protected static int lastReaderHash;
  -
  -    /**
  -	 * Contains the cache of sort information, mapping
  -	 * String (field names) to ScoreDocComparator.
  -	 */
  -    protected static final Hashtable fieldCache = new Hashtable();
  -
  -	/** The pattern used to detect integer values in a field */
  -	/** removed for java 1.3 compatibility
  -		protected static final Pattern pIntegers = Pattern.compile ("[0-9\\-]+");
  -	**/
  -
  -	/** The pattern used to detect float values in a field */
  -	/** removed for java 1.3 compatibility
  -		protected static final Object pFloats = Pattern.compile ("[0-9+\\-\\.eEfFdD]+");
  -	**/
  -
  -
  -	/**
  -	 * Returns a comparator for the given field.  If there is already one in the cache, it is returned.
  -	 * Otherwise one is created and put into the cache.  If <code>reader</code> is different than the
  -	 * one used for the current cache, or has changed size, the cache is cleared first.
  -	 * @param reader  Index to use.
  -	 * @param field   Field to sort by.
  -	 * @return  Comparator; never <code>null</code>.
  -	 * @throws IOException  If an error occurs reading the index.
  -	 * @see #determineComparator
  -	 */
  -	static ScoreDocComparator getCachedComparator (final IndexReader reader, final String field, final int type, final SortComparatorSource factory)
  -	throws IOException {
  -
  -		if (type == SortField.DOC) return ScoreDocComparator.INDEXORDER;
  -		if (type == SortField.SCORE) return ScoreDocComparator.RELEVANCE;
  -
  -		// see if we have already generated a comparator for this field
  -		if (reader.hashCode() == lastReaderHash) {
  -			ScoreDocLookupComparator comparer = (ScoreDocLookupComparator) fieldCache.get (field);
  -			if (comparer != null && comparer.sizeMatches(reader.maxDoc())) {
  -				return comparer;
  -			}
  -		} else {
  -			lastReaderHash = reader.hashCode();
  -			fieldCache.clear();
  -		}
  -
  -		ScoreDocComparator comparer = null;
  -		switch (type) {
  -			case SortField.AUTO:   comparer = determineComparator (reader, field); break;
  -			case SortField.STRING: comparer = StringSortedHitQueue.comparator (reader, field); break;
  -			case SortField.INT:    comparer = IntegerSortedHitQueue.comparator (reader, field); break;
  -			case SortField.FLOAT:  comparer = FloatSortedHitQueue.comparator (reader, field); break;
  -			case SortField.CUSTOM: comparer = factory.newComparator (reader, field); break;
  -			default:
  -				throw new RuntimeException ("invalid sort field type: "+type);
  -		}
  -
  -		// store the comparator in the cache for reuse
  -		fieldCache.put (field, comparer);
  -
  -		return comparer;
  -	}
  -
  -
  -	/** Clears the static cache of sorting information. */
  -	static void clearCache() {
  -		fieldCache.clear();
  -	}
  -
  -
  -	/**
  -	 * Returns a FieldSortedHitQueue sorted by the given ScoreDocComparator.
  -	 * @param comparator Comparator to use.
  -	 * @param size       Number of hits to retain.
  -	 * @return  Hit queue sorted using the given comparator.
  -	 */
  -	static FieldSortedHitQueue getInstance (ScoreDocComparator comparator, int size) {
  -		return new FieldSortedHitQueue (comparator, size) {
  -			// dummy out the abstract method
  -			protected ScoreDocLookupComparator createComparator (IndexReader reader, String field) throws IOException {
  -				return null;
  -			}
  -		};
  -	}
  -
  -
  -	/**
  -	 * Looks at the actual values in the field and determines whether
  -	 * they contain Integers, Floats or Strings.  Only the first term in the field
  -	 * is looked at.
  -	 * <p>The following patterns are used to determine the content of the terms:
  -	 * <p><table border="1" cellspacing="0" cellpadding="3">
  -	 * <tr><th>Sequence</th><th>Pattern</th><th>Type</th></tr>
  -	 * <tr><td>1</td><td>[0-9\-]+</td><td>Integer</td></tr>
  -	 * <tr><td>2</td><td>[0-9+\-\.eEfFdD]+</td><td>Float</td></tr>
  -	 * <tr><td>3</td><td><i>(none - default)</i></td><td>String</td></tr>
  -	 * </table>
  -	 *
  -	 * @param reader  Index to use.
  -	 * @param field   Field to create comparator for.
  -	 * @return  Comparator appropriate for the terms in the given field.
  -	 * @throws IOException  If an error occurs reading the index.
  -	 */
  -	protected static ScoreDocComparator determineComparator (IndexReader reader, String field)
  -	throws IOException {
  -		field = field.intern();
  -		TermEnum enumerator = reader.terms (new Term (field, ""));
  -		try {
  -			Term term = enumerator.term();
  -			if (term == null) {
  -				throw new RuntimeException ("no terms in field "+field+" - cannot determine sort type");
  -			}
  -			if (term.field() == field) {
  -				String termtext = term.text().trim();
  -
  -				/**
  -				 * Java 1.4 level code:
  -
  -				if (pIntegers.matcher(termtext).matches())
  -					return IntegerSortedHitQueue.comparator (reader, enumerator, field);
  -
  -				else if (pFloats.matcher(termtext).matches())
  -					return FloatSortedHitQueue.comparator (reader, enumerator, field);
  -				 */
  -
  -				// Java 1.3 level code:
  -				try {
  -					Integer.parseInt (termtext);
  -					return IntegerSortedHitQueue.comparator (reader, enumerator, field);
  -				} catch (NumberFormatException nfe) {
  -					// nothing
  -				}
  -				try {
  -					Float.parseFloat (termtext);
  -					return FloatSortedHitQueue.comparator (reader, enumerator, field);
  -				} catch (NumberFormatException nfe) {
  -					// nothing
  -				}
  -				
  -				return StringSortedHitQueue.comparator (reader, enumerator, field);
  -
  -			} else {
  -				throw new RuntimeException ("field \""+field+"\" does not appear to be indexed");
  -			}
  -		} finally {
  -			enumerator.close();
  -		}
  -	}
  -
  -	/**
  -	 * The sorting priority used.  The first element is set by the constructors.
  -	 * The result is that sorting is done by field value, then by index order.
  -	 */
  -	private final ScoreDocComparator[] comparators = new ScoreDocComparator[] {
  -		null, ScoreDocComparator.INDEXORDER
  -	};
  -
  -
  -    /**
  -     * Creates a hit queue sorted by the given field.  Hits are sorted by the field, then
  -	 * by index order.
  -     * @param reader  IndexReader to use.
  -     * @param field   Field to sort by.
  -     * @param size    Number of hits to return - see {@link PriorityQueue#initialize(int) initialize}
  -     * @throws IOException  If the internal term enumerator fails.
  -     */
  -    FieldSortedHitQueue (IndexReader reader, String field, int size)
  -    throws IOException {
  -
  -		// reset the cache if we have a new reader
  -        int hash = reader.hashCode();
  -        if (hash != lastReaderHash) {
  -            lastReaderHash = hash;
  -            fieldCache.clear();
  -        }
  +  /**
  +   * Creates a hit queue sorted by the given list of fields.
  +   * @param reader  Index to use.
  +   * @param fields Field names, in priority order (highest priority first).  Cannot be <code>null</code> or empty.
  +   * @param size  The number of hits to retain.  Must be greater than zero.
  +   * @throws IOException
  +   */
  +  FieldSortedHitQueue (IndexReader reader, SortField[] fields, int size)
  +  throws IOException {
  +    final int n = fields.length;
  +    comparators = new ScoreDocComparator[n];
  +    this.fields = new SortField[n];
  +    for (int i=0; i<n; ++i) {
  +      String fieldname = fields[i].getField();
  +      comparators[i] = getCachedComparator (reader, fieldname, fields[i].getType(), fields[i].getFactory());
  +      this.fields[i] = new SortField (fieldname, comparators[i].sortType(), fields[i].getReverse());
  +    }
  +    initialize (size);
  +  }
  +
   
  -		// initialize the PriorityQueue
  -        initialize (size);
  +  /** Stores a comparator corresponding to each field being sorted by */
  +  protected ScoreDocComparator[] comparators;
   
  -		// set the sort
  -        comparators[0] = initializeSort (reader, field);
  +  /** Stores the sort criteria being used. */
  +  protected SortField[] fields;
  +
  +  /** Stores the maximum score value encountered, for normalizing.
  +   *  we only care about scores greater than 1.0 - if all the scores
  +   *  are less than 1.0, we don't have to normalize. */
  +  protected float maxscore = 1.0f;
  +
  +
  +  /**
  +   * Returns whether <code>a</code> is less relevant than <code>b</code>.
  +   * @param a ScoreDoc
  +   * @param b ScoreDoc
  +   * @return <code>true</code> if document <code>a</code> should be sorted after document <code>b</code>.
  +   */
  +  protected final boolean lessThan (final Object a, final Object b) {
  +    final ScoreDoc docA = (ScoreDoc) a;
  +    final ScoreDoc docB = (ScoreDoc) b;
  +
  +    // keep track of maximum score
  +    if (docA.score > maxscore) maxscore = docA.score;
  +    if (docB.score > maxscore) maxscore = docB.score;
  +
  +    // run comparators
  +    final int n = comparators.length;
  +    int c = 0;
  +    for (int i=0; i<n && c==0; ++i) {
  +      c = (fields[i].reverse) ? comparators[i].compare (docB, docA)
  +                              : comparators[i].compare (docA, docB);
       }
  +    return c > 0;
  +  }
  +
   
  +  /**
  +   * Given a FieldDoc object, stores the values used
  +   * to sort the given document.  These values are not the raw
  +   * values out of the index, but the internal representation
  +   * of them.  This is so the given search hit can be collated
  +   * by a MultiSearcher with other search hits.
  +   * @param  doc  The FieldDoc to store sort values into.
  +   * @return  The same FieldDoc passed in.
  +   * @see Searchable#search(Query,Filter,int,Sort)
  +   */
  +  FieldDoc fillFields (final FieldDoc doc) {
  +    final int n = comparators.length;
  +    final Comparable[] fields = new Comparable[n];
  +    for (int i=0; i<n; ++i)
  +      fields[i] = comparators[i].sortValue(doc);
  +    doc.fields = fields;
  +    if (maxscore > 1.0f) doc.score /= maxscore;   // normalize scores
  +    return doc;
  +  }
  +
  +
  +  /** Returns the SortFields being used by this hit queue. */
  +  SortField[] getFields() {
  +    return fields;
  +  }
  +
  +  /** Internal cache of comparators. Similar to FieldCache, only
  +   *  caches comparators instead of term values. */
  +  static final Map Comparators = new WeakHashMap();
  +
  +  /** Returns a comparator if it is in the cache. */
  +  static ScoreDocComparator lookup (IndexReader reader, String field, int type, Object factory) {
  +    FieldCacheImpl.Entry entry = (factory != null) ? new FieldCacheImpl.Entry (reader, field, factory)
  +                                                   : new FieldCacheImpl.Entry (reader, field, type);
  +    synchronized (Comparators) {
  +      return (ScoreDocComparator) Comparators.get (entry);
  +    }
  +  }
   
  -	/**
  -	 * Creates a sorted hit queue based on an existing comparator.  The hits
  -	 * are sorted by the given comparator, then by index order.
  -	 * @param comparator  Comparator used to sort hits.
  -	 * @param size        Number of hits to retain.
  -	 */
  -	protected FieldSortedHitQueue (ScoreDocComparator comparator, int size) {
  -		initialize (size);          // initialize the PriorityQueue
  -		comparators[0] = comparator;    // set the sort
  -	}
  -
  -
  -	/**
  -	 * Returns whether <code>a</code> is less relevant than <code>b</code>
  -	 * @param a ScoreDoc
  -	 * @param b ScoreDoc
  -	 * @return <code>true</code> if document <code>a</code> should be sorted after document <code>b</code>.
  -	 */
  -	protected final boolean lessThan (final Object a, final Object b) {
  -		final ScoreDoc docA = (ScoreDoc) a;
  -		final ScoreDoc docB = (ScoreDoc) b;
  -		final int n = comparators.length;
  -		int c = 0;
  -		for (int i=0; i<n && c==0; ++i) {
  -			c = comparators[i].compare (docA, docB);
  -		}
  -		return c > 0;
  -	}
  -
  -
  -    /**
  -     * Initializes the cache of sort information.  <code>fieldCache</code> is queried
  -     * to see if it has the term information for the given field.
  -     * If so, and if the reader still has the same value for maxDoc()
  -     * (note that we assume new IndexReaders are caught during the
  -     * constructor), the existing data is used.  If not, all the term values
  -     * for the given field are fetched.  The value of the term is assumed
  -     * to indicate the sort order for any documents containing the term.  Documents
  -     * should only have one term in the given field.  Multiple documents
  -     * can share the same term if desired, in which case they will be
  -	 * considered equal during the sort.
  -     * @param reader  The document index.
  -     * @param field   The field to sort by.
  -     * @throws IOException  If createComparator(IndexReader,String) fails - usually caused by the term enumerator failing.
  -     */
  -    protected final ScoreDocComparator initializeSort (IndexReader reader, String field)
  -    throws IOException {
  -
  -		ScoreDocLookupComparator comparer = (ScoreDocLookupComparator) fieldCache.get (field);
  -		if (comparer == null || !comparer.sizeMatches(reader.maxDoc())) {
  -			comparer = createComparator (reader, field);
  -            fieldCache.put (field, comparer);
  -		}
  -		return comparer;
  +  /** Stores a comparator into the cache. */
  +  static Object store (IndexReader reader, String field, int type, Object factory, Object value) {
  +    FieldCacheImpl.Entry entry = (factory != null) ? new FieldCacheImpl.Entry (reader, field, factory)
  +                                                   : new FieldCacheImpl.Entry (reader, field, type);
  +    synchronized (Comparators) {
  +      return Comparators.put (entry, value);
       }
  +  }
   
  +  static ScoreDocComparator getCachedComparator (IndexReader reader, String fieldname, int type, SortComparatorSource factory)
  +  throws IOException {
  +    if (type == SortField.DOC) return ScoreDocComparator.INDEXORDER;
  +    if (type == SortField.SCORE) return ScoreDocComparator.RELEVANCE;
  +    ScoreDocComparator comparator = lookup (reader, fieldname, type, factory);
  +    if (comparator == null) {
  +      switch (type) {
  +        case SortField.AUTO:
  +          comparator = comparatorAuto (reader, fieldname);
  +          break;
  +        case SortField.INT:
  +          comparator = comparatorInt (reader, fieldname);
  +          break;
  +        case SortField.FLOAT:
  +          comparator = comparatorFloat (reader, fieldname);
  +          break;
  +        case SortField.STRING:
  +          comparator = comparatorString (reader, fieldname);
  +          break;
  +        case SortField.CUSTOM:
  +          comparator = factory.newComparator (reader, fieldname);
  +          break;
  +        default:
  +          throw new RuntimeException ("unknown field type: "+type);
  +      }
  +      store (reader, fieldname, type, factory, comparator);
  +    }
  +    return comparator;
  +  }
   
  -	/**
  -	 * Subclasses should implement this method to provide an appropriate ScoreDocLookupComparator.
  -	 * @param reader  Index to use.
  -	 * @param field   Field to use for sorting.
  -	 * @return Comparator to use to sort hits.
  -	 * @throws IOException  If an error occurs reading the index.
  -	 */
  -	protected abstract ScoreDocLookupComparator createComparator (IndexReader reader, String field)
  -	throws IOException;
  -}
  \ No newline at end of file
  +  /**
  +   * Returns a comparator for sorting hits according to a field containing integers.
  +   * @param reader  Index to use.
  +   * @param fieldname  Field containg integer values.
  +   * @return  Comparator for sorting hits.
  +   * @throws IOException If an error occurs reading the index.
  +   */
  +  static ScoreDocComparator comparatorInt (final IndexReader reader, final String fieldname)
  +  throws IOException {
  +    final String field = fieldname.intern();
  +    return new ScoreDocComparator() {
  +
  +      final int[] fieldOrder = FieldCache.DEFAULT.getInts (reader, field);
  +
  +      public final int compare (final ScoreDoc i, final ScoreDoc j) {
  +        final int fi = fieldOrder[i.doc];
  +        final int fj = fieldOrder[j.doc];
  +        if (fi < fj) return -1;
  +        if (fi > fj) return 1;
  +        return 0;
  +      }
  +
  +      public Comparable sortValue (final ScoreDoc i) {
  +        return new Integer (fieldOrder[i.doc]);
  +      }
  +
  +      public int sortType() {
  +        return SortField.INT;
  +      }
  +    };
  +  }
  +
  +  /**
  +   * Returns a comparator for sorting hits according to a field containing floats.
  +   * @param reader  Index to use.
  +   * @param fieldname  Field containg float values.
  +   * @return  Comparator for sorting hits.
  +   * @throws IOException If an error occurs reading the index.
  +   */
  +  static ScoreDocComparator comparatorFloat (final IndexReader reader, final String fieldname)
  +  throws IOException {
  +    final String field = fieldname.intern();
  +    return new ScoreDocComparator () {
  +
  +      protected final float[] fieldOrder = FieldCache.DEFAULT.getFloats (reader, field);
  +
  +      public final int compare (final ScoreDoc i, final ScoreDoc j) {
  +        final float fi = fieldOrder[i.doc];
  +        final float fj = fieldOrder[j.doc];
  +        if (fi < fj) return -1;
  +        if (fi > fj) return 1;
  +        return 0;
  +      }
  +
  +      public Comparable sortValue (final ScoreDoc i) {
  +        return new Float (fieldOrder[i.doc]);
  +      }
  +
  +      public int sortType() {
  +        return SortField.FLOAT;
  +      }
  +    };
  +  }
  +
  +  /**
  +   * Returns a comparator for sorting hits according to a field containing strings.
  +   * @param reader  Index to use.
  +   * @param fieldname  Field containg string values.
  +   * @return  Comparator for sorting hits.
  +   * @throws IOException If an error occurs reading the index.
  +   */
  +  static ScoreDocComparator comparatorString (final IndexReader reader, final String fieldname)
  +  throws IOException {
  +    final String field = fieldname.intern();
  +    return new ScoreDocComparator () {
  +      final FieldCache.StringIndex index = FieldCache.DEFAULT.getStringIndex (reader, field);
  +
  +      public final int compare (final ScoreDoc i, final ScoreDoc j) {
  +        final int fi = index.order[i.doc];
  +        final int fj = index.order[j.doc];
  +        if (fi < fj) return -1;
  +        if (fi > fj) return 1;
  +        return 0;
  +      }
  +
  +      public Comparable sortValue (final ScoreDoc i) {
  +        return index.lookup[index.order[i.doc]];
  +      }
  +
  +      public int sortType() {
  +        return SortField.STRING;
  +      }
  +    };
  +  }
  +
  +  /**
  +   * Returns a comparator for sorting hits according to values in the given field.
  +   * The terms in the field are looked at to determine whether they contain integers,
  +   * floats or strings.  Once the type is determined, one of the other static methods
  +   * in this class is called to get the comparator.
  +   * @param reader  Index to use.
  +   * @param fieldname  Field containg values.
  +   * @return  Comparator for sorting hits.
  +   * @throws IOException If an error occurs reading the index.
  +   */
  +  static ScoreDocComparator comparatorAuto (final IndexReader reader, final String fieldname)
  +  throws IOException {
  +    final String field = fieldname.intern();
  +    Object lookupArray = FieldCache.DEFAULT.getAuto (reader, field);
  +    if (lookupArray instanceof FieldCache.StringIndex) {
  +      return comparatorString (reader, field);
  +    } else if (lookupArray instanceof int[]) {
  +      return comparatorInt (reader, field);
  +    } else if (lookupArray instanceof float[]) {
  +      return comparatorFloat (reader, field);
  +    } else if (lookupArray instanceof String[]) {
  +      return comparatorString (reader, field);
  +    } else {
  +      throw new RuntimeException ("unknown data type in field '"+field+"'");
  +    }
  +  }
  +}
  
  
  
  1.19      +2 -2      jakarta-lucene/src/java/org/apache/lucene/search/IndexSearcher.java
  
  Index: IndexSearcher.java
  ===================================================================
  RCS file: /home/cvs/jakarta-lucene/src/java/org/apache/lucene/search/IndexSearcher.java,v
  retrieving revision 1.18
  retrieving revision 1.19
  diff -u -r1.18 -r1.19
  --- IndexSearcher.java	11 May 2004 17:18:04 -0000	1.18
  +++ IndexSearcher.java	19 May 2004 23:05:27 -0000	1.19
  @@ -115,8 +115,8 @@
         return new TopFieldDocs(0, new ScoreDoc[0], sort.fields);
   
       final BitSet bits = filter != null ? filter.bits(reader) : null;
  -    final MultiFieldSortedHitQueue hq =
  -      new MultiFieldSortedHitQueue(reader, sort.fields, nDocs);
  +    final FieldSortedHitQueue hq =
  +      new FieldSortedHitQueue(reader, sort.fields, nDocs);
       final int[] totalHits = new int[1];
       scorer.score(new HitCollector() {
           public final void collect(int doc, float score) {
  
  
  
  1.5       +4 -23     jakarta-lucene/src/java/org/apache/lucene/search/ScoreDocComparator.java
  
  Index: ScoreDocComparator.java
  ===================================================================
  RCS file: /home/cvs/jakarta-lucene/src/java/org/apache/lucene/search/ScoreDocComparator.java,v
  retrieving revision 1.4
  retrieving revision 1.5
  diff -u -r1.4 -r1.5
  --- ScoreDocComparator.java	23 Apr 2004 14:46:34 -0000	1.4
  +++ ScoreDocComparator.java	19 May 2004 23:05:27 -0000	1.5
  @@ -35,11 +35,6 @@
   			if (i.score < j.score) return 1;
   			return 0;
   		}
  -		public int compareReverse (ScoreDoc i, ScoreDoc j) {
  -			if (i.score < j.score) return -1;
  -			if (i.score > j.score) return 1;
  -			return 0;
  -		}
   		public Comparable sortValue (ScoreDoc i) {
   			return new Float (i.score);
   		}
  @@ -56,11 +51,6 @@
   			if (i.doc > j.doc) return 1;
   			return 0;
   		}
  -		public int compareReverse (ScoreDoc i, ScoreDoc j) {
  -			if (i.doc > j.doc) return -1;
  -			if (i.doc < j.doc) return 1;
  -			return 0;
  -		}
   		public Comparable sortValue (ScoreDoc i) {
   			return new Integer (i.doc);
   		}
  @@ -82,20 +72,10 @@
   
   
   	/**
  -	 * Compares two ScoreDoc objects and returns a result indicating their
  -	 * sort order in reverse.
  -	 * @param i First ScoreDoc
  -	 * @param j Second ScoreDoc
  -	 * @return <code>-1</code> if <code>j</code> should come before <code>i</code><br><code>1</code> if <code>j</code> should come after <code>i</code><br><code>0</code> if they are equal
  -	 * @see java.util.Comparator
  -	 */
  -	int compareReverse (ScoreDoc i, ScoreDoc j);
  -
  -
  -	/**
   	 * Returns the value used to sort the given document.  The
   	 * object returned must implement the java.io.Serializable
  -	 * interface.
  +	 * interface.  This is used by multisearchers to determine how to collate results from their searchers.
  +	 * @see FieldDoc
   	 * @param i Document
   	 * @return Serializable object
   	 */
  @@ -105,6 +85,7 @@
   	/**
   	 * Returns the type of sort.  Should return <code>SortField.SCORE</code>, <code>SortField.DOC</code>, <code>SortField.STRING</code>, <code>SortField.INTEGER</code>, 
   	 * <code>SortField.FLOAT</code> or <code>SortField.CUSTOM</code>.  It is not valid to return <code>SortField.AUTO</code>.
  +	 * This is used by multisearchers to determine how to collate results from their searchers.
   	 * @return One of the constants in SortField.
   	 * @see SortField
   	 */
  
  
  
  1.2       +2 -2      jakarta-lucene/src/java/org/apache/lucene/search/SortComparatorSource.java
  
  Index: SortComparatorSource.java
  ===================================================================
  RCS file: /home/cvs/jakarta-lucene/src/java/org/apache/lucene/search/SortComparatorSource.java,v
  retrieving revision 1.1
  retrieving revision 1.2
  diff -u -r1.1 -r1.2
  --- SortComparatorSource.java	22 Apr 2004 22:23:14 -0000	1.1
  +++ SortComparatorSource.java	19 May 2004 23:05:27 -0000	1.2
  @@ -23,6 +23,6 @@
      * @return Comparator of ScoreDoc objects.
      * @throws IOException If an error occurs reading the index.
      */
  -  ScoreDocLookupComparator newComparator (IndexReader reader, String fieldname)
  +  ScoreDocComparator newComparator (IndexReader reader, String fieldname)
     throws IOException;
   }
  
  
  
  1.8       +6 -1      jakarta-lucene/src/java/org/apache/lucene/search/SortField.java
  
  Index: SortField.java
  ===================================================================
  RCS file: /home/cvs/jakarta-lucene/src/java/org/apache/lucene/search/SortField.java,v
  retrieving revision 1.7
  retrieving revision 1.8
  diff -u -r1.7 -r1.8
  --- SortField.java	8 May 2004 19:54:12 -0000	1.7
  +++ SortField.java	19 May 2004 23:05:27 -0000	1.8
  @@ -62,6 +62,11 @@
      * sorting is done according to natural order. */
     public static final int CUSTOM = 9;
   
  +  // IMPLEMENTATION NOTE: the FieldCache.STRING_INDEX is in the same "namespace"
  +  // as the above static int values.  Any new values must not have the same value
  +  // as FieldCache.STRING_INDEX.
  +
  +	
     /** Represents sorting by document score (relevancy). */
     public static final SortField FIELD_SCORE = new SortField (null, SCORE);
   
  
  
  
  1.1                  jakarta-lucene/src/java/org/apache/lucene/search/FieldCache.java
  
  Index: FieldCache.java
  ===================================================================
  package org.apache.lucene.search;
  
  /**
   * Copyright 2004 The Apache Software Foundation
   *
   * Licensed under the Apache License, Version 2.0 (the "License");
   * you may not use this file except in compliance with the License.
   * You may obtain a copy of the License at
   *
   *     http://www.apache.org/licenses/LICENSE-2.0
   *
   * Unless required by applicable law or agreed to in writing, software
   * distributed under the License is distributed on an "AS IS" BASIS,
   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
   * See the License for the specific language governing permissions and
   * limitations under the License.
   */
  
  import org.apache.lucene.index.IndexReader;
  import java.io.IOException;
  
  /**
   * Expert: Maintains caches of term values.
   *
   * <p>Created: May 19, 2004 11:13:14 AM
   *
   * @author  Tim Jones (Nacimiento Software)
   * @since   lucene 1.4
   * @version $Id: FieldCache.java,v 1.1 2004/05/19 23:05:27 tjones Exp $
   */
  public interface FieldCache {
  
    /** Indicator for StringIndex values in the cache. */
    // NOTE: the value assigned to this constant must not be
    // the same as any of those in SortField!!
    public static final int STRING_INDEX = -1;
  
  
    /** Expert: Stores term text values and document ordering data. */
    public static class StringIndex {
  
      /** All the term values, in natural order. */
      public final String[] lookup;
  
      /** For each document, an index into the lookup array. */
      public final int[] order;
  
      /** Creates one of these objects */
      public StringIndex (int[] values, String[] lookup) {
        this.order = values;
        this.lookup = lookup;
      }
    }
  
  
    /** Expert: The cache used internally by sorting and range query classes. */
    public static FieldCache DEFAULT = new FieldCacheImpl();
  
  
    /** Checks the internal cache for an appropriate entry, and if none is
     * found, reads the terms in <code>field</code> as integers and returns an array
     * of size <code>reader.maxDoc()</code> of the value each document
     * has in the given field.
     * @param reader  Used to get field values.
     * @param field   Which field contains the integers.
     * @return The values in the given field for each document.
     * @throws IOException  If any error occurs.
     */
    public int[] getInts (IndexReader reader, String field)
    throws IOException;
  
    /** Checks the internal cache for an appropriate entry, and if
     * none is found, reads the terms in <code>field</code> as floats and returns an array
     * of size <code>reader.maxDoc()</code> of the value each document
     * has in the given field.
     * @param reader  Used to get field values.
     * @param field   Which field contains the floats.
     * @return The values in the given field for each document.
     * @throws IOException  If any error occurs.
     */
    public float[] getFloats (IndexReader reader, String field)
    throws IOException;
  
    /** Checks the internal cache for an appropriate entry, and if none
     * is found, reads the term values in <code>field</code> and returns an array
     * of size <code>reader.maxDoc()</code> containing the value each document
     * has in the given field.
     * @param reader  Used to get field values.
     * @param field   Which field contains the strings.
     * @return The values in the given field for each document.
     * @throws IOException  If any error occurs.
     */
    public String[] getStrings (IndexReader reader, String field)
    throws IOException;
  
    /** Checks the internal cache for an appropriate entry, and if none
     * is found reads the term values in <code>field</code> and returns
     * an array of them in natural order, along with an array telling
     * which element in the term array each document uses.
     * @param reader  Used to get field values.
     * @param field   Which field contains the strings.
     * @return Array of terms and index into the array for each document.
     * @throws IOException  If any error occurs.
     */
    public StringIndex getStringIndex (IndexReader reader, String field)
    throws IOException;
  
    /** Checks the internal cache for an appropriate entry, and if
     * none is found reads <code>field</code> to see if it contains integers, floats
     * or strings, and then calls one of the other methods in this class to get the
     * values.  For string values, a StringIndex is returned.  After
     * calling this method, there is an entry in the cache for both
     * type <code>AUTO</code> and the actual found type.
     * @param reader  Used to get field values.
     * @param field   Which field contains the values.
     * @return int[], float[] or StringIndex.
     * @throws IOException  If any error occurs.
     */
    public Object getAuto (IndexReader reader, String field)
    throws IOException;
  
    /** Checks the internal cache for an appropriate entry, and if none
     * is found reads the terms out of <code>field</code> and calls the given SortComparator
     * to get the sort values.  A hit in the cache will happen if <code>reader</code>,
     * <code>field</code>, and <code>comparator</code> are the same (using <code>equals()</code>)
     * as a previous call to this method.
     * @param reader  Used to get field values.
     * @param field   Which field contains the values.
     * @param comparator Used to convert terms into something to sort by.
     * @return Array of sort objects, one for each document.
     * @throws IOException  If any error occurs.
     */
    public Comparable[] getCustom (IndexReader reader, String field, SortComparator comparator)
    throws IOException;
  }
  
  
  
  1.1                  jakarta-lucene/src/java/org/apache/lucene/search/FieldCacheImpl.java
  
  Index: FieldCacheImpl.java
  ===================================================================
  package org.apache.lucene.search;
  
  /**
   * Copyright 2004 The Apache Software Foundation
   *
   * Licensed under the Apache License, Version 2.0 (the "License");
   * you may not use this file except in compliance with the License.
   * You may obtain a copy of the License at
   *
   *     http://www.apache.org/licenses/LICENSE-2.0
   *
   * Unless required by applicable law or agreed to in writing, software
   * distributed under the License is distributed on an "AS IS" BASIS,
   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
   * See the License for the specific language governing permissions and
   * limitations under the License.
   */
  
  import org.apache.lucene.index.IndexReader;
  import org.apache.lucene.index.Term;
  import org.apache.lucene.index.TermDocs;
  import org.apache.lucene.index.TermEnum;
  
  import java.io.IOException;
  import java.util.Map;
  import java.util.WeakHashMap;
  
  /**
   * Expert: The default cache implementation, storing all values in memory.
   * A WeakHashMap is used for storage.
   *
   * <p>Created: May 19, 2004 4:40:36 PM
   *
   * @author  Tim Jones (Nacimiento Software)
   * @since   lucene 1.4
   * @version $Id: FieldCacheImpl.java,v 1.1 2004/05/19 23:05:27 tjones Exp $
   */
  class FieldCacheImpl
  implements FieldCache {
  
    /** Expert: Every key in the internal cache is of this type. */
    static class Entry {
      final IndexReader reader;  // which Reader
      final String field;        // which Field
      final int type;            // which SortField type
      final Object custom;       // which custom comparator
      final int hashcode;        // unique for this object
  
      /** Creates one of these objects. */
      Entry (IndexReader reader, String field, int type) {
        this.reader = reader;
        this.field = field.intern();
        this.type = type;
        this.custom = null;
        this.hashcode = reader.hashCode() ^ field.hashCode() ^ type;
      }
  
      /** Creates one of these objects for a custom comparator. */
      Entry (IndexReader reader, String field, Object custom) {
        this.reader = reader;
        this.field = field.intern();
        this.type = SortField.CUSTOM;
        this.custom = custom;
        this.hashcode = reader.hashCode() ^ field.hashCode() ^ type ^ custom.hashCode();
      }
  
      /** Two of these are equal iff they reference the same reader, field and type. */
      public boolean equals (Object o) {
        if (o instanceof Entry) {
          Entry other = (Entry) o;
          if (other.reader == reader && other.field == field && other.type == type) {
            if (other.custom == null) {
              if (custom == null) return true;
            } else if (other.custom.equals (custom)) {
              return true;
            }
          }
        }
        return false;
      }
  
      /** Composes a hashcode based on the referenced reader, field and type. */
      public int hashCode() {
        return hashcode;
      }
    }
  
  
    /** The internal cache. Maps Entry to array of interpreted term values. **/
    final Map cache = new WeakHashMap();
  
    /** See if an object is in the cache. */
    Object lookup (IndexReader reader, String field, int type) {
      Entry entry = new Entry (reader, field, type);
      synchronized (this) {
        return cache.get (entry);
      }
    }
  
    /** See if a custom object is in the cache. */
    Object lookup (IndexReader reader, String field, Object comparer) {
      Entry entry = new Entry (reader, field, comparer);
      synchronized (this) {
        return cache.get (entry);
      }
    }
  
    /** Put an object into the cache. */
    Object store (IndexReader reader, String field, int type, Object value) {
      Entry entry = new Entry (reader, field, type);
      synchronized (this) {
        return cache.put (entry, value);
      }
    }
  
    /** Put a custom object into the cache. */
    Object store (IndexReader reader, String field, Object comparer, Object value) {
      Entry entry = new Entry (reader, field, comparer);
      synchronized (this) {
        return cache.put (entry, value);
      }
    }
  
    // inherit javadocs
    public int[] getInts (IndexReader reader, String field)
    throws IOException {
      field = field.intern();
      Object ret = lookup (reader, field, SortField.INT);
      if (ret == null) {
        final int[] retArray = new int[reader.maxDoc()];
        if (retArray.length > 0) {
          TermDocs termDocs = reader.termDocs();
          TermEnum termEnum = reader.terms (new Term (field, ""));
          try {
            if (termEnum.term() == null) {
              throw new RuntimeException ("no terms in field " + field);
            }
            do {
              Term term = termEnum.term();
              if (term.field() != field) break;
              int termval = Integer.parseInt (term.text());
              termDocs.seek (termEnum);
              while (termDocs.next()) {
                retArray[termDocs.doc()] = termval;
              }
            } while (termEnum.next());
          } finally {
            termDocs.close();
            termEnum.close();
          }
        }
        store (reader, field, SortField.INT, retArray);
        return retArray;
      }
      return (int[]) ret;
    }
  
    // inherit javadocs
    public float[] getFloats (IndexReader reader, String field)
    throws IOException {
      field = field.intern();
      Object ret = lookup (reader, field, SortField.FLOAT);
      if (ret == null) {
        final float[] retArray = new float[reader.maxDoc()];
        if (retArray.length > 0) {
          TermDocs termDocs = reader.termDocs();
          TermEnum termEnum = reader.terms (new Term (field, ""));
          try {
            if (termEnum.term() == null) {
              throw new RuntimeException ("no terms in field " + field);
            }
            do {
              Term term = termEnum.term();
              if (term.field() != field) break;
              float termval = Float.parseFloat (term.text());
              termDocs.seek (termEnum);
              while (termDocs.next()) {
                retArray[termDocs.doc()] = termval;
              }
            } while (termEnum.next());
          } finally {
            termDocs.close();
            termEnum.close();
          }
        }
        store (reader, field, SortField.FLOAT, retArray);
        return retArray;
      }
      return (float[]) ret;
    }
  
    // inherit javadocs
    public String[] getStrings (IndexReader reader, String field)
    throws IOException {
      field = field.intern();
      Object ret = lookup (reader, field, SortField.STRING);
      if (ret == null) {
        final String[] retArray = new String[reader.maxDoc()];
        if (retArray.length > 0) {
          TermDocs termDocs = reader.termDocs();
          TermEnum termEnum = reader.terms (new Term (field, ""));
          try {
            if (termEnum.term() == null) {
              throw new RuntimeException ("no terms in field " + field);
            }
            do {
              Term term = termEnum.term();
              if (term.field() != field) break;
              String termval = term.text();
              termDocs.seek (termEnum);
              while (termDocs.next()) {
                retArray[termDocs.doc()] = termval;
              }
            } while (termEnum.next());
          } finally {
            termDocs.close();
            termEnum.close();
          }
        }
        store (reader, field, SortField.STRING, retArray);
        return retArray;
      }
      return (String[]) ret;
    }
  
    // inherit javadocs
    public StringIndex getStringIndex (IndexReader reader, String field)
    throws IOException {
      field = field.intern();
      Object ret = lookup (reader, field, STRING_INDEX);
      if (ret == null) {
        final int[] retArray = new int[reader.maxDoc()];
        String[] mterms = new String[reader.maxDoc()];
        if (retArray.length > 0) {
          TermDocs termDocs = reader.termDocs();
          TermEnum termEnum = reader.terms (new Term (field, ""));
          int t = 0;  // current term number
          try {
            if (termEnum.term() == null) {
              throw new RuntimeException ("no terms in field " + field);
            }
            do {
              Term term = termEnum.term();
              if (term.field() != field) break;
  
              // store term text
              // we expect that there is at most one term per document
              if (t >= mterms.length) throw new RuntimeException ("there are more terms than documents in field \"" + field + "\"");
              mterms[t] = term.text();
  
              termDocs.seek (termEnum);
              while (termDocs.next()) {
                retArray[termDocs.doc()] = t;
              }
  
              t++;
            } while (termEnum.next());
          } finally {
            termDocs.close();
            termEnum.close();
          }
  
          if (t == 0) {
            // if there are no terms, make the term array
            // have a single null entry
            mterms = new String[1];
  		} else if (t < mterms.length) {
            // if there are less terms than documents,
            // trim off the dead array space
            String[] terms = new String[t];
            System.arraycopy (mterms, 0, terms, 0, t);
            mterms = terms;
          }
        }
        StringIndex value = new StringIndex (retArray, mterms);
        store (reader, field, STRING_INDEX, value);
        return value;
      }
      return (StringIndex) ret;
    }
  
    /** The pattern used to detect integer values in a field */
    /** removed for java 1.3 compatibility
     protected static final Pattern pIntegers = Pattern.compile ("[0-9\\-]+");
     **/
  
    /** The pattern used to detect float values in a field */
    /**
     * removed for java 1.3 compatibility
     * protected static final Object pFloats = Pattern.compile ("[0-9+\\-\\.eEfFdD]+");
     */
  
    // inherit javadocs
    public Object getAuto (IndexReader reader, String field)
    throws IOException {
      field = field.intern();
      Object ret = lookup (reader, field, SortField.AUTO);
      if (ret == null) {
        TermEnum enumerator = reader.terms (new Term (field, ""));
        try {
          Term term = enumerator.term();
          if (term == null) {
            throw new RuntimeException ("no terms in field " + field + " - cannot determine sort type");
          }
          if (term.field() == field) {
            String termtext = term.text().trim();
  
            /**
             * Java 1.4 level code:
  
             if (pIntegers.matcher(termtext).matches())
             return IntegerSortedHitQueue.comparator (reader, enumerator, field);
  
             else if (pFloats.matcher(termtext).matches())
             return FloatSortedHitQueue.comparator (reader, enumerator, field);
             */
  
            // Java 1.3 level code:
            try {
              Integer.parseInt (termtext);
              ret = getInts (reader, field);
            } catch (NumberFormatException nfe1) {
              try {
                Float.parseFloat (termtext);
                ret = getFloats (reader, field);
              } catch (NumberFormatException nfe2) {
                ret = getStringIndex (reader, field);
              }
            }
            if (ret != null) {
              store (reader, field, SortField.AUTO, ret);
            }
          } else {
            throw new RuntimeException ("field \"" + field + "\" does not appear to be indexed");
          }
        } finally {
          enumerator.close();
        }
  
      }
      return ret;
    }
  
    // inherit javadocs
    public Comparable[] getCustom (IndexReader reader, String field, SortComparator comparator)
    throws IOException {
      field = field.intern();
      Object ret = lookup (reader, field, comparator);
      if (ret == null) {
        final Comparable[] retArray = new Comparable[reader.maxDoc()];
        if (retArray.length > 0) {
          TermDocs termDocs = reader.termDocs();
          TermEnum termEnum = reader.terms (new Term (field, ""));
          try {
            if (termEnum.term() == null) {
              throw new RuntimeException ("no terms in field " + field);
            }
            do {
              Term term = termEnum.term();
              if (term.field() != field) break;
              Comparable termval = comparator.getComparable (term.text());
              termDocs.seek (termEnum);
              while (termDocs.next()) {
                retArray[termDocs.doc()] = termval;
              }
            } while (termEnum.next());
          } finally {
            termDocs.close();
            termEnum.close();
          }
        }
        store (reader, field, SortField.CUSTOM, retArray);
        return retArray;
      }
      return (String[]) ret;
    }
  
  }
  
  
  
  1.1                  jakarta-lucene/src/java/org/apache/lucene/search/SortComparator.java
  
  Index: SortComparator.java
  ===================================================================
  package org.apache.lucene.search;
  
  import org.apache.lucene.index.IndexReader;
  import org.apache.lucene.index.Term;
  import org.apache.lucene.index.TermDocs;
  import org.apache.lucene.index.TermEnum;
  
  import java.io.IOException;
  import java.io.Serializable;
  
  /**
   * Abstract base class for sorting hits returned by a Query.
   *
   * <p>This class should only be used if the other SortField
   * types (SCORE, DOC, STRING, INT, FLOAT) do not provide an
   * adequate sorting.  It maintains an internal cache of values which
   * could be quite large.  The cache is an array of Comparable,
   * one for each document in the index.  There is a distinct
   * Comparable for each unique term in the field - if
   * some documents have the same term in the field, the cache
   * array will have entries which reference the same Comparable.
   *
   * <p>Created: Apr 21, 2004 5:08:38 PM
   *
   * @author  Tim Jones
   * @version $Id: SortComparator.java,v 1.1 2004/05/19 23:05:27 tjones Exp $
   * @since   1.4
   */
  public abstract class SortComparator
  implements SortComparatorSource {
  
    // inherit javadocs
    public ScoreDocComparator newComparator (final IndexReader reader, final String fieldname)
    throws IOException {
      final String field = fieldname.intern();
      return new ScoreDocComparator() {
        protected Comparable[] cachedValues = FieldCache.DEFAULT.getCustom (reader, field, SortComparator.this);
  
        public int compare (ScoreDoc i, ScoreDoc j) {
          return cachedValues[i.doc].compareTo (cachedValues[j.doc]);
        }
  
        public Comparable sortValue (ScoreDoc i) {
          return cachedValues[i.doc];
        }
  
        public int sortType(){
          return SortField.CUSTOM;
        }
      };
    }
  
    /**
     * Returns an object which, when sorted according to natural order,
     * will order the Term values in the correct order.
     * <p>For example, if the Terms contained integer values, this method
     * would return <code>new Integer(termtext)</code>.  Note that this
     * might not always be the most efficient implementation - for this
     * particular example, a better implementation might be to make a
     * ScoreDocLookupComparator that uses an internal lookup table of int.
     * @param termtext The textual value of the term.
     * @return An object representing <code>termtext</code> that sorts according to the natural order of <code>termtext</code>.
     * @see Comparable
     * @see ScoreDocComparator
     */
    protected abstract Comparable getComparable (String termtext);
  
  }
  
  
  1.3       +12 -12    jakarta-lucene/src/test/org/apache/lucene/search/SampleComparable.java
  
  Index: SampleComparable.java
  ===================================================================
  RCS file: /home/cvs/jakarta-lucene/src/test/org/apache/lucene/search/SampleComparable.java,v
  retrieving revision 1.2
  retrieving revision 1.3
  diff -u -r1.2 -r1.3
  --- SampleComparable.java	23 Apr 2004 14:44:37 -0000	1.2
  +++ SampleComparable.java	19 May 2004 23:05:27 -0000	1.3
  @@ -64,28 +64,20 @@
       return i;
     }
   
  -  public static SortComparatorSource getComparator () {
  +  public static SortComparatorSource getComparatorSource () {
       return new SortComparatorSource () {
  -      public ScoreDocLookupComparator newComparator (final IndexReader reader, String fieldname)
  +      public ScoreDocComparator newComparator (final IndexReader reader, String fieldname)
         throws IOException {
           final String field = fieldname.intern ();
           final TermEnum enumerator = reader.terms (new Term (fieldname, ""));
           try {
  -          return new ScoreDocLookupComparator () {
  +          return new ScoreDocComparator () {
               protected Comparable[] cachedValues = fillCache (reader, enumerator, field);
   
  -            public boolean sizeMatches (int n) {
  -              return (cachedValues.length == n);
  -            }
  -
               public int compare (ScoreDoc i, ScoreDoc j) {
                 return cachedValues[i.doc].compareTo (cachedValues[j.doc]);
               }
   
  -            public int compareReverse (ScoreDoc i, ScoreDoc j) {
  -              return cachedValues[j.doc].compareTo (cachedValues[i.doc]);
  -            }
  -
               public Comparable sortValue (ScoreDoc i) {
                 return cachedValues[i.doc];
               }
  @@ -136,6 +128,14 @@
         }
   
         Comparable getComparable (String termtext) {
  +        return new SampleComparable (termtext);
  +      }
  +    };
  +  }
  +
  +  public static SortComparator getComparator() {
  +    return new SortComparator() {
  +      protected Comparable getComparable (String termtext) {
           return new SampleComparable (termtext);
         }
       };
  
  
  
  1.6       +15 -5     jakarta-lucene/src/test/org/apache/lucene/search/TestSort.java
  
  Index: TestSort.java
  ===================================================================
  RCS file: /home/cvs/jakarta-lucene/src/test/org/apache/lucene/search/TestSort.java,v
  retrieving revision 1.5
  retrieving revision 1.6
  diff -u -r1.5 -r1.6
  --- TestSort.java	22 Apr 2004 22:23:15 -0000	1.5
  +++ TestSort.java	19 May 2004 23:05:27 -0000	1.6
  @@ -253,9 +253,14 @@
   
   
   	public void testCustomSorts() throws Exception {
  -		sort.setSort (new SortField ("custom", SampleComparable.getComparator()));
  +		sort.setSort (new SortField ("custom", SampleComparable.getComparatorSource()));
   		assertMatches (full, queryX, sort, "CAIEG");
  -		sort.setSort (new SortField ("custom", SampleComparable.getComparator(), true));
  +		sort.setSort (new SortField ("custom", SampleComparable.getComparatorSource(), true));
  +		assertMatches (full, queryY, sort, "HJDBF");
  +		SortComparator custom = SampleComparable.getComparator();
  +		sort.setSort (new SortField ("custom", custom));
  +		assertMatches (full, queryX, sort, "CAIEG");
  +		sort.setSort (new SortField ("custom", custom, true));
   		assertMatches (full, queryY, sort, "HJDBF");
   	}
   
  @@ -281,9 +286,14 @@
   	public void testRemoteCustomSort() throws Exception {
   		Searchable searcher = getRemote();
   		MultiSearcher multi = new MultiSearcher (new Searchable[] { searcher });
  -		sort.setSort (new SortField ("custom", SampleComparable.getComparator()));
  +		sort.setSort (new SortField ("custom", SampleComparable.getComparatorSource()));
  +		assertMatches (multi, queryX, sort, "CAIEG");
  +		sort.setSort (new SortField ("custom", SampleComparable.getComparatorSource(), true));
  +		assertMatches (multi, queryY, sort, "HJDBF");
  +		SortComparator custom = SampleComparable.getComparator();
  +		sort.setSort (new SortField ("custom", custom));
   		assertMatches (multi, queryX, sort, "CAIEG");
  -		sort.setSort (new SortField ("custom", SampleComparable.getComparator(), true));
  +		sort.setSort (new SortField ("custom", custom, true));
   		assertMatches (multi, queryY, sort, "HJDBF");
   	}
   
  
  
  

---------------------------------------------------------------------
To unsubscribe, e-mail: lucene-dev-unsubscribe@jakarta.apache.org
For additional commands, e-mail: lucene-dev-help@jakarta.apache.org