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 [3/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...

Added: lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/collation/tokenattributes/CollatedTermAttributeImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/collation/tokenattributes/CollatedTermAttributeImpl.java?rev=1075210&view=auto
==============================================================================
--- lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/collation/tokenattributes/CollatedTermAttributeImpl.java (added)
+++ lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/collation/tokenattributes/CollatedTermAttributeImpl.java Mon Feb 28 05:15:50 2011
@@ -0,0 +1,48 @@
+package org.apache.lucene.collation.tokenattributes;
+
+/**
+ * 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;
+
+import org.apache.lucene.analysis.tokenattributes.CharTermAttributeImpl;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Extension of {@link CharTermAttributeImpl} that encodes the term
+ * text as a binary Unicode collation key instead of as UTF-8 bytes.
+ */
+public class CollatedTermAttributeImpl extends CharTermAttributeImpl {
+  private final Collator collator;
+
+  /**
+   * Create a new CollatedTermAttributeImpl
+   * @param collator Collation key generator
+   */
+  public CollatedTermAttributeImpl(Collator collator) {
+    this.collator = collator;
+  }
+  
+  @Override
+  public int toBytesRef(BytesRef target) {
+    target.bytes = collator.getCollationKey(toString()).toByteArray();
+    target.offset = 0;
+    target.length = target.bytes.length;
+    return target.hashCode();
+  }
+
+}

Modified: lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/collation/CollationTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/collation/CollationTestBase.java?rev=1075210&r1=1075209&r2=1075210&view=diff
==============================================================================
--- lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/collation/CollationTestBase.java (original)
+++ lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/collation/CollationTestBase.java Mon Feb 28 05:15:50 2011
@@ -36,6 +36,7 @@ import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.Document;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IndexableBinaryStringTools;
 import org.apache.lucene.util.LuceneTestCase;
 
@@ -56,7 +57,9 @@ public abstract class CollationTestBase 
    * @param keyBits the result from 
    *  collator.getCollationKey(original).toByteArray()
    * @return The encoded collation key for the original String
+   * @deprecated only for testing deprecated filters
    */
+  @Deprecated
   protected String encodeCollationKey(byte[] keyBits) {
     // Ensure that the backing char[] array is large enough to hold the encoded
     // Binary String
@@ -65,10 +68,10 @@ public abstract class CollationTestBase 
     IndexableBinaryStringTools.encode(keyBits, 0, keyBits.length, encodedBegArray, 0, encodedLength);
     return new String(encodedBegArray);
   }
-    
-  public void testFarsiRangeFilterCollating(Analyzer analyzer, String firstBeg, 
-                                            String firstEnd, String secondBeg,
-                                            String secondEnd) throws Exception {
+  
+  public void testFarsiRangeFilterCollating(Analyzer analyzer, BytesRef firstBeg, 
+                                            BytesRef firstEnd, BytesRef secondBeg,
+                                            BytesRef secondEnd) throws Exception {
     RAMDirectory ramDir = new RAMDirectory();
     IndexWriter writer = new IndexWriter(ramDir, new IndexWriterConfig(
         TEST_VERSION_CURRENT, analyzer));
@@ -98,9 +101,9 @@ public abstract class CollationTestBase 
     searcher.close();
   }
  
-  public void testFarsiRangeQueryCollating(Analyzer analyzer, String firstBeg, 
-                                            String firstEnd, String secondBeg,
-                                            String secondEnd) throws Exception {
+  public void testFarsiRangeQueryCollating(Analyzer analyzer, BytesRef firstBeg, 
+                                            BytesRef firstEnd, BytesRef secondBeg,
+                                            BytesRef secondEnd) throws Exception {
     RAMDirectory ramDir = new RAMDirectory();
     IndexWriter writer = new IndexWriter(ramDir, new IndexWriterConfig(
         TEST_VERSION_CURRENT, analyzer));
@@ -126,8 +129,8 @@ public abstract class CollationTestBase 
     searcher.close();
   }
 
-  public void testFarsiTermRangeQuery(Analyzer analyzer, String firstBeg,
-      String firstEnd, String secondBeg, String secondEnd) throws Exception {
+  public void testFarsiTermRangeQuery(Analyzer analyzer, BytesRef firstBeg,
+      BytesRef firstEnd, BytesRef secondBeg, BytesRef secondEnd) throws Exception {
 
     RAMDirectory farsiIndex = new RAMDirectory();
     IndexWriter writer = new IndexWriter(farsiIndex, new IndexWriterConfig(

Modified: lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/collation/TestCollationKeyAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/collation/TestCollationKeyAnalyzer.java?rev=1075210&r1=1075209&r2=1075210&view=diff
==============================================================================
--- lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/collation/TestCollationKeyAnalyzer.java (original)
+++ lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/collation/TestCollationKeyAnalyzer.java Mon Feb 28 05:15:50 2011
@@ -19,6 +19,8 @@ package org.apache.lucene.collation;
 
 
 import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.index.codecs.CodecProvider;
+import org.apache.lucene.util.BytesRef;
 
 import java.text.Collator;
 import java.util.Locale;
@@ -34,17 +36,19 @@ public class TestCollationKeyAnalyzer ex
   // RuleBasedCollator.  However, the Arabic Locale seems to order the Farsi
   // characters properly.
   private Collator collator = Collator.getInstance(new Locale("ar"));
-  private Analyzer analyzer = new CollationKeyAnalyzer(collator);
+  private Analyzer analyzer = new CollationKeyAnalyzer(TEST_VERSION_CURRENT, collator);
 
-  private String firstRangeBeginning = encodeCollationKey
-    (collator.getCollationKey(firstRangeBeginningOriginal).toByteArray());
-  private String firstRangeEnd = encodeCollationKey
-    (collator.getCollationKey(firstRangeEndOriginal).toByteArray());
-  private String secondRangeBeginning = encodeCollationKey
-    (collator.getCollationKey(secondRangeBeginningOriginal).toByteArray());
-  private String secondRangeEnd = encodeCollationKey
-    (collator.getCollationKey(secondRangeEndOriginal).toByteArray());
+  private BytesRef firstRangeBeginning = new BytesRef(collator.getCollationKey(firstRangeBeginningOriginal).toByteArray());
+  private BytesRef firstRangeEnd = new BytesRef(collator.getCollationKey(firstRangeEndOriginal).toByteArray());
+  private BytesRef secondRangeBeginning = new BytesRef(collator.getCollationKey(secondRangeBeginningOriginal).toByteArray());
+  private BytesRef secondRangeEnd = new BytesRef(collator.getCollationKey(secondRangeEndOriginal).toByteArray());
   
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    assumeFalse("preflex format only supports UTF-8 encoded bytes", "PreFlex".equals(CodecProvider.getDefault().getDefaultFieldCodec()));
+  }
+
   public void testFarsiRangeFilterCollating() throws Exception {
     testFarsiRangeFilterCollating
       (analyzer, firstRangeBeginning, firstRangeEnd, 
@@ -65,13 +69,13 @@ public class TestCollationKeyAnalyzer ex
   
   public void testCollationKeySort() throws Exception {
     Analyzer usAnalyzer 
-      = new CollationKeyAnalyzer(Collator.getInstance(Locale.US));
+      = new CollationKeyAnalyzer(TEST_VERSION_CURRENT, Collator.getInstance(Locale.US));
     Analyzer franceAnalyzer 
-      = new CollationKeyAnalyzer(Collator.getInstance(Locale.FRANCE));
+      = new CollationKeyAnalyzer(TEST_VERSION_CURRENT, Collator.getInstance(Locale.FRANCE));
     Analyzer swedenAnalyzer 
-      = new CollationKeyAnalyzer(Collator.getInstance(new Locale("sv", "se")));
+      = new CollationKeyAnalyzer(TEST_VERSION_CURRENT, Collator.getInstance(new Locale("sv", "se")));
     Analyzer denmarkAnalyzer 
-      = new CollationKeyAnalyzer(Collator.getInstance(new Locale("da", "dk")));
+      = new CollationKeyAnalyzer(TEST_VERSION_CURRENT, Collator.getInstance(new Locale("da", "dk")));
     
     // The ICU Collator and Sun java.text.Collator implementations differ in their
     // orderings - "BFJDH" is the ordering for java.text.Collator for Locale.US.

Modified: lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/collation/TestCollationKeyFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/collation/TestCollationKeyFilter.java?rev=1075210&r1=1075209&r2=1075210&view=diff
==============================================================================
--- lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/collation/TestCollationKeyFilter.java (original)
+++ lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/collation/TestCollationKeyFilter.java Mon Feb 28 05:15:50 2011
@@ -21,12 +21,16 @@ package org.apache.lucene.collation;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.core.KeywordTokenizer;
+import org.apache.lucene.util.BytesRef;
 
 import java.text.Collator;
 import java.util.Locale;
 import java.io.Reader;
 
-
+/**
+ * @deprecated remove when CollationKeyFilter is removed.
+ */
+@Deprecated
 public class TestCollationKeyFilter extends CollationTestBase {
   // the sort order of Ø versus U depends on the version of the rules being used
   // for the inherited root locale: Ø's order isnt specified in Locale.US since 
@@ -39,14 +43,14 @@ public class TestCollationKeyFilter exte
   private Collator collator = Collator.getInstance(new Locale("ar"));
   private Analyzer analyzer = new TestAnalyzer(collator);
 
-  private String firstRangeBeginning = encodeCollationKey
-    (collator.getCollationKey(firstRangeBeginningOriginal).toByteArray());
-  private String firstRangeEnd = encodeCollationKey
-    (collator.getCollationKey(firstRangeEndOriginal).toByteArray());
-  private String secondRangeBeginning = encodeCollationKey
-    (collator.getCollationKey(secondRangeBeginningOriginal).toByteArray());
-  private String secondRangeEnd = encodeCollationKey
-    (collator.getCollationKey(secondRangeEndOriginal).toByteArray());
+  private BytesRef firstRangeBeginning = new BytesRef(encodeCollationKey
+    (collator.getCollationKey(firstRangeBeginningOriginal).toByteArray()));
+  private BytesRef firstRangeEnd = new BytesRef(encodeCollationKey
+    (collator.getCollationKey(firstRangeEndOriginal).toByteArray()));
+  private BytesRef secondRangeBeginning = new BytesRef(encodeCollationKey
+    (collator.getCollationKey(secondRangeBeginningOriginal).toByteArray()));
+  private BytesRef secondRangeEnd = new BytesRef(encodeCollationKey
+    (collator.getCollationKey(secondRangeEndOriginal).toByteArray()));
 
   
   public final class TestAnalyzer extends Analyzer {

Added: lucene/dev/trunk/modules/analysis/icu/src/java/org/apache/lucene/collation/ICUCollationAttributeFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/icu/src/java/org/apache/lucene/collation/ICUCollationAttributeFactory.java?rev=1075210&view=auto
==============================================================================
--- lucene/dev/trunk/modules/analysis/icu/src/java/org/apache/lucene/collation/ICUCollationAttributeFactory.java (added)
+++ lucene/dev/trunk/modules/analysis/icu/src/java/org/apache/lucene/collation/ICUCollationAttributeFactory.java Mon Feb 28 05:15:50 2011
@@ -0,0 +1,96 @@
+package org.apache.lucene.collation;
+
+/**
+ * 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 org.apache.lucene.collation.tokenattributes.ICUCollatedTermAttributeImpl;
+import org.apache.lucene.util.Attribute;
+import org.apache.lucene.util.AttributeImpl;
+import org.apache.lucene.util.AttributeSource;
+
+import com.ibm.icu.text.Collator;
+
+/**
+ * <p>
+ *   Converts each token into its {@link com.ibm.icu.text.CollationKey}, and
+ *   then encodes bytes as an index term.
+ * </p>
+ * <p>
+ *   <strong>WARNING:</strong> Make sure you use exactly the same Collator at
+ *   index and query time -- CollationKeys are only comparable when produced by
+ *   the same Collator.  {@link com.ibm.icu.text.RuleBasedCollator}s are 
+ *   independently versioned, so it is safe to search against stored
+ *   CollationKeys if the following are exactly the same (best practice is
+ *   to store this information with the index and check that they remain the
+ *   same at query time):
+ * </p>
+ * <ol>
+ *   <li>
+ *     Collator version - see {@link Collator#getVersion()}
+ *   </li>
+ *   <li>
+ *     The collation strength used - see {@link Collator#setStrength(int)}
+ *   </li>
+ * </ol> 
+ * <p>
+ *   CollationKeys generated by ICU Collators are not compatible with those
+ *   generated by java.text.Collators.  Specifically, if you use 
+ *   ICUCollationAttributeFactory to generate index terms, do not use 
+ *   {@link CollationAttributeFactory} on the query side, or vice versa.
+ * </p>
+ * <p>
+ *   ICUCollationAttributeFactory is significantly faster and generates significantly
+ *   shorter keys than CollationAttributeFactory.  See
+ *   <a href="http://site.icu-project.org/charts/collation-icu4j-sun"
+ *   >http://site.icu-project.org/charts/collation-icu4j-sun</a> for key
+ *   generation timing and key length comparisons between ICU4J and
+ *   java.text.Collator over several languages.
+ * </p>
+ */
+public class ICUCollationAttributeFactory extends AttributeSource.AttributeFactory {
+  private final Collator collator;
+  private final AttributeSource.AttributeFactory delegate;
+  
+  /**
+   * Create an ICUCollationAttributeFactory, using 
+   * {@link AttributeSource.AttributeFactory#DEFAULT_ATTRIBUTE_FACTORY} as the
+   * factory for all other attributes.
+   * @param collator CollationKey generator
+   */
+  public ICUCollationAttributeFactory(Collator collator) {
+    this(AttributeSource.AttributeFactory.DEFAULT_ATTRIBUTE_FACTORY, collator);
+  }
+  
+  /**
+   * Create an ICUCollationAttributeFactory, using the supplied Attribute 
+   * Factory as the factory for all other attributes.
+   * @param delegate Attribute Factory
+   * @param collator CollationKey generator
+   */
+  public ICUCollationAttributeFactory(AttributeSource.AttributeFactory delegate, Collator collator) {
+    this.delegate = delegate;
+    this.collator = collator;
+  }
+  
+  @Override
+  public AttributeImpl createAttributeInstance(
+      Class<? extends Attribute> attClass) {
+    return attClass.isAssignableFrom(ICUCollatedTermAttributeImpl.class)
+      ? new ICUCollatedTermAttributeImpl(collator)
+      : delegate.createAttributeInstance(attClass);
+  }
+}

Modified: lucene/dev/trunk/modules/analysis/icu/src/java/org/apache/lucene/collation/ICUCollationKeyAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/icu/src/java/org/apache/lucene/collation/ICUCollationKeyAnalyzer.java?rev=1075210&r1=1075209&r2=1075210&view=diff
==============================================================================
--- lucene/dev/trunk/modules/analysis/icu/src/java/org/apache/lucene/collation/ICUCollationKeyAnalyzer.java (original)
+++ lucene/dev/trunk/modules/analysis/icu/src/java/org/apache/lucene/collation/ICUCollationKeyAnalyzer.java Mon Feb 28 05:15:50 2011
@@ -19,24 +19,20 @@ package org.apache.lucene.collation;
 
 
 import com.ibm.icu.text.Collator;
-import org.apache.lucene.analysis.Analyzer;
-import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.core.KeywordTokenizer;
-import org.apache.lucene.analysis.Tokenizer;
-
+import org.apache.lucene.analysis.util.ReusableAnalyzerBase;
 import org.apache.lucene.collation.CollationKeyAnalyzer; // javadocs
+import org.apache.lucene.util.Version;
 
 import java.io.Reader;
-import java.io.IOException;
-
 
 /**
  * <p>
  *   Filters {@link KeywordTokenizer} with {@link ICUCollationKeyFilter}.
  * <p>
  *   Converts the token into its {@link com.ibm.icu.text.CollationKey}, and
- *   then encodes the CollationKey with 
- *   {@link org.apache.lucene.util.IndexableBinaryStringTools}, to allow it to
+ *   then encodes the CollationKey either directly or with 
+ *   {@link IndexableBinaryStringTools} (see <a href="#version">below</a>), to allow it to
  *   be stored as an index term.
  * </p>
  * <p>
@@ -70,39 +66,48 @@ import java.io.IOException;
  *   generation timing and key length comparisons between ICU4J and
  *   java.text.Collator over several languages.
  * </p>
+ * <a name="version"/>
+ * <p>You must specify the required {@link Version}
+ * compatibility when creating ICUCollationKeyAnalyzer:
+ * <ul>
+ *   <li> As of 4.0, Collation Keys are directly encoded as bytes. Previous
+ *   versions will encode the bytes with {@link IndexableBinaryStringTools}.
+ * </ul>
  */
-public final class ICUCollationKeyAnalyzer extends Analyzer {
-  private Collator collator;
-
-  public ICUCollationKeyAnalyzer(Collator collator) {
+public final class ICUCollationKeyAnalyzer extends ReusableAnalyzerBase {
+  private final Collator collator;
+  private final ICUCollationAttributeFactory factory;
+  private final Version matchVersion;
+
+  /**
+   * Create a new ICUCollationKeyAnalyzer, using the specified collator.
+   * 
+   * @param matchVersion See <a href="#version">above</a>
+   * @param collator CollationKey generator
+   */
+  public ICUCollationKeyAnalyzer(Version matchVersion, Collator collator) {
+    this.matchVersion = matchVersion;
     this.collator = collator;
+    this.factory = new ICUCollationAttributeFactory(collator);
   }
 
-  @Override
-  public TokenStream tokenStream(String fieldName, Reader reader) {
-    TokenStream result = new KeywordTokenizer(reader);
-    result = new ICUCollationKeyFilter(result, collator);
-    return result;
-  }
-  
-  private class SavedStreams {
-    Tokenizer source;
-    TokenStream result;
+  /**
+   * @deprecated Use {@link ICUCollationKeyAnalyzer#ICUCollationKeyAnalyzer(Version, Collator)}
+   *   and specify a version instead. This ctor will be removed in Lucene 5.0
+   */
+  @Deprecated
+  public ICUCollationKeyAnalyzer(Collator collator) {
+    this(Version.LUCENE_31, collator);
   }
-  
+
   @Override
-  public TokenStream reusableTokenStream(String fieldName, Reader reader) 
-    throws IOException {
-    
-    SavedStreams streams = (SavedStreams)getPreviousTokenStream();
-    if (streams == null) {
-      streams = new SavedStreams();
-      streams.source = new KeywordTokenizer(reader);
-      streams.result = new ICUCollationKeyFilter(streams.source, collator);
-      setPreviousTokenStream(streams);
+  protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+    if (matchVersion.onOrAfter(Version.LUCENE_40)) {
+      KeywordTokenizer tokenizer = new KeywordTokenizer(factory, reader, KeywordTokenizer.DEFAULT_BUFFER_SIZE);
+      return new TokenStreamComponents(tokenizer, tokenizer);
     } else {
-      streams.source.reset(reader);
+      KeywordTokenizer tokenizer = new KeywordTokenizer(reader);
+      return new TokenStreamComponents(tokenizer, new ICUCollationKeyFilter(tokenizer, collator));
     }
-    return streams.result;
   }
 }

Modified: lucene/dev/trunk/modules/analysis/icu/src/java/org/apache/lucene/collation/ICUCollationKeyFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/icu/src/java/org/apache/lucene/collation/ICUCollationKeyFilter.java?rev=1075210&r1=1075209&r2=1075210&view=diff
==============================================================================
--- lucene/dev/trunk/modules/analysis/icu/src/java/org/apache/lucene/collation/ICUCollationKeyFilter.java (original)
+++ lucene/dev/trunk/modules/analysis/icu/src/java/org/apache/lucene/collation/ICUCollationKeyFilter.java Mon Feb 28 05:15:50 2011
@@ -68,7 +68,10 @@ import java.io.IOException;
  *   generation timing and key length comparisons between ICU4J and
  *   java.text.Collator over several languages.
  * </p>
+ *  @deprecated Use {@link ICUCollationAttributeFactory} instead, which encodes
+ *  terms directly as bytes. This filter will be removed in Lucene 5.0
  */
+@Deprecated
 public final class ICUCollationKeyFilter extends TokenFilter {
   private Collator collator = null;
   private RawCollationKey reusableKey = new RawCollationKey();

Added: lucene/dev/trunk/modules/analysis/icu/src/java/org/apache/lucene/collation/tokenattributes/ICUCollatedTermAttributeImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/icu/src/java/org/apache/lucene/collation/tokenattributes/ICUCollatedTermAttributeImpl.java?rev=1075210&view=auto
==============================================================================
--- lucene/dev/trunk/modules/analysis/icu/src/java/org/apache/lucene/collation/tokenattributes/ICUCollatedTermAttributeImpl.java (added)
+++ lucene/dev/trunk/modules/analysis/icu/src/java/org/apache/lucene/collation/tokenattributes/ICUCollatedTermAttributeImpl.java Mon Feb 28 05:15:50 2011
@@ -0,0 +1,50 @@
+package org.apache.lucene.collation.tokenattributes;
+
+/**
+ * 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 org.apache.lucene.analysis.tokenattributes.CharTermAttributeImpl;
+import org.apache.lucene.util.BytesRef;
+
+import com.ibm.icu.text.Collator;
+import com.ibm.icu.text.RawCollationKey;
+
+/**
+ * Extension of {@link CharTermAttributeImpl} that encodes the term
+ * text as a binary Unicode collation key instead of as UTF-8 bytes.
+ */
+public class ICUCollatedTermAttributeImpl extends CharTermAttributeImpl {
+  private final Collator collator;
+  private final RawCollationKey key = new RawCollationKey();
+  
+  /**
+   * Create a new ICUCollatedTermAttributeImpl
+   * @param collator Collation key generator
+   */
+  public ICUCollatedTermAttributeImpl(Collator collator) {
+    this.collator = collator;
+  }
+  
+  @Override
+  public int toBytesRef(BytesRef target) {
+    collator.getRawCollationKey(toString(), key);
+    target.bytes = key.bytes;
+    target.offset = 0;
+    target.length = key.size;
+    return target.hashCode();
+  }
+}

Modified: lucene/dev/trunk/modules/analysis/icu/src/java/overview.html
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/icu/src/java/overview.html?rev=1075210&r1=1075209&r2=1075210&view=diff
==============================================================================
--- lucene/dev/trunk/modules/analysis/icu/src/java/overview.html (original)
+++ lucene/dev/trunk/modules/analysis/icu/src/java/overview.html Mon Feb 28 05:15:50 2011
@@ -112,11 +112,10 @@ algorithm.
 
 <h3>Farsi Range Queries</h3>
 <code><pre>
-  Collator collator = Collator.getInstance(new Locale("ar"));
-  ICUCollationKeyAnalyzer analyzer = new ICUCollationKeyAnalyzer(collator);
+  Collator collator = Collator.getInstance(new ULocale("ar"));
+  ICUCollationKeyAnalyzer analyzer = new ICUCollationKeyAnalyzer(Version.LUCENE_40, collator);
   RAMDirectory ramDir = new RAMDirectory();
-  IndexWriter writer = new IndexWriter
-    (ramDir, analyzer, true, IndexWriter.MaxFieldLength.LIMITED);
+  IndexWriter writer = new IndexWriter(ramDir, new IndexWriterConfig(Version.LUCENE_40, analyzer));
   Document doc = new Document();
   doc.add(new Field("content", "\u0633\u0627\u0628", 
                     Field.Store.YES, Field.Index.ANALYZED));
@@ -124,12 +123,9 @@ algorithm.
   writer.close();
   IndexSearcher is = new IndexSearcher(ramDir, true);
 
-  // The AnalyzingQueryParser in Lucene's contrib allows terms in range queries
-  // to be passed through an analyzer - Lucene's standard QueryParser does not
-  // allow this.
-  AnalyzingQueryParser aqp = new AnalyzingQueryParser("content", analyzer);
-  aqp.setLowercaseExpandedTerms(false);
-  
+  QueryParser aqp = new QueryParser(Version.LUCENE_40, "content", analyzer);
+  aqp.setAnalyzeRangeTerms(true);
+    
   // 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
   // indexed Term above should NOT be returned by a ConstantScoreRangeQuery
@@ -143,10 +139,9 @@ algorithm.
 <h3>Danish Sorting</h3>
 <code><pre>
   Analyzer analyzer 
-    = new ICUCollationKeyAnalyzer(Collator.getInstance(new Locale("da", "dk")));
+    = new ICUCollationKeyAnalyzer(Version.LUCENE_40, Collator.getInstance(new ULocale("da", "dk")));
   RAMDirectory indexStore = new RAMDirectory();
-  IndexWriter writer = new IndexWriter 
-    (indexStore, analyzer, true, IndexWriter.MaxFieldLength.LIMITED);
+  IndexWriter writer = new IndexWriter(indexStore, new IndexWriterConfig(Version.LUCENE_40, analyzer));
   String[] tracer = new String[] { "A", "B", "C", "D", "E" };
   String[] data = new String[] { "HAT", "HUT", "H\u00C5T", "H\u00D8T", "HOT" };
   String[] sortedTracerOrder = new String[] { "A", "E", "B", "D", "C" };
@@ -157,7 +152,7 @@ algorithm.
     writer.addDocument(doc);
   }
   writer.close();
-  Searcher searcher = new IndexSearcher(indexStore, true);
+  IndexSearcher searcher = new IndexSearcher(indexStore, true);
   Sort sort = new Sort();
   sort.setSort(new SortField("contents", SortField.STRING));
   Query query = new MatchAllDocsQuery();
@@ -170,18 +165,17 @@ algorithm.
 
 <h3>Turkish Case Normalization</h3>
 <code><pre>
-  Collator collator = Collator.getInstance(new Locale("tr", "TR"));
+  Collator collator = Collator.getInstance(new ULocale("tr", "TR"));
   collator.setStrength(Collator.PRIMARY);
-  Analyzer analyzer = new ICUCollationKeyAnalyzer(collator);
+  Analyzer analyzer = new ICUCollationKeyAnalyzer(Version.LUCENE_40, collator);
   RAMDirectory ramDir = new RAMDirectory();
-  IndexWriter writer = new IndexWriter
-    (ramDir, analyzer, true, IndexWriter.MaxFieldLength.LIMITED);
+  IndexWriter writer = new IndexWriter(ramDir, new IndexWriterConfig(Version.LUCENE_40, analyzer));
   Document doc = new Document();
   doc.add(new Field("contents", "DIGY", Field.Store.NO, Field.Index.ANALYZED));
   writer.addDocument(doc);
   writer.close();
   IndexSearcher is = new IndexSearcher(ramDir, true);
-  QueryParser parser = new QueryParser("contents", analyzer);
+  QueryParser parser = new QueryParser(Version.LUCENE_40, "contents", analyzer);
   Query query = parser.parse("d\u0131gy");   // U+0131: dotless i
   ScoreDoc[] result = is.search(query, null, 1000).scoreDocs;
   assertEquals("The index Term should be included.", 1, result.length);

Modified: lucene/dev/trunk/modules/analysis/icu/src/test/org/apache/lucene/collation/TestICUCollationKeyAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/icu/src/test/org/apache/lucene/collation/TestICUCollationKeyAnalyzer.java?rev=1075210&r1=1075209&r2=1075210&view=diff
==============================================================================
--- lucene/dev/trunk/modules/analysis/icu/src/test/org/apache/lucene/collation/TestICUCollationKeyAnalyzer.java (original)
+++ lucene/dev/trunk/modules/analysis/icu/src/test/org/apache/lucene/collation/TestICUCollationKeyAnalyzer.java Mon Feb 28 05:15:50 2011
@@ -20,6 +20,8 @@ package org.apache.lucene.collation;
 
 import com.ibm.icu.text.Collator;
 import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.index.codecs.CodecProvider;
+import org.apache.lucene.util.BytesRef;
 
 import java.util.Locale;
 
@@ -27,17 +29,23 @@ import java.util.Locale;
 public class TestICUCollationKeyAnalyzer extends CollationTestBase {
 
   private Collator collator = Collator.getInstance(new Locale("fa"));
-  private Analyzer analyzer = new ICUCollationKeyAnalyzer(collator);
+  private Analyzer analyzer = new ICUCollationKeyAnalyzer(TEST_VERSION_CURRENT, collator);
 
-  private String firstRangeBeginning = encodeCollationKey
+  private BytesRef firstRangeBeginning = new BytesRef
     (collator.getCollationKey(firstRangeBeginningOriginal).toByteArray());
-  private String firstRangeEnd = encodeCollationKey
+  private BytesRef firstRangeEnd = new BytesRef
     (collator.getCollationKey(firstRangeEndOriginal).toByteArray());
-  private String secondRangeBeginning = encodeCollationKey
+  private BytesRef secondRangeBeginning = new BytesRef
     (collator.getCollationKey(secondRangeBeginningOriginal).toByteArray());
-  private String secondRangeEnd = encodeCollationKey
+  private BytesRef secondRangeEnd = new BytesRef
     (collator.getCollationKey(secondRangeEndOriginal).toByteArray());
-  
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    assumeFalse("preflex format only supports UTF-8 encoded bytes", "PreFlex".equals(CodecProvider.getDefault().getDefaultFieldCodec()));
+  }
+
   public void testFarsiRangeFilterCollating() throws Exception {
     testFarsiRangeFilterCollating(analyzer, firstRangeBeginning, firstRangeEnd, 
                                   secondRangeBeginning, secondRangeEnd);
@@ -62,13 +70,13 @@ public class TestICUCollationKeyAnalyzer
   //  
   public void testCollationKeySort() throws Exception {
     Analyzer usAnalyzer = new ICUCollationKeyAnalyzer
-      (Collator.getInstance(Locale.US));
+      (TEST_VERSION_CURRENT, Collator.getInstance(Locale.US));
     Analyzer franceAnalyzer = new ICUCollationKeyAnalyzer
-      (Collator.getInstance(Locale.FRANCE));
+      (TEST_VERSION_CURRENT, Collator.getInstance(Locale.FRANCE));
     Analyzer swedenAnalyzer = new ICUCollationKeyAnalyzer
-      (Collator.getInstance(new Locale("sv", "se")));
+      (TEST_VERSION_CURRENT, Collator.getInstance(new Locale("sv", "se")));
     Analyzer denmarkAnalyzer = new ICUCollationKeyAnalyzer
-      (Collator.getInstance(new Locale("da", "dk")));
+      (TEST_VERSION_CURRENT, Collator.getInstance(new Locale("da", "dk")));
 
     // The ICU Collator and java.text.Collator implementations differ in their
     // orderings - "BFJHD" is the ordering for the ICU Collator for Locale.US.

Modified: lucene/dev/trunk/modules/analysis/icu/src/test/org/apache/lucene/collation/TestICUCollationKeyFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/icu/src/test/org/apache/lucene/collation/TestICUCollationKeyFilter.java?rev=1075210&r1=1075209&r2=1075210&view=diff
==============================================================================
--- lucene/dev/trunk/modules/analysis/icu/src/test/org/apache/lucene/collation/TestICUCollationKeyFilter.java (original)
+++ lucene/dev/trunk/modules/analysis/icu/src/test/org/apache/lucene/collation/TestICUCollationKeyFilter.java Mon Feb 28 05:15:50 2011
@@ -22,24 +22,26 @@ import com.ibm.icu.text.Collator;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.core.KeywordTokenizer;
+import org.apache.lucene.util.BytesRef;
 
 import java.io.Reader;
 import java.util.Locale;
 
-
+/** @deprecated remove this when ICUCollationKeyFilter is removed */
+@Deprecated
 public class TestICUCollationKeyFilter extends CollationTestBase {
 
   private Collator collator = Collator.getInstance(new Locale("fa"));
   private Analyzer analyzer = new TestAnalyzer(collator);
 
-  private String firstRangeBeginning = encodeCollationKey
-    (collator.getCollationKey(firstRangeBeginningOriginal).toByteArray());
-  private String firstRangeEnd = encodeCollationKey
-    (collator.getCollationKey(firstRangeEndOriginal).toByteArray());
-  private String secondRangeBeginning = encodeCollationKey
-    (collator.getCollationKey(secondRangeBeginningOriginal).toByteArray());
-  private String secondRangeEnd = encodeCollationKey
-    (collator.getCollationKey(secondRangeEndOriginal).toByteArray());
+  private BytesRef firstRangeBeginning = new BytesRef(encodeCollationKey
+    (collator.getCollationKey(firstRangeBeginningOriginal).toByteArray()));
+  private BytesRef firstRangeEnd = new BytesRef(encodeCollationKey
+    (collator.getCollationKey(firstRangeEndOriginal).toByteArray()));
+  private BytesRef secondRangeBeginning = new BytesRef(encodeCollationKey
+    (collator.getCollationKey(secondRangeBeginningOriginal).toByteArray()));
+  private BytesRef secondRangeEnd = new BytesRef(encodeCollationKey
+    (collator.getCollationKey(secondRangeEndOriginal).toByteArray()));
 
   
   public final class TestAnalyzer extends Analyzer {

Modified: lucene/dev/trunk/solr/src/java/org/apache/solr/handler/component/ShardDoc.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/handler/component/ShardDoc.java?rev=1075210&r1=1075209&r2=1075210&view=diff
==============================================================================
--- lucene/dev/trunk/solr/src/java/org/apache/solr/handler/component/ShardDoc.java (original)
+++ lucene/dev/trunk/solr/src/java/org/apache/solr/handler/component/ShardDoc.java Mon Feb 28 05:15:50 2011
@@ -95,10 +95,10 @@ class ShardFieldSortedHitQueue extends P
 
       String fieldname = fields[i].getField();
       comparators[i] = getCachedComparator(fieldname, fields[i]
-          .getType(), fields[i].getLocale(), fields[i].getComparatorSource());
+          .getType(), fields[i].getComparatorSource());
 
      if (fields[i].getType() == SortField.STRING) {
-        this.fields[i] = new SortField(fieldname, fields[i].getLocale(),
+        this.fields[i] = new SortField(fieldname, SortField.STRING, 
             fields[i].getReverse());
       } else {
         this.fields[i] = new SortField(fieldname, fields[i].getType(),
@@ -145,17 +145,14 @@ class ShardFieldSortedHitQueue extends P
     return c < 0;
   }
 
-  Comparator getCachedComparator(String fieldname, int type, Locale locale, FieldComparatorSource factory) {
+  Comparator getCachedComparator(String fieldname, int type, FieldComparatorSource factory) {
     Comparator comparator = null;
     switch (type) {
     case SortField.SCORE:
       comparator = comparatorScore(fieldname);
       break;
     case SortField.STRING:
-      if (locale != null)
-        comparator = comparatorStringLocale(fieldname, locale);
-      else
-        comparator = comparatorNatural(fieldname);
+      comparator = comparatorNatural(fieldname);
       break;
     case SortField.CUSTOM:
       if (factory instanceof MissingStringLastComparatorSource){

Modified: lucene/dev/trunk/solr/src/java/org/apache/solr/schema/DateField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/schema/DateField.java?rev=1075210&r1=1075209&r2=1075210&view=diff
==============================================================================
--- lucene/dev/trunk/solr/src/java/org/apache/solr/schema/DateField.java (original)
+++ lucene/dev/trunk/solr/src/java/org/apache/solr/schema/DateField.java Mon Feb 28 05:15:50 2011
@@ -410,7 +410,7 @@ public class DateField extends FieldType
 
   /** DateField specific range query */
   public Query getRangeQuery(QParser parser, SchemaField sf, Date part1, Date part2, boolean minInclusive, boolean maxInclusive) {
-    return new TermRangeQuery(
+    return TermRangeQuery.newStringRange(
             sf.getName(),
             part1 == null ? null : toInternal(part1),
             part2 == null ? null : toInternal(part2),

Modified: lucene/dev/trunk/solr/src/java/org/apache/solr/schema/FieldType.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/schema/FieldType.java?rev=1075210&r1=1075209&r2=1075210&view=diff
==============================================================================
--- lucene/dev/trunk/solr/src/java/org/apache/solr/schema/FieldType.java (original)
+++ lucene/dev/trunk/solr/src/java/org/apache/solr/schema/FieldType.java Mon Feb 28 05:15:50 2011
@@ -521,7 +521,7 @@ public abstract class FieldType extends 
    */
   public Query getRangeQuery(QParser parser, SchemaField field, String part1, String part2, boolean minInclusive, boolean maxInclusive) {
     // constant score mode is now enabled per default
-    return new TermRangeQuery(
+    return TermRangeQuery.newStringRange(
             field.getName(),
             part1 == null ? null : toInternal(part1),
             part2 == null ? null : toInternal(part2),

Modified: lucene/dev/trunk/solr/src/java/org/apache/solr/search/QueryParsing.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/search/QueryParsing.java?rev=1075210&r1=1075209&r2=1075210&view=diff
==============================================================================
--- lucene/dev/trunk/solr/src/java/org/apache/solr/search/QueryParsing.java (original)
+++ lucene/dev/trunk/solr/src/java/org/apache/solr/search/QueryParsing.java Mon Feb 28 05:15:50 2011
@@ -398,8 +398,8 @@ public class QueryParsing {
       String fname = q.getField();
       FieldType ft = writeFieldName(fname, schema, out, flags);
       out.append(q.includesLower() ? '[' : '{');
-      String lt = q.getLowerTerm();
-      String ut = q.getUpperTerm();
+      String lt = q.getLowerTerm().utf8ToString();
+      String ut = q.getUpperTerm().utf8ToString();
       if (lt == null) {
         out.append('*');
       } else {