You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2011/02/28 06:15:52 UTC

svn commit: r1075210 [1/3] - in /lucene/dev/trunk: lucene/ lucene/contrib/highlighter/src/test/org/apache/lucene/search/highlight/ lucene/contrib/queries/src/java/org/apache/lucene/search/ lucene/contrib/queries/src/test/org/apache/lucene/search/ lucen...

Author: rmuir
Date: Mon Feb 28 05:15:50 2011
New Revision: 1075210

URL: http://svn.apache.org/viewvc?rev=1075210&view=rev
Log:
LUCENE-2514, LUCENE-2551: collation uses byte[] keys, deprecate old unscalable locale sort/range, termrangequery/filter work on bytes

Added:
    lucene/dev/trunk/lucene/contrib/queries/src/java/org/apache/lucene/search/SlowCollatedStringComparator.java   (with props)
    lucene/dev/trunk/lucene/contrib/queries/src/java/org/apache/lucene/search/SlowCollatedTermRangeFilter.java   (with props)
    lucene/dev/trunk/lucene/contrib/queries/src/java/org/apache/lucene/search/SlowCollatedTermRangeQuery.java   (with props)
    lucene/dev/trunk/lucene/contrib/queries/src/java/org/apache/lucene/search/SlowCollatedTermRangeTermsEnum.java   (with props)
    lucene/dev/trunk/lucene/contrib/queries/src/test/org/apache/lucene/search/TestSlowCollationMethods.java   (with props)
    lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/collation/CollationAttributeFactory.java   (with props)
    lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/collation/tokenattributes/
    lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/collation/tokenattributes/CollatedTermAttributeImpl.java   (with props)
    lucene/dev/trunk/modules/analysis/icu/src/java/org/apache/lucene/collation/ICUCollationAttributeFactory.java   (with props)
    lucene/dev/trunk/modules/analysis/icu/src/java/org/apache/lucene/collation/tokenattributes/
    lucene/dev/trunk/modules/analysis/icu/src/java/org/apache/lucene/collation/tokenattributes/ICUCollatedTermAttributeImpl.java   (with props)
Removed:
    lucene/dev/trunk/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/config/RangeCollatorAttribute.java
    lucene/dev/trunk/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/config/RangeCollatorAttributeImpl.java
Modified:
    lucene/dev/trunk/lucene/MIGRATE.txt
    lucene/dev/trunk/lucene/contrib/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java
    lucene/dev/trunk/lucene/contrib/queries/src/test/org/apache/lucene/search/BooleanFilterTest.java
    lucene/dev/trunk/lucene/contrib/queries/src/test/org/apache/lucene/search/ChainedFilterTest.java
    lucene/dev/trunk/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/complexPhrase/ComplexPhraseQueryParser.java
    lucene/dev/trunk/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/StandardQueryParser.java
    lucene/dev/trunk/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/builders/RangeQueryNodeBuilder.java
    lucene/dev/trunk/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/config/StandardQueryConfigHandler.java
    lucene/dev/trunk/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/nodes/RangeQueryNode.java
    lucene/dev/trunk/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/processors/ParametricRangeQueryNodeProcessor.java
    lucene/dev/trunk/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/standard/TestQPHelper.java
    lucene/dev/trunk/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/standard/config/TestAttributes.java
    lucene/dev/trunk/lucene/contrib/xml-query-parser/src/java/org/apache/lucene/xmlparser/builders/RangeFilterBuilder.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/analysis/tokenattributes/CharTermAttributeImpl.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/queryParser/QueryParserBase.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldComparator.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldDocSortedHitQueue.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/SortField.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermRangeFilter.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermRangeQuery.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermRangeTermsEnum.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/IndexableBinaryStringTools.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/queryParser/TestQueryParser.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestConstantScoreQuery.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestDateFilter.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestFilteredQuery.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestMultiTermConstantScore.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestMultiTermQueryRewrites.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestMultiValuedNumericRangeQuery.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestNumericRangeQuery32.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestNumericRangeQuery64.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestSort.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestTermRangeFilter.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestTermRangeQuery.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestIndexableBinaryStringTools.java
    lucene/dev/trunk/modules/analysis/CHANGES.txt
    lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/core/KeywordTokenizer.java
    lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/collation/CollationKeyAnalyzer.java
    lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/collation/CollationKeyFilter.java
    lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/collation/package.html
    lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/collation/CollationTestBase.java
    lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/collation/TestCollationKeyAnalyzer.java
    lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/collation/TestCollationKeyFilter.java
    lucene/dev/trunk/modules/analysis/icu/src/java/org/apache/lucene/collation/ICUCollationKeyAnalyzer.java
    lucene/dev/trunk/modules/analysis/icu/src/java/org/apache/lucene/collation/ICUCollationKeyFilter.java
    lucene/dev/trunk/modules/analysis/icu/src/java/overview.html
    lucene/dev/trunk/modules/analysis/icu/src/test/org/apache/lucene/collation/TestICUCollationKeyAnalyzer.java
    lucene/dev/trunk/modules/analysis/icu/src/test/org/apache/lucene/collation/TestICUCollationKeyFilter.java
    lucene/dev/trunk/solr/src/java/org/apache/solr/handler/component/ShardDoc.java
    lucene/dev/trunk/solr/src/java/org/apache/solr/schema/DateField.java
    lucene/dev/trunk/solr/src/java/org/apache/solr/schema/FieldType.java
    lucene/dev/trunk/solr/src/java/org/apache/solr/search/QueryParsing.java

Modified: lucene/dev/trunk/lucene/MIGRATE.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/MIGRATE.txt?rev=1075210&r1=1075209&r2=1075210&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/MIGRATE.txt (original)
+++ lucene/dev/trunk/lucene/MIGRATE.txt Mon Feb 28 05:15:50 2011
@@ -313,6 +313,21 @@ LUCENE-1458, LUCENE-2111: Flexible Index
     - o.a.l.analysis.StopwordAnalyzerBase -> o.a.l.analysis.util.StopwordAnalyzerBase
     - o.a.l.analysis.WordListLoader -> o.a.l.analysis.util.WordListLoader
 
+* LUCENE-2514: The option to use a Collator's order (instead of binary order) for
+  sorting and range queries has been moved to contrib/queries.
+
+  The Collated TermRangeQuery/Filter has been moved to SlowCollatedTermRangeQuery/Filter, 
+  and the collated sorting has been moved to SlowCollatedStringComparator.
+
+  Note: this functionality isn't very scalable and if you are using it, consider 
+  indexing collation keys with the collation support in the analysis module instead.
+  
+  To perform collated range queries, use a suitable collating analyzer: CollationKeyAnalyzer 
+  or ICUCollationKeyAnalyzer, and set qp.setAnalyzeRangeTerms(true).
+  
+  TermRangeQuery and TermRangeFilter now work purely on bytes. Both have helper factory methods
+  (newStringRange) similar to the NumericRange API, to easily perform range queries on Strings.
+  
 * LUCENE-2691: The near-real-time API has moved from IndexWriter to
   IndexReader.  Instead of IndexWriter.getReader(), call
   IndexReader.open(IndexWriter) or IndexReader.reopen(IndexWriter).

Modified: lucene/dev/trunk/lucene/contrib/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java?rev=1075210&r1=1075209&r2=1075210&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java (original)
+++ lucene/dev/trunk/lucene/contrib/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java Mon Feb 28 05:15:50 2011
@@ -828,7 +828,7 @@ public class HighlighterTest extends Bas
       @Override
       public void run() throws Exception {
         numHighlights = 0;
-        TermRangeFilter rf = new TermRangeFilter("contents", "john", "john", true, true);
+        TermRangeFilter rf = TermRangeFilter.newStringRange("contents", "john", "john", true, true);
         SpanQuery clauses[] = { new SpanTermQuery(new Term("contents", "john")),
             new SpanTermQuery(new Term("contents", "kennedy")), };
         SpanNearQuery snq = new SpanNearQuery(clauses, 1, true);
@@ -851,7 +851,7 @@ public class HighlighterTest extends Bas
       @Override
       public void run() throws Exception {
         numHighlights = 0;
-        TermRangeFilter rf = new TermRangeFilter("contents", "john", "john", true, true);
+        TermRangeFilter rf = TermRangeFilter.newStringRange("contents", "john", "john", true, true);
         PhraseQuery pq = new PhraseQuery();
         pq.add(new Term("contents", "john"));
         pq.add(new Term("contents", "kennedy"));

Added: lucene/dev/trunk/lucene/contrib/queries/src/java/org/apache/lucene/search/SlowCollatedStringComparator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/queries/src/java/org/apache/lucene/search/SlowCollatedStringComparator.java?rev=1075210&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/contrib/queries/src/java/org/apache/lucene/search/SlowCollatedStringComparator.java (added)
+++ lucene/dev/trunk/lucene/contrib/queries/src/java/org/apache/lucene/search/SlowCollatedStringComparator.java Mon Feb 28 05:15:50 2011
@@ -0,0 +1,106 @@
+package org.apache.lucene.search;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.text.Collator;
+
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.search.FieldCache.DocTerms;
+import org.apache.lucene.util.BytesRef;
+
+/** Sorts by a field's value using the given Collator
+ *
+ * <p><b>WARNING</b>: this is very slow; you'll
+ * get much better performance using the
+ * CollationKeyAnalyzer or ICUCollationKeyAnalyzer. 
+ * @deprecated Index collation keys with CollationKeyAnalyzer or ICUCollationKeyAnalyzer instead.
+ * This class will be removed in Lucene 5.0
+ */
+@Deprecated
+public final class SlowCollatedStringComparator extends FieldComparator {
+
+  private final String[] values;
+  private DocTerms currentDocTerms;
+  private final String field;
+  final Collator collator;
+  private String bottom;
+  private final BytesRef tempBR = new BytesRef();
+
+  public SlowCollatedStringComparator(int numHits, String field, Collator collator) {
+    values = new String[numHits];
+    this.field = field;
+    this.collator = collator;
+  }
+
+  @Override
+  public int compare(int slot1, int slot2) {
+    final String val1 = values[slot1];
+    final String val2 = values[slot2];
+    if (val1 == null) {
+      if (val2 == null) {
+        return 0;
+      }
+      return -1;
+    } else if (val2 == null) {
+      return 1;
+    }
+    return collator.compare(val1, val2);
+  }
+
+  @Override
+  public int compareBottom(int doc) {
+    final String val2 = currentDocTerms.getTerm(doc, tempBR).utf8ToString();
+    if (bottom == null) {
+      if (val2 == null) {
+        return 0;
+      }
+      return -1;
+    } else if (val2 == null) {
+      return 1;
+    }
+    return collator.compare(bottom, val2);
+  }
+
+  @Override
+  public void copy(int slot, int doc) {
+    final BytesRef br = currentDocTerms.getTerm(doc, tempBR);
+    if (br == null) {
+      values[slot] = null;
+    } else {
+      values[slot] = br.utf8ToString();
+    }
+  }
+
+  @Override
+  public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
+    currentDocTerms = FieldCache.DEFAULT.getTerms(context.reader, field);
+    return this;
+  }
+  
+  @Override
+  public void setBottom(final int bottom) {
+    this.bottom = values[bottom];
+  }
+
+  @Override
+  public Comparable<?> value(int slot) {
+    final String s = values[slot];
+    return s == null ? null : new BytesRef(values[slot]);
+  }
+}

Added: lucene/dev/trunk/lucene/contrib/queries/src/java/org/apache/lucene/search/SlowCollatedTermRangeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/queries/src/java/org/apache/lucene/search/SlowCollatedTermRangeFilter.java?rev=1075210&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/contrib/queries/src/java/org/apache/lucene/search/SlowCollatedTermRangeFilter.java (added)
+++ lucene/dev/trunk/lucene/contrib/queries/src/java/org/apache/lucene/search/SlowCollatedTermRangeFilter.java Mon Feb 28 05:15:50 2011
@@ -0,0 +1,70 @@
+package org.apache.lucene.search;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.text.Collator;
+
+/**
+ * A Filter that restricts search results to a range of term
+ * values in a given field.
+ *
+ * <p>This filter matches the documents looking for terms that fall into the
+ * supplied range according to {@link
+ * String#compareTo(String)}, unless a <code>Collator</code> is provided. It is not intended
+ * for numerical ranges; use {@link NumericRangeFilter} instead.
+ *
+ * <p>If you construct a large number of range filters with different ranges but on the 
+ * same field, {@link FieldCacheRangeFilter} may have significantly better performance. 
+ * @deprecated Index collation keys with CollationKeyAnalyzer or ICUCollationKeyAnalyzer instead.
+ * This class will be removed in Lucene 5.0
+ */
+@Deprecated
+public class SlowCollatedTermRangeFilter extends MultiTermQueryWrapperFilter<SlowCollatedTermRangeQuery> {
+  /**
+   *
+   * @param lowerTerm The lower bound on this range
+   * @param upperTerm The upper bound on this range
+   * @param includeLower Does this range include the lower bound?
+   * @param includeUpper Does this range include the upper bound?
+   * @param collator The collator to use when determining range inclusion; set
+   *  to null to use Unicode code point ordering instead of collation.
+   * @throws IllegalArgumentException if both terms are null or if
+   *  lowerTerm is null and includeLower is true (similar for upperTerm
+   *  and includeUpper)
+   */
+  public SlowCollatedTermRangeFilter(String fieldName, String lowerTerm, String upperTerm,
+                     boolean includeLower, boolean includeUpper,
+                     Collator collator) {
+      super(new SlowCollatedTermRangeQuery(fieldName, lowerTerm, upperTerm, includeLower, includeUpper, collator));
+  }
+  
+  /** Returns the lower value of this range filter */
+  public String getLowerTerm() { return query.getLowerTerm(); }
+
+  /** Returns the upper value of this range filter */
+  public String getUpperTerm() { return query.getUpperTerm(); }
+  
+  /** Returns <code>true</code> if the lower endpoint is inclusive */
+  public boolean includesLower() { return query.includesLower(); }
+  
+  /** Returns <code>true</code> if the upper endpoint is inclusive */
+  public boolean includesUpper() { return query.includesUpper(); }
+
+  /** Returns the collator used to determine range inclusion, if any. */
+  public Collator getCollator() { return query.getCollator(); }
+}

Added: lucene/dev/trunk/lucene/contrib/queries/src/java/org/apache/lucene/search/SlowCollatedTermRangeQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/queries/src/java/org/apache/lucene/search/SlowCollatedTermRangeQuery.java?rev=1075210&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/contrib/queries/src/java/org/apache/lucene/search/SlowCollatedTermRangeQuery.java (added)
+++ lucene/dev/trunk/lucene/contrib/queries/src/java/org/apache/lucene/search/SlowCollatedTermRangeQuery.java Mon Feb 28 05:15:50 2011
@@ -0,0 +1,176 @@
+package org.apache.lucene.search;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.text.Collator;
+
+import org.apache.lucene.index.Terms;
+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.
+ *
+ * <p>This query matches the documents looking for terms that fall into the
+ * supplied range according to {@link
+ * String#compareTo(String)}, unless a <code>Collator</code> is provided. It is not intended
+ * for numerical ranges; use {@link NumericRangeQuery} instead.
+ *
+ * <p>This query uses the {@link
+ * MultiTermQuery#CONSTANT_SCORE_AUTO_REWRITE_DEFAULT}
+ * rewrite method.
+ * @deprecated Index collation keys with CollationKeyAnalyzer or ICUCollationKeyAnalyzer instead.
+ * This class will be removed in Lucene 5.0
+ */
+@Deprecated
+public class SlowCollatedTermRangeQuery extends MultiTermQuery {
+  private String lowerTerm;
+  private String upperTerm;
+  private boolean includeLower;
+  private boolean includeUpper;
+  private Collator collator;
+
+  /** Constructs a query selecting all terms greater/equal than
+   * <code>lowerTerm</code> but less/equal than <code>upperTerm</code>.
+   * <p>
+   * If an endpoint is null, it is said 
+   * to be "open". Either or both endpoints may be open.  Open endpoints may not 
+   * be exclusive (you can't select all but the first or last term without 
+   * explicitly specifying the term to exclude.)
+   * <p>
+   *
+   * @param lowerTerm The Term text at the lower end of the range
+   * @param upperTerm The Term text at the upper end of the range
+   * @param includeLower
+   *          If true, the <code>lowerTerm</code> is
+   *          included in the range.
+   * @param includeUpper
+   *          If true, the <code>upperTerm</code> is
+   *          included in the range.
+   * @param collator The collator to use to collate index Terms, to determine
+   *  their membership in the range bounded by <code>lowerTerm</code> and
+   *  <code>upperTerm</code>.
+   */
+  public SlowCollatedTermRangeQuery(String field, String lowerTerm, String upperTerm, 
+      boolean includeLower, boolean includeUpper,  Collator collator) {
+    super(field);
+    this.lowerTerm = lowerTerm;
+    this.upperTerm = upperTerm;
+    this.includeLower = includeLower;
+    this.includeUpper = includeUpper;
+    this.collator = collator;
+  }
+
+  /** Returns the lower value of this range query */
+  public String getLowerTerm() { return lowerTerm; }
+
+  /** Returns the upper value of this range query */
+  public String getUpperTerm() { return upperTerm; }
+  
+  /** Returns <code>true</code> if the lower endpoint is inclusive */
+  public boolean includesLower() { return includeLower; }
+  
+  /** Returns <code>true</code> if the upper endpoint is inclusive */
+  public boolean includesUpper() { return includeUpper; }
+
+  /** Returns the collator used to determine range inclusion */
+  public Collator getCollator() { return collator; }
+  
+  @Override
+  protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
+    if (lowerTerm != null && upperTerm != null && collator.compare(lowerTerm, upperTerm) > 0) {
+      return TermsEnum.EMPTY;
+    }
+    
+    TermsEnum tenum = terms.iterator();
+
+    if (lowerTerm == null && upperTerm == null) {
+      return tenum;
+    }
+    return new SlowCollatedTermRangeTermsEnum(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) {
+      StringBuilder buffer = new StringBuilder();
+      if (!getField().equals(field)) {
+          buffer.append(getField());
+          buffer.append(":");
+      }
+      buffer.append(includeLower ? '[' : '{');
+      buffer.append(lowerTerm != null ? lowerTerm : "*");
+      buffer.append(" TO ");
+      buffer.append(upperTerm != null ? upperTerm : "*");
+      buffer.append(includeUpper ? ']' : '}');
+      buffer.append(ToStringUtils.boost(getBoost()));
+      return buffer.toString();
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = super.hashCode();
+    result = prime * result + ((collator == null) ? 0 : collator.hashCode());
+    result = prime * result + (includeLower ? 1231 : 1237);
+    result = prime * result + (includeUpper ? 1231 : 1237);
+    result = prime * result + ((lowerTerm == null) ? 0 : lowerTerm.hashCode());
+    result = prime * result + ((upperTerm == null) ? 0 : upperTerm.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (!super.equals(obj))
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    SlowCollatedTermRangeQuery other = (SlowCollatedTermRangeQuery) obj;
+    if (collator == null) {
+      if (other.collator != null)
+        return false;
+    } else if (!collator.equals(other.collator))
+      return false;
+    if (includeLower != other.includeLower)
+      return false;
+    if (includeUpper != other.includeUpper)
+      return false;
+    if (lowerTerm == null) {
+      if (other.lowerTerm != null)
+        return false;
+    } else if (!lowerTerm.equals(other.lowerTerm))
+      return false;
+    if (upperTerm == null) {
+      if (other.upperTerm != null)
+        return false;
+    } else if (!upperTerm.equals(other.upperTerm))
+      return false;
+    return true;
+  }
+}

Added: lucene/dev/trunk/lucene/contrib/queries/src/java/org/apache/lucene/search/SlowCollatedTermRangeTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/queries/src/java/org/apache/lucene/search/SlowCollatedTermRangeTermsEnum.java?rev=1075210&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/contrib/queries/src/java/org/apache/lucene/search/SlowCollatedTermRangeTermsEnum.java (added)
+++ lucene/dev/trunk/lucene/contrib/queries/src/java/org/apache/lucene/search/SlowCollatedTermRangeTermsEnum.java Mon Feb 28 05:15:50 2011
@@ -0,0 +1,102 @@
+package org.apache.lucene.search;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.text.Collator;
+
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Subclass of FilteredTermEnum for enumerating all terms that match the
+ * specified range parameters.
+ * <p>Term enumerations are always ordered by
+ * {@link #getComparator}.  Each term in the enumeration is
+ * greater than all that precede it.</p>
+ * @deprecated Index collation keys with CollationKeyAnalyzer or ICUCollationKeyAnalyzer instead.
+ *  This class will be removed in Lucene 5.0
+ */
+@Deprecated
+public class SlowCollatedTermRangeTermsEnum extends FilteredTermsEnum {
+  private Collator collator;
+  private String upperTermText;
+  private String lowerTermText;
+  private boolean includeLower;
+  private boolean includeUpper;
+
+  /**
+   * Enumerates all terms greater/equal than <code>lowerTerm</code>
+   * but less/equal than <code>upperTerm</code>. 
+   * 
+   * If an endpoint is null, it is said to be "open". Either or both 
+   * endpoints may be open.  Open endpoints may not be exclusive 
+   * (you can't select all but the first or last term without 
+   * explicitly specifying the term to exclude.)
+   * 
+   * @param tenum
+   * @param lowerTermText
+   *          The term text at the lower end of the range
+   * @param upperTermText
+   *          The term text at the upper end of the range
+   * @param includeLower
+   *          If true, the <code>lowerTerm</code> is included in the range.
+   * @param includeUpper
+   *          If true, the <code>upperTerm</code> is included in the range.
+   * @param collator
+   *          The collator to use to collate index Terms, to determine their
+   *          membership in the range bounded by <code>lowerTerm</code> and
+   *          <code>upperTerm</code>.
+   * 
+   * @throws IOException
+   */
+  public SlowCollatedTermRangeTermsEnum(TermsEnum tenum, String lowerTermText, String upperTermText, 
+    boolean includeLower, boolean includeUpper, Collator collator) throws IOException {
+    super(tenum);
+    this.collator = collator;
+    this.upperTermText = upperTermText;
+    this.lowerTermText = lowerTermText;
+    this.includeLower = includeLower;
+    this.includeUpper = includeUpper;
+
+    // do a little bit of normalization...
+    // open ended range queries should always be inclusive.
+    if (this.lowerTermText == null) {
+      this.lowerTermText = "";
+      this.includeLower = true;
+    }
+
+    // TODO: optimize
+    BytesRef startBytesRef = new BytesRef("");
+    setInitialSeekTerm(startBytesRef);
+  }
+
+  @Override
+  protected AcceptStatus accept(BytesRef term) {
+    if ((includeLower
+         ? collator.compare(term.utf8ToString(), lowerTermText) >= 0
+         : collator.compare(term.utf8ToString(), lowerTermText) > 0)
+        && (upperTermText == null
+            || (includeUpper
+                ? collator.compare(term.utf8ToString(), upperTermText) <= 0
+                : collator.compare(term.utf8ToString(), upperTermText) < 0))) {
+      return AcceptStatus.YES;
+    }
+    return AcceptStatus.NO;
+  }
+}

Modified: lucene/dev/trunk/lucene/contrib/queries/src/test/org/apache/lucene/search/BooleanFilterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/queries/src/test/org/apache/lucene/search/BooleanFilterTest.java?rev=1075210&r1=1075209&r2=1075210&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/queries/src/test/org/apache/lucene/search/BooleanFilterTest.java (original)
+++ lucene/dev/trunk/lucene/contrib/queries/src/test/org/apache/lucene/search/BooleanFilterTest.java Mon Feb 28 05:15:50 2011
@@ -70,7 +70,7 @@ public class BooleanFilterTest extends L
 	
   private Filter getRangeFilter(String field,String lowerPrice, String upperPrice)
 	{
-    Filter f = new TermRangeFilter(field,lowerPrice,upperPrice,true,true);
+    Filter f = TermRangeFilter.newStringRange(field,lowerPrice,upperPrice,true,true);
     return f;
 	}
   private Filter getTermsFilter(String field,String text)

Modified: lucene/dev/trunk/lucene/contrib/queries/src/test/org/apache/lucene/search/ChainedFilterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/queries/src/test/org/apache/lucene/search/ChainedFilterTest.java?rev=1075210&r1=1075209&r2=1075210&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/queries/src/test/org/apache/lucene/search/ChainedFilterTest.java (original)
+++ lucene/dev/trunk/lucene/contrib/queries/src/test/org/apache/lucene/search/ChainedFilterTest.java Mon Feb 28 05:15:50 2011
@@ -84,7 +84,7 @@ public class ChainedFilterTest extends L
     //Date pastTheEnd = parseDate("2099 Jan 1");
     // dateFilter = DateFilter.Before("date", pastTheEnd);
     // just treat dates as strings and select the whole range for now...
-    dateFilter = new TermRangeFilter("date","","ZZZZ",true,true);
+    dateFilter = TermRangeFilter.newStringRange("date","","ZZZZ",true,true);
 
     bobFilter = new QueryWrapperFilter(
         new TermQuery(new Term("owner", "bob")));

Added: lucene/dev/trunk/lucene/contrib/queries/src/test/org/apache/lucene/search/TestSlowCollationMethods.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/queries/src/test/org/apache/lucene/search/TestSlowCollationMethods.java?rev=1075210&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/contrib/queries/src/test/org/apache/lucene/search/TestSlowCollationMethods.java (added)
+++ lucene/dev/trunk/lucene/contrib/queries/src/test/org/apache/lucene/search/TestSlowCollationMethods.java Mon Feb 28 05:15:50 2011
@@ -0,0 +1,137 @@
+package org.apache.lucene.search;
+
+import java.io.IOException;
+import java.text.Collator;
+import java.util.Locale;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Tests SlowCollatedStringComparator, SlowCollatedTermRangeQuery, and SlowCollatedTermRangeFilter
+ */
+public class TestSlowCollationMethods extends LuceneTestCase {
+  private static Collator collator;
+  private static IndexSearcher searcher;
+  private static IndexReader reader;
+  private static Directory dir;
+  private static int numDocs;
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    final Locale locale = LuceneTestCase.randomLocale(random);
+    collator = Collator.getInstance(locale);
+    collator.setStrength(Collator.IDENTICAL);
+    collator.setDecomposition(Collator.NO_DECOMPOSITION);
+
+    numDocs = 1000 * RANDOM_MULTIPLIER;
+    dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random, dir);
+    for (int i = 0; i < numDocs; i++) {
+      Document doc = new Document();
+      String value = _TestUtil.randomUnicodeString(random);
+      Field field = newField("field", value, Field.Store.YES, Field.Index.NOT_ANALYZED_NO_NORMS);
+      doc.add(field);
+      iw.addDocument(doc);
+    }
+    reader = iw.getReader();
+    iw.close();
+
+    // TODO: we should be able to use newSearcher, but custom sorts are broken if IS has an executorservice
+    // see LUCENE-2941
+    //searcher = newSearcher(reader);
+    searcher = new IndexSearcher(reader); 
+  }
+  
+  @AfterClass
+  public static void afterClass() throws Exception {
+    searcher.close();
+    reader.close();
+    dir.close();
+    collator = null;
+    searcher = null;
+    reader = null;
+    dir = null;
+  }
+  
+  public void testSort() throws Exception {
+    SortField sf = new SortField("field", new FieldComparatorSource() {
+      @Override
+      public FieldComparator newComparator(String fieldname, int numHits, int sortPos, boolean reversed) throws IOException {
+        return new SlowCollatedStringComparator(numHits, fieldname, collator);
+      }
+    });
+    TopFieldDocs docs = searcher.search(new MatchAllDocsQuery(), null, numDocs, new Sort(sf));
+    String prev = "";
+    for (ScoreDoc doc : docs.scoreDocs) {
+      String value = reader.document(doc.doc).get("field");
+      assertTrue(collator.compare(value, prev) >= 0);
+      prev = value;
+    }
+  }
+  
+  private void doTestRanges(String startPoint, String endPoint, Query query) throws Exception {
+    // positive test
+    TopDocs docs = searcher.search(query, numDocs);
+    for (ScoreDoc doc : docs.scoreDocs) {
+      String value = reader.document(doc.doc).get("field");
+      assertTrue(collator.compare(value, startPoint) >= 0);
+      assertTrue(collator.compare(value, endPoint) <= 0);
+    }
+    
+    // negative test
+    BooleanQuery bq = new BooleanQuery();
+    bq.add(new MatchAllDocsQuery(), Occur.SHOULD);
+    bq.add(query, Occur.MUST_NOT);
+    docs = searcher.search(bq, numDocs);
+    for (ScoreDoc doc : docs.scoreDocs) {
+      String value = reader.document(doc.doc).get("field");
+      assertTrue(collator.compare(value, startPoint) < 0 || collator.compare(value, endPoint) > 0);
+    }
+  }
+  
+  public void testRangeQuery() throws Exception {
+    int numQueries = 50*RANDOM_MULTIPLIER;
+    for (int i = 0; i < numQueries; i++) {
+      String startPoint = _TestUtil.randomUnicodeString(random);
+      String endPoint = _TestUtil.randomUnicodeString(random);
+      Query query = new SlowCollatedTermRangeQuery("field", startPoint, endPoint, true, true, collator);
+      doTestRanges(startPoint, endPoint, query);
+    }
+  }
+  
+  public void testRangeFilter() throws Exception {
+    int numQueries = 50*RANDOM_MULTIPLIER;
+    for (int i = 0; i < numQueries; i++) {
+      String startPoint = _TestUtil.randomUnicodeString(random);
+      String endPoint = _TestUtil.randomUnicodeString(random);
+      Query query = new ConstantScoreQuery(new SlowCollatedTermRangeFilter("field", startPoint, endPoint, true, true, collator));
+      doTestRanges(startPoint, endPoint, query);
+    }
+  }
+}

Modified: lucene/dev/trunk/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/complexPhrase/ComplexPhraseQueryParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/complexPhrase/ComplexPhraseQueryParser.java?rev=1075210&r1=1075209&r2=1075210&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/complexPhrase/ComplexPhraseQueryParser.java (original)
+++ lucene/dev/trunk/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/complexPhrase/ComplexPhraseQueryParser.java Mon Feb 28 05:15:50 2011
@@ -174,8 +174,7 @@ public class ComplexPhraseQueryParser ex
     if (isPass2ResolvingPhrases) {
       // Must use old-style RangeQuery in order to produce a BooleanQuery
       // that can be turned into SpanOr clause
-      TermRangeQuery rangeQuery = new TermRangeQuery(field, part1, part2, startInclusive, endInclusive,
-          getRangeCollator());
+      TermRangeQuery rangeQuery = TermRangeQuery.newStringRange(field, part1, part2, startInclusive, endInclusive);
       rangeQuery.setRewriteMethod(MultiTermQuery.SCORING_BOOLEAN_QUERY_REWRITE);
       return rangeQuery;
     }

Modified: lucene/dev/trunk/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/StandardQueryParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/StandardQueryParser.java?rev=1075210&r1=1075209&r2=1075210&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/StandardQueryParser.java (original)
+++ lucene/dev/trunk/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/StandardQueryParser.java Mon Feb 28 05:15:50 2011
@@ -17,7 +17,6 @@ package org.apache.lucene.queryParser.st
  * limitations under the License.
  */
 
-import java.text.Collator;
 import java.util.Locale;
 import java.util.Map;
 import java.util.TooManyListenersException;
@@ -41,10 +40,8 @@ import org.apache.lucene.queryParser.sta
 import org.apache.lucene.queryParser.standard.config.MultiFieldAttribute;
 import org.apache.lucene.queryParser.standard.config.MultiTermRewriteMethodAttribute;
 import org.apache.lucene.queryParser.standard.config.PositionIncrementsAttribute;
-import org.apache.lucene.queryParser.standard.config.RangeCollatorAttribute;
 import org.apache.lucene.queryParser.standard.config.StandardQueryConfigHandler;
 import org.apache.lucene.queryParser.standard.config.DefaultOperatorAttribute.Operator;
-import org.apache.lucene.queryParser.standard.nodes.RangeQueryNode;
 import org.apache.lucene.queryParser.standard.parser.StandardSyntaxParser;
 import org.apache.lucene.queryParser.standard.processors.StandardQueryNodeProcessorPipeline;
 import org.apache.lucene.search.FuzzyQuery;
@@ -188,32 +185,6 @@ public class StandardQueryParser extends
   }
 
   /**
-   * Sets the collator used to determine index term inclusion in ranges for
-   * RangeQuerys.
-   * <p/>
-   * <strong>WARNING:</strong> Setting the rangeCollator to a non-null collator
-   * using this method will cause every single index Term in the Field
-   * referenced by lowerTerm and/or upperTerm to be examined. Depending on the
-   * number of index Terms in this Field, the operation could be very slow.
-   * 
-   * @param collator
-   *          the collator to use when constructing {@link RangeQueryNode}s
-   */
-  public void setRangeCollator(Collator collator) {
-    RangeCollatorAttribute attr = getQueryConfigHandler().getAttribute(RangeCollatorAttribute.class);
-    attr.setDateResolution(collator);
-  }
-
-  /**
-   * @return the collator used to determine index term inclusion in ranges for
-   *         RangeQuerys.
-   */
-  public Collator getRangeCollator() {
-    RangeCollatorAttribute attr = getQueryConfigHandler().getAttribute(RangeCollatorAttribute.class);
-    return attr.getRangeCollator();
-  }
-
-  /**
    * Sets the boolean operator of the QueryParser. In default mode (
    * {@link Operator#OR}) terms without any modifiers are considered optional:
    * for example <code>capital of Hungary</code> is equal to

Modified: lucene/dev/trunk/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/builders/RangeQueryNodeBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/builders/RangeQueryNodeBuilder.java?rev=1075210&r1=1075209&r2=1075210&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/builders/RangeQueryNodeBuilder.java (original)
+++ lucene/dev/trunk/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/builders/RangeQueryNodeBuilder.java Mon Feb 28 05:15:50 2011
@@ -53,9 +53,7 @@ public class RangeQueryNodeBuilder imple
 
     String field = rangeNode.getField().toString();
 
-    TermRangeQuery rangeQuery = new TermRangeQuery(field, lower
-        .getTextAsString(), upper.getTextAsString(), lowerInclusive,
-        upperInclusive, rangeNode.getCollator());
+    TermRangeQuery rangeQuery = TermRangeQuery.newStringRange(field, lower.getTextAsString(), upper.getTextAsString(), lowerInclusive, upperInclusive);
     
     MultiTermQuery.RewriteMethod method = (MultiTermQuery.RewriteMethod)queryNode.getTag(MultiTermRewriteMethodAttribute.TAG_ID);
     if (method != null) {

Modified: lucene/dev/trunk/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/config/StandardQueryConfigHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/config/StandardQueryConfigHandler.java?rev=1075210&r1=1075209&r2=1075210&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/config/StandardQueryConfigHandler.java (original)
+++ lucene/dev/trunk/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/config/StandardQueryConfigHandler.java Mon Feb 28 05:15:50 2011
@@ -38,7 +38,6 @@ public class StandardQueryConfigHandler 
     addFieldConfigListener(new FieldDateResolutionFCListener(this));
 
     // Default Values
-    addAttribute(RangeCollatorAttribute.class);
     addAttribute(DefaultOperatorAttribute.class);
     addAttribute(AnalyzerAttribute.class);
     addAttribute(FuzzyAttribute.class);

Modified: lucene/dev/trunk/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/nodes/RangeQueryNode.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/nodes/RangeQueryNode.java?rev=1075210&r1=1075209&r2=1075210&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/nodes/RangeQueryNode.java (original)
+++ lucene/dev/trunk/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/nodes/RangeQueryNode.java Mon Feb 28 05:15:50 2011
@@ -17,34 +17,24 @@ package org.apache.lucene.queryParser.st
  * limitations under the License.
  */
 
-import java.text.Collator;
-
 import org.apache.lucene.queryParser.core.nodes.ParametricQueryNode;
 import org.apache.lucene.queryParser.core.nodes.ParametricRangeQueryNode;
-import org.apache.lucene.queryParser.standard.config.RangeCollatorAttribute;
 import org.apache.lucene.queryParser.standard.processors.ParametricRangeQueryNodeProcessor;
 
 /**
- * This query node represents a range query. It also holds which collator will
- * be used by the range query and if the constant score rewrite is enabled. <br/>
+ * This query node represents a range query. 
  * 
  * @see ParametricRangeQueryNodeProcessor
- * @see RangeCollatorAttribute
  * @see org.apache.lucene.search.TermRangeQuery
  */
 public class RangeQueryNode extends ParametricRangeQueryNode {
 
-  private Collator collator;
-
   /**
    * @param lower
    * @param upper
    */
-  public RangeQueryNode(ParametricQueryNode lower, ParametricQueryNode upper, Collator collator) {
+  public RangeQueryNode(ParametricQueryNode lower, ParametricQueryNode upper) {
     super(lower, upper);
-
-    this.collator = collator;
-
   }
 
   @Override
@@ -57,12 +47,4 @@ public class RangeQueryNode extends Para
     return sb.toString();
 
   }
-
-  /**
-   * @return the collator
-   */
-  public Collator getCollator() {
-    return this.collator;
-  }
-
 }

Modified: lucene/dev/trunk/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/processors/ParametricRangeQueryNodeProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/processors/ParametricRangeQueryNodeProcessor.java?rev=1075210&r1=1075209&r2=1075210&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/processors/ParametricRangeQueryNodeProcessor.java (original)
+++ lucene/dev/trunk/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/processors/ParametricRangeQueryNodeProcessor.java Mon Feb 28 05:15:50 2011
@@ -17,7 +17,6 @@ package org.apache.lucene.queryParser.st
  * limitations under the License.
  */
 
-import java.text.Collator;
 import java.text.DateFormat;
 import java.util.Calendar;
 import java.util.Date;
@@ -36,7 +35,6 @@ import org.apache.lucene.queryParser.cor
 import org.apache.lucene.queryParser.core.processors.QueryNodeProcessorImpl;
 import org.apache.lucene.queryParser.standard.config.DateResolutionAttribute;
 import org.apache.lucene.queryParser.standard.config.LocaleAttribute;
-import org.apache.lucene.queryParser.standard.config.RangeCollatorAttribute;
 import org.apache.lucene.queryParser.standard.nodes.RangeQueryNode;
 
 /**
@@ -54,12 +52,7 @@ import org.apache.lucene.queryParser.sta
  * If a {@link DateResolutionAttribute} is defined and the {@link Resolution} is
  * not <code>null</code> it will also be used to parse the date value. <br/>
  * <br/>
- * This processor will also try to retrieve a {@link RangeCollatorAttribute}
- * from the {@link QueryConfigHandler}. If a {@link RangeCollatorAttribute} is
- * found and the {@link Collator} is not <code>null</code>, it's set on the
- * {@link RangeQueryNode}. <br/>
  * 
- * @see RangeCollatorAttribute
  * @see DateResolutionAttribute
  * @see LocaleAttribute
  * @see RangeQueryNode
@@ -79,17 +72,9 @@ public class ParametricRangeQueryNodePro
       ParametricQueryNode upper = parametricRangeNode.getUpperBound();
       ParametricQueryNode lower = parametricRangeNode.getLowerBound();
       Locale locale = Locale.getDefault();
-      Collator collator = null;
       DateTools.Resolution dateRes = null;
       boolean inclusive = false;
 
-      if (getQueryConfigHandler().hasAttribute(RangeCollatorAttribute.class)) {
-
-        collator = getQueryConfigHandler().getAttribute(
-            RangeCollatorAttribute.class).getRangeCollator();
-
-      }
-
       if (getQueryConfigHandler().hasAttribute(LocaleAttribute.class)) {
 
         locale = getQueryConfigHandler().getAttribute(LocaleAttribute.class)
@@ -155,7 +140,7 @@ public class ParametricRangeQueryNodePro
       lower.setText(part1);
       upper.setText(part2);
 
-      return new RangeQueryNode(lower, upper, collator);
+      return new RangeQueryNode(lower, upper);
 
     }
 

Modified: lucene/dev/trunk/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/standard/TestQPHelper.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/standard/TestQPHelper.java?rev=1075210&r1=1075209&r2=1075210&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/standard/TestQPHelper.java (original)
+++ lucene/dev/trunk/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/standard/TestQPHelper.java Mon Feb 28 05:15:50 2011
@@ -642,55 +642,6 @@ public class TestQPHelper extends Lucene
         "gack (bar blar {a TO z})");
   }
 
-  public void testFarsiRangeCollating() throws Exception {
-    Directory ramDir = newDirectory();
-    IndexWriter iw = new IndexWriter(ramDir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(MockTokenizer.WHITESPACE, false)));
-    Document doc = new Document();
-    doc.add(newField("content", "\u0633\u0627\u0628", Field.Store.YES,
-        Field.Index.NOT_ANALYZED));
-    iw.addDocument(doc);
-    iw.close();
-    IndexSearcher is = new IndexSearcher(ramDir, true);
-
-    StandardQueryParser qp = new StandardQueryParser();
-    qp.setAnalyzer(new MockAnalyzer(MockTokenizer.WHITESPACE, false));
-
-    // Neither Java 1.4.2 nor 1.5.0 has Farsi Locale collation available in
-    // RuleBasedCollator. However, the Arabic Locale seems to order the
-    // Farsi
-    // characters properly.
-    Collator c = Collator.getInstance(new Locale("ar"));
-    qp.setRangeCollator(c);
-
-    // Unicode order would include U+0633 in [ U+062F - U+0698 ], but Farsi
-    // orders the U+0698 character before the U+0633 character, so the
-    // single
-    // index Term below should NOT be returned by a ConstantScoreRangeQuery
-    // with a Farsi Collator (or an Arabic one for the case when Farsi is
-    // not
-    // supported).
-
-    // Test ConstantScoreRangeQuery
-    qp.setMultiTermRewriteMethod(MultiTermQuery.CONSTANT_SCORE_FILTER_REWRITE);
-    ScoreDoc[] result = is.search(qp.parse("[ \u062F TO \u0698 ]", "content"),
-        null, 1000).scoreDocs;
-    assertEquals("The index Term should not be included.", 0, result.length);
-
-    result = is.search(qp.parse("[ \u0633 TO \u0638 ]", "content"), null, 1000).scoreDocs;
-    assertEquals("The index Term should be included.", 1, result.length);
-
-    // Test RangeQuery
-    qp.setMultiTermRewriteMethod(MultiTermQuery.SCORING_BOOLEAN_QUERY_REWRITE);
-    result = is.search(qp.parse("[ \u062F TO \u0698 ]", "content"), null, 1000).scoreDocs;
-    assertEquals("The index Term should not be included.", 0, result.length);
-
-    result = is.search(qp.parse("[ \u0633 TO \u0638 ]", "content"), null, 1000).scoreDocs;
-    assertEquals("The index Term should be included.", 1, result.length);
-
-    is.close();
-    ramDir.close();
-  }
-
   /** for testing DateTools support */
   private String getDate(String s, DateTools.Resolution resolution)
       throws Exception {

Modified: lucene/dev/trunk/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/standard/config/TestAttributes.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/standard/config/TestAttributes.java?rev=1075210&r1=1075209&r2=1075210&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/standard/config/TestAttributes.java (original)
+++ lucene/dev/trunk/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/standard/config/TestAttributes.java Mon Feb 28 05:15:50 2011
@@ -60,8 +60,6 @@ public class TestAttributes extends Luce
       Collections.singletonMap(MultiTermRewriteMethodAttribute.class.getName()+"#multiTermRewriteMethod", MultiTermQuery.CONSTANT_SCORE_AUTO_REWRITE_DEFAULT));
     _TestUtil.assertAttributeReflection(new PositionIncrementsAttributeImpl(),
       Collections.singletonMap(PositionIncrementsAttribute.class.getName()+"#positionIncrementsEnabled", false));
-    _TestUtil.assertAttributeReflection(new RangeCollatorAttributeImpl(),
-      Collections.singletonMap(RangeCollatorAttribute.class.getName()+"#rangeCollator", null));
   }
 
 }

Modified: lucene/dev/trunk/lucene/contrib/xml-query-parser/src/java/org/apache/lucene/xmlparser/builders/RangeFilterBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/xml-query-parser/src/java/org/apache/lucene/xmlparser/builders/RangeFilterBuilder.java?rev=1075210&r1=1075209&r2=1075210&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/xml-query-parser/src/java/org/apache/lucene/xmlparser/builders/RangeFilterBuilder.java (original)
+++ lucene/dev/trunk/lucene/contrib/xml-query-parser/src/java/org/apache/lucene/xmlparser/builders/RangeFilterBuilder.java Mon Feb 28 05:15:50 2011
@@ -41,7 +41,7 @@ public class RangeFilterBuilder implemen
 		String upperTerm=e.getAttribute("upperTerm");
 		boolean includeLower=DOMUtils.getAttribute(e,"includeLower",true);
 		boolean includeUpper=DOMUtils.getAttribute(e,"includeUpper",true);
-		return new TermRangeFilter(fieldName,lowerTerm,upperTerm,includeLower,includeUpper);
+		return TermRangeFilter.newStringRange(fieldName,lowerTerm,upperTerm,includeLower,includeUpper);
 	}
 
 }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/analysis/tokenattributes/CharTermAttributeImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/analysis/tokenattributes/CharTermAttributeImpl.java?rev=1075210&r1=1075209&r2=1075210&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/analysis/tokenattributes/CharTermAttributeImpl.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/analysis/tokenattributes/CharTermAttributeImpl.java Mon Feb 28 05:15:50 2011
@@ -77,7 +77,7 @@ public class CharTermAttributeImpl exten
   }
   
   // *** TermToBytesRefAttribute interface ***
-  public final int toBytesRef(BytesRef target) {
+  public int toBytesRef(BytesRef target) {
     return UnicodeUtil.UTF16toUTF8WithHash(termBuffer, 0, termLength, target);
   }
   

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/queryParser/QueryParserBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/queryParser/QueryParserBase.java?rev=1075210&r1=1075209&r2=1075210&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/queryParser/QueryParserBase.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/queryParser/QueryParserBase.java Mon Feb 28 05:15:50 2011
@@ -78,9 +78,9 @@ public abstract class QueryParserBase {
   // maps field names to date resolutions
   Map<String,DateTools.Resolution> fieldToDateResolution = null;
 
-  // The collator to use when determining range inclusion,
-  // for use when constructing RangeQuerys.
-  Collator rangeCollator = null;
+  //Whether or not to analyze range terms when constructing RangeQuerys
+  // (For example, analyzing terms into collation keys for locale-sensitive RangeQuery)
+  boolean analyzeRangeTerms = false;
 
   boolean autoGeneratePhraseQueries;
 
@@ -391,27 +391,21 @@ public abstract class QueryParserBase {
   }
 
   /**
-   * Sets the collator used to determine index term inclusion in ranges
-   * for RangeQuerys.
-   * <p/>
-   * <strong>WARNING:</strong> Setting the rangeCollator to a non-null
-   * collator using this method will cause every single index Term in the
-   * Field referenced by lowerTerm and/or upperTerm to be examined.
-   * Depending on the number of index Terms in this Field, the operation could
-   * be very slow.
-   *
-   *  @param rc  the collator to use when constructing RangeQuerys
+   * Set whether or not to analyze range terms when constructing RangeQuerys.
+   * For example, setting this to true can enable analyzing terms into 
+   * collation keys for locale-sensitive RangeQuery.
+   * 
+   * @param analyzeRangeTerms whether or not terms should be analyzed for RangeQuerys
    */
-  public void setRangeCollator(Collator rc) {
-    rangeCollator = rc;
+  public void setAnalyzeRangeTerms(boolean analyzeRangeTerms) {
+    this.analyzeRangeTerms = analyzeRangeTerms;
   }
 
   /**
-   * @return the collator used to determine index term inclusion in ranges
-   * for RangeQuerys.
+   * @return whether or not to analyze range terms when constructing RangeQuerys.
    */
-  public Collator getRangeCollator() {
-    return rangeCollator;
+  public boolean getAnalyzeRangeTerms() {
+    return analyzeRangeTerms;
   }
 
   protected void addClause(List<BooleanClause> clauses, int conj, int mods, Query q) {
@@ -792,6 +786,36 @@ public abstract class QueryParserBase {
     return new FuzzyQuery(term,minimumSimilarity,prefixLength);
   }
 
+  private BytesRef analyzeRangePart(String field, String part) {
+    TokenStream source;
+      
+    try {
+      source = analyzer.reusableTokenStream(field, new StringReader(part));
+      source.reset();
+    } catch (IOException e) {
+      source = analyzer.tokenStream(field, new StringReader(part));
+    }
+      
+    BytesRef result = new BytesRef();
+    TermToBytesRefAttribute termAtt = source.getAttribute(TermToBytesRefAttribute.class);
+      
+    try {
+      if (!source.incrementToken())
+        throw new IllegalArgumentException("analyzer returned no terms for range part: " + part);
+      termAtt.toBytesRef(result);
+      if (source.incrementToken())
+        throw new IllegalArgumentException("analyzer returned too many terms for range part: " + part);
+    } catch (IOException e) {
+      throw new RuntimeException("error analyzing range part: " + part, e);
+    }
+      
+    try {
+      source.close();
+    } catch (IOException ignored) {}
+      
+    return result;
+  }
+
   /**
    * Builds a new TermRangeQuery instance
    * @param field Field
@@ -802,7 +826,23 @@ public abstract class QueryParserBase {
    * @return new TermRangeQuery instance
    */
   protected Query newRangeQuery(String field, String part1, String part2, boolean startInclusive, boolean endInclusive) {
-    final TermRangeQuery query = new TermRangeQuery(field, part1, part2, startInclusive, endInclusive, rangeCollator);
+    final BytesRef start;
+    final BytesRef end;
+     
+    if (part1 == null) {
+      start = null;
+    } else {
+      start = analyzeRangeTerms ? analyzeRangePart(field, part1) : new BytesRef(part1);
+    }
+     
+    if (part2 == null) {
+      end = null;
+    } else {
+      end = analyzeRangeTerms ? analyzeRangePart(field, part2) : new BytesRef(part2);
+    }
+      
+    final TermRangeQuery query = new TermRangeQuery(field, start, end, startInclusive, endInclusive);
+
     query.setRewriteMethod(multiTermRewriteMethod);
     return query;
   }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldComparator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldComparator.java?rev=1075210&r1=1075209&r2=1075210&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldComparator.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldComparator.java Mon Feb 28 05:15:50 2011
@@ -18,8 +18,6 @@ package org.apache.lucene.search;
  */
 
 import java.io.IOException;
-import java.text.Collator;
-import java.util.Locale;
 
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.search.FieldCache.DocTermsIndex;
@@ -718,85 +716,6 @@ public abstract class FieldComparator {
     }
   }
   
-  
-  /** Sorts by a field's value using the Collator for a
-   *  given Locale.
-   *
-   * <p><b>WARNING</b>: this is likely very slow; you'll
-   * get much better performance using the
-   * CollationKeyAnalyzer or ICUCollationKeyAnalyzer. */
-  public static final class StringComparatorLocale extends FieldComparator {
-
-    private final String[] values;
-    private DocTerms currentDocTerms;
-    private final String field;
-    final Collator collator;
-    private String bottom;
-    private final BytesRef tempBR = new BytesRef();
-
-    StringComparatorLocale(int numHits, String field, Locale locale) {
-      values = new String[numHits];
-      this.field = field;
-      collator = Collator.getInstance(locale);
-    }
-
-    @Override
-    public int compare(int slot1, int slot2) {
-      final String val1 = values[slot1];
-      final String val2 = values[slot2];
-      if (val1 == null) {
-        if (val2 == null) {
-          return 0;
-        }
-        return -1;
-      } else if (val2 == null) {
-        return 1;
-      }
-      return collator.compare(val1, val2);
-    }
-
-    @Override
-    public int compareBottom(int doc) {
-      final String val2 = currentDocTerms.getTerm(doc, tempBR).utf8ToString();
-      if (bottom == null) {
-        if (val2 == null) {
-          return 0;
-        }
-        return -1;
-      } else if (val2 == null) {
-        return 1;
-      }
-      return collator.compare(bottom, val2);
-    }
-
-    @Override
-    public void copy(int slot, int doc) {
-      final BytesRef br = currentDocTerms.getTerm(doc, tempBR);
-      if (br == null) {
-        values[slot] = null;
-      } else {
-        values[slot] = br.utf8ToString();
-      }
-    }
-
-    @Override
-    public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
-      currentDocTerms = FieldCache.DEFAULT.getTerms(context.reader, field);
-      return this;
-    }
-    
-    @Override
-    public void setBottom(final int bottom) {
-      this.bottom = values[bottom];
-    }
-
-    @Override
-    public Comparable<?> value(int slot) {
-      final String s = values[slot];
-      return s == null ? null : new BytesRef(values[slot]);
-    }
-  }
-
   /** Sorts by field's natural Term sort order, using
    *  ordinals.  This is functionally equivalent to {@link
    *  TermValComparator}, but it first resolves the string

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldDocSortedHitQueue.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldDocSortedHitQueue.java?rev=1075210&r1=1075209&r2=1075210&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldDocSortedHitQueue.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldDocSortedHitQueue.java Mon Feb 28 05:15:50 2011
@@ -20,9 +20,6 @@ package org.apache.lucene.search;
 import org.apache.lucene.util.PriorityQueue;
 import org.apache.lucene.util.BytesRef;
 
-import java.text.Collator;
-import java.util.Locale;
-
 /**
  * Expert: Collects sorted results from Searchable's and collates them.
  * The elements put into this queue must be of type FieldDoc.
@@ -35,11 +32,6 @@ class FieldDocSortedHitQueue extends Pri
 
   volatile SortField[] fields = null;
 
-  // used in the case where the fields are sorted by locale
-  // based strings
-  volatile Collator[] collators = null;
-
-
   /**
    * Creates a hit queue sorted by the given list of fields.
    * @param fields Fieldable names, in priority order (highest priority first).
@@ -60,7 +52,6 @@ class FieldDocSortedHitQueue extends Pri
    */
   void setFields (SortField[] fields) {
     this.fields = fields;
-    this.collators = hasCollators (fields);
   }
 
 
@@ -69,24 +60,6 @@ class FieldDocSortedHitQueue extends Pri
     return fields;
   }
 
-
-  /** Returns an array of collators, possibly <code>null</code>.  The collators
-   * correspond to any SortFields which were given a specific locale.
-   * @param fields Array of sort fields.
-   * @return Array, possibly <code>null</code>.
-   */
-  private Collator[] hasCollators (final SortField[] fields) {
-    if (fields == null) return null;
-    Collator[] ret = new Collator[fields.length];
-    for (int i=0; i<fields.length; ++i) {
-      Locale locale = fields[i].getLocale();
-      if (locale != null)
-        ret[i] = Collator.getInstance (locale);
-    }
-    return ret;
-  }
-
-
   /**
    * Returns whether <code>a</code> is less relevant than <code>b</code>.
    * @param a ScoreDoc
@@ -109,11 +82,9 @@ class FieldDocSortedHitQueue extends Pri
           c = (s2 == null) ? 0 : -1;
         } else if (s2 == null) {
           c = 1;
-        } else if (fields[i].getLocale() == null) {
-          c = s1.compareTo(s2);
         } else {
-          c = collators[i].compare(s1.utf8ToString(), s2.utf8ToString());
-        }
+          c = s1.compareTo(s2);
+        } 
       } else {
         c = docA.fields[i].compareTo(docB.fields[i]);
         if (type == SortField.SCORE) {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/SortField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/SortField.java?rev=1075210&r1=1075209&r2=1075210&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/SortField.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/SortField.java Mon Feb 28 05:15:50 2011
@@ -18,7 +18,6 @@ package org.apache.lucene.search;
  */
 
 import java.io.IOException;
-import java.util.Locale;
 
 import org.apache.lucene.search.cache.*;
 import org.apache.lucene.util.StringHelper;
@@ -90,7 +89,6 @@ public class SortField {
 
   private String field;
   private int type;  // defaults to determining type dynamically
-  private Locale locale;    // defaults to "natural order" (no Locale)
   boolean reverse = false;  // defaults to natural order
   private CachedArrayCreator<?> creator;
   public Object missingValue = null; // used for 'sortMissingFirst/Last'
@@ -213,28 +211,6 @@ public class SortField {
     }
     return this;
   }
-  
-
-  /** Creates a sort by terms in the given field sorted
-   * according to the given locale.
-   * @param field  Name of field to sort by, cannot be <code>null</code>.
-   * @param locale Locale of values in the field.
-   */
-  public SortField (String field, Locale locale) {
-    initFieldType(field, STRING);
-    this.locale = locale;
-  }
-
-  /** Creates a sort, possibly in reverse, by terms in the given field sorted
-   * according to the given locale.
-   * @param field  Name of field to sort by, cannot be <code>null</code>.
-   * @param locale Locale of values in the field.
-   */
-  public SortField (String field, Locale locale, boolean reverse) {
-    initFieldType(field, STRING);
-    this.locale = locale;
-    this.reverse = reverse;
-  }
 
   /** Creates a sort with a custom comparison function.
    * @param field Name of field to sort by; cannot be <code>null</code>.
@@ -295,14 +271,6 @@ public class SortField {
     return type;
   }
 
-  /** Returns the Locale by which term values are interpreted.
-   * May return <code>null</code> if no Locale was specified.
-   * @return Locale, or <code>null</code>.
-   */
-  public Locale getLocale() {
-    return locale;
-  }
-
   /** 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>.
@@ -384,7 +352,6 @@ public class SortField {
         break;
     }
 
-    if (locale != null) buffer.append('(').append(locale).append(')');
     if (creator != null) buffer.append('(').append(creator).append(')');
     if (reverse) buffer.append('!');
 
@@ -404,7 +371,6 @@ public class SortField {
       other.field == this.field // field is always interned
       && other.type == this.type
       && other.reverse == this.reverse
-      && (other.locale == null ? this.locale == null : other.locale.equals(this.locale))
       && (other.comparatorSource == null ? this.comparatorSource == null : other.comparatorSource.equals(this.comparatorSource))
       && (other.creator == null ? this.creator == null : other.creator.equals(this.creator))
     );
@@ -419,7 +385,6 @@ public class SortField {
   public int hashCode() {
     int hash=type^0x346565dd + Boolean.valueOf(reverse).hashCode()^0xaf5998bb;
     if (field != null) hash += field.hashCode()^0xff5685dd;
-    if (locale != null) hash += locale.hashCode()^0x08150815;
     if (comparatorSource != null) hash += comparatorSource.hashCode();
     if (creator != null) hash += creator.hashCode()^0x3aaf56ff;
     return hash;
@@ -439,13 +404,6 @@ public class SortField {
    */
   public FieldComparator getComparator(final int numHits, final int sortPos) throws IOException {
 
-    if (locale != null) {
-      // TODO: it'd be nice to allow FieldCache.getStringIndex
-      // to optionally accept a Locale so sorting could then use
-      // the faster StringComparator impls
-      return new FieldComparator.StringComparatorLocale(numHits, field, locale);
-    }
-
     switch (type) {
     case SortField.SCORE:
       return new FieldComparator.RelevanceComparator(numHits);

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermRangeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermRangeFilter.java?rev=1075210&r1=1075209&r2=1075210&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermRangeFilter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermRangeFilter.java Mon Feb 28 05:15:50 2011
@@ -1,5 +1,7 @@
 package org.apache.lucene.search;
 
+import org.apache.lucene.util.BytesRef;
+
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -17,15 +19,13 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
-import java.text.Collator;
-
 /**
  * A Filter that restricts search results to a range of term
  * values in a given field.
  *
  * <p>This filter matches the documents looking for terms that fall into the
  * supplied range according to {@link
- * String#compareTo(String)}, unless a <code>Collator</code> is provided. It is not intended
+ * Byte#compareTo(Byte)},  It is not intended
  * for numerical ranges; use {@link NumericRangeFilter} instead.
  *
  * <p>If you construct a large number of range filters with different ranges but on the 
@@ -44,39 +44,25 @@ public class TermRangeFilter extends Mul
    *  lowerTerm is null and includeLower is true (similar for upperTerm
    *  and includeUpper)
    */
-  public TermRangeFilter(String fieldName, String lowerTerm, String upperTerm,
+  public TermRangeFilter(String fieldName, BytesRef lowerTerm, BytesRef upperTerm,
                      boolean includeLower, boolean includeUpper) {
       super(new TermRangeQuery(fieldName, lowerTerm, upperTerm, includeLower, includeUpper));
   }
 
   /**
-   * <strong>WARNING:</strong> Using this constructor and supplying a non-null
-   * value in the <code>collator</code> parameter will cause every single 
-   * index Term in the Field referenced by lowerTerm and/or upperTerm to be
-   * examined.  Depending on the number of index Terms in this Field, the 
-   * operation could be very slow.
-   *
-   * @param lowerTerm The lower bound on this range
-   * @param upperTerm The upper bound on this range
-   * @param includeLower Does this range include the lower bound?
-   * @param includeUpper Does this range include the upper bound?
-   * @param collator The collator to use when determining range inclusion; set
-   *  to null to use Unicode code point ordering instead of collation.
-   * @throws IllegalArgumentException if both terms are null or if
-   *  lowerTerm is null and includeLower is true (similar for upperTerm
-   *  and includeUpper)
+   * Factory that creates a new TermRangeFilter using Strings for term text.
    */
-  public TermRangeFilter(String fieldName, String lowerTerm, String upperTerm,
-                     boolean includeLower, boolean includeUpper,
-                     Collator collator) {
-      super(new TermRangeQuery(fieldName, lowerTerm, upperTerm, includeLower, includeUpper, collator));
+  public static TermRangeFilter newStringRange(String field, String lowerTerm, String upperTerm, boolean includeLower, boolean includeUpper) {
+    BytesRef lower = lowerTerm == null ? null : new BytesRef(lowerTerm);
+    BytesRef upper = upperTerm == null ? null : new BytesRef(upperTerm);
+    return new TermRangeFilter(field, lower, upper, includeLower, includeUpper);
   }
-
+  
   /**
    * Constructs a filter for field <code>fieldName</code> matching
    * less than or equal to <code>upperTerm</code>.
    */
-  public static TermRangeFilter Less(String fieldName, String upperTerm) {
+  public static TermRangeFilter Less(String fieldName, BytesRef upperTerm) {
       return new TermRangeFilter(fieldName, null, upperTerm, false, true);
   }
 
@@ -84,22 +70,19 @@ public class TermRangeFilter extends Mul
    * Constructs a filter for field <code>fieldName</code> matching
    * greater than or equal to <code>lowerTerm</code>.
    */
-  public static TermRangeFilter More(String fieldName, String lowerTerm) {
+  public static TermRangeFilter More(String fieldName, BytesRef lowerTerm) {
       return new TermRangeFilter(fieldName, lowerTerm, null, true, false);
   }
   
   /** Returns the lower value of this range filter */
-  public String getLowerTerm() { return query.getLowerTerm(); }
+  public BytesRef getLowerTerm() { return query.getLowerTerm(); }
 
   /** Returns the upper value of this range filter */
-  public String getUpperTerm() { return query.getUpperTerm(); }
+  public BytesRef getUpperTerm() { return query.getUpperTerm(); }
   
   /** Returns <code>true</code> if the lower endpoint is inclusive */
   public boolean includesLower() { return query.includesLower(); }
   
   /** Returns <code>true</code> if the upper endpoint is inclusive */
   public boolean includesUpper() { return query.includesUpper(); }
-
-  /** Returns the collator used to determine range inclusion, if any. */
-  public Collator getCollator() { return query.getCollator(); }
 }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermRangeQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermRangeQuery.java?rev=1075210&r1=1075209&r2=1075210&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermRangeQuery.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermRangeQuery.java Mon Feb 28 05:15:50 2011
@@ -18,11 +18,11 @@ package org.apache.lucene.search;
  */
 
 import java.io.IOException;
-import java.text.Collator;
 
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.util.AttributeSource;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.ToStringUtils;
 
 /**
@@ -30,7 +30,7 @@ import org.apache.lucene.util.ToStringUt
  *
  * <p>This query matches the documents looking for terms that fall into the
  * supplied range according to {@link
- * String#compareTo(String)}, unless a <code>Collator</code> is provided. It is not intended
+ * Byte#compareTo(Byte)}. It is not intended
  * for numerical ranges; use {@link NumericRangeQuery} instead.
  *
  * <p>This query uses the {@link
@@ -40,9 +40,8 @@ import org.apache.lucene.util.ToStringUt
  */
 
 public class TermRangeQuery extends MultiTermQuery {
-  private String lowerTerm;
-  private String upperTerm;
-  private Collator collator;
+  private BytesRef lowerTerm;
+  private BytesRef upperTerm;
   private boolean includeLower;
   private boolean includeUpper;
 
@@ -69,78 +68,48 @@ public class TermRangeQuery extends Mult
    *          If true, the <code>upperTerm</code> is
    *          included in the range.
    */
-  public TermRangeQuery(String field, String lowerTerm, String upperTerm, boolean includeLower, boolean includeUpper) {
-    this(field, lowerTerm, upperTerm, includeLower, includeUpper, null);
-  }
-
-  /** Constructs a query selecting all terms greater/equal than
-   * <code>lowerTerm</code> but less/equal than <code>upperTerm</code>.
-   * <p>
-   * If an endpoint is null, it is said 
-   * to be "open". Either or both endpoints may be open.  Open endpoints may not 
-   * be exclusive (you can't select all but the first or last term without 
-   * explicitly specifying the term to exclude.)
-   * <p>
-   * If <code>collator</code> is not null, it will be used to decide whether
-   * index terms are within the given range, rather than using the Unicode code
-   * point order in which index terms are stored.
-   * <p>
-   * <strong>WARNING:</strong> Using this constructor and supplying a non-null
-   * value in the <code>collator</code> parameter will cause every single 
-   * index Term in the Field referenced by lowerTerm and/or upperTerm to be
-   * examined.  Depending on the number of index Terms in this Field, the 
-   * operation could be very slow.
-   *
-   * @param lowerTerm The Term text at the lower end of the range
-   * @param upperTerm The Term text at the upper end of the range
-   * @param includeLower
-   *          If true, the <code>lowerTerm</code> is
-   *          included in the range.
-   * @param includeUpper
-   *          If true, the <code>upperTerm</code> is
-   *          included in the range.
-   * @param collator The collator to use to collate index Terms, to determine
-   *  their membership in the range bounded by <code>lowerTerm</code> and
-   *  <code>upperTerm</code>.
-   */
-  public TermRangeQuery(String field, String lowerTerm, String upperTerm, boolean includeLower, boolean includeUpper,
-                    Collator collator) {
+  public TermRangeQuery(String field, BytesRef lowerTerm, BytesRef upperTerm, boolean includeLower, boolean includeUpper) {
     super(field);
     this.lowerTerm = lowerTerm;
     this.upperTerm = upperTerm;
     this.includeLower = includeLower;
     this.includeUpper = includeUpper;
-    this.collator = collator;
+  }
+
+  /**
+   * Factory that creates a new TermRangeQuery using Strings for term text.
+   */
+  public static TermRangeQuery newStringRange(String field, String lowerTerm, String upperTerm, boolean includeLower, boolean includeUpper) {
+    BytesRef lower = lowerTerm == null ? null : new BytesRef(lowerTerm);
+    BytesRef upper = upperTerm == null ? null : new BytesRef(upperTerm);
+    return new TermRangeQuery(field, lower, upper, includeLower, includeUpper);
   }
 
   /** Returns the lower value of this range query */
-  public String getLowerTerm() { return lowerTerm; }
+  public BytesRef getLowerTerm() { return lowerTerm; }
 
   /** Returns the upper value of this range query */
-  public String getUpperTerm() { return upperTerm; }
+  public BytesRef getUpperTerm() { return upperTerm; }
   
   /** Returns <code>true</code> if the lower endpoint is inclusive */
   public boolean includesLower() { return includeLower; }
   
   /** Returns <code>true</code> if the upper endpoint is inclusive */
   public boolean includesUpper() { return includeUpper; }
-
-  /** Returns the collator used to determine range inclusion, if any. */
-  public Collator getCollator() { return collator; }
   
   @Override
   protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
-    if (collator == null && lowerTerm != null && upperTerm != null && lowerTerm.compareTo(upperTerm) > 0) {
+    if (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) {
+    if ((lowerTerm == null || (includeLower && lowerTerm.length == 0)) && upperTerm == null) {
       return tenum;
     }
     return new TermRangeTermsEnum(tenum,
-        lowerTerm, upperTerm, includeLower, includeUpper, collator);
+        lowerTerm, upperTerm, includeLower, includeUpper);
   }
 
   /** Prints a user-readable version of this query. */
@@ -152,9 +121,10 @@ public class TermRangeQuery extends Mult
           buffer.append(":");
       }
       buffer.append(includeLower ? '[' : '{');
-      buffer.append(lowerTerm != null ? ("*".equals(lowerTerm) ? "\\*" : lowerTerm)  : "*");
+      // TODO: all these toStrings for queries should just output the bytes, it might not be UTF-8!
+      buffer.append(lowerTerm != null ? ("*".equals(lowerTerm.utf8ToString()) ? "\\*" : lowerTerm.utf8ToString())  : "*");
       buffer.append(" TO ");
-      buffer.append(upperTerm != null ? ("*".equals(upperTerm) ? "\\*" : upperTerm) : "*");
+      buffer.append(upperTerm != null ? ("*".equals(upperTerm.utf8ToString()) ? "\\*" : upperTerm.utf8ToString()) : "*");
       buffer.append(includeUpper ? ']' : '}');
       buffer.append(ToStringUtils.boost(getBoost()));
       return buffer.toString();
@@ -164,7 +134,6 @@ public class TermRangeQuery extends Mult
   public int hashCode() {
     final int prime = 31;
     int result = super.hashCode();
-    result = prime * result + ((collator == null) ? 0 : collator.hashCode());
     result = prime * result + (includeLower ? 1231 : 1237);
     result = prime * result + (includeUpper ? 1231 : 1237);
     result = prime * result + ((lowerTerm == null) ? 0 : lowerTerm.hashCode());
@@ -181,11 +150,6 @@ public class TermRangeQuery extends Mult
     if (getClass() != obj.getClass())
       return false;
     TermRangeQuery other = (TermRangeQuery) obj;
-    if (collator == null) {
-      if (other.collator != null)
-        return false;
-    } else if (!collator.equals(other.collator))
-      return false;
     if (includeLower != other.includeLower)
       return false;
     if (includeUpper != other.includeUpper)

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermRangeTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermRangeTermsEnum.java?rev=1075210&r1=1075209&r2=1075210&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermRangeTermsEnum.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermRangeTermsEnum.java Mon Feb 28 05:15:50 2011
@@ -18,7 +18,6 @@ package org.apache.lucene.search;
  */
 
 import java.io.IOException;
-import java.text.Collator;
 import java.util.Comparator;
 
 import org.apache.lucene.index.TermsEnum;
@@ -33,11 +32,8 @@ import org.apache.lucene.util.BytesRef;
  */
 public class TermRangeTermsEnum extends FilteredTermsEnum {
 
-  private Collator collator;
-  private String upperTermText;
-  private String lowerTermText;
-  private boolean includeLower;
-  private boolean includeUpper;
+  final private boolean includeLower;
+  final private boolean includeUpper;
   final private BytesRef lowerBytesRef;
   final private BytesRef upperBytesRef;
   private final Comparator<BytesRef> termComp;
@@ -53,79 +49,61 @@ public class TermRangeTermsEnum extends 
    * 
    * @param tenum
    *          TermsEnum to filter
-   * @param lowerTermText
+   * @param lowerTerm
    *          The term text at the lower end of the range
-   * @param upperTermText
+   * @param upperTerm
    *          The term text at the upper end of the range
    * @param includeLower
    *          If true, the <code>lowerTerm</code> is included in the range.
    * @param includeUpper
    *          If true, the <code>upperTerm</code> is included in the range.
-   * @param collator
-   *          The collator to use to collate index Terms, to determine their
-   *          membership in the range bounded by <code>lowerTerm</code> and
-   *          <code>upperTerm</code>.
    * 
    * @throws IOException
    */
-  public TermRangeTermsEnum(TermsEnum tenum, String lowerTermText, String upperTermText, 
-    boolean includeLower, boolean includeUpper, Collator collator) throws IOException {
+  public TermRangeTermsEnum(TermsEnum tenum, BytesRef lowerTerm, BytesRef upperTerm, 
+    boolean includeLower, boolean includeUpper) throws IOException {
     super(tenum);
-    this.collator = collator;
-    this.upperTermText = upperTermText;
-    this.lowerTermText = lowerTermText;
-    this.includeLower = includeLower;
-    this.includeUpper = includeUpper;
 
     // do a little bit of normalization...
     // open ended range queries should always be inclusive.
-    if (this.lowerTermText == null) {
-      this.lowerTermText = "";
+    if (lowerTerm == null) {
+      this.lowerBytesRef = new BytesRef();
       this.includeLower = true;
+    } else {
+      this.lowerBytesRef = lowerTerm;
+      this.includeLower = includeLower;
     }
-    lowerBytesRef = new BytesRef(this.lowerTermText);
 
-    if (this.upperTermText == null) {
+    if (upperTerm == null) {
       this.includeUpper = true;
       upperBytesRef = null;
     } else {
-      upperBytesRef = new BytesRef(upperTermText);
+      this.includeUpper = includeUpper;
+      upperBytesRef = upperTerm;
     }
 
-    BytesRef startBytesRef = (collator == null) ? lowerBytesRef : new BytesRef("");
-    setInitialSeekTerm(startBytesRef);
+    setInitialSeekTerm(lowerBytesRef);
     termComp = getComparator();
   }
 
   @Override
   protected AcceptStatus accept(BytesRef term) {
-    if (collator == null) {
-      if (!this.includeLower && term.equals(lowerBytesRef))
-        return AcceptStatus.NO;
-      // Use this field's default sort ordering
-      if (upperBytesRef != null) {
-        final int cmp = termComp.compare(upperBytesRef, term);
-        /*
-         * if beyond the upper term, or is exclusive and this is equal to
-         * the upper term, break out
-         */
-        if ((cmp < 0) ||
-            (!includeUpper && cmp==0)) {
-          return AcceptStatus.END;
-        }
-      }
-      return AcceptStatus.YES;
-    } else {
-      if ((includeLower
-           ? collator.compare(term.utf8ToString(), lowerTermText) >= 0
-           : collator.compare(term.utf8ToString(), lowerTermText) > 0)
-          && (upperTermText == null
-              || (includeUpper
-                  ? collator.compare(term.utf8ToString(), upperTermText) <= 0
-                  : collator.compare(term.utf8ToString(), upperTermText) < 0))) {
-        return AcceptStatus.YES;
-      }
+    if (!this.includeLower && term.equals(lowerBytesRef))
       return AcceptStatus.NO;
+    
+    // Use this field's default sort ordering
+    if (upperBytesRef != null) {
+      final int cmp = termComp.compare(upperBytesRef, term);
+      /*
+       * if beyond the upper term, or is exclusive and this is equal to
+       * the upper term, break out
+       */
+      if ((cmp < 0) ||
+          (!includeUpper && cmp==0)) {
+        return AcceptStatus.END;
+      }
     }
+
+    return AcceptStatus.YES;
   }
 }