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 2012/06/28 15:20:16 UTC

svn commit: r1355001 - in /lucene/dev/trunk/lucene: ./ queryparser/src/java/org/apache/lucene/queryparser/analyzing/ queryparser/src/test/org/apache/lucene/queryparser/analyzing/ test-framework/src/java/org/apache/lucene/analysis/

Author: rmuir
Date: Thu Jun 28 13:20:15 2012
New Revision: 1355001

URL: http://svn.apache.org/viewvc?rev=1355001&view=rev
Log:
LUCENE-4176: fix AnalyzingQueryParser to analyze range endpoints as bytes

Added:
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockBytesAnalyzer.java   (with props)
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockBytesAttributeFactory.java   (with props)
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockUTF16TermAttributeImpl.java   (with props)
Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/analyzing/AnalyzingQueryParser.java
    lucene/dev/trunk/lucene/queryparser/src/test/org/apache/lucene/queryparser/analyzing/TestAnalyzingQueryParser.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1355001&r1=1355000&r2=1355001&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Thu Jun 28 13:20:15 2012
@@ -16,6 +16,11 @@ API Changes
   has a different API (carries a list of tags instead of a compound tag). Upgrade
   of embedded morfologik dictionaries to version 1.9. (Dawid Weiss)
 
+Bug Fixes
+
+* LUCENE-4176: Fix AnalyzingQueryParser to analyze range endpoints as bytes,
+  so that it works correctly with Analyzers that produce binary non-UTF-8 terms
+  such as CollationAnalyzer. (Nattapong Sirilappanich via Robert Muir) 
 
 ======================= Lucene 4.0.0-ALPHA =======================
 

Modified: lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/analyzing/AnalyzingQueryParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/analyzing/AnalyzingQueryParser.java?rev=1355001&r1=1355000&r2=1355001&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/analyzing/AnalyzingQueryParser.java (original)
+++ lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/analyzing/AnalyzingQueryParser.java Thu Jun 28 13:20:15 2012
@@ -50,6 +50,7 @@ public class AnalyzingQueryParser extend
    */
   public AnalyzingQueryParser(Version matchVersion, String field, Analyzer analyzer) {
     super(matchVersion, field, analyzer);
+    setAnalyzeRangeTerms(true);
   }
 
   /**
@@ -278,72 +279,4 @@ public class AnalyzingQueryParser extend
 
     return (nextToken == null) ? null : super.getFuzzyQuery(field, nextToken, minSimilarity);
   }
-
-  /**
-   * Overrides super class, by passing terms through analyzer.
-   * @exception ParseException
-   */
-  @Override
-  protected Query getRangeQuery(String field, String part1, String part2, boolean startInclusive, boolean endInclusive)
-      throws ParseException {
-    // get Analyzer from superclass and tokenize the terms
-    TokenStream source = null;
-    CharTermAttribute termAtt = null;
-    boolean multipleTokens = false;
-
-    if (part1 != null) {
-      // part1
-      try {
-        source = getAnalyzer().tokenStream(field, new StringReader(part1));
-        termAtt = source.addAttribute(CharTermAttribute.class);
-        source.reset();
-        multipleTokens = false;
-
-
-        if (source.incrementToken()) {
-          part1 = termAtt.toString();
-        }
-        multipleTokens = source.incrementToken();
-      } catch (IOException e) {
-        // ignore
-      }
-      try {
-        source.end();
-        source.close();
-      } catch (IOException e) {
-        // ignore
-      }
-      if (multipleTokens) {
-        throw new ParseException("Cannot build RangeQuery with analyzer " + getAnalyzer().getClass()
-            + " - tokens were added to part1");
-      }
-    }
-
-    if (part2 != null) {
-      try {
-        // part2
-        source = getAnalyzer().tokenStream(field, new StringReader(part2));
-        termAtt = source.addAttribute(CharTermAttribute.class);
-        source.reset();
-        if (source.incrementToken()) {
-          part2 = termAtt.toString();
-        }
-        multipleTokens = source.incrementToken();
-      } catch (IOException e) {
-        // ignore
-      }
-      try {
-        source.end();
-        source.close();
-      } catch (IOException e) {
-        // ignore
-      }
-      if (multipleTokens) {
-        throw new ParseException("Cannot build RangeQuery with analyzer " + getAnalyzer().getClass()
-            + " - tokens were added to part2");
-      }
-    }
-    return super.getRangeQuery(field, part1, part2, startInclusive, endInclusive);
-  }
-
 }

Modified: lucene/dev/trunk/lucene/queryparser/src/test/org/apache/lucene/queryparser/analyzing/TestAnalyzingQueryParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/queryparser/src/test/org/apache/lucene/queryparser/analyzing/TestAnalyzingQueryParser.java?rev=1355001&r1=1355000&r2=1355001&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/queryparser/src/test/org/apache/lucene/queryparser/analyzing/TestAnalyzingQueryParser.java (original)
+++ lucene/dev/trunk/lucene/queryparser/src/test/org/apache/lucene/queryparser/analyzing/TestAnalyzingQueryParser.java Thu Jun 28 13:20:15 2012
@@ -22,7 +22,16 @@ import java.io.Reader;
 
 import org.apache.lucene.analysis.*;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.queryparser.classic.ParseException;
+import org.apache.lucene.queryparser.classic.QueryParser;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
 
 /**
@@ -138,5 +147,28 @@ public class TestAnalyzingQueryParser ex
       Tokenizer result = new MockTokenizer(reader, MockTokenizer.SIMPLE, true);
       return new TokenStreamComponents(result, new FoldingFilter(result));
     }
-  }  
+  }
+  
+  // LUCENE-4176
+  public void testByteTerms() throws Exception {
+    Directory ramDir = newDirectory();
+    Analyzer analyzer = new MockBytesAnalyzer();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), ramDir, analyzer);
+    Document doc = new Document();
+    FieldType fieldType = new FieldType();
+    fieldType.setIndexed(true);
+    fieldType.setTokenized(true);
+    fieldType.setStored(true);
+    Field field = new Field("content","เข", fieldType);
+    doc.add(field);
+    writer.addDocument(doc);
+    writer.close();
+    DirectoryReader ir = DirectoryReader.open(ramDir);
+    IndexSearcher is = new IndexSearcher(ir);
+    QueryParser qp = new AnalyzingQueryParser(TEST_VERSION_CURRENT, "content", analyzer);
+    Query q = qp.parse("[เข TO เข]");
+    assertEquals(1, is.search(q, 10).totalHits);
+    ir.close();
+    ramDir.close();
+  }
 }
\ No newline at end of file

Added: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockBytesAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockBytesAnalyzer.java?rev=1355001&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockBytesAnalyzer.java (added)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockBytesAnalyzer.java Thu Jun 28 13:20:15 2012
@@ -0,0 +1,33 @@
+package org.apache.lucene.analysis;
+
+/*
+ * 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.Reader;
+
+/**
+ * Analyzer for testing that encodes terms as UTF-16 bytes.
+ */
+public class MockBytesAnalyzer extends Analyzer {
+  private final MockBytesAttributeFactory factory = new MockBytesAttributeFactory();
+  
+  @Override
+  protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+    Tokenizer t = new MockTokenizer(factory, reader, MockTokenizer.KEYWORD, false, MockTokenizer.DEFAULT_MAX_TOKEN_LENGTH);
+    return new TokenStreamComponents(t);
+  }
+}

Added: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockBytesAttributeFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockBytesAttributeFactory.java?rev=1355001&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockBytesAttributeFactory.java (added)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockBytesAttributeFactory.java Thu Jun 28 13:20:15 2012
@@ -0,0 +1,40 @@
+package org.apache.lucene.analysis;
+
+/*
+ * 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.util.Attribute;
+import org.apache.lucene.util.AttributeImpl;
+import org.apache.lucene.util.AttributeSource;
+
+/**
+ * Attribute factory that implements CharTermAttribute with 
+ * {@link MockUTF16TermAttributeImpl}
+ */
+public class MockBytesAttributeFactory extends AttributeSource.AttributeFactory {
+  private final AttributeSource.AttributeFactory delegate =
+      AttributeSource.AttributeFactory.DEFAULT_ATTRIBUTE_FACTORY;
+  
+  @Override
+  public AttributeImpl createAttributeInstance(
+      Class<? extends Attribute> attClass) {
+    return attClass.isAssignableFrom(MockUTF16TermAttributeImpl.class)
+      ? new MockUTF16TermAttributeImpl()
+      : delegate.createAttributeInstance(attClass);
+  }
+  
+}

Added: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockUTF16TermAttributeImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockUTF16TermAttributeImpl.java?rev=1355001&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockUTF16TermAttributeImpl.java (added)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockUTF16TermAttributeImpl.java Thu Jun 28 13:20:15 2012
@@ -0,0 +1,41 @@
+package org.apache.lucene.analysis;
+
+/*
+ * 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.nio.charset.Charset;
+
+import org.apache.lucene.analysis.tokenattributes.CharTermAttributeImpl;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Extension of {@link CharTermAttributeImpl} that encodes the term
+ * text as UTF-16 bytes instead of as UTF-8 bytes.
+ */
+public class MockUTF16TermAttributeImpl extends CharTermAttributeImpl {
+  static final Charset charset = Charset.forName("UTF-16LE");
+  
+  @Override
+  public int fillBytesRef() {
+    BytesRef bytes = getBytesRef();
+    byte[] utf16 = toString().getBytes(charset);
+    bytes.bytes = utf16;
+    bytes.offset = 0;
+    bytes.length = utf16.length;
+    return bytes.hashCode();
+  }
+}