mirror of https://github.com/apache/lucene.git
LUCENE-5097: Analyzer now has an additional tokenStream(String fieldName, String text) method, so wrapping by StringReader for common use is no longer needed. This method uses an internal reuseable reader, which was previously only used by the Field class.
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1500862 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
33d014f3aa
commit
f092795fe9
|
@ -300,6 +300,11 @@ API Changes
|
||||||
an overhead parameter, so you can easily pass a different value other than
|
an overhead parameter, so you can easily pass a different value other than
|
||||||
PackedInts.FASTEST from your own codec. (Robert Muir)
|
PackedInts.FASTEST from your own codec. (Robert Muir)
|
||||||
|
|
||||||
|
* LUCENE-5097: Analyzer now has an additional tokenStream(String fieldName,
|
||||||
|
String text) method, so wrapping by StringReader for common use is no
|
||||||
|
longer needed. This method uses an internal reuseable reader, which was
|
||||||
|
previously only used by the Field class. (Uwe Schindler, Robert Muir)
|
||||||
|
|
||||||
Build
|
Build
|
||||||
|
|
||||||
* LUCENE-4987: Upgrade randomized testing to version 2.0.10:
|
* LUCENE-4987: Upgrade randomized testing to version 2.0.10:
|
||||||
|
|
|
@ -18,7 +18,6 @@ package org.apache.lucene.analysis.synonym;
|
||||||
*/
|
*/
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.StringReader;
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
|
@ -112,7 +111,7 @@ public class SynonymMap {
|
||||||
* separates by {@link SynonymMap#WORD_SEPARATOR}.
|
* separates by {@link SynonymMap#WORD_SEPARATOR}.
|
||||||
* reuse and its chars must not be null. */
|
* reuse and its chars must not be null. */
|
||||||
public static CharsRef analyze(Analyzer analyzer, String text, CharsRef reuse) throws IOException {
|
public static CharsRef analyze(Analyzer analyzer, String text, CharsRef reuse) throws IOException {
|
||||||
TokenStream ts = analyzer.tokenStream("", new StringReader(text));
|
TokenStream ts = analyzer.tokenStream("", text);
|
||||||
CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class);
|
CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class);
|
||||||
PositionIncrementAttribute posIncAtt = ts.addAttribute(PositionIncrementAttribute.class);
|
PositionIncrementAttribute posIncAtt = ts.addAttribute(PositionIncrementAttribute.class);
|
||||||
ts.reset();
|
ts.reset();
|
||||||
|
|
|
@ -24,7 +24,6 @@ import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||||
import org.apache.lucene.analysis.util.CharArraySet;
|
import org.apache.lucene.analysis.util.CharArraySet;
|
||||||
import org.apache.lucene.util.Version;
|
import org.apache.lucene.util.Version;
|
||||||
|
|
||||||
import java.io.StringReader;
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
@ -47,8 +46,7 @@ public class TestStopAnalyzer extends BaseTokenStreamTestCase {
|
||||||
|
|
||||||
public void testDefaults() throws IOException {
|
public void testDefaults() throws IOException {
|
||||||
assertTrue(stop != null);
|
assertTrue(stop != null);
|
||||||
StringReader reader = new StringReader("This is a test of the english stop analyzer");
|
TokenStream stream = stop.tokenStream("test", "This is a test of the english stop analyzer");
|
||||||
TokenStream stream = stop.tokenStream("test", reader);
|
|
||||||
assertTrue(stream != null);
|
assertTrue(stream != null);
|
||||||
CharTermAttribute termAtt = stream.getAttribute(CharTermAttribute.class);
|
CharTermAttribute termAtt = stream.getAttribute(CharTermAttribute.class);
|
||||||
stream.reset();
|
stream.reset();
|
||||||
|
@ -61,8 +59,7 @@ public class TestStopAnalyzer extends BaseTokenStreamTestCase {
|
||||||
public void testStopList() throws IOException {
|
public void testStopList() throws IOException {
|
||||||
CharArraySet stopWordsSet = new CharArraySet(TEST_VERSION_CURRENT, asSet("good", "test", "analyzer"), false);
|
CharArraySet stopWordsSet = new CharArraySet(TEST_VERSION_CURRENT, asSet("good", "test", "analyzer"), false);
|
||||||
StopAnalyzer newStop = new StopAnalyzer(Version.LUCENE_40, stopWordsSet);
|
StopAnalyzer newStop = new StopAnalyzer(Version.LUCENE_40, stopWordsSet);
|
||||||
StringReader reader = new StringReader("This is a good test of the english stop analyzer");
|
TokenStream stream = newStop.tokenStream("test", "This is a good test of the english stop analyzer");
|
||||||
TokenStream stream = newStop.tokenStream("test", reader);
|
|
||||||
assertNotNull(stream);
|
assertNotNull(stream);
|
||||||
CharTermAttribute termAtt = stream.getAttribute(CharTermAttribute.class);
|
CharTermAttribute termAtt = stream.getAttribute(CharTermAttribute.class);
|
||||||
|
|
||||||
|
@ -76,9 +73,9 @@ public class TestStopAnalyzer extends BaseTokenStreamTestCase {
|
||||||
public void testStopListPositions() throws IOException {
|
public void testStopListPositions() throws IOException {
|
||||||
CharArraySet stopWordsSet = new CharArraySet(TEST_VERSION_CURRENT, asSet("good", "test", "analyzer"), false);
|
CharArraySet stopWordsSet = new CharArraySet(TEST_VERSION_CURRENT, asSet("good", "test", "analyzer"), false);
|
||||||
StopAnalyzer newStop = new StopAnalyzer(TEST_VERSION_CURRENT, stopWordsSet);
|
StopAnalyzer newStop = new StopAnalyzer(TEST_VERSION_CURRENT, stopWordsSet);
|
||||||
StringReader reader = new StringReader("This is a good test of the english stop analyzer with positions");
|
String s = "This is a good test of the english stop analyzer with positions";
|
||||||
int expectedIncr[] = { 1, 1, 1, 3, 1, 1, 1, 2, 1};
|
int expectedIncr[] = { 1, 1, 1, 3, 1, 1, 1, 2, 1};
|
||||||
TokenStream stream = newStop.tokenStream("test", reader);
|
TokenStream stream = newStop.tokenStream("test", s);
|
||||||
assertNotNull(stream);
|
assertNotNull(stream);
|
||||||
int i = 0;
|
int i = 0;
|
||||||
CharTermAttribute termAtt = stream.getAttribute(CharTermAttribute.class);
|
CharTermAttribute termAtt = stream.getAttribute(CharTermAttribute.class);
|
||||||
|
|
|
@ -18,13 +18,10 @@ package org.apache.lucene.analysis.miscellaneous;
|
||||||
*/
|
*/
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.StringReader;
|
|
||||||
|
|
||||||
import org.apache.lucene.analysis.Analyzer;
|
import org.apache.lucene.analysis.Analyzer;
|
||||||
import org.apache.lucene.analysis.BaseTokenStreamTestCase;
|
import org.apache.lucene.analysis.BaseTokenStreamTestCase;
|
||||||
import org.apache.lucene.analysis.MockAnalyzer;
|
import org.apache.lucene.analysis.MockAnalyzer;
|
||||||
import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
|
|
||||||
import org.apache.lucene.analysis.standard.StandardAnalyzer;
|
|
||||||
import org.apache.lucene.document.Document;
|
import org.apache.lucene.document.Document;
|
||||||
import org.apache.lucene.document.Field;
|
import org.apache.lucene.document.Field;
|
||||||
import org.apache.lucene.index.DirectoryReader;
|
import org.apache.lucene.index.DirectoryReader;
|
||||||
|
@ -47,14 +44,14 @@ public class TestLimitTokenCountAnalyzer extends BaseTokenStreamTestCase {
|
||||||
Analyzer a = new LimitTokenCountAnalyzer(mock, 2, consumeAll);
|
Analyzer a = new LimitTokenCountAnalyzer(mock, 2, consumeAll);
|
||||||
|
|
||||||
// dont use assertAnalyzesTo here, as the end offset is not the end of the string (unless consumeAll is true, in which case its correct)!
|
// dont use assertAnalyzesTo here, as the end offset is not the end of the string (unless consumeAll is true, in which case its correct)!
|
||||||
assertTokenStreamContents(a.tokenStream("dummy", new StringReader("1 2 3 4 5")), new String[] { "1", "2" }, new int[] { 0, 3 }, new int[] { 1, 4 }, consumeAll ? 16 : null);
|
assertTokenStreamContents(a.tokenStream("dummy", "1 2 3 4 5"), new String[] { "1", "2" }, new int[] { 0, 3 }, new int[] { 1, 4 }, consumeAll ? 16 : null);
|
||||||
assertTokenStreamContents(a.tokenStream("dummy", new StringReader("1 2 3 4 5")), new String[] { "1", "2" }, new int[] { 0, 2 }, new int[] { 1, 3 }, consumeAll ? 9 : null);
|
assertTokenStreamContents(a.tokenStream("dummy", "1 2 3 4 5"), new String[] { "1", "2" }, new int[] { 0, 2 }, new int[] { 1, 3 }, consumeAll ? 9 : null);
|
||||||
|
|
||||||
// less than the limit, ensure we behave correctly
|
// less than the limit, ensure we behave correctly
|
||||||
assertTokenStreamContents(a.tokenStream("dummy", new StringReader("1 ")), new String[] { "1" }, new int[] { 0 }, new int[] { 1 }, consumeAll ? 3 : null);
|
assertTokenStreamContents(a.tokenStream("dummy", "1 "), new String[] { "1" }, new int[] { 0 }, new int[] { 1 }, consumeAll ? 3 : null);
|
||||||
|
|
||||||
// equal to limit
|
// equal to limit
|
||||||
assertTokenStreamContents(a.tokenStream("dummy", new StringReader("1 2 ")), new String[] { "1", "2" }, new int[] { 0, 3 }, new int[] { 1, 4 }, consumeAll ? 6 : null);
|
assertTokenStreamContents(a.tokenStream("dummy", "1 2 "), new String[] { "1", "2" }, new int[] { 0, 3 }, new int[] { 1, 4 }, consumeAll ? 6 : null);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -43,17 +43,17 @@ public class TestLimitTokenPositionFilter extends BaseTokenStreamTestCase {
|
||||||
};
|
};
|
||||||
|
|
||||||
// dont use assertAnalyzesTo here, as the end offset is not the end of the string (unless consumeAll is true, in which case its correct)!
|
// dont use assertAnalyzesTo here, as the end offset is not the end of the string (unless consumeAll is true, in which case its correct)!
|
||||||
assertTokenStreamContents(a.tokenStream("dummy", new StringReader("1 2 3 4 5")),
|
assertTokenStreamContents(a.tokenStream("dummy", "1 2 3 4 5"),
|
||||||
new String[] { "1", "2" }, new int[] { 0, 3 }, new int[] { 1, 4 }, consumeAll ? 16 : null);
|
new String[] { "1", "2" }, new int[] { 0, 3 }, new int[] { 1, 4 }, consumeAll ? 16 : null);
|
||||||
assertTokenStreamContents(a.tokenStream("dummy", new StringReader("1 2 3 4 5")),
|
assertTokenStreamContents(a.tokenStream("dummy", new StringReader("1 2 3 4 5")),
|
||||||
new String[] { "1", "2" }, new int[] { 0, 2 }, new int[] { 1, 3 }, consumeAll ? 9 : null);
|
new String[] { "1", "2" }, new int[] { 0, 2 }, new int[] { 1, 3 }, consumeAll ? 9 : null);
|
||||||
|
|
||||||
// less than the limit, ensure we behave correctly
|
// less than the limit, ensure we behave correctly
|
||||||
assertTokenStreamContents(a.tokenStream("dummy", new StringReader("1 ")),
|
assertTokenStreamContents(a.tokenStream("dummy", "1 "),
|
||||||
new String[] { "1" }, new int[] { 0 }, new int[] { 1 }, consumeAll ? 3 : null);
|
new String[] { "1" }, new int[] { 0 }, new int[] { 1 }, consumeAll ? 3 : null);
|
||||||
|
|
||||||
// equal to limit
|
// equal to limit
|
||||||
assertTokenStreamContents(a.tokenStream("dummy", new StringReader("1 2 ")),
|
assertTokenStreamContents(a.tokenStream("dummy", "1 2 "),
|
||||||
new String[] { "1", "2" }, new int[] { 0, 3 }, new int[] { 1, 4 }, consumeAll ? 6 : null);
|
new String[] { "1", "2" }, new int[] { 0, 3 }, new int[] { 1, 4 }, consumeAll ? 6 : null);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -1,7 +1,6 @@
|
||||||
package org.apache.lucene.analysis.miscellaneous;
|
package org.apache.lucene.analysis.miscellaneous;
|
||||||
|
|
||||||
import java.io.Reader;
|
import java.io.Reader;
|
||||||
import java.io.StringReader;
|
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
@ -38,8 +37,7 @@ public class TestPerFieldAnalyzerWrapper extends BaseTokenStreamTestCase {
|
||||||
PerFieldAnalyzerWrapper analyzer =
|
PerFieldAnalyzerWrapper analyzer =
|
||||||
new PerFieldAnalyzerWrapper(new WhitespaceAnalyzer(TEST_VERSION_CURRENT), analyzerPerField);
|
new PerFieldAnalyzerWrapper(new WhitespaceAnalyzer(TEST_VERSION_CURRENT), analyzerPerField);
|
||||||
|
|
||||||
TokenStream tokenStream = analyzer.tokenStream("field",
|
TokenStream tokenStream = analyzer.tokenStream("field", text);
|
||||||
new StringReader(text));
|
|
||||||
CharTermAttribute termAtt = tokenStream.getAttribute(CharTermAttribute.class);
|
CharTermAttribute termAtt = tokenStream.getAttribute(CharTermAttribute.class);
|
||||||
tokenStream.reset();
|
tokenStream.reset();
|
||||||
|
|
||||||
|
@ -48,8 +46,7 @@ public class TestPerFieldAnalyzerWrapper extends BaseTokenStreamTestCase {
|
||||||
"Qwerty",
|
"Qwerty",
|
||||||
termAtt.toString());
|
termAtt.toString());
|
||||||
|
|
||||||
tokenStream = analyzer.tokenStream("special",
|
tokenStream = analyzer.tokenStream("special", text);
|
||||||
new StringReader(text));
|
|
||||||
termAtt = tokenStream.getAttribute(CharTermAttribute.class);
|
termAtt = tokenStream.getAttribute(CharTermAttribute.class);
|
||||||
tokenStream.reset();
|
tokenStream.reset();
|
||||||
|
|
||||||
|
|
|
@ -26,7 +26,6 @@ import org.apache.lucene.index.IndexWriter;
|
||||||
import org.apache.lucene.index.IndexWriterConfig;
|
import org.apache.lucene.index.IndexWriterConfig;
|
||||||
import org.apache.lucene.store.RAMDirectory;
|
import org.apache.lucene.store.RAMDirectory;
|
||||||
|
|
||||||
import java.io.StringReader;
|
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
|
|
||||||
|
@ -66,44 +65,44 @@ public class QueryAutoStopWordAnalyzerTest extends BaseTokenStreamTestCase {
|
||||||
public void testNoStopwords() throws Exception {
|
public void testNoStopwords() throws Exception {
|
||||||
// Note: an empty list of fields passed in
|
// Note: an empty list of fields passed in
|
||||||
protectedAnalyzer = new QueryAutoStopWordAnalyzer(TEST_VERSION_CURRENT, appAnalyzer, reader, Collections.<String>emptyList(), 1);
|
protectedAnalyzer = new QueryAutoStopWordAnalyzer(TEST_VERSION_CURRENT, appAnalyzer, reader, Collections.<String>emptyList(), 1);
|
||||||
TokenStream protectedTokenStream = protectedAnalyzer.tokenStream("variedField", new StringReader("quick"));
|
TokenStream protectedTokenStream = protectedAnalyzer.tokenStream("variedField", "quick");
|
||||||
assertTokenStreamContents(protectedTokenStream, new String[]{"quick"});
|
assertTokenStreamContents(protectedTokenStream, new String[]{"quick"});
|
||||||
|
|
||||||
protectedTokenStream = protectedAnalyzer.tokenStream("repetitiveField", new StringReader("boring"));
|
protectedTokenStream = protectedAnalyzer.tokenStream("repetitiveField", "boring");
|
||||||
assertTokenStreamContents(protectedTokenStream, new String[]{"boring"});
|
assertTokenStreamContents(protectedTokenStream, new String[]{"boring"});
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testDefaultStopwordsAllFields() throws Exception {
|
public void testDefaultStopwordsAllFields() throws Exception {
|
||||||
protectedAnalyzer = new QueryAutoStopWordAnalyzer(TEST_VERSION_CURRENT, appAnalyzer, reader);
|
protectedAnalyzer = new QueryAutoStopWordAnalyzer(TEST_VERSION_CURRENT, appAnalyzer, reader);
|
||||||
TokenStream protectedTokenStream = protectedAnalyzer.tokenStream("repetitiveField", new StringReader("boring"));
|
TokenStream protectedTokenStream = protectedAnalyzer.tokenStream("repetitiveField", "boring");
|
||||||
assertTokenStreamContents(protectedTokenStream, new String[0]); // Default stop word filtering will remove boring
|
assertTokenStreamContents(protectedTokenStream, new String[0]); // Default stop word filtering will remove boring
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testStopwordsAllFieldsMaxPercentDocs() throws Exception {
|
public void testStopwordsAllFieldsMaxPercentDocs() throws Exception {
|
||||||
protectedAnalyzer = new QueryAutoStopWordAnalyzer(TEST_VERSION_CURRENT, appAnalyzer, reader, 1f / 2f);
|
protectedAnalyzer = new QueryAutoStopWordAnalyzer(TEST_VERSION_CURRENT, appAnalyzer, reader, 1f / 2f);
|
||||||
|
|
||||||
TokenStream protectedTokenStream = protectedAnalyzer.tokenStream("repetitiveField", new StringReader("boring"));
|
TokenStream protectedTokenStream = protectedAnalyzer.tokenStream("repetitiveField", "boring");
|
||||||
// A filter on terms in > one half of docs remove boring
|
// A filter on terms in > one half of docs remove boring
|
||||||
assertTokenStreamContents(protectedTokenStream, new String[0]);
|
assertTokenStreamContents(protectedTokenStream, new String[0]);
|
||||||
|
|
||||||
protectedTokenStream = protectedAnalyzer.tokenStream("repetitiveField", new StringReader("vaguelyboring"));
|
protectedTokenStream = protectedAnalyzer.tokenStream("repetitiveField", "vaguelyboring");
|
||||||
// A filter on terms in > half of docs should not remove vaguelyBoring
|
// A filter on terms in > half of docs should not remove vaguelyBoring
|
||||||
assertTokenStreamContents(protectedTokenStream, new String[]{"vaguelyboring"});
|
assertTokenStreamContents(protectedTokenStream, new String[]{"vaguelyboring"});
|
||||||
|
|
||||||
protectedAnalyzer = new QueryAutoStopWordAnalyzer(TEST_VERSION_CURRENT, appAnalyzer, reader, 1f / 4f);
|
protectedAnalyzer = new QueryAutoStopWordAnalyzer(TEST_VERSION_CURRENT, appAnalyzer, reader, 1f / 4f);
|
||||||
protectedTokenStream = protectedAnalyzer.tokenStream("repetitiveField", new StringReader("vaguelyboring"));
|
protectedTokenStream = protectedAnalyzer.tokenStream("repetitiveField", "vaguelyboring");
|
||||||
// A filter on terms in > quarter of docs should remove vaguelyBoring
|
// A filter on terms in > quarter of docs should remove vaguelyBoring
|
||||||
assertTokenStreamContents(protectedTokenStream, new String[0]);
|
assertTokenStreamContents(protectedTokenStream, new String[0]);
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testStopwordsPerFieldMaxPercentDocs() throws Exception {
|
public void testStopwordsPerFieldMaxPercentDocs() throws Exception {
|
||||||
protectedAnalyzer = new QueryAutoStopWordAnalyzer(TEST_VERSION_CURRENT, appAnalyzer, reader, Arrays.asList("variedField"), 1f / 2f);
|
protectedAnalyzer = new QueryAutoStopWordAnalyzer(TEST_VERSION_CURRENT, appAnalyzer, reader, Arrays.asList("variedField"), 1f / 2f);
|
||||||
TokenStream protectedTokenStream = protectedAnalyzer.tokenStream("repetitiveField", new StringReader("boring"));
|
TokenStream protectedTokenStream = protectedAnalyzer.tokenStream("repetitiveField", "boring");
|
||||||
// A filter on one Field should not affect queries on another
|
// A filter on one Field should not affect queries on another
|
||||||
assertTokenStreamContents(protectedTokenStream, new String[]{"boring"});
|
assertTokenStreamContents(protectedTokenStream, new String[]{"boring"});
|
||||||
|
|
||||||
protectedAnalyzer = new QueryAutoStopWordAnalyzer(TEST_VERSION_CURRENT, appAnalyzer, reader, Arrays.asList("variedField", "repetitiveField"), 1f / 2f);
|
protectedAnalyzer = new QueryAutoStopWordAnalyzer(TEST_VERSION_CURRENT, appAnalyzer, reader, Arrays.asList("variedField", "repetitiveField"), 1f / 2f);
|
||||||
protectedTokenStream = protectedAnalyzer.tokenStream("repetitiveField", new StringReader("boring"));
|
protectedTokenStream = protectedAnalyzer.tokenStream("repetitiveField", "boring");
|
||||||
// A filter on the right Field should affect queries on it
|
// A filter on the right Field should affect queries on it
|
||||||
assertTokenStreamContents(protectedTokenStream, new String[0]);
|
assertTokenStreamContents(protectedTokenStream, new String[0]);
|
||||||
}
|
}
|
||||||
|
@ -121,11 +120,11 @@ public class QueryAutoStopWordAnalyzerTest extends BaseTokenStreamTestCase {
|
||||||
public void testNoFieldNamePollution() throws Exception {
|
public void testNoFieldNamePollution() throws Exception {
|
||||||
protectedAnalyzer = new QueryAutoStopWordAnalyzer(TEST_VERSION_CURRENT, appAnalyzer, reader, Arrays.asList("repetitiveField"), 10);
|
protectedAnalyzer = new QueryAutoStopWordAnalyzer(TEST_VERSION_CURRENT, appAnalyzer, reader, Arrays.asList("repetitiveField"), 10);
|
||||||
|
|
||||||
TokenStream protectedTokenStream = protectedAnalyzer.tokenStream("repetitiveField", new StringReader("boring"));
|
TokenStream protectedTokenStream = protectedAnalyzer.tokenStream("repetitiveField", "boring");
|
||||||
// Check filter set up OK
|
// Check filter set up OK
|
||||||
assertTokenStreamContents(protectedTokenStream, new String[0]);
|
assertTokenStreamContents(protectedTokenStream, new String[0]);
|
||||||
|
|
||||||
protectedTokenStream = protectedAnalyzer.tokenStream("variedField", new StringReader("boring"));
|
protectedTokenStream = protectedAnalyzer.tokenStream("variedField", "boring");
|
||||||
// Filter should not prevent stopwords in one field being used in another
|
// Filter should not prevent stopwords in one field being used in another
|
||||||
assertTokenStreamContents(protectedTokenStream, new String[]{"boring"});
|
assertTokenStreamContents(protectedTokenStream, new String[]{"boring"});
|
||||||
}
|
}
|
||||||
|
@ -134,7 +133,7 @@ public class QueryAutoStopWordAnalyzerTest extends BaseTokenStreamTestCase {
|
||||||
QueryAutoStopWordAnalyzer a = new QueryAutoStopWordAnalyzer(
|
QueryAutoStopWordAnalyzer a = new QueryAutoStopWordAnalyzer(
|
||||||
TEST_VERSION_CURRENT,
|
TEST_VERSION_CURRENT,
|
||||||
new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false), reader, 10);
|
new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false), reader, 10);
|
||||||
TokenStream ts = a.tokenStream("repetitiveField", new StringReader("this boring"));
|
TokenStream ts = a.tokenStream("repetitiveField", "this boring");
|
||||||
assertTokenStreamContents(ts, new String[] { "this" });
|
assertTokenStreamContents(ts, new String[] { "this" });
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -17,8 +17,6 @@ package org.apache.lucene.analysis.shingle;
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
import java.io.StringReader;
|
|
||||||
|
|
||||||
import org.apache.lucene.analysis.Analyzer;
|
import org.apache.lucene.analysis.Analyzer;
|
||||||
import org.apache.lucene.analysis.BaseTokenStreamTestCase;
|
import org.apache.lucene.analysis.BaseTokenStreamTestCase;
|
||||||
import org.apache.lucene.analysis.MockAnalyzer;
|
import org.apache.lucene.analysis.MockAnalyzer;
|
||||||
|
@ -97,7 +95,7 @@ public class ShingleAnalyzerWrapperTest extends BaseTokenStreamTestCase {
|
||||||
public void testShingleAnalyzerWrapperPhraseQuery() throws Exception {
|
public void testShingleAnalyzerWrapperPhraseQuery() throws Exception {
|
||||||
PhraseQuery q = new PhraseQuery();
|
PhraseQuery q = new PhraseQuery();
|
||||||
|
|
||||||
TokenStream ts = analyzer.tokenStream("content", new StringReader("this sentence"));
|
TokenStream ts = analyzer.tokenStream("content", "this sentence");
|
||||||
int j = -1;
|
int j = -1;
|
||||||
|
|
||||||
PositionIncrementAttribute posIncrAtt = ts.addAttribute(PositionIncrementAttribute.class);
|
PositionIncrementAttribute posIncrAtt = ts.addAttribute(PositionIncrementAttribute.class);
|
||||||
|
@ -123,7 +121,7 @@ public class ShingleAnalyzerWrapperTest extends BaseTokenStreamTestCase {
|
||||||
public void testShingleAnalyzerWrapperBooleanQuery() throws Exception {
|
public void testShingleAnalyzerWrapperBooleanQuery() throws Exception {
|
||||||
BooleanQuery q = new BooleanQuery();
|
BooleanQuery q = new BooleanQuery();
|
||||||
|
|
||||||
TokenStream ts = analyzer.tokenStream("content", new StringReader("test sentence"));
|
TokenStream ts = analyzer.tokenStream("content", "test sentence");
|
||||||
|
|
||||||
CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class);
|
CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class);
|
||||||
|
|
||||||
|
|
|
@ -91,7 +91,7 @@ public class TestTeeSinkTokenFilter extends BaseTokenStreamTestCase {
|
||||||
Analyzer analyzer = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false);
|
Analyzer analyzer = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false);
|
||||||
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer));
|
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer));
|
||||||
Document doc = new Document();
|
Document doc = new Document();
|
||||||
TokenStream tokenStream = analyzer.tokenStream("field", new StringReader("abcd "));
|
TokenStream tokenStream = analyzer.tokenStream("field", "abcd ");
|
||||||
TeeSinkTokenFilter tee = new TeeSinkTokenFilter(tokenStream);
|
TeeSinkTokenFilter tee = new TeeSinkTokenFilter(tokenStream);
|
||||||
TokenStream sink = tee.newSinkTokenStream();
|
TokenStream sink = tee.newSinkTokenStream();
|
||||||
FieldType ft = new FieldType(TextField.TYPE_NOT_STORED);
|
FieldType ft = new FieldType(TextField.TYPE_NOT_STORED);
|
||||||
|
|
|
@ -19,7 +19,6 @@ package org.apache.lucene.analysis.th;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.Reader;
|
import java.io.Reader;
|
||||||
import java.io.StringReader;
|
|
||||||
import java.util.Random;
|
import java.util.Random;
|
||||||
|
|
||||||
import org.apache.lucene.analysis.Analyzer;
|
import org.apache.lucene.analysis.Analyzer;
|
||||||
|
@ -30,7 +29,6 @@ import org.apache.lucene.analysis.core.KeywordTokenizer;
|
||||||
import org.apache.lucene.analysis.core.StopAnalyzer;
|
import org.apache.lucene.analysis.core.StopAnalyzer;
|
||||||
import org.apache.lucene.analysis.tokenattributes.FlagsAttribute;
|
import org.apache.lucene.analysis.tokenattributes.FlagsAttribute;
|
||||||
import org.apache.lucene.analysis.util.CharArraySet;
|
import org.apache.lucene.analysis.util.CharArraySet;
|
||||||
import org.apache.lucene.util.Version;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Test case for ThaiAnalyzer, modified from TestFrenchAnalyzer
|
* Test case for ThaiAnalyzer, modified from TestFrenchAnalyzer
|
||||||
|
@ -122,10 +120,10 @@ public class TestThaiAnalyzer extends BaseTokenStreamTestCase {
|
||||||
public void testAttributeReuse() throws Exception {
|
public void testAttributeReuse() throws Exception {
|
||||||
ThaiAnalyzer analyzer = new ThaiAnalyzer(TEST_VERSION_CURRENT);
|
ThaiAnalyzer analyzer = new ThaiAnalyzer(TEST_VERSION_CURRENT);
|
||||||
// just consume
|
// just consume
|
||||||
TokenStream ts = analyzer.tokenStream("dummy", new StringReader("ภาษาไทย"));
|
TokenStream ts = analyzer.tokenStream("dummy", "ภาษาไทย");
|
||||||
assertTokenStreamContents(ts, new String[] { "ภาษา", "ไทย" });
|
assertTokenStreamContents(ts, new String[] { "ภาษา", "ไทย" });
|
||||||
// this consumer adds flagsAtt, which this analyzer does not use.
|
// this consumer adds flagsAtt, which this analyzer does not use.
|
||||||
ts = analyzer.tokenStream("dummy", new StringReader("ภาษาไทย"));
|
ts = analyzer.tokenStream("dummy", "ภาษาไทย");
|
||||||
ts.addAttribute(FlagsAttribute.class);
|
ts.addAttribute(FlagsAttribute.class);
|
||||||
assertTokenStreamContents(ts, new String[] { "ภาษา", "ไทย" });
|
assertTokenStreamContents(ts, new String[] { "ภาษา", "ไทย" });
|
||||||
}
|
}
|
||||||
|
|
|
@ -123,7 +123,7 @@ public class TestCharTokenizers extends BaseTokenStreamTestCase {
|
||||||
int num = 1000 * RANDOM_MULTIPLIER;
|
int num = 1000 * RANDOM_MULTIPLIER;
|
||||||
for (int i = 0; i < num; i++) {
|
for (int i = 0; i < num; i++) {
|
||||||
String s = _TestUtil.randomUnicodeString(random());
|
String s = _TestUtil.randomUnicodeString(random());
|
||||||
TokenStream ts = analyzer.tokenStream("foo", new StringReader(s));
|
TokenStream ts = analyzer.tokenStream("foo", s);
|
||||||
ts.reset();
|
ts.reset();
|
||||||
OffsetAttribute offsetAtt = ts.addAttribute(OffsetAttribute.class);
|
OffsetAttribute offsetAtt = ts.addAttribute(OffsetAttribute.class);
|
||||||
while (ts.incrementToken()) {
|
while (ts.incrementToken()) {
|
||||||
|
@ -161,7 +161,7 @@ public class TestCharTokenizers extends BaseTokenStreamTestCase {
|
||||||
int num = 1000 * RANDOM_MULTIPLIER;
|
int num = 1000 * RANDOM_MULTIPLIER;
|
||||||
for (int i = 0; i < num; i++) {
|
for (int i = 0; i < num; i++) {
|
||||||
String s = _TestUtil.randomUnicodeString(random());
|
String s = _TestUtil.randomUnicodeString(random());
|
||||||
TokenStream ts = analyzer.tokenStream("foo", new StringReader(s));
|
TokenStream ts = analyzer.tokenStream("foo", s);
|
||||||
ts.reset();
|
ts.reset();
|
||||||
OffsetAttribute offsetAtt = ts.addAttribute(OffsetAttribute.class);
|
OffsetAttribute offsetAtt = ts.addAttribute(OffsetAttribute.class);
|
||||||
while (ts.incrementToken()) {
|
while (ts.incrementToken()) {
|
||||||
|
|
|
@ -249,7 +249,7 @@ public class TestICUTokenizer extends BaseTokenStreamTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testTokenAttributes() throws Exception {
|
public void testTokenAttributes() throws Exception {
|
||||||
TokenStream ts = a.tokenStream("dummy", new StringReader("This is a test"));
|
TokenStream ts = a.tokenStream("dummy", "This is a test");
|
||||||
ScriptAttribute scriptAtt = ts.addAttribute(ScriptAttribute.class);
|
ScriptAttribute scriptAtt = ts.addAttribute(ScriptAttribute.class);
|
||||||
ts.reset();
|
ts.reset();
|
||||||
while (ts.incrementToken()) {
|
while (ts.incrementToken()) {
|
||||||
|
|
|
@ -19,7 +19,6 @@ package org.apache.lucene.analysis.ja;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.Reader;
|
import java.io.Reader;
|
||||||
import java.io.StringReader;
|
|
||||||
import java.util.Random;
|
import java.util.Random;
|
||||||
|
|
||||||
import org.apache.lucene.analysis.Analyzer;
|
import org.apache.lucene.analysis.Analyzer;
|
||||||
|
@ -54,7 +53,7 @@ public class TestExtendedMode extends BaseTokenStreamTestCase {
|
||||||
int numIterations = atLeast(1000);
|
int numIterations = atLeast(1000);
|
||||||
for (int i = 0; i < numIterations; i++) {
|
for (int i = 0; i < numIterations; i++) {
|
||||||
String s = _TestUtil.randomUnicodeString(random(), 100);
|
String s = _TestUtil.randomUnicodeString(random(), 100);
|
||||||
TokenStream ts = analyzer.tokenStream("foo", new StringReader(s));
|
TokenStream ts = analyzer.tokenStream("foo", s);
|
||||||
CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class);
|
CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class);
|
||||||
ts.reset();
|
ts.reset();
|
||||||
while (ts.incrementToken()) {
|
while (ts.incrementToken()) {
|
||||||
|
|
|
@ -18,7 +18,6 @@ package org.apache.lucene.analysis.ja;
|
||||||
*/
|
*/
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.StringReader;
|
|
||||||
import java.util.Random;
|
import java.util.Random;
|
||||||
|
|
||||||
import org.apache.lucene.analysis.Analyzer;
|
import org.apache.lucene.analysis.Analyzer;
|
||||||
|
@ -151,7 +150,7 @@ public class TestJapaneseAnalyzer extends BaseTokenStreamTestCase {
|
||||||
Mode.SEARCH,
|
Mode.SEARCH,
|
||||||
JapaneseAnalyzer.getDefaultStopSet(),
|
JapaneseAnalyzer.getDefaultStopSet(),
|
||||||
JapaneseAnalyzer.getDefaultStopTags());
|
JapaneseAnalyzer.getDefaultStopTags());
|
||||||
assertTokenStreamContents(a.tokenStream("foo", new StringReader("abcd")),
|
assertTokenStreamContents(a.tokenStream("foo", "abcd"),
|
||||||
new String[] { "a", "b", "cd" },
|
new String[] { "a", "b", "cd" },
|
||||||
new int[] { 0, 1, 2 },
|
new int[] { 0, 1, 2 },
|
||||||
new int[] { 1, 2, 4 },
|
new int[] { 1, 2, 4 },
|
||||||
|
|
|
@ -22,7 +22,6 @@ import java.io.InputStream;
|
||||||
import java.io.InputStreamReader;
|
import java.io.InputStreamReader;
|
||||||
import java.io.LineNumberReader;
|
import java.io.LineNumberReader;
|
||||||
import java.io.Reader;
|
import java.io.Reader;
|
||||||
import java.io.StringReader;
|
|
||||||
import java.util.Random;
|
import java.util.Random;
|
||||||
|
|
||||||
import org.apache.lucene.analysis.Analyzer;
|
import org.apache.lucene.analysis.Analyzer;
|
||||||
|
@ -142,7 +141,7 @@ public class TestJapaneseTokenizer extends BaseTokenStreamTestCase {
|
||||||
* ideally the test would actually fail instead of hanging...
|
* ideally the test would actually fail instead of hanging...
|
||||||
*/
|
*/
|
||||||
public void testDecomposition5() throws Exception {
|
public void testDecomposition5() throws Exception {
|
||||||
TokenStream ts = analyzer.tokenStream("bogus", new StringReader("くよくよくよくよくよくよくよくよくよくよくよくよくよくよくよくよくよくよくよくよ"));
|
TokenStream ts = analyzer.tokenStream("bogus", "くよくよくよくよくよくよくよくよくよくよくよくよくよくよくよくよくよくよくよくよ");
|
||||||
ts.reset();
|
ts.reset();
|
||||||
while (ts.incrementToken()) {
|
while (ts.incrementToken()) {
|
||||||
|
|
||||||
|
@ -166,8 +165,8 @@ public class TestJapaneseTokenizer extends BaseTokenStreamTestCase {
|
||||||
/** Tests that sentence offset is incorporated into the resulting offsets */
|
/** Tests that sentence offset is incorporated into the resulting offsets */
|
||||||
public void testTwoSentences() throws Exception {
|
public void testTwoSentences() throws Exception {
|
||||||
/*
|
/*
|
||||||
//TokenStream ts = a.tokenStream("foo", new StringReader("妹の咲子です。俺と年子で、今受験生です。"));
|
//TokenStream ts = a.tokenStream("foo", "妹の咲子です。俺と年子で、今受験生です。");
|
||||||
TokenStream ts = analyzer.tokenStream("foo", new StringReader("�<!--\"<!--#<!--;?><!--#<!--#><!---->?>-->;"));
|
TokenStream ts = analyzer.tokenStream("foo", "�<!--\"<!--#<!--;?><!--#<!--#><!---->?>-->;");
|
||||||
ts.reset();
|
ts.reset();
|
||||||
CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class);
|
CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class);
|
||||||
while(ts.incrementToken()) {
|
while(ts.incrementToken()) {
|
||||||
|
@ -214,7 +213,7 @@ public class TestJapaneseTokenizer extends BaseTokenStreamTestCase {
|
||||||
public void testLargeDocReliability() throws Exception {
|
public void testLargeDocReliability() throws Exception {
|
||||||
for (int i = 0; i < 100; i++) {
|
for (int i = 0; i < 100; i++) {
|
||||||
String s = _TestUtil.randomUnicodeString(random(), 10000);
|
String s = _TestUtil.randomUnicodeString(random(), 10000);
|
||||||
TokenStream ts = analyzer.tokenStream("foo", new StringReader(s));
|
TokenStream ts = analyzer.tokenStream("foo", s);
|
||||||
ts.reset();
|
ts.reset();
|
||||||
while (ts.incrementToken()) {
|
while (ts.incrementToken()) {
|
||||||
}
|
}
|
||||||
|
@ -235,7 +234,7 @@ public class TestJapaneseTokenizer extends BaseTokenStreamTestCase {
|
||||||
System.out.println("\nTEST: iter=" + i);
|
System.out.println("\nTEST: iter=" + i);
|
||||||
}
|
}
|
||||||
String s = _TestUtil.randomUnicodeString(random(), 100);
|
String s = _TestUtil.randomUnicodeString(random(), 100);
|
||||||
TokenStream ts = analyzer.tokenStream("foo", new StringReader(s));
|
TokenStream ts = analyzer.tokenStream("foo", s);
|
||||||
CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class);
|
CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class);
|
||||||
ts.reset();
|
ts.reset();
|
||||||
while (ts.incrementToken()) {
|
while (ts.incrementToken()) {
|
||||||
|
@ -245,14 +244,14 @@ public class TestJapaneseTokenizer extends BaseTokenStreamTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testOnlyPunctuation() throws IOException {
|
public void testOnlyPunctuation() throws IOException {
|
||||||
TokenStream ts = analyzerNoPunct.tokenStream("foo", new StringReader("。、。。"));
|
TokenStream ts = analyzerNoPunct.tokenStream("foo", "。、。。");
|
||||||
ts.reset();
|
ts.reset();
|
||||||
assertFalse(ts.incrementToken());
|
assertFalse(ts.incrementToken());
|
||||||
ts.end();
|
ts.end();
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testOnlyPunctuationExtended() throws IOException {
|
public void testOnlyPunctuationExtended() throws IOException {
|
||||||
TokenStream ts = extendedModeAnalyzerNoPunct.tokenStream("foo", new StringReader("......"));
|
TokenStream ts = extendedModeAnalyzerNoPunct.tokenStream("foo", "......");
|
||||||
ts.reset();
|
ts.reset();
|
||||||
assertFalse(ts.incrementToken());
|
assertFalse(ts.incrementToken());
|
||||||
ts.end();
|
ts.end();
|
||||||
|
@ -261,14 +260,14 @@ public class TestJapaneseTokenizer extends BaseTokenStreamTestCase {
|
||||||
// note: test is kinda silly since kuromoji emits punctuation tokens.
|
// note: test is kinda silly since kuromoji emits punctuation tokens.
|
||||||
// but, when/if we filter these out it will be useful.
|
// but, when/if we filter these out it will be useful.
|
||||||
public void testEnd() throws Exception {
|
public void testEnd() throws Exception {
|
||||||
assertTokenStreamContents(analyzerNoPunct.tokenStream("foo", new StringReader("これは本ではない")),
|
assertTokenStreamContents(analyzerNoPunct.tokenStream("foo", "これは本ではない"),
|
||||||
new String[] { "これ", "は", "本", "で", "は", "ない" },
|
new String[] { "これ", "は", "本", "で", "は", "ない" },
|
||||||
new int[] { 0, 2, 3, 4, 5, 6 },
|
new int[] { 0, 2, 3, 4, 5, 6 },
|
||||||
new int[] { 2, 3, 4, 5, 6, 8 },
|
new int[] { 2, 3, 4, 5, 6, 8 },
|
||||||
new Integer(8)
|
new Integer(8)
|
||||||
);
|
);
|
||||||
|
|
||||||
assertTokenStreamContents(analyzerNoPunct.tokenStream("foo", new StringReader("これは本ではない ")),
|
assertTokenStreamContents(analyzerNoPunct.tokenStream("foo", "これは本ではない "),
|
||||||
new String[] { "これ", "は", "本", "で", "は", "ない" },
|
new String[] { "これ", "は", "本", "で", "は", "ない" },
|
||||||
new int[] { 0, 2, 3, 4, 5, 6, 8 },
|
new int[] { 0, 2, 3, 4, 5, 6, 8 },
|
||||||
new int[] { 2, 3, 4, 5, 6, 8, 9 },
|
new int[] { 2, 3, 4, 5, 6, 8, 9 },
|
||||||
|
@ -279,7 +278,7 @@ public class TestJapaneseTokenizer extends BaseTokenStreamTestCase {
|
||||||
public void testUserDict() throws Exception {
|
public void testUserDict() throws Exception {
|
||||||
// Not a great test because w/o userdict.txt the
|
// Not a great test because w/o userdict.txt the
|
||||||
// segmentation is the same:
|
// segmentation is the same:
|
||||||
assertTokenStreamContents(analyzer.tokenStream("foo", new StringReader("関西国際空港に行った")),
|
assertTokenStreamContents(analyzer.tokenStream("foo", "関西国際空港に行った"),
|
||||||
new String[] { "関西", "国際", "空港", "に", "行っ", "た" },
|
new String[] { "関西", "国際", "空港", "に", "行っ", "た" },
|
||||||
new int[] { 0, 2, 4, 6, 7, 9 },
|
new int[] { 0, 2, 4, 6, 7, 9 },
|
||||||
new int[] { 2, 4, 6, 7, 9, 10 },
|
new int[] { 2, 4, 6, 7, 9, 10 },
|
||||||
|
@ -289,7 +288,7 @@ public class TestJapaneseTokenizer extends BaseTokenStreamTestCase {
|
||||||
|
|
||||||
public void testUserDict2() throws Exception {
|
public void testUserDict2() throws Exception {
|
||||||
// Better test: w/o userdict the segmentation is different:
|
// Better test: w/o userdict the segmentation is different:
|
||||||
assertTokenStreamContents(analyzer.tokenStream("foo", new StringReader("朝青龍")),
|
assertTokenStreamContents(analyzer.tokenStream("foo", "朝青龍"),
|
||||||
new String[] { "朝青龍" },
|
new String[] { "朝青龍" },
|
||||||
new int[] { 0 },
|
new int[] { 0 },
|
||||||
new int[] { 3 },
|
new int[] { 3 },
|
||||||
|
@ -299,7 +298,7 @@ public class TestJapaneseTokenizer extends BaseTokenStreamTestCase {
|
||||||
|
|
||||||
public void testUserDict3() throws Exception {
|
public void testUserDict3() throws Exception {
|
||||||
// Test entry that breaks into multiple tokens:
|
// Test entry that breaks into multiple tokens:
|
||||||
assertTokenStreamContents(analyzer.tokenStream("foo", new StringReader("abcd")),
|
assertTokenStreamContents(analyzer.tokenStream("foo", "abcd"),
|
||||||
new String[] { "a", "b", "cd" },
|
new String[] { "a", "b", "cd" },
|
||||||
new int[] { 0, 1, 2 },
|
new int[] { 0, 1, 2 },
|
||||||
new int[] { 1, 2, 4 },
|
new int[] { 1, 2, 4 },
|
||||||
|
@ -315,7 +314,7 @@ public class TestJapaneseTokenizer extends BaseTokenStreamTestCase {
|
||||||
/*
|
/*
|
||||||
public void testUserDict4() throws Exception {
|
public void testUserDict4() throws Exception {
|
||||||
// Test entry that has another entry as prefix
|
// Test entry that has another entry as prefix
|
||||||
assertTokenStreamContents(analyzer.tokenStream("foo", new StringReader("abcdefghij")),
|
assertTokenStreamContents(analyzer.tokenStream("foo", "abcdefghij"),
|
||||||
new String[] { "ab", "cd", "efg", "hij" },
|
new String[] { "ab", "cd", "efg", "hij" },
|
||||||
new int[] { 0, 2, 4, 7 },
|
new int[] { 0, 2, 4, 7 },
|
||||||
new int[] { 2, 4, 7, 10 },
|
new int[] { 2, 4, 7, 10 },
|
||||||
|
@ -366,7 +365,7 @@ public class TestJapaneseTokenizer extends BaseTokenStreamTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
private void assertReadings(String input, String... readings) throws IOException {
|
private void assertReadings(String input, String... readings) throws IOException {
|
||||||
TokenStream ts = analyzer.tokenStream("ignored", new StringReader(input));
|
TokenStream ts = analyzer.tokenStream("ignored", input);
|
||||||
ReadingAttribute readingAtt = ts.addAttribute(ReadingAttribute.class);
|
ReadingAttribute readingAtt = ts.addAttribute(ReadingAttribute.class);
|
||||||
ts.reset();
|
ts.reset();
|
||||||
for(String reading : readings) {
|
for(String reading : readings) {
|
||||||
|
@ -378,7 +377,7 @@ public class TestJapaneseTokenizer extends BaseTokenStreamTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
private void assertPronunciations(String input, String... pronunciations) throws IOException {
|
private void assertPronunciations(String input, String... pronunciations) throws IOException {
|
||||||
TokenStream ts = analyzer.tokenStream("ignored", new StringReader(input));
|
TokenStream ts = analyzer.tokenStream("ignored", input);
|
||||||
ReadingAttribute readingAtt = ts.addAttribute(ReadingAttribute.class);
|
ReadingAttribute readingAtt = ts.addAttribute(ReadingAttribute.class);
|
||||||
ts.reset();
|
ts.reset();
|
||||||
for(String pronunciation : pronunciations) {
|
for(String pronunciation : pronunciations) {
|
||||||
|
@ -390,7 +389,7 @@ public class TestJapaneseTokenizer extends BaseTokenStreamTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
private void assertBaseForms(String input, String... baseForms) throws IOException {
|
private void assertBaseForms(String input, String... baseForms) throws IOException {
|
||||||
TokenStream ts = analyzer.tokenStream("ignored", new StringReader(input));
|
TokenStream ts = analyzer.tokenStream("ignored", input);
|
||||||
BaseFormAttribute baseFormAtt = ts.addAttribute(BaseFormAttribute.class);
|
BaseFormAttribute baseFormAtt = ts.addAttribute(BaseFormAttribute.class);
|
||||||
ts.reset();
|
ts.reset();
|
||||||
for(String baseForm : baseForms) {
|
for(String baseForm : baseForms) {
|
||||||
|
@ -402,7 +401,7 @@ public class TestJapaneseTokenizer extends BaseTokenStreamTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
private void assertInflectionTypes(String input, String... inflectionTypes) throws IOException {
|
private void assertInflectionTypes(String input, String... inflectionTypes) throws IOException {
|
||||||
TokenStream ts = analyzer.tokenStream("ignored", new StringReader(input));
|
TokenStream ts = analyzer.tokenStream("ignored", input);
|
||||||
InflectionAttribute inflectionAtt = ts.addAttribute(InflectionAttribute.class);
|
InflectionAttribute inflectionAtt = ts.addAttribute(InflectionAttribute.class);
|
||||||
ts.reset();
|
ts.reset();
|
||||||
for(String inflectionType : inflectionTypes) {
|
for(String inflectionType : inflectionTypes) {
|
||||||
|
@ -414,7 +413,7 @@ public class TestJapaneseTokenizer extends BaseTokenStreamTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
private void assertInflectionForms(String input, String... inflectionForms) throws IOException {
|
private void assertInflectionForms(String input, String... inflectionForms) throws IOException {
|
||||||
TokenStream ts = analyzer.tokenStream("ignored", new StringReader(input));
|
TokenStream ts = analyzer.tokenStream("ignored", input);
|
||||||
InflectionAttribute inflectionAtt = ts.addAttribute(InflectionAttribute.class);
|
InflectionAttribute inflectionAtt = ts.addAttribute(InflectionAttribute.class);
|
||||||
ts.reset();
|
ts.reset();
|
||||||
for(String inflectionForm : inflectionForms) {
|
for(String inflectionForm : inflectionForms) {
|
||||||
|
@ -426,7 +425,7 @@ public class TestJapaneseTokenizer extends BaseTokenStreamTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
private void assertPartsOfSpeech(String input, String... partsOfSpeech) throws IOException {
|
private void assertPartsOfSpeech(String input, String... partsOfSpeech) throws IOException {
|
||||||
TokenStream ts = analyzer.tokenStream("ignored", new StringReader(input));
|
TokenStream ts = analyzer.tokenStream("ignored", input);
|
||||||
PartOfSpeechAttribute partOfSpeechAtt = ts.addAttribute(PartOfSpeechAttribute.class);
|
PartOfSpeechAttribute partOfSpeechAtt = ts.addAttribute(PartOfSpeechAttribute.class);
|
||||||
ts.reset();
|
ts.reset();
|
||||||
for(String partOfSpeech : partsOfSpeech) {
|
for(String partOfSpeech : partsOfSpeech) {
|
||||||
|
@ -619,7 +618,7 @@ public class TestJapaneseTokenizer extends BaseTokenStreamTestCase {
|
||||||
if (numIterations > 1) {
|
if (numIterations > 1) {
|
||||||
// warmup
|
// warmup
|
||||||
for (int i = 0; i < numIterations; i++) {
|
for (int i = 0; i < numIterations; i++) {
|
||||||
final TokenStream ts = analyzer.tokenStream("ignored", new StringReader(line));
|
final TokenStream ts = analyzer.tokenStream("ignored", line);
|
||||||
ts.reset();
|
ts.reset();
|
||||||
while(ts.incrementToken());
|
while(ts.incrementToken());
|
||||||
}
|
}
|
||||||
|
@ -628,7 +627,7 @@ public class TestJapaneseTokenizer extends BaseTokenStreamTestCase {
|
||||||
|
|
||||||
long totalStart = System.currentTimeMillis();
|
long totalStart = System.currentTimeMillis();
|
||||||
for (int i = 0; i < numIterations; i++) {
|
for (int i = 0; i < numIterations; i++) {
|
||||||
final TokenStream ts = analyzer.tokenStream("ignored", new StringReader(line));
|
final TokenStream ts = analyzer.tokenStream("ignored", line);
|
||||||
ts.reset();
|
ts.reset();
|
||||||
while(ts.incrementToken());
|
while(ts.incrementToken());
|
||||||
}
|
}
|
||||||
|
@ -640,7 +639,7 @@ public class TestJapaneseTokenizer extends BaseTokenStreamTestCase {
|
||||||
totalStart = System.currentTimeMillis();
|
totalStart = System.currentTimeMillis();
|
||||||
for (int i = 0; i < numIterations; i++) {
|
for (int i = 0; i < numIterations; i++) {
|
||||||
for (String sentence: sentences) {
|
for (String sentence: sentences) {
|
||||||
final TokenStream ts = analyzer.tokenStream("ignored", new StringReader(sentence));
|
final TokenStream ts = analyzer.tokenStream("ignored", sentence);
|
||||||
ts.reset();
|
ts.reset();
|
||||||
while(ts.incrementToken());
|
while(ts.incrementToken());
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,7 +19,6 @@ package org.apache.lucene.analysis.morfologik;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.Reader;
|
import java.io.Reader;
|
||||||
import java.io.StringReader;
|
|
||||||
import java.util.TreeSet;
|
import java.util.TreeSet;
|
||||||
|
|
||||||
import org.apache.lucene.analysis.Analyzer;
|
import org.apache.lucene.analysis.Analyzer;
|
||||||
|
@ -73,7 +72,7 @@ public class TestMorfologikAnalyzer extends BaseTokenStreamTestCase {
|
||||||
|
|
||||||
@SuppressWarnings("unused")
|
@SuppressWarnings("unused")
|
||||||
private void dumpTokens(String input) throws IOException {
|
private void dumpTokens(String input) throws IOException {
|
||||||
TokenStream ts = getTestAnalyzer().tokenStream("dummy", new StringReader(input));
|
TokenStream ts = getTestAnalyzer().tokenStream("dummy", input);
|
||||||
ts.reset();
|
ts.reset();
|
||||||
|
|
||||||
MorphosyntacticTagsAttribute attribute = ts.getAttribute(MorphosyntacticTagsAttribute.class);
|
MorphosyntacticTagsAttribute attribute = ts.getAttribute(MorphosyntacticTagsAttribute.class);
|
||||||
|
@ -86,7 +85,7 @@ public class TestMorfologikAnalyzer extends BaseTokenStreamTestCase {
|
||||||
/** Test reuse of MorfologikFilter with leftover stems. */
|
/** Test reuse of MorfologikFilter with leftover stems. */
|
||||||
public final void testLeftoverStems() throws IOException {
|
public final void testLeftoverStems() throws IOException {
|
||||||
Analyzer a = getTestAnalyzer();
|
Analyzer a = getTestAnalyzer();
|
||||||
TokenStream ts_1 = a.tokenStream("dummy", new StringReader("liście"));
|
TokenStream ts_1 = a.tokenStream("dummy", "liście");
|
||||||
CharTermAttribute termAtt_1 = ts_1.getAttribute(CharTermAttribute.class);
|
CharTermAttribute termAtt_1 = ts_1.getAttribute(CharTermAttribute.class);
|
||||||
ts_1.reset();
|
ts_1.reset();
|
||||||
ts_1.incrementToken();
|
ts_1.incrementToken();
|
||||||
|
@ -94,7 +93,7 @@ public class TestMorfologikAnalyzer extends BaseTokenStreamTestCase {
|
||||||
ts_1.end();
|
ts_1.end();
|
||||||
ts_1.close();
|
ts_1.close();
|
||||||
|
|
||||||
TokenStream ts_2 = a.tokenStream("dummy", new StringReader("danych"));
|
TokenStream ts_2 = a.tokenStream("dummy", "danych");
|
||||||
CharTermAttribute termAtt_2 = ts_2.getAttribute(CharTermAttribute.class);
|
CharTermAttribute termAtt_2 = ts_2.getAttribute(CharTermAttribute.class);
|
||||||
ts_2.reset();
|
ts_2.reset();
|
||||||
ts_2.incrementToken();
|
ts_2.incrementToken();
|
||||||
|
@ -141,7 +140,7 @@ public class TestMorfologikAnalyzer extends BaseTokenStreamTestCase {
|
||||||
|
|
||||||
/** Test morphosyntactic annotations. */
|
/** Test morphosyntactic annotations. */
|
||||||
public final void testPOSAttribute() throws IOException {
|
public final void testPOSAttribute() throws IOException {
|
||||||
TokenStream ts = getTestAnalyzer().tokenStream("dummy", new StringReader("liście"));
|
TokenStream ts = getTestAnalyzer().tokenStream("dummy", "liście");
|
||||||
|
|
||||||
ts.reset();
|
ts.reset();
|
||||||
assertPOSToken(ts, "liście",
|
assertPOSToken(ts, "liście",
|
||||||
|
|
|
@ -19,7 +19,6 @@ package org.apache.lucene.analysis.cn.smart;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.Reader;
|
import java.io.Reader;
|
||||||
import java.io.StringReader;
|
|
||||||
import java.util.Random;
|
import java.util.Random;
|
||||||
|
|
||||||
import org.apache.lucene.analysis.BaseTokenStreamTestCase;
|
import org.apache.lucene.analysis.BaseTokenStreamTestCase;
|
||||||
|
@ -185,7 +184,7 @@ public class TestSmartChineseAnalyzer extends BaseTokenStreamTestCase {
|
||||||
sb.append("我购买了道具和服装。");
|
sb.append("我购买了道具和服装。");
|
||||||
}
|
}
|
||||||
Analyzer analyzer = new SmartChineseAnalyzer(TEST_VERSION_CURRENT);
|
Analyzer analyzer = new SmartChineseAnalyzer(TEST_VERSION_CURRENT);
|
||||||
TokenStream stream = analyzer.tokenStream("", new StringReader(sb.toString()));
|
TokenStream stream = analyzer.tokenStream("", sb.toString());
|
||||||
stream.reset();
|
stream.reset();
|
||||||
while (stream.incrementToken()) {
|
while (stream.incrementToken()) {
|
||||||
}
|
}
|
||||||
|
@ -198,7 +197,7 @@ public class TestSmartChineseAnalyzer extends BaseTokenStreamTestCase {
|
||||||
sb.append("我购买了道具和服装");
|
sb.append("我购买了道具和服装");
|
||||||
}
|
}
|
||||||
Analyzer analyzer = new SmartChineseAnalyzer(TEST_VERSION_CURRENT);
|
Analyzer analyzer = new SmartChineseAnalyzer(TEST_VERSION_CURRENT);
|
||||||
TokenStream stream = analyzer.tokenStream("", new StringReader(sb.toString()));
|
TokenStream stream = analyzer.tokenStream("", sb.toString());
|
||||||
stream.reset();
|
stream.reset();
|
||||||
while (stream.incrementToken()) {
|
while (stream.incrementToken()) {
|
||||||
}
|
}
|
||||||
|
|
|
@ -35,7 +35,6 @@ import org.junit.After;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
import java.io.StringReader;
|
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
|
@ -62,7 +61,7 @@ public class UIMABaseAnalyzerTest extends BaseTokenStreamTestCase {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void baseUIMAAnalyzerStreamTest() throws Exception {
|
public void baseUIMAAnalyzerStreamTest() throws Exception {
|
||||||
TokenStream ts = analyzer.tokenStream("text", new StringReader("the big brown fox jumped on the wood"));
|
TokenStream ts = analyzer.tokenStream("text", "the big brown fox jumped on the wood");
|
||||||
assertTokenStreamContents(ts, new String[]{"the", "big", "brown", "fox", "jumped", "on", "the", "wood"});
|
assertTokenStreamContents(ts, new String[]{"the", "big", "brown", "fox", "jumped", "on", "the", "wood"});
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -23,8 +23,6 @@ import org.junit.After;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
import java.io.StringReader;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Testcase for {@link UIMATypeAwareAnalyzer}
|
* Testcase for {@link UIMATypeAwareAnalyzer}
|
||||||
*/
|
*/
|
||||||
|
@ -51,7 +49,7 @@ public class UIMATypeAwareAnalyzerTest extends BaseTokenStreamTestCase {
|
||||||
public void baseUIMATypeAwareAnalyzerStreamTest() throws Exception {
|
public void baseUIMATypeAwareAnalyzerStreamTest() throws Exception {
|
||||||
|
|
||||||
// create a token stream
|
// create a token stream
|
||||||
TokenStream ts = analyzer.tokenStream("text", new StringReader("the big brown fox jumped on the wood"));
|
TokenStream ts = analyzer.tokenStream("text", "the big brown fox jumped on the wood");
|
||||||
|
|
||||||
// check that 'the big brown fox jumped on the wood' tokens have the expected PoS types
|
// check that 'the big brown fox jumped on the wood' tokens have the expected PoS types
|
||||||
assertTokenStreamContents(ts,
|
assertTokenStreamContents(ts,
|
||||||
|
|
|
@ -21,7 +21,6 @@ import java.io.BufferedReader;
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.FileInputStream;
|
import java.io.FileInputStream;
|
||||||
import java.io.InputStreamReader;
|
import java.io.InputStreamReader;
|
||||||
import java.io.StringReader;
|
|
||||||
import java.text.Collator;
|
import java.text.Collator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Locale;
|
import java.util.Locale;
|
||||||
|
@ -979,8 +978,8 @@ public class TestPerfTasksLogic extends BenchmarkTestCase {
|
||||||
|
|
||||||
private void assertEqualCollation(Analyzer a1, Analyzer a2, String text)
|
private void assertEqualCollation(Analyzer a1, Analyzer a2, String text)
|
||||||
throws Exception {
|
throws Exception {
|
||||||
TokenStream ts1 = a1.tokenStream("bogus", new StringReader(text));
|
TokenStream ts1 = a1.tokenStream("bogus", text);
|
||||||
TokenStream ts2 = a2.tokenStream("bogus", new StringReader(text));
|
TokenStream ts2 = a2.tokenStream("bogus", text);
|
||||||
ts1.reset();
|
ts1.reset();
|
||||||
ts2.reset();
|
ts2.reset();
|
||||||
TermToBytesRefAttribute termAtt1 = ts1.addAttribute(TermToBytesRefAttribute.class);
|
TermToBytesRefAttribute termAtt1 = ts1.addAttribute(TermToBytesRefAttribute.class);
|
||||||
|
@ -1030,7 +1029,7 @@ public class TestPerfTasksLogic extends BenchmarkTestCase {
|
||||||
Benchmark benchmark = execBenchmark(getAnalyzerFactoryConfig
|
Benchmark benchmark = execBenchmark(getAnalyzerFactoryConfig
|
||||||
("shingle-analyzer", "StandardTokenizer,ShingleFilter"));
|
("shingle-analyzer", "StandardTokenizer,ShingleFilter"));
|
||||||
benchmark.getRunData().getAnalyzer().tokenStream
|
benchmark.getRunData().getAnalyzer().tokenStream
|
||||||
("bogus", new StringReader(text)).close();
|
("bogus", text).close();
|
||||||
BaseTokenStreamTestCase.assertAnalyzesTo(benchmark.getRunData().getAnalyzer(), text,
|
BaseTokenStreamTestCase.assertAnalyzesTo(benchmark.getRunData().getAnalyzer(), text,
|
||||||
new String[] { "one", "one two", "two", "two three",
|
new String[] { "one", "one two", "two", "two three",
|
||||||
"three", "three four", "four", "four five",
|
"three", "three four", "four", "four five",
|
||||||
|
|
|
@ -33,7 +33,6 @@ import org.apache.lucene.search.WildcardQuery;
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.StringReader;
|
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.LinkedList;
|
import java.util.LinkedList;
|
||||||
|
|
||||||
|
@ -86,7 +85,7 @@ public class SimpleNaiveBayesClassifier implements Classifier<BytesRef> {
|
||||||
|
|
||||||
private String[] tokenizeDoc(String doc) throws IOException {
|
private String[] tokenizeDoc(String doc) throws IOException {
|
||||||
Collection<String> result = new LinkedList<String>();
|
Collection<String> result = new LinkedList<String>();
|
||||||
TokenStream tokenStream = analyzer.tokenStream(textFieldName, new StringReader(doc));
|
TokenStream tokenStream = analyzer.tokenStream(textFieldName, doc);
|
||||||
CharTermAttribute charTermAttribute = tokenStream.addAttribute(CharTermAttribute.class);
|
CharTermAttribute charTermAttribute = tokenStream.addAttribute(CharTermAttribute.class);
|
||||||
tokenStream.reset();
|
tokenStream.reset();
|
||||||
while (tokenStream.incrementToken()) {
|
while (tokenStream.incrementToken()) {
|
||||||
|
|
|
@ -120,11 +120,16 @@ public abstract class Analyzer implements Closeable {
|
||||||
* See the {@link org.apache.lucene.analysis Analysis package documentation} for
|
* See the {@link org.apache.lucene.analysis Analysis package documentation} for
|
||||||
* some examples demonstrating this.
|
* some examples demonstrating this.
|
||||||
*
|
*
|
||||||
|
* <b>NOTE:</b> If your data is available as a {@code String}, use
|
||||||
|
* {@link #tokenStream(String, String)} which reuses a {@code StringReader}-like
|
||||||
|
* instance internally.
|
||||||
|
*
|
||||||
* @param fieldName the name of the field the created TokenStream is used for
|
* @param fieldName the name of the field the created TokenStream is used for
|
||||||
* @param reader the reader the streams source reads from
|
* @param reader the reader the streams source reads from
|
||||||
* @return TokenStream for iterating the analyzed content of <code>reader</code>
|
* @return TokenStream for iterating the analyzed content of <code>reader</code>
|
||||||
* @throws AlreadyClosedException if the Analyzer is closed.
|
* @throws AlreadyClosedException if the Analyzer is closed.
|
||||||
* @throws IOException if an i/o error occurs.
|
* @throws IOException if an i/o error occurs.
|
||||||
|
* @see #tokenStream(String, String)
|
||||||
*/
|
*/
|
||||||
public final TokenStream tokenStream(final String fieldName,
|
public final TokenStream tokenStream(final String fieldName,
|
||||||
final Reader reader) throws IOException {
|
final Reader reader) throws IOException {
|
||||||
|
@ -139,6 +144,45 @@ public abstract class Analyzer implements Closeable {
|
||||||
return components.getTokenStream();
|
return components.getTokenStream();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns a TokenStream suitable for <code>fieldName</code>, tokenizing
|
||||||
|
* the contents of <code>text</code>.
|
||||||
|
* <p>
|
||||||
|
* This method uses {@link #createComponents(String, Reader)} to obtain an
|
||||||
|
* instance of {@link TokenStreamComponents}. It returns the sink of the
|
||||||
|
* components and stores the components internally. Subsequent calls to this
|
||||||
|
* method will reuse the previously stored components after resetting them
|
||||||
|
* through {@link TokenStreamComponents#setReader(Reader)}.
|
||||||
|
* <p>
|
||||||
|
* <b>NOTE:</b> After calling this method, the consumer must follow the
|
||||||
|
* workflow described in {@link TokenStream} to properly consume its contents.
|
||||||
|
* See the {@link org.apache.lucene.analysis Analysis package documentation} for
|
||||||
|
* some examples demonstrating this.
|
||||||
|
*
|
||||||
|
* @param fieldName the name of the field the created TokenStream is used for
|
||||||
|
* @param text the String the streams source reads from
|
||||||
|
* @return TokenStream for iterating the analyzed content of <code>reader</code>
|
||||||
|
* @throws AlreadyClosedException if the Analyzer is closed.
|
||||||
|
* @throws IOException if an i/o error occurs (may rarely happen for strings).
|
||||||
|
* @see #tokenStream(String, Reader)
|
||||||
|
*/
|
||||||
|
public final TokenStream tokenStream(final String fieldName, final String text) throws IOException {
|
||||||
|
TokenStreamComponents components = reuseStrategy.getReusableComponents(fieldName);
|
||||||
|
@SuppressWarnings("resource") final ReusableStringReader strReader =
|
||||||
|
(components == null || components.reusableStringReader == null) ?
|
||||||
|
new ReusableStringReader() : components.reusableStringReader;
|
||||||
|
strReader.setValue(text);
|
||||||
|
final Reader r = initReader(fieldName, strReader);
|
||||||
|
if (components == null) {
|
||||||
|
components = createComponents(fieldName, r);
|
||||||
|
reuseStrategy.setReusableComponents(fieldName, components);
|
||||||
|
} else {
|
||||||
|
components.setReader(r);
|
||||||
|
}
|
||||||
|
components.reusableStringReader = strReader;
|
||||||
|
return components.getTokenStream();
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Override this if you want to add a CharFilter chain.
|
* Override this if you want to add a CharFilter chain.
|
||||||
* <p>
|
* <p>
|
||||||
|
@ -208,6 +252,9 @@ public abstract class Analyzer implements Closeable {
|
||||||
* the chain. This can be the source if there are no filters.
|
* the chain. This can be the source if there are no filters.
|
||||||
*/
|
*/
|
||||||
protected final TokenStream sink;
|
protected final TokenStream sink;
|
||||||
|
|
||||||
|
/** Internal cache only used by {@link Analyzer#tokenStream(String, String)}. */
|
||||||
|
transient ReusableStringReader reusableStringReader;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates a new {@link TokenStreamComponents} instance.
|
* Creates a new {@link TokenStreamComponents} instance.
|
||||||
|
|
|
@ -0,0 +1,61 @@
|
||||||
|
package org.apache.lucene.analysis;
|
||||||
|
|
||||||
|
import java.io.Reader;
|
||||||
|
|
||||||
|
/*
|
||||||
|
* 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.
|
||||||
|
*/
|
||||||
|
|
||||||
|
/** Internal class to enable reuse of the string reader by {@link Analyzer#tokenStream(String,String)} */
|
||||||
|
final class ReusableStringReader extends Reader {
|
||||||
|
private int pos = 0, size = 0;
|
||||||
|
private String s = null;
|
||||||
|
|
||||||
|
void setValue(String s) {
|
||||||
|
this.s = s;
|
||||||
|
this.size = s.length();
|
||||||
|
this.pos = 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int read() {
|
||||||
|
if (pos < size) {
|
||||||
|
return s.charAt(pos++);
|
||||||
|
} else {
|
||||||
|
s = null;
|
||||||
|
return -1;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int read(char[] c, int off, int len) {
|
||||||
|
if (pos < size) {
|
||||||
|
len = Math.min(len, size-pos);
|
||||||
|
s.getChars(pos, pos+len, c, off);
|
||||||
|
pos += len;
|
||||||
|
return len;
|
||||||
|
} else {
|
||||||
|
s = null;
|
||||||
|
return -1;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() {
|
||||||
|
pos = size; // this prevents NPE when reading after close!
|
||||||
|
s = null;
|
||||||
|
}
|
||||||
|
}
|
|
@ -75,7 +75,6 @@ public class Field implements IndexableField, StorableField {
|
||||||
protected TokenStream tokenStream;
|
protected TokenStream tokenStream;
|
||||||
|
|
||||||
private transient TokenStream internalTokenStream;
|
private transient TokenStream internalTokenStream;
|
||||||
private transient ReusableStringReader internalReader;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Field's boost
|
* Field's boost
|
||||||
|
@ -552,56 +551,12 @@ public class Field implements IndexableField, StorableField {
|
||||||
} else if (readerValue() != null) {
|
} else if (readerValue() != null) {
|
||||||
return analyzer.tokenStream(name(), readerValue());
|
return analyzer.tokenStream(name(), readerValue());
|
||||||
} else if (stringValue() != null) {
|
} else if (stringValue() != null) {
|
||||||
if (internalReader == null) {
|
return analyzer.tokenStream(name(), stringValue());
|
||||||
internalReader = new ReusableStringReader();
|
|
||||||
}
|
|
||||||
internalReader.setValue(stringValue());
|
|
||||||
return analyzer.tokenStream(name(), internalReader);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
throw new IllegalArgumentException("Field must have either TokenStream, String, Reader or Number value");
|
throw new IllegalArgumentException("Field must have either TokenStream, String, Reader or Number value");
|
||||||
}
|
}
|
||||||
|
|
||||||
static final class ReusableStringReader extends Reader {
|
|
||||||
private int pos = 0, size = 0;
|
|
||||||
private String s = null;
|
|
||||||
|
|
||||||
void setValue(String s) {
|
|
||||||
this.s = s;
|
|
||||||
this.size = s.length();
|
|
||||||
this.pos = 0;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public int read() {
|
|
||||||
if (pos < size) {
|
|
||||||
return s.charAt(pos++);
|
|
||||||
} else {
|
|
||||||
s = null;
|
|
||||||
return -1;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public int read(char[] c, int off, int len) {
|
|
||||||
if (pos < size) {
|
|
||||||
len = Math.min(len, size-pos);
|
|
||||||
s.getChars(pos, pos+len, c, off);
|
|
||||||
pos += len;
|
|
||||||
return len;
|
|
||||||
} else {
|
|
||||||
s = null;
|
|
||||||
return -1;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void close() {
|
|
||||||
pos = size; // this prevents NPE when reading after close!
|
|
||||||
s = null;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
static final class StringTokenStream extends TokenStream {
|
static final class StringTokenStream extends TokenStream {
|
||||||
private final CharTermAttribute termAttribute = addAttribute(CharTermAttribute.class);
|
private final CharTermAttribute termAttribute = addAttribute(CharTermAttribute.class);
|
||||||
private final OffsetAttribute offsetAttribute = addAttribute(OffsetAttribute.class);
|
private final OffsetAttribute offsetAttribute = addAttribute(OffsetAttribute.class);
|
||||||
|
|
|
@ -96,7 +96,7 @@ public class TestMockAnalyzer extends BaseTokenStreamTestCase {
|
||||||
String testString = "t";
|
String testString = "t";
|
||||||
|
|
||||||
Analyzer analyzer = new MockAnalyzer(random());
|
Analyzer analyzer = new MockAnalyzer(random());
|
||||||
TokenStream stream = analyzer.tokenStream("dummy", new StringReader(testString));
|
TokenStream stream = analyzer.tokenStream("dummy", testString);
|
||||||
stream.reset();
|
stream.reset();
|
||||||
while (stream.incrementToken()) {
|
while (stream.incrementToken()) {
|
||||||
// consume
|
// consume
|
||||||
|
|
|
@ -0,0 +1,61 @@
|
||||||
|
package org.apache.lucene.analysis;
|
||||||
|
|
||||||
|
import java.nio.CharBuffer;
|
||||||
|
|
||||||
|
import org.apache.lucene.util.LuceneTestCase;
|
||||||
|
|
||||||
|
/*
|
||||||
|
* 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.
|
||||||
|
*/
|
||||||
|
public class TestReusableStringReader extends LuceneTestCase {
|
||||||
|
|
||||||
|
public void test() throws Exception {
|
||||||
|
ReusableStringReader reader = new ReusableStringReader();
|
||||||
|
assertEquals(-1, reader.read());
|
||||||
|
assertEquals(-1, reader.read(new char[1]));
|
||||||
|
assertEquals(-1, reader.read(new char[2], 1, 1));
|
||||||
|
assertEquals(-1, reader.read(CharBuffer.wrap(new char[2])));
|
||||||
|
|
||||||
|
reader.setValue("foobar");
|
||||||
|
char[] buf = new char[4];
|
||||||
|
assertEquals(4, reader.read(buf));
|
||||||
|
assertEquals("foob", new String(buf));
|
||||||
|
assertEquals(2, reader.read(buf));
|
||||||
|
assertEquals("ar", new String(buf, 0, 2));
|
||||||
|
assertEquals(-1, reader.read(buf));
|
||||||
|
reader.close();
|
||||||
|
|
||||||
|
reader.setValue("foobar");
|
||||||
|
assertEquals(0, reader.read(buf, 1, 0));
|
||||||
|
assertEquals(3, reader.read(buf, 1, 3));
|
||||||
|
assertEquals("foo", new String(buf, 1, 3));
|
||||||
|
assertEquals(2, reader.read(CharBuffer.wrap(buf, 2, 2)));
|
||||||
|
assertEquals("ba", new String(buf, 2, 2));
|
||||||
|
assertEquals('r', (char) reader.read());
|
||||||
|
assertEquals(-1, reader.read(buf));
|
||||||
|
reader.close();
|
||||||
|
|
||||||
|
reader.setValue("foobar");
|
||||||
|
StringBuilder sb = new StringBuilder();
|
||||||
|
int ch;
|
||||||
|
while ((ch = reader.read()) != -1) {
|
||||||
|
sb.append((char) ch);
|
||||||
|
}
|
||||||
|
reader.close();
|
||||||
|
assertEquals("foobar", sb.toString());
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -18,11 +18,8 @@ package org.apache.lucene.document;
|
||||||
*/
|
*/
|
||||||
|
|
||||||
import java.io.StringReader;
|
import java.io.StringReader;
|
||||||
import java.nio.CharBuffer;
|
|
||||||
|
|
||||||
import org.apache.lucene.analysis.CannedTokenStream;
|
import org.apache.lucene.analysis.CannedTokenStream;
|
||||||
import org.apache.lucene.analysis.Token;
|
import org.apache.lucene.analysis.Token;
|
||||||
import org.apache.lucene.document.Field.ReusableStringReader;
|
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
import org.apache.lucene.util.LuceneTestCase;
|
import org.apache.lucene.util.LuceneTestCase;
|
||||||
|
|
||||||
|
@ -518,39 +515,4 @@ public class TestField extends LuceneTestCase {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testReusableStringReader() throws Exception {
|
|
||||||
ReusableStringReader reader = new ReusableStringReader();
|
|
||||||
assertEquals(-1, reader.read());
|
|
||||||
assertEquals(-1, reader.read(new char[1]));
|
|
||||||
assertEquals(-1, reader.read(new char[2], 1, 1));
|
|
||||||
assertEquals(-1, reader.read(CharBuffer.wrap(new char[2])));
|
|
||||||
|
|
||||||
reader.setValue("foobar");
|
|
||||||
char[] buf = new char[4];
|
|
||||||
assertEquals(4, reader.read(buf));
|
|
||||||
assertEquals("foob", new String(buf));
|
|
||||||
assertEquals(2, reader.read(buf));
|
|
||||||
assertEquals("ar", new String(buf, 0, 2));
|
|
||||||
assertEquals(-1, reader.read(buf));
|
|
||||||
reader.close();
|
|
||||||
|
|
||||||
reader.setValue("foobar");
|
|
||||||
assertEquals(0, reader.read(buf, 1, 0));
|
|
||||||
assertEquals(3, reader.read(buf, 1, 3));
|
|
||||||
assertEquals("foo", new String(buf, 1, 3));
|
|
||||||
assertEquals(2, reader.read(CharBuffer.wrap(buf, 2, 2)));
|
|
||||||
assertEquals("ba", new String(buf, 2, 2));
|
|
||||||
assertEquals('r', (char) reader.read());
|
|
||||||
assertEquals(-1, reader.read(buf));
|
|
||||||
reader.close();
|
|
||||||
|
|
||||||
reader.setValue("foobar");
|
|
||||||
StringBuilder sb = new StringBuilder();
|
|
||||||
int ch;
|
|
||||||
while ((ch = reader.read()) != -1) {
|
|
||||||
sb.append((char) ch);
|
|
||||||
}
|
|
||||||
reader.close();
|
|
||||||
assertEquals("foobar", sb.toString());
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -18,8 +18,6 @@ package org.apache.lucene.index;
|
||||||
*/
|
*/
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.StringReader;
|
|
||||||
|
|
||||||
import org.apache.lucene.analysis.Analyzer;
|
import org.apache.lucene.analysis.Analyzer;
|
||||||
import org.apache.lucene.analysis.MockAnalyzer;
|
import org.apache.lucene.analysis.MockAnalyzer;
|
||||||
import org.apache.lucene.analysis.TokenStream;
|
import org.apache.lucene.analysis.TokenStream;
|
||||||
|
@ -49,7 +47,7 @@ public class TestLongPostings extends LuceneTestCase {
|
||||||
if (other != null && s.equals(other)) {
|
if (other != null && s.equals(other)) {
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
final TokenStream ts = a.tokenStream("foo", new StringReader(s));
|
final TokenStream ts = a.tokenStream("foo", s);
|
||||||
final TermToBytesRefAttribute termAtt = ts.getAttribute(TermToBytesRefAttribute.class);
|
final TermToBytesRefAttribute termAtt = ts.getAttribute(TermToBytesRefAttribute.class);
|
||||||
final BytesRef termBytes = termAtt.getBytesRef();
|
final BytesRef termBytes = termAtt.getBytesRef();
|
||||||
ts.reset();
|
ts.reset();
|
||||||
|
|
|
@ -18,8 +18,6 @@ package org.apache.lucene.index;
|
||||||
*/
|
*/
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.StringReader;
|
|
||||||
|
|
||||||
import org.apache.lucene.analysis.Analyzer;
|
import org.apache.lucene.analysis.Analyzer;
|
||||||
import org.apache.lucene.analysis.CachingTokenFilter;
|
import org.apache.lucene.analysis.CachingTokenFilter;
|
||||||
import org.apache.lucene.analysis.MockAnalyzer;
|
import org.apache.lucene.analysis.MockAnalyzer;
|
||||||
|
@ -176,7 +174,7 @@ public class TestTermVectorsWriter extends LuceneTestCase {
|
||||||
Analyzer analyzer = new MockAnalyzer(random());
|
Analyzer analyzer = new MockAnalyzer(random());
|
||||||
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, analyzer));
|
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, analyzer));
|
||||||
Document doc = new Document();
|
Document doc = new Document();
|
||||||
TokenStream stream = analyzer.tokenStream("field", new StringReader("abcd "));
|
TokenStream stream = analyzer.tokenStream("field", "abcd ");
|
||||||
stream.reset(); // TODO: weird to reset before wrapping with CachingTokenFilter... correct?
|
stream.reset(); // TODO: weird to reset before wrapping with CachingTokenFilter... correct?
|
||||||
stream = new CachingTokenFilter(stream);
|
stream = new CachingTokenFilter(stream);
|
||||||
FieldType customType = new FieldType(TextField.TYPE_NOT_STORED);
|
FieldType customType = new FieldType(TextField.TYPE_NOT_STORED);
|
||||||
|
|
|
@ -617,7 +617,7 @@ public class TestPhraseQuery extends LuceneTestCase {
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
TokenStream ts = analyzer.tokenStream("ignore", new StringReader(term));
|
TokenStream ts = analyzer.tokenStream("ignore", term);
|
||||||
CharTermAttribute termAttr = ts.addAttribute(CharTermAttribute.class);
|
CharTermAttribute termAttr = ts.addAttribute(CharTermAttribute.class);
|
||||||
ts.reset();
|
ts.reset();
|
||||||
while(ts.incrementToken()) {
|
while(ts.incrementToken()) {
|
||||||
|
|
|
@ -17,7 +17,6 @@ package org.apache.lucene.search.highlight;
|
||||||
*/
|
*/
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.StringReader;
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
|
|
||||||
|
@ -25,7 +24,6 @@ import org.apache.lucene.analysis.Analyzer;
|
||||||
import org.apache.lucene.analysis.TokenStream;
|
import org.apache.lucene.analysis.TokenStream;
|
||||||
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
|
||||||
import org.apache.lucene.util.PriorityQueue;
|
import org.apache.lucene.util.PriorityQueue;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -78,7 +76,7 @@ public class Highlighter
|
||||||
public final String getBestFragment(Analyzer analyzer, String fieldName,String text)
|
public final String getBestFragment(Analyzer analyzer, String fieldName,String text)
|
||||||
throws IOException, InvalidTokenOffsetsException
|
throws IOException, InvalidTokenOffsetsException
|
||||||
{
|
{
|
||||||
TokenStream tokenStream = analyzer.tokenStream(fieldName, new StringReader(text));
|
TokenStream tokenStream = analyzer.tokenStream(fieldName, text);
|
||||||
return getBestFragment(tokenStream, text);
|
return getBestFragment(tokenStream, text);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -130,7 +128,7 @@ public class Highlighter
|
||||||
int maxNumFragments)
|
int maxNumFragments)
|
||||||
throws IOException, InvalidTokenOffsetsException
|
throws IOException, InvalidTokenOffsetsException
|
||||||
{
|
{
|
||||||
TokenStream tokenStream = analyzer.tokenStream(fieldName, new StringReader(text));
|
TokenStream tokenStream = analyzer.tokenStream(fieldName, text);
|
||||||
return getBestFragments(tokenStream, text, maxNumFragments);
|
return getBestFragments(tokenStream, text, maxNumFragments);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -36,7 +36,6 @@ import org.apache.lucene.util.ArrayUtil;
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.StringReader;
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Comparator;
|
import java.util.Comparator;
|
||||||
|
|
||||||
|
@ -314,7 +313,7 @@ public class TokenSources {
|
||||||
public static TokenStream getTokenStream(String field, String contents,
|
public static TokenStream getTokenStream(String field, String contents,
|
||||||
Analyzer analyzer) {
|
Analyzer analyzer) {
|
||||||
try {
|
try {
|
||||||
return analyzer.tokenStream(field, new StringReader(contents));
|
return analyzer.tokenStream(field, contents);
|
||||||
} catch (IOException ex) {
|
} catch (IOException ex) {
|
||||||
throw new RuntimeException(ex);
|
throw new RuntimeException(ex);
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,7 +20,6 @@ package org.apache.lucene.search.highlight;
|
||||||
import java.io.ByteArrayInputStream;
|
import java.io.ByteArrayInputStream;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.Reader;
|
import java.io.Reader;
|
||||||
import java.io.StringReader;
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
|
@ -248,7 +247,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
|
||||||
private String highlightField(Query query, String fieldName, String text)
|
private String highlightField(Query query, String fieldName, String text)
|
||||||
throws IOException, InvalidTokenOffsetsException {
|
throws IOException, InvalidTokenOffsetsException {
|
||||||
TokenStream tokenStream = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true, MockTokenFilter.ENGLISH_STOPSET)
|
TokenStream tokenStream = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true, MockTokenFilter.ENGLISH_STOPSET)
|
||||||
.tokenStream(fieldName, new StringReader(text));
|
.tokenStream(fieldName, text);
|
||||||
// Assuming "<B>", "</B>" used to highlight
|
// Assuming "<B>", "</B>" used to highlight
|
||||||
SimpleHTMLFormatter formatter = new SimpleHTMLFormatter();
|
SimpleHTMLFormatter formatter = new SimpleHTMLFormatter();
|
||||||
QueryScorer scorer = new QueryScorer(query, fieldName, FIELD_NAME);
|
QueryScorer scorer = new QueryScorer(query, fieldName, FIELD_NAME);
|
||||||
|
@ -269,8 +268,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
|
||||||
|
|
||||||
for (int i = 0; i < hits.totalHits; i++) {
|
for (int i = 0; i < hits.totalHits; i++) {
|
||||||
String text = searcher.doc(hits.scoreDocs[i].doc).get(FIELD_NAME);
|
String text = searcher.doc(hits.scoreDocs[i].doc).get(FIELD_NAME);
|
||||||
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME,
|
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, text);
|
||||||
new StringReader(text));
|
|
||||||
highlighter.setTextFragmenter(new SimpleFragmenter(40));
|
highlighter.setTextFragmenter(new SimpleFragmenter(40));
|
||||||
|
|
||||||
String result = highlighter.getBestFragments(tokenStream, text, maxNumFragmentsRequired,
|
String result = highlighter.getBestFragments(tokenStream, text, maxNumFragmentsRequired,
|
||||||
|
@ -348,7 +346,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
|
||||||
|
|
||||||
for (int i = 0; i < hits.totalHits; i++) {
|
for (int i = 0; i < hits.totalHits; i++) {
|
||||||
String text = searcher.doc(hits.scoreDocs[i].doc).get(FIELD_NAME);
|
String text = searcher.doc(hits.scoreDocs[i].doc).get(FIELD_NAME);
|
||||||
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, new StringReader(text));
|
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, text);
|
||||||
|
|
||||||
highlighter.setTextFragmenter(new SimpleFragmenter(40));
|
highlighter.setTextFragmenter(new SimpleFragmenter(40));
|
||||||
|
|
||||||
|
@ -377,7 +375,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
|
||||||
|
|
||||||
for (int i = 0; i < hits.totalHits; i++) {
|
for (int i = 0; i < hits.totalHits; i++) {
|
||||||
String text = searcher.doc(hits.scoreDocs[i].doc).get(FIELD_NAME);
|
String text = searcher.doc(hits.scoreDocs[i].doc).get(FIELD_NAME);
|
||||||
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, new StringReader(text));
|
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, text);
|
||||||
|
|
||||||
highlighter.setTextFragmenter(new SimpleFragmenter(40));
|
highlighter.setTextFragmenter(new SimpleFragmenter(40));
|
||||||
|
|
||||||
|
@ -406,7 +404,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
|
||||||
|
|
||||||
for (int i = 0; i < hits.totalHits; i++) {
|
for (int i = 0; i < hits.totalHits; i++) {
|
||||||
String text = searcher.doc(hits.scoreDocs[i].doc).get(FIELD_NAME);
|
String text = searcher.doc(hits.scoreDocs[i].doc).get(FIELD_NAME);
|
||||||
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, new StringReader(text));
|
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, text);
|
||||||
|
|
||||||
highlighter.setTextFragmenter(new SimpleFragmenter(40));
|
highlighter.setTextFragmenter(new SimpleFragmenter(40));
|
||||||
|
|
||||||
|
@ -431,7 +429,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
|
||||||
|
|
||||||
for (int i = 0; i < hits.totalHits; i++) {
|
for (int i = 0; i < hits.totalHits; i++) {
|
||||||
String text = searcher.doc(hits.scoreDocs[i].doc).get(FIELD_NAME);
|
String text = searcher.doc(hits.scoreDocs[i].doc).get(FIELD_NAME);
|
||||||
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, new StringReader(text));
|
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, text);
|
||||||
|
|
||||||
highlighter.setTextFragmenter(new SimpleFragmenter(40));
|
highlighter.setTextFragmenter(new SimpleFragmenter(40));
|
||||||
|
|
||||||
|
@ -455,7 +453,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
|
||||||
|
|
||||||
for (int i = 0; i < hits.totalHits; i++) {
|
for (int i = 0; i < hits.totalHits; i++) {
|
||||||
String text = searcher.doc(hits.scoreDocs[i].doc).get(FIELD_NAME);
|
String text = searcher.doc(hits.scoreDocs[i].doc).get(FIELD_NAME);
|
||||||
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, new StringReader(text));
|
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, text);
|
||||||
|
|
||||||
highlighter.setTextFragmenter(new SimpleFragmenter(40));
|
highlighter.setTextFragmenter(new SimpleFragmenter(40));
|
||||||
|
|
||||||
|
@ -479,7 +477,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
|
||||||
|
|
||||||
for (int i = 0; i < hits.totalHits; i++) {
|
for (int i = 0; i < hits.totalHits; i++) {
|
||||||
String text = searcher.doc(hits.scoreDocs[i].doc).get(FIELD_NAME);
|
String text = searcher.doc(hits.scoreDocs[i].doc).get(FIELD_NAME);
|
||||||
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, new StringReader(text));
|
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, text);
|
||||||
|
|
||||||
highlighter.setTextFragmenter(new SimpleFragmenter(40));
|
highlighter.setTextFragmenter(new SimpleFragmenter(40));
|
||||||
|
|
||||||
|
@ -505,7 +503,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
|
||||||
|
|
||||||
for (int i = 0; i < hits.totalHits; i++) {
|
for (int i = 0; i < hits.totalHits; i++) {
|
||||||
String text = searcher.doc(hits.scoreDocs[i].doc).getField(NUMERIC_FIELD_NAME).numericValue().toString();
|
String text = searcher.doc(hits.scoreDocs[i].doc).getField(NUMERIC_FIELD_NAME).numericValue().toString();
|
||||||
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, new StringReader(text));
|
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, text);
|
||||||
|
|
||||||
highlighter.setTextFragmenter(new SimpleFragmenter(40));
|
highlighter.setTextFragmenter(new SimpleFragmenter(40));
|
||||||
|
|
||||||
|
@ -533,7 +531,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
|
||||||
|
|
||||||
for (int i = 0; i < hits.totalHits; i++) {
|
for (int i = 0; i < hits.totalHits; i++) {
|
||||||
String text = searcher.doc(hits.scoreDocs[i].doc).get(FIELD_NAME);
|
String text = searcher.doc(hits.scoreDocs[i].doc).get(FIELD_NAME);
|
||||||
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, new StringReader(text));
|
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, text);
|
||||||
|
|
||||||
String result = highlighter.getBestFragments(tokenStream, text, maxNumFragmentsRequired,
|
String result = highlighter.getBestFragments(tokenStream, text, maxNumFragmentsRequired,
|
||||||
"...");
|
"...");
|
||||||
|
@ -555,7 +553,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
|
||||||
|
|
||||||
for (int i = 0; i < hits.totalHits; i++) {
|
for (int i = 0; i < hits.totalHits; i++) {
|
||||||
String text = searcher.doc(hits.scoreDocs[i].doc).get(FIELD_NAME);
|
String text = searcher.doc(hits.scoreDocs[i].doc).get(FIELD_NAME);
|
||||||
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, new StringReader(text));
|
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, text);
|
||||||
QueryScorer scorer = new QueryScorer(query, FIELD_NAME);
|
QueryScorer scorer = new QueryScorer(query, FIELD_NAME);
|
||||||
Highlighter highlighter = new Highlighter(this, scorer);
|
Highlighter highlighter = new Highlighter(this, scorer);
|
||||||
|
|
||||||
|
@ -585,7 +583,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
|
||||||
|
|
||||||
for (int i = 0; i < hits.totalHits; i++) {
|
for (int i = 0; i < hits.totalHits; i++) {
|
||||||
String text = searcher.doc(hits.scoreDocs[i].doc).get(FIELD_NAME);
|
String text = searcher.doc(hits.scoreDocs[i].doc).get(FIELD_NAME);
|
||||||
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, new StringReader(text));
|
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, text);
|
||||||
|
|
||||||
highlighter.setTextFragmenter(new SimpleSpanFragmenter(scorer, 5));
|
highlighter.setTextFragmenter(new SimpleSpanFragmenter(scorer, 5));
|
||||||
|
|
||||||
|
@ -608,7 +606,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
|
||||||
|
|
||||||
for (int i = 0; i < hits.totalHits; i++) {
|
for (int i = 0; i < hits.totalHits; i++) {
|
||||||
String text = searcher.doc(hits.scoreDocs[i].doc).get(FIELD_NAME);
|
String text = searcher.doc(hits.scoreDocs[i].doc).get(FIELD_NAME);
|
||||||
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, new StringReader(text));
|
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, text);
|
||||||
|
|
||||||
highlighter.setTextFragmenter(new SimpleSpanFragmenter(scorer, 20));
|
highlighter.setTextFragmenter(new SimpleSpanFragmenter(scorer, 20));
|
||||||
|
|
||||||
|
@ -639,7 +637,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
|
||||||
|
|
||||||
for (int i = 0; i < hits.totalHits; i++) {
|
for (int i = 0; i < hits.totalHits; i++) {
|
||||||
String text = searcher.doc(hits.scoreDocs[i].doc).get(FIELD_NAME);
|
String text = searcher.doc(hits.scoreDocs[i].doc).get(FIELD_NAME);
|
||||||
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, new StringReader(text));
|
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, text);
|
||||||
|
|
||||||
highlighter.setTextFragmenter(new SimpleFragmenter(40));
|
highlighter.setTextFragmenter(new SimpleFragmenter(40));
|
||||||
|
|
||||||
|
@ -710,7 +708,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
|
||||||
int maxNumFragmentsRequired = 2;
|
int maxNumFragmentsRequired = 2;
|
||||||
for (int i = 0; i < hits.totalHits; i++) {
|
for (int i = 0; i < hits.totalHits; i++) {
|
||||||
String text = searcher.doc(hits.scoreDocs[i].doc).get(FIELD_NAME);
|
String text = searcher.doc(hits.scoreDocs[i].doc).get(FIELD_NAME);
|
||||||
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, new StringReader(text));
|
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, text);
|
||||||
|
|
||||||
String result = highlighter.getBestFragments(tokenStream, text, maxNumFragmentsRequired,
|
String result = highlighter.getBestFragments(tokenStream, text, maxNumFragmentsRequired,
|
||||||
"...");
|
"...");
|
||||||
|
@ -907,7 +905,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
|
||||||
int maxNumFragmentsRequired = 2;
|
int maxNumFragmentsRequired = 2;
|
||||||
String fragmentSeparator = "...";
|
String fragmentSeparator = "...";
|
||||||
QueryScorer scorer = new QueryScorer(query, HighlighterTest.FIELD_NAME);
|
QueryScorer scorer = new QueryScorer(query, HighlighterTest.FIELD_NAME);
|
||||||
TokenStream tokenStream = analyzer.tokenStream(HighlighterTest.FIELD_NAME, new StringReader(text));
|
TokenStream tokenStream = analyzer.tokenStream(HighlighterTest.FIELD_NAME, text);
|
||||||
|
|
||||||
Highlighter highlighter = new Highlighter(this, scorer);
|
Highlighter highlighter = new Highlighter(this, scorer);
|
||||||
|
|
||||||
|
@ -931,7 +929,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
|
||||||
int maxNumFragmentsRequired = 2;
|
int maxNumFragmentsRequired = 2;
|
||||||
String fragmentSeparator = "...";
|
String fragmentSeparator = "...";
|
||||||
QueryScorer scorer = new QueryScorer(query, null);
|
QueryScorer scorer = new QueryScorer(query, null);
|
||||||
TokenStream tokenStream = analyzer.tokenStream(HighlighterTest.FIELD_NAME, new StringReader(text));
|
TokenStream tokenStream = analyzer.tokenStream(HighlighterTest.FIELD_NAME, text);
|
||||||
|
|
||||||
Highlighter highlighter = new Highlighter(this, scorer);
|
Highlighter highlighter = new Highlighter(this, scorer);
|
||||||
|
|
||||||
|
@ -955,7 +953,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
|
||||||
int maxNumFragmentsRequired = 2;
|
int maxNumFragmentsRequired = 2;
|
||||||
String fragmentSeparator = "...";
|
String fragmentSeparator = "...";
|
||||||
QueryScorer scorer = new QueryScorer(query, "random_field", HighlighterTest.FIELD_NAME);
|
QueryScorer scorer = new QueryScorer(query, "random_field", HighlighterTest.FIELD_NAME);
|
||||||
TokenStream tokenStream = analyzer.tokenStream(HighlighterTest.FIELD_NAME, new StringReader(text));
|
TokenStream tokenStream = analyzer.tokenStream(HighlighterTest.FIELD_NAME, text);
|
||||||
|
|
||||||
Highlighter highlighter = new Highlighter(this, scorer);
|
Highlighter highlighter = new Highlighter(this, scorer);
|
||||||
|
|
||||||
|
@ -1126,7 +1124,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
|
||||||
numHighlights = 0;
|
numHighlights = 0;
|
||||||
for (int i = 0; i < hits.totalHits; i++) {
|
for (int i = 0; i < hits.totalHits; i++) {
|
||||||
String text = searcher.doc(hits.scoreDocs[i].doc).get(FIELD_NAME);
|
String text = searcher.doc(hits.scoreDocs[i].doc).get(FIELD_NAME);
|
||||||
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, new StringReader(text));
|
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, text);
|
||||||
|
|
||||||
Highlighter highlighter = getHighlighter(query, FIELD_NAME,
|
Highlighter highlighter = getHighlighter(query, FIELD_NAME,
|
||||||
HighlighterTest.this);
|
HighlighterTest.this);
|
||||||
|
@ -1187,7 +1185,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
|
||||||
Highlighter highlighter = getHighlighter(wTerms, HighlighterTest.this);// new
|
Highlighter highlighter = getHighlighter(wTerms, HighlighterTest.this);// new
|
||||||
// Highlighter(new
|
// Highlighter(new
|
||||||
// QueryTermScorer(wTerms));
|
// QueryTermScorer(wTerms));
|
||||||
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, new StringReader(texts[0]));
|
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, texts[0]);
|
||||||
highlighter.setTextFragmenter(new SimpleFragmenter(2));
|
highlighter.setTextFragmenter(new SimpleFragmenter(2));
|
||||||
|
|
||||||
String result = highlighter.getBestFragment(tokenStream, texts[0]).trim();
|
String result = highlighter.getBestFragment(tokenStream, texts[0]).trim();
|
||||||
|
@ -1196,7 +1194,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
|
||||||
|
|
||||||
// readjust weights
|
// readjust weights
|
||||||
wTerms[1].setWeight(50f);
|
wTerms[1].setWeight(50f);
|
||||||
tokenStream = analyzer.tokenStream(FIELD_NAME, new StringReader(texts[0]));
|
tokenStream = analyzer.tokenStream(FIELD_NAME, texts[0]);
|
||||||
highlighter = getHighlighter(wTerms, HighlighterTest.this);
|
highlighter = getHighlighter(wTerms, HighlighterTest.this);
|
||||||
highlighter.setTextFragmenter(new SimpleFragmenter(2));
|
highlighter.setTextFragmenter(new SimpleFragmenter(2));
|
||||||
|
|
||||||
|
@ -1232,7 +1230,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
|
||||||
Highlighter highlighter = getHighlighter(query, null, HighlighterTest.this);
|
Highlighter highlighter = getHighlighter(query, null, HighlighterTest.this);
|
||||||
|
|
||||||
// Get 3 best fragments and separate with a "..."
|
// Get 3 best fragments and separate with a "..."
|
||||||
TokenStream tokenStream = analyzer.tokenStream(null, new StringReader(s));
|
TokenStream tokenStream = analyzer.tokenStream(null, s);
|
||||||
|
|
||||||
String result = highlighter.getBestFragments(tokenStream, s, 3, "...");
|
String result = highlighter.getBestFragments(tokenStream, s, 3, "...");
|
||||||
String expectedResult = "<B>football</B>-<B>soccer</B> in the euro 2004 <B>footie</B> competition";
|
String expectedResult = "<B>football</B>-<B>soccer</B> in the euro 2004 <B>footie</B> competition";
|
||||||
|
@ -1257,7 +1255,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
|
||||||
|
|
||||||
for (int i = 0; i < hits.totalHits; i++) {
|
for (int i = 0; i < hits.totalHits; i++) {
|
||||||
String text = searcher.doc(hits.scoreDocs[i].doc).get(FIELD_NAME);
|
String text = searcher.doc(hits.scoreDocs[i].doc).get(FIELD_NAME);
|
||||||
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, new StringReader(text));
|
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, text);
|
||||||
Highlighter highlighter = getHighlighter(query, FIELD_NAME,
|
Highlighter highlighter = getHighlighter(query, FIELD_NAME,
|
||||||
HighlighterTest.this);
|
HighlighterTest.this);
|
||||||
String result = highlighter.getBestFragment(tokenStream, text);
|
String result = highlighter.getBestFragment(tokenStream, text);
|
||||||
|
@ -1280,7 +1278,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
|
||||||
|
|
||||||
for (int i = 0; i < hits.totalHits; i++) {
|
for (int i = 0; i < hits.totalHits; i++) {
|
||||||
String text = searcher.doc(hits.scoreDocs[i].doc).get(FIELD_NAME);
|
String text = searcher.doc(hits.scoreDocs[i].doc).get(FIELD_NAME);
|
||||||
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, new StringReader(text));
|
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, text);
|
||||||
|
|
||||||
Highlighter highlighter = getHighlighter(query, FIELD_NAME,
|
Highlighter highlighter = getHighlighter(query, FIELD_NAME,
|
||||||
HighlighterTest.this);// new Highlighter(this, new
|
HighlighterTest.this);// new Highlighter(this, new
|
||||||
|
@ -1288,7 +1286,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
|
||||||
highlighter.setTextFragmenter(new SimpleFragmenter(20));
|
highlighter.setTextFragmenter(new SimpleFragmenter(20));
|
||||||
String stringResults[] = highlighter.getBestFragments(tokenStream, text, 10);
|
String stringResults[] = highlighter.getBestFragments(tokenStream, text, 10);
|
||||||
|
|
||||||
tokenStream = analyzer.tokenStream(FIELD_NAME, new StringReader(text));
|
tokenStream = analyzer.tokenStream(FIELD_NAME, text);
|
||||||
TextFragment fragmentResults[] = highlighter.getBestTextFragments(tokenStream, text,
|
TextFragment fragmentResults[] = highlighter.getBestTextFragments(tokenStream, text,
|
||||||
true, 10);
|
true, 10);
|
||||||
|
|
||||||
|
@ -1318,7 +1316,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
|
||||||
public void run() throws Exception {
|
public void run() throws Exception {
|
||||||
numHighlights = 0;
|
numHighlights = 0;
|
||||||
doSearching(new TermQuery(new Term(FIELD_NAME, "meat")));
|
doSearching(new TermQuery(new Term(FIELD_NAME, "meat")));
|
||||||
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, new StringReader(texts[0]));
|
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, texts[0]);
|
||||||
Highlighter highlighter = getHighlighter(query, FIELD_NAME,
|
Highlighter highlighter = getHighlighter(query, FIELD_NAME,
|
||||||
HighlighterTest.this);// new Highlighter(this, new
|
HighlighterTest.this);// new Highlighter(this, new
|
||||||
// QueryTermScorer(query));
|
// QueryTermScorer(query));
|
||||||
|
@ -1432,7 +1430,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
|
||||||
|
|
||||||
for (int i = 0; i < hits.totalHits; i++) {
|
for (int i = 0; i < hits.totalHits; i++) {
|
||||||
String text = searcher.doc(hits.scoreDocs[i].doc).get(FIELD_NAME);
|
String text = searcher.doc(hits.scoreDocs[i].doc).get(FIELD_NAME);
|
||||||
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, new StringReader(text));
|
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, text);
|
||||||
Highlighter highlighter = getHighlighter(query, FIELD_NAME, HighlighterTest.this, false);
|
Highlighter highlighter = getHighlighter(query, FIELD_NAME, HighlighterTest.this, false);
|
||||||
|
|
||||||
highlighter.setTextFragmenter(new SimpleFragmenter(40));
|
highlighter.setTextFragmenter(new SimpleFragmenter(40));
|
||||||
|
@ -1461,7 +1459,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
|
||||||
doSearching(new TermQuery(new Term(FIELD_NAME, "aninvalidquerywhichshouldyieldnoresults")));
|
doSearching(new TermQuery(new Term(FIELD_NAME, "aninvalidquerywhichshouldyieldnoresults")));
|
||||||
|
|
||||||
for (String text : texts) {
|
for (String text : texts) {
|
||||||
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, new StringReader(text));
|
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, text);
|
||||||
Highlighter highlighter = getHighlighter(query, FIELD_NAME,
|
Highlighter highlighter = getHighlighter(query, FIELD_NAME,
|
||||||
HighlighterTest.this);
|
HighlighterTest.this);
|
||||||
String result = highlighter.getBestFragment(tokenStream, text);
|
String result = highlighter.getBestFragment(tokenStream, text);
|
||||||
|
@ -1503,7 +1501,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
highlighter.setTextFragmenter(new SimpleFragmenter(2000));
|
highlighter.setTextFragmenter(new SimpleFragmenter(2000));
|
||||||
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, new StringReader(rawDocContent));
|
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, rawDocContent);
|
||||||
|
|
||||||
String encodedSnippet = highlighter.getBestFragments(tokenStream, rawDocContent, 1, "");
|
String encodedSnippet = highlighter.getBestFragments(tokenStream, rawDocContent, 1, "");
|
||||||
// An ugly bit of XML creation:
|
// An ugly bit of XML creation:
|
||||||
|
@ -1828,7 +1826,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
|
||||||
*
|
*
|
||||||
* for (int i = 0; i < hits.totalHits; i++) { String text =
|
* for (int i = 0; i < hits.totalHits; i++) { String text =
|
||||||
* searcher.doc2(hits.scoreDocs[i].doc).get(FIELD_NAME); TokenStream
|
* searcher.doc2(hits.scoreDocs[i].doc).get(FIELD_NAME); TokenStream
|
||||||
* tokenStream=bigramAnalyzer.tokenStream(FIELD_NAME,new StringReader(text));
|
* tokenStream=bigramAnalyzer.tokenStream(FIELD_NAME,text);
|
||||||
* String highlightedText = highlighter.getBestFragment(tokenStream,text);
|
* String highlightedText = highlighter.getBestFragment(tokenStream,text);
|
||||||
* System.out.println(highlightedText); } }
|
* System.out.println(highlightedText); } }
|
||||||
*/
|
*/
|
||||||
|
@ -1855,7 +1853,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
|
||||||
final int expectedHighlights) throws Exception {
|
final int expectedHighlights) throws Exception {
|
||||||
for (int i = 0; i < hits.totalHits; i++) {
|
for (int i = 0; i < hits.totalHits; i++) {
|
||||||
String text = searcher.doc(hits.scoreDocs[i].doc).get(FIELD_NAME);
|
String text = searcher.doc(hits.scoreDocs[i].doc).get(FIELD_NAME);
|
||||||
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, new StringReader(text));
|
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, text);
|
||||||
QueryScorer scorer = new QueryScorer(query, FIELD_NAME);
|
QueryScorer scorer = new QueryScorer(query, FIELD_NAME);
|
||||||
Highlighter highlighter = new Highlighter(this, scorer);
|
Highlighter highlighter = new Highlighter(this, scorer);
|
||||||
|
|
||||||
|
@ -2104,7 +2102,7 @@ final class SynonymTokenizer extends TokenStream {
|
||||||
int maxNumFragmentsRequired = 2;
|
int maxNumFragmentsRequired = 2;
|
||||||
String fragmentSeparator = "...";
|
String fragmentSeparator = "...";
|
||||||
Scorer scorer = null;
|
Scorer scorer = null;
|
||||||
TokenStream tokenStream = analyzer.tokenStream(HighlighterTest.FIELD_NAME, new StringReader(text));
|
TokenStream tokenStream = analyzer.tokenStream(HighlighterTest.FIELD_NAME, text);
|
||||||
if (mode == QUERY) {
|
if (mode == QUERY) {
|
||||||
scorer = new QueryScorer(query);
|
scorer = new QueryScorer(query);
|
||||||
} else if (mode == QUERY_TERM) {
|
} else if (mode == QUERY_TERM) {
|
||||||
|
|
|
@ -17,7 +17,6 @@ package org.apache.lucene.search.highlight.custom;
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.StringReader;
|
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
import org.apache.lucene.analysis.MockAnalyzer;
|
import org.apache.lucene.analysis.MockAnalyzer;
|
||||||
|
@ -89,8 +88,7 @@ public class HighlightCustomQueryTest extends LuceneTestCase {
|
||||||
private String highlightField(Query query, String fieldName,
|
private String highlightField(Query query, String fieldName,
|
||||||
String text) throws IOException, InvalidTokenOffsetsException {
|
String text) throws IOException, InvalidTokenOffsetsException {
|
||||||
TokenStream tokenStream = new MockAnalyzer(random(), MockTokenizer.SIMPLE,
|
TokenStream tokenStream = new MockAnalyzer(random(), MockTokenizer.SIMPLE,
|
||||||
true, MockTokenFilter.ENGLISH_STOPSET).tokenStream(fieldName,
|
true, MockTokenFilter.ENGLISH_STOPSET).tokenStream(fieldName, text);
|
||||||
new StringReader(text));
|
|
||||||
// Assuming "<B>", "</B>" used to highlight
|
// Assuming "<B>", "</B>" used to highlight
|
||||||
SimpleHTMLFormatter formatter = new SimpleHTMLFormatter();
|
SimpleHTMLFormatter formatter = new SimpleHTMLFormatter();
|
||||||
MyQueryScorer scorer = new MyQueryScorer(query, fieldName, FIELD_NAME);
|
MyQueryScorer scorer = new MyQueryScorer(query, fieldName, FIELD_NAME);
|
||||||
|
|
|
@ -19,7 +19,6 @@ package org.apache.lucene.search.vectorhighlight;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.Reader;
|
import java.io.Reader;
|
||||||
import java.io.StringReader;
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
@ -171,7 +170,7 @@ public abstract class AbstractTestCase extends LuceneTestCase {
|
||||||
protected List<BytesRef> analyze(String text, String field, Analyzer analyzer) throws IOException {
|
protected List<BytesRef> analyze(String text, String field, Analyzer analyzer) throws IOException {
|
||||||
List<BytesRef> bytesRefs = new ArrayList<BytesRef>();
|
List<BytesRef> bytesRefs = new ArrayList<BytesRef>();
|
||||||
|
|
||||||
TokenStream tokenStream = analyzer.tokenStream(field, new StringReader(text));
|
TokenStream tokenStream = analyzer.tokenStream(field, text);
|
||||||
TermToBytesRefAttribute termAttribute = tokenStream.getAttribute(TermToBytesRefAttribute.class);
|
TermToBytesRefAttribute termAttribute = tokenStream.getAttribute(TermToBytesRefAttribute.class);
|
||||||
|
|
||||||
BytesRef bytesRef = termAttribute.getBytesRef();
|
BytesRef bytesRef = termAttribute.getBytesRef();
|
||||||
|
|
|
@ -18,7 +18,6 @@ package org.apache.lucene.index.memory;
|
||||||
*/
|
*/
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.StringReader;
|
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.Comparator;
|
import java.util.Comparator;
|
||||||
|
@ -291,7 +290,7 @@ public class MemoryIndex {
|
||||||
|
|
||||||
TokenStream stream;
|
TokenStream stream;
|
||||||
try {
|
try {
|
||||||
stream = analyzer.tokenStream(fieldName, new StringReader(text));
|
stream = analyzer.tokenStream(fieldName, text);
|
||||||
} catch (IOException ex) {
|
} catch (IOException ex) {
|
||||||
throw new RuntimeException(ex);
|
throw new RuntimeException(ex);
|
||||||
}
|
}
|
||||||
|
|
|
@ -22,7 +22,6 @@ import java.io.IOException;
|
||||||
import java.io.InputStream;
|
import java.io.InputStream;
|
||||||
import java.io.InputStreamReader;
|
import java.io.InputStreamReader;
|
||||||
import java.io.Reader;
|
import java.io.Reader;
|
||||||
import java.io.StringReader;
|
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
|
||||||
|
@ -378,7 +377,7 @@ public class MemoryIndexTest extends BaseTokenStreamTestCase {
|
||||||
SpanQuery wrappedquery = new SpanMultiTermQueryWrapper<RegexpQuery>(regex);
|
SpanQuery wrappedquery = new SpanMultiTermQueryWrapper<RegexpQuery>(regex);
|
||||||
|
|
||||||
MemoryIndex mindex = new MemoryIndex(random().nextBoolean(), random().nextInt(50) * 1024 * 1024);
|
MemoryIndex mindex = new MemoryIndex(random().nextBoolean(), random().nextInt(50) * 1024 * 1024);
|
||||||
mindex.addField("field", new MockAnalyzer(random()).tokenStream("field", new StringReader("hello there")));
|
mindex.addField("field", new MockAnalyzer(random()).tokenStream("field", "hello there"));
|
||||||
|
|
||||||
// This throws an NPE
|
// This throws an NPE
|
||||||
assertEquals(0, mindex.search(wrappedquery), 0.00001f);
|
assertEquals(0, mindex.search(wrappedquery), 0.00001f);
|
||||||
|
@ -390,7 +389,7 @@ public class MemoryIndexTest extends BaseTokenStreamTestCase {
|
||||||
SpanQuery wrappedquery = new SpanOrQuery(new SpanMultiTermQueryWrapper<RegexpQuery>(regex));
|
SpanQuery wrappedquery = new SpanOrQuery(new SpanMultiTermQueryWrapper<RegexpQuery>(regex));
|
||||||
|
|
||||||
MemoryIndex mindex = new MemoryIndex(random().nextBoolean(), random().nextInt(50) * 1024 * 1024);
|
MemoryIndex mindex = new MemoryIndex(random().nextBoolean(), random().nextInt(50) * 1024 * 1024);
|
||||||
mindex.addField("field", new MockAnalyzer(random()).tokenStream("field", new StringReader("hello there")));
|
mindex.addField("field", new MockAnalyzer(random()).tokenStream("field", "hello there"));
|
||||||
|
|
||||||
// This passes though
|
// This passes though
|
||||||
assertEquals(0, mindex.search(wrappedquery), 0.00001f);
|
assertEquals(0, mindex.search(wrappedquery), 0.00001f);
|
||||||
|
|
|
@ -18,7 +18,6 @@ package org.apache.lucene.queryparser.analyzing;
|
||||||
*/
|
*/
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.StringReader;
|
|
||||||
import java.util.regex.Matcher;
|
import java.util.regex.Matcher;
|
||||||
import java.util.regex.Pattern;
|
import java.util.regex.Pattern;
|
||||||
|
|
||||||
|
@ -165,7 +164,7 @@ public class AnalyzingQueryParser extends org.apache.lucene.queryparser.classic.
|
||||||
String analyzed = null;
|
String analyzed = null;
|
||||||
TokenStream stream = null;
|
TokenStream stream = null;
|
||||||
try{
|
try{
|
||||||
stream = getAnalyzer().tokenStream(field, new StringReader(chunk));
|
stream = getAnalyzer().tokenStream(field, chunk);
|
||||||
stream.reset();
|
stream.reset();
|
||||||
CharTermAttribute termAtt = stream.getAttribute(CharTermAttribute.class);
|
CharTermAttribute termAtt = stream.getAttribute(CharTermAttribute.class);
|
||||||
// get first and hopefully only output token
|
// get first and hopefully only output token
|
||||||
|
|
|
@ -500,7 +500,7 @@ public abstract class QueryParserBase implements CommonQueryParserConfiguration
|
||||||
|
|
||||||
TokenStream source;
|
TokenStream source;
|
||||||
try {
|
try {
|
||||||
source = analyzer.tokenStream(field, new StringReader(queryText));
|
source = analyzer.tokenStream(field, queryText);
|
||||||
source.reset();
|
source.reset();
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
ParseException p = new ParseException("Unable to initialize TokenStream to analyze query text");
|
ParseException p = new ParseException("Unable to initialize TokenStream to analyze query text");
|
||||||
|
@ -844,7 +844,7 @@ public abstract class QueryParserBase implements CommonQueryParserConfiguration
|
||||||
if (analyzerIn == null) analyzerIn = analyzer;
|
if (analyzerIn == null) analyzerIn = analyzer;
|
||||||
|
|
||||||
try {
|
try {
|
||||||
source = analyzerIn.tokenStream(field, new StringReader(part));
|
source = analyzerIn.tokenStream(field, part);
|
||||||
source.reset();
|
source.reset();
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
throw new RuntimeException("Unable to initialize TokenStream to analyze multiTerm term: " + part, e);
|
throw new RuntimeException("Unable to initialize TokenStream to analyze multiTerm term: " + part, e);
|
||||||
|
|
|
@ -18,7 +18,6 @@ package org.apache.lucene.queryparser.flexible.standard.processors;
|
||||||
*/
|
*/
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.StringReader;
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.LinkedList;
|
import java.util.LinkedList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
@ -116,7 +115,7 @@ public class AnalyzerQueryNodeProcessor extends QueryNodeProcessorImpl {
|
||||||
|
|
||||||
TokenStream source;
|
TokenStream source;
|
||||||
try {
|
try {
|
||||||
source = this.analyzer.tokenStream(field, new StringReader(text));
|
source = this.analyzer.tokenStream(field, text);
|
||||||
source.reset();
|
source.reset();
|
||||||
} catch (IOException e1) {
|
} catch (IOException e1) {
|
||||||
throw new RuntimeException(e1);
|
throw new RuntimeException(e1);
|
||||||
|
|
|
@ -4,7 +4,6 @@
|
||||||
package org.apache.lucene.queryparser.xml.builders;
|
package org.apache.lucene.queryparser.xml.builders;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.StringReader;
|
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
|
||||||
|
@ -75,7 +74,7 @@ public class LikeThisQueryBuilder implements QueryBuilder {
|
||||||
stopWordsSet = new HashSet<String>();
|
stopWordsSet = new HashSet<String>();
|
||||||
for (String field : fields) {
|
for (String field : fields) {
|
||||||
try {
|
try {
|
||||||
TokenStream ts = analyzer.tokenStream(field, new StringReader(stopWords));
|
TokenStream ts = analyzer.tokenStream(field, stopWords);
|
||||||
CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class);
|
CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class);
|
||||||
ts.reset();
|
ts.reset();
|
||||||
while (ts.incrementToken()) {
|
while (ts.incrementToken()) {
|
||||||
|
|
|
@ -13,9 +13,9 @@ import org.apache.lucene.queryparser.xml.ParserException;
|
||||||
import org.w3c.dom.Element;
|
import org.w3c.dom.Element;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.StringReader;
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
* contributor license agreements. See the NOTICE file distributed with
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
@ -51,7 +51,7 @@ public class SpanOrTermsBuilder extends SpanBuilderBase {
|
||||||
|
|
||||||
try {
|
try {
|
||||||
List<SpanQuery> clausesList = new ArrayList<SpanQuery>();
|
List<SpanQuery> clausesList = new ArrayList<SpanQuery>();
|
||||||
TokenStream ts = analyzer.tokenStream(fieldName, new StringReader(value));
|
TokenStream ts = analyzer.tokenStream(fieldName, value);
|
||||||
TermToBytesRefAttribute termAtt = ts.addAttribute(TermToBytesRefAttribute.class);
|
TermToBytesRefAttribute termAtt = ts.addAttribute(TermToBytesRefAttribute.class);
|
||||||
BytesRef bytes = termAtt.getBytesRef();
|
BytesRef bytes = termAtt.getBytesRef();
|
||||||
ts.reset();
|
ts.reset();
|
||||||
|
|
|
@ -3,7 +3,6 @@ package org.apache.lucene.queryparser.xml.builders;
|
||||||
import org.apache.lucene.analysis.Analyzer;
|
import org.apache.lucene.analysis.Analyzer;
|
||||||
import org.apache.lucene.analysis.TokenStream;
|
import org.apache.lucene.analysis.TokenStream;
|
||||||
import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
|
import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
|
||||||
import org.apache.lucene.index.Term;
|
|
||||||
import org.apache.lucene.search.Filter;
|
import org.apache.lucene.search.Filter;
|
||||||
import org.apache.lucene.queries.TermsFilter;
|
import org.apache.lucene.queries.TermsFilter;
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
|
@ -13,7 +12,6 @@ import org.apache.lucene.queryparser.xml.ParserException;
|
||||||
import org.w3c.dom.Element;
|
import org.w3c.dom.Element;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.StringReader;
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
|
@ -57,9 +55,8 @@ public class TermsFilterBuilder implements FilterBuilder {
|
||||||
String fieldName = DOMUtils.getAttributeWithInheritanceOrFail(e, "fieldName");
|
String fieldName = DOMUtils.getAttributeWithInheritanceOrFail(e, "fieldName");
|
||||||
|
|
||||||
try {
|
try {
|
||||||
TokenStream ts = analyzer.tokenStream(fieldName, new StringReader(text));
|
TokenStream ts = analyzer.tokenStream(fieldName, text);
|
||||||
TermToBytesRefAttribute termAtt = ts.addAttribute(TermToBytesRefAttribute.class);
|
TermToBytesRefAttribute termAtt = ts.addAttribute(TermToBytesRefAttribute.class);
|
||||||
Term term = null;
|
|
||||||
BytesRef bytes = termAtt.getBytesRef();
|
BytesRef bytes = termAtt.getBytesRef();
|
||||||
ts.reset();
|
ts.reset();
|
||||||
while (ts.incrementToken()) {
|
while (ts.incrementToken()) {
|
||||||
|
|
|
@ -15,7 +15,6 @@ import org.apache.lucene.queryparser.xml.QueryBuilder;
|
||||||
import org.w3c.dom.Element;
|
import org.w3c.dom.Element;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.StringReader;
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
@ -53,7 +52,7 @@ public class TermsQueryBuilder implements QueryBuilder {
|
||||||
BooleanQuery bq = new BooleanQuery(DOMUtils.getAttribute(e, "disableCoord", false));
|
BooleanQuery bq = new BooleanQuery(DOMUtils.getAttribute(e, "disableCoord", false));
|
||||||
bq.setMinimumNumberShouldMatch(DOMUtils.getAttribute(e, "minimumNumberShouldMatch", 0));
|
bq.setMinimumNumberShouldMatch(DOMUtils.getAttribute(e, "minimumNumberShouldMatch", 0));
|
||||||
try {
|
try {
|
||||||
TokenStream ts = analyzer.tokenStream(fieldName, new StringReader(text));
|
TokenStream ts = analyzer.tokenStream(fieldName, text);
|
||||||
TermToBytesRefAttribute termAtt = ts.addAttribute(TermToBytesRefAttribute.class);
|
TermToBytesRefAttribute termAtt = ts.addAttribute(TermToBytesRefAttribute.class);
|
||||||
Term term = null;
|
Term term = null;
|
||||||
BytesRef bytes = termAtt.getBytesRef();
|
BytesRef bytes = termAtt.getBytesRef();
|
||||||
|
|
|
@ -18,7 +18,6 @@ package org.apache.lucene.sandbox.queries;
|
||||||
*/
|
*/
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.StringReader;
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
|
@ -194,7 +193,7 @@ public class FuzzyLikeThisQuery extends Query
|
||||||
|
|
||||||
private void addTerms(IndexReader reader, FieldVals f) throws IOException {
|
private void addTerms(IndexReader reader, FieldVals f) throws IOException {
|
||||||
if (f.queryString == null) return;
|
if (f.queryString == null) return;
|
||||||
TokenStream ts = analyzer.tokenStream(f.fieldName, new StringReader(f.queryString));
|
TokenStream ts = analyzer.tokenStream(f.fieldName, f.queryString);
|
||||||
CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class);
|
CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class);
|
||||||
|
|
||||||
int corpusNumDocs = reader.numDocs();
|
int corpusNumDocs = reader.numDocs();
|
||||||
|
|
|
@ -21,7 +21,6 @@ import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.InputStream;
|
import java.io.InputStream;
|
||||||
import java.io.OutputStream;
|
import java.io.OutputStream;
|
||||||
import java.io.StringReader;
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.Comparator;
|
import java.util.Comparator;
|
||||||
|
@ -856,7 +855,7 @@ public class AnalyzingSuggester extends Lookup {
|
||||||
|
|
||||||
final Set<IntsRef> toFiniteStrings(final BytesRef surfaceForm, final TokenStreamToAutomaton ts2a) throws IOException {
|
final Set<IntsRef> toFiniteStrings(final BytesRef surfaceForm, final TokenStreamToAutomaton ts2a) throws IOException {
|
||||||
// Analyze surface form:
|
// Analyze surface form:
|
||||||
TokenStream ts = indexAnalyzer.tokenStream("", new StringReader(surfaceForm.utf8ToString()));
|
TokenStream ts = indexAnalyzer.tokenStream("", surfaceForm.utf8ToString());
|
||||||
|
|
||||||
// Create corresponding automaton: labels are bytes
|
// Create corresponding automaton: labels are bytes
|
||||||
// from each analyzed token, with byte 0 used as
|
// from each analyzed token, with byte 0 used as
|
||||||
|
@ -881,7 +880,7 @@ public class AnalyzingSuggester extends Lookup {
|
||||||
final Automaton toLookupAutomaton(final CharSequence key) throws IOException {
|
final Automaton toLookupAutomaton(final CharSequence key) throws IOException {
|
||||||
// TODO: is there a Reader from a CharSequence?
|
// TODO: is there a Reader from a CharSequence?
|
||||||
// Turn tokenstream into automaton:
|
// Turn tokenstream into automaton:
|
||||||
TokenStream ts = queryAnalyzer.tokenStream("", new StringReader(key.toString()));
|
TokenStream ts = queryAnalyzer.tokenStream("", key.toString());
|
||||||
Automaton automaton = (getTokenStreamToAutomaton()).toAutomaton(ts);
|
Automaton automaton = (getTokenStreamToAutomaton()).toAutomaton(ts);
|
||||||
ts.close();
|
ts.close();
|
||||||
|
|
||||||
|
|
|
@ -295,15 +295,15 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
public static void assertAnalyzesTo(Analyzer a, String input, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[]) throws IOException {
|
public static void assertAnalyzesTo(Analyzer a, String input, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[]) throws IOException {
|
||||||
assertTokenStreamContents(a.tokenStream("dummy", new StringReader(input)), output, startOffsets, endOffsets, types, posIncrements, null, input.length());
|
assertTokenStreamContents(a.tokenStream("dummy", input), output, startOffsets, endOffsets, types, posIncrements, null, input.length());
|
||||||
}
|
}
|
||||||
|
|
||||||
public static void assertAnalyzesTo(Analyzer a, String input, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[], int posLengths[]) throws IOException {
|
public static void assertAnalyzesTo(Analyzer a, String input, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[], int posLengths[]) throws IOException {
|
||||||
assertTokenStreamContents(a.tokenStream("dummy", new StringReader(input)), output, startOffsets, endOffsets, types, posIncrements, posLengths, input.length());
|
assertTokenStreamContents(a.tokenStream("dummy", input), output, startOffsets, endOffsets, types, posIncrements, posLengths, input.length());
|
||||||
}
|
}
|
||||||
|
|
||||||
public static void assertAnalyzesTo(Analyzer a, String input, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[], int posLengths[], boolean offsetsAreCorrect) throws IOException {
|
public static void assertAnalyzesTo(Analyzer a, String input, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[], int posLengths[], boolean offsetsAreCorrect) throws IOException {
|
||||||
assertTokenStreamContents(a.tokenStream("dummy", new StringReader(input)), output, startOffsets, endOffsets, types, posIncrements, posLengths, input.length(), offsetsAreCorrect);
|
assertTokenStreamContents(a.tokenStream("dummy", input), output, startOffsets, endOffsets, types, posIncrements, posLengths, input.length(), offsetsAreCorrect);
|
||||||
}
|
}
|
||||||
|
|
||||||
public static void assertAnalyzesTo(Analyzer a, String input, String[] output) throws IOException {
|
public static void assertAnalyzesTo(Analyzer a, String input, String[] output) throws IOException {
|
||||||
|
@ -332,7 +332,7 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
|
||||||
|
|
||||||
|
|
||||||
public static void assertAnalyzesToReuse(Analyzer a, String input, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[]) throws IOException {
|
public static void assertAnalyzesToReuse(Analyzer a, String input, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[]) throws IOException {
|
||||||
assertTokenStreamContents(a.tokenStream("dummy", new StringReader(input)), output, startOffsets, endOffsets, types, posIncrements, null, input.length());
|
assertTokenStreamContents(a.tokenStream("dummy", input), output, startOffsets, endOffsets, types, posIncrements, null, input.length());
|
||||||
}
|
}
|
||||||
|
|
||||||
public static void assertAnalyzesToReuse(Analyzer a, String input, String[] output) throws IOException {
|
public static void assertAnalyzesToReuse(Analyzer a, String input, String[] output) throws IOException {
|
||||||
|
@ -891,7 +891,7 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
|
||||||
|
|
||||||
protected String toDot(Analyzer a, String inputText) throws IOException {
|
protected String toDot(Analyzer a, String inputText) throws IOException {
|
||||||
final StringWriter sw = new StringWriter();
|
final StringWriter sw = new StringWriter();
|
||||||
final TokenStream ts = a.tokenStream("field", new StringReader(inputText));
|
final TokenStream ts = a.tokenStream("field", inputText);
|
||||||
ts.reset();
|
ts.reset();
|
||||||
new TokenStreamToDot(inputText, ts, new PrintWriter(sw)).toDot();
|
new TokenStreamToDot(inputText, ts, new PrintWriter(sw)).toDot();
|
||||||
return sw.toString();
|
return sw.toString();
|
||||||
|
@ -899,7 +899,7 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
|
||||||
|
|
||||||
protected void toDotFile(Analyzer a, String inputText, String localFileName) throws IOException {
|
protected void toDotFile(Analyzer a, String inputText, String localFileName) throws IOException {
|
||||||
Writer w = new OutputStreamWriter(new FileOutputStream(localFileName), "UTF-8");
|
Writer w = new OutputStreamWriter(new FileOutputStream(localFileName), "UTF-8");
|
||||||
final TokenStream ts = a.tokenStream("field", new StringReader(inputText));
|
final TokenStream ts = a.tokenStream("field", inputText);
|
||||||
ts.reset();
|
ts.reset();
|
||||||
new TokenStreamToDot(inputText, ts, new PrintWriter(w)).toDot();
|
new TokenStreamToDot(inputText, ts, new PrintWriter(w)).toDot();
|
||||||
w.close();
|
w.close();
|
||||||
|
|
|
@ -19,7 +19,6 @@ package org.apache.lucene.analysis;
|
||||||
|
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.StringReader;
|
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
|
@ -33,7 +32,6 @@ import org.apache.lucene.index.DirectoryReader;
|
||||||
import org.apache.lucene.index.IndexReader;
|
import org.apache.lucene.index.IndexReader;
|
||||||
import org.apache.lucene.index.IndexWriter;
|
import org.apache.lucene.index.IndexWriter;
|
||||||
import org.apache.lucene.index.IndexWriterConfig;
|
import org.apache.lucene.index.IndexWriterConfig;
|
||||||
import org.apache.lucene.index.IndexableField;
|
|
||||||
import org.apache.lucene.index.StorableField;
|
import org.apache.lucene.index.StorableField;
|
||||||
import org.apache.lucene.index.StoredDocument;
|
import org.apache.lucene.index.StoredDocument;
|
||||||
import org.apache.lucene.index.Term;
|
import org.apache.lucene.index.Term;
|
||||||
|
@ -199,13 +197,13 @@ public abstract class CollationTestBase extends LuceneTestCase {
|
||||||
doc.add(new Field("tracer", sortData[i][0], customType));
|
doc.add(new Field("tracer", sortData[i][0], customType));
|
||||||
doc.add(new TextField("contents", sortData[i][1], Field.Store.NO));
|
doc.add(new TextField("contents", sortData[i][1], Field.Store.NO));
|
||||||
if (sortData[i][2] != null)
|
if (sortData[i][2] != null)
|
||||||
doc.add(new TextField("US", usAnalyzer.tokenStream("US", new StringReader(sortData[i][2]))));
|
doc.add(new TextField("US", usAnalyzer.tokenStream("US", sortData[i][2])));
|
||||||
if (sortData[i][3] != null)
|
if (sortData[i][3] != null)
|
||||||
doc.add(new TextField("France", franceAnalyzer.tokenStream("France", new StringReader(sortData[i][3]))));
|
doc.add(new TextField("France", franceAnalyzer.tokenStream("France", sortData[i][3])));
|
||||||
if (sortData[i][4] != null)
|
if (sortData[i][4] != null)
|
||||||
doc.add(new TextField("Sweden", swedenAnalyzer.tokenStream("Sweden", new StringReader(sortData[i][4]))));
|
doc.add(new TextField("Sweden", swedenAnalyzer.tokenStream("Sweden", sortData[i][4])));
|
||||||
if (sortData[i][5] != null)
|
if (sortData[i][5] != null)
|
||||||
doc.add(new TextField("Denmark", denmarkAnalyzer.tokenStream("Denmark", new StringReader(sortData[i][5]))));
|
doc.add(new TextField("Denmark", denmarkAnalyzer.tokenStream("Denmark", sortData[i][5])));
|
||||||
writer.addDocument(doc);
|
writer.addDocument(doc);
|
||||||
}
|
}
|
||||||
writer.forceMerge(1);
|
writer.forceMerge(1);
|
||||||
|
@ -260,7 +258,7 @@ public abstract class CollationTestBase extends LuceneTestCase {
|
||||||
|
|
||||||
for (int i = 0; i < numTestPoints; i++) {
|
for (int i = 0; i < numTestPoints; i++) {
|
||||||
String term = _TestUtil.randomSimpleString(random());
|
String term = _TestUtil.randomSimpleString(random());
|
||||||
TokenStream ts = analyzer.tokenStream("fake", new StringReader(term));
|
TokenStream ts = analyzer.tokenStream("fake", term);
|
||||||
TermToBytesRefAttribute termAtt = ts.addAttribute(TermToBytesRefAttribute.class);
|
TermToBytesRefAttribute termAtt = ts.addAttribute(TermToBytesRefAttribute.class);
|
||||||
BytesRef bytes = termAtt.getBytesRef();
|
BytesRef bytes = termAtt.getBytesRef();
|
||||||
ts.reset();
|
ts.reset();
|
||||||
|
@ -279,7 +277,7 @@ public abstract class CollationTestBase extends LuceneTestCase {
|
||||||
for (Map.Entry<String,BytesRef> mapping : map.entrySet()) {
|
for (Map.Entry<String,BytesRef> mapping : map.entrySet()) {
|
||||||
String term = mapping.getKey();
|
String term = mapping.getKey();
|
||||||
BytesRef expected = mapping.getValue();
|
BytesRef expected = mapping.getValue();
|
||||||
TokenStream ts = analyzer.tokenStream("fake", new StringReader(term));
|
TokenStream ts = analyzer.tokenStream("fake", term);
|
||||||
TermToBytesRefAttribute termAtt = ts.addAttribute(TermToBytesRefAttribute.class);
|
TermToBytesRefAttribute termAtt = ts.addAttribute(TermToBytesRefAttribute.class);
|
||||||
BytesRef bytes = termAtt.getBytesRef();
|
BytesRef bytes = termAtt.getBytesRef();
|
||||||
ts.reset();
|
ts.reset();
|
||||||
|
|
|
@ -19,7 +19,6 @@ package org.apache.solr.schema;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.InputStream;
|
import java.io.InputStream;
|
||||||
import java.io.StringReader;
|
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
import org.apache.commons.io.IOUtils;
|
import org.apache.commons.io.IOUtils;
|
||||||
|
@ -27,8 +26,6 @@ import org.apache.lucene.analysis.Analyzer;
|
||||||
import org.apache.lucene.analysis.TokenStream;
|
import org.apache.lucene.analysis.TokenStream;
|
||||||
import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
|
import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
|
||||||
import org.apache.lucene.collation.ICUCollationKeyAnalyzer;
|
import org.apache.lucene.collation.ICUCollationKeyAnalyzer;
|
||||||
import org.apache.lucene.index.GeneralField;
|
|
||||||
import org.apache.lucene.index.IndexableField;
|
|
||||||
import org.apache.lucene.index.StorableField;
|
import org.apache.lucene.index.StorableField;
|
||||||
import org.apache.lucene.search.Query;
|
import org.apache.lucene.search.Query;
|
||||||
import org.apache.lucene.search.SortField;
|
import org.apache.lucene.search.SortField;
|
||||||
|
@ -240,7 +237,7 @@ public class ICUCollationField extends FieldType {
|
||||||
TokenStream source;
|
TokenStream source;
|
||||||
|
|
||||||
try {
|
try {
|
||||||
source = analyzer.tokenStream(field, new StringReader(part));
|
source = analyzer.tokenStream(field, part);
|
||||||
source.reset();
|
source.reset();
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
throw new RuntimeException("Unable to initialize TokenStream to analyze range part: " + part, e);
|
throw new RuntimeException("Unable to initialize TokenStream to analyze range part: " + part, e);
|
||||||
|
|
|
@ -86,7 +86,7 @@ public abstract class AnalysisRequestHandlerBase extends RequestHandlerBase {
|
||||||
|
|
||||||
TokenStream tokenStream = null;
|
TokenStream tokenStream = null;
|
||||||
try {
|
try {
|
||||||
tokenStream = analyzer.tokenStream(context.getFieldName(), new StringReader(value));
|
tokenStream = analyzer.tokenStream(context.getFieldName(), value);
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
|
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
|
||||||
}
|
}
|
||||||
|
@ -140,7 +140,7 @@ public abstract class AnalysisRequestHandlerBase extends RequestHandlerBase {
|
||||||
protected Set<BytesRef> getQueryTokenSet(String query, Analyzer analyzer) {
|
protected Set<BytesRef> getQueryTokenSet(String query, Analyzer analyzer) {
|
||||||
try {
|
try {
|
||||||
final Set<BytesRef> tokens = new HashSet<BytesRef>();
|
final Set<BytesRef> tokens = new HashSet<BytesRef>();
|
||||||
final TokenStream tokenStream = analyzer.tokenStream("", new StringReader(query));
|
final TokenStream tokenStream = analyzer.tokenStream("", query);
|
||||||
final TermToBytesRefAttribute bytesAtt = tokenStream.getAttribute(TermToBytesRefAttribute.class);
|
final TermToBytesRefAttribute bytesAtt = tokenStream.getAttribute(TermToBytesRefAttribute.class);
|
||||||
final BytesRef bytes = bytesAtt.getBytesRef();
|
final BytesRef bytes = bytesAtt.getBytesRef();
|
||||||
|
|
||||||
|
|
|
@ -72,7 +72,6 @@ import javax.xml.xpath.XPathFactory;
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.InputStream;
|
import java.io.InputStream;
|
||||||
import java.io.StringReader;
|
|
||||||
import java.net.MalformedURLException;
|
import java.net.MalformedURLException;
|
||||||
import java.net.URL;
|
import java.net.URL;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
@ -344,7 +343,7 @@ public class QueryElevationComponent extends SearchComponent implements SolrCore
|
||||||
return query;
|
return query;
|
||||||
}
|
}
|
||||||
StringBuilder norm = new StringBuilder();
|
StringBuilder norm = new StringBuilder();
|
||||||
TokenStream tokens = analyzer.tokenStream("", new StringReader(query));
|
TokenStream tokens = analyzer.tokenStream("", query);
|
||||||
tokens.reset();
|
tokens.reset();
|
||||||
|
|
||||||
CharTermAttribute termAtt = tokens.addAttribute(CharTermAttribute.class);
|
CharTermAttribute termAtt = tokens.addAttribute(CharTermAttribute.class);
|
||||||
|
|
|
@ -18,7 +18,6 @@
|
||||||
package org.apache.solr.handler.component;
|
package org.apache.solr.handler.component;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.StringReader;
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
|
@ -464,7 +463,7 @@ public class SpellCheckComponent extends SearchComponent implements SolrCoreAwar
|
||||||
private Collection<Token> getTokens(String q, Analyzer analyzer) throws IOException {
|
private Collection<Token> getTokens(String q, Analyzer analyzer) throws IOException {
|
||||||
Collection<Token> result = new ArrayList<Token>();
|
Collection<Token> result = new ArrayList<Token>();
|
||||||
assert analyzer != null;
|
assert analyzer != null;
|
||||||
TokenStream ts = analyzer.tokenStream("", new StringReader(q));
|
TokenStream ts = analyzer.tokenStream("", q);
|
||||||
ts.reset();
|
ts.reset();
|
||||||
// TODO: support custom attributes
|
// TODO: support custom attributes
|
||||||
CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class);
|
CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class);
|
||||||
|
|
|
@ -48,7 +48,6 @@ import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.StringReader;
|
|
||||||
import java.util.*;
|
import java.util.*;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -636,7 +635,7 @@ public class DefaultSolrHighlighter extends SolrHighlighter implements PluginInf
|
||||||
private TokenStream createAnalyzerTStream(IndexSchema schema, String fieldName, String docText) throws IOException {
|
private TokenStream createAnalyzerTStream(IndexSchema schema, String fieldName, String docText) throws IOException {
|
||||||
|
|
||||||
TokenStream tstream;
|
TokenStream tstream;
|
||||||
TokenStream ts = schema.getAnalyzer().tokenStream(fieldName, new StringReader(docText));
|
TokenStream ts = schema.getAnalyzer().tokenStream(fieldName, docText);
|
||||||
ts.reset();
|
ts.reset();
|
||||||
tstream = new TokenOrderingFilter(ts, 10);
|
tstream = new TokenOrderingFilter(ts, 10);
|
||||||
return tstream;
|
return tstream;
|
||||||
|
|
|
@ -405,7 +405,7 @@ public abstract class SolrQueryParserBase {
|
||||||
|
|
||||||
TokenStream source;
|
TokenStream source;
|
||||||
try {
|
try {
|
||||||
source = analyzer.tokenStream(field, new StringReader(queryText));
|
source = analyzer.tokenStream(field, queryText);
|
||||||
source.reset();
|
source.reset();
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
throw new SyntaxError("Unable to initialize TokenStream to analyze query text", e);
|
throw new SyntaxError("Unable to initialize TokenStream to analyze query text", e);
|
||||||
|
|
|
@ -19,7 +19,6 @@ package org.apache.solr.schema;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.InputStream;
|
import java.io.InputStream;
|
||||||
import java.io.StringReader;
|
|
||||||
import java.text.Collator;
|
import java.text.Collator;
|
||||||
import java.text.ParseException;
|
import java.text.ParseException;
|
||||||
import java.text.RuleBasedCollator;
|
import java.text.RuleBasedCollator;
|
||||||
|
@ -31,8 +30,6 @@ import org.apache.lucene.analysis.Analyzer;
|
||||||
import org.apache.lucene.analysis.TokenStream;
|
import org.apache.lucene.analysis.TokenStream;
|
||||||
import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
|
import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
|
||||||
import org.apache.lucene.collation.CollationKeyAnalyzer;
|
import org.apache.lucene.collation.CollationKeyAnalyzer;
|
||||||
import org.apache.lucene.index.GeneralField;
|
|
||||||
import org.apache.lucene.index.IndexableField;
|
|
||||||
import org.apache.lucene.index.StorableField;
|
import org.apache.lucene.index.StorableField;
|
||||||
import org.apache.lucene.search.Query;
|
import org.apache.lucene.search.Query;
|
||||||
import org.apache.lucene.search.SortField;
|
import org.apache.lucene.search.SortField;
|
||||||
|
@ -216,7 +213,7 @@ public class CollationField extends FieldType {
|
||||||
TokenStream source;
|
TokenStream source;
|
||||||
|
|
||||||
try {
|
try {
|
||||||
source = analyzer.tokenStream(field, new StringReader(part));
|
source = analyzer.tokenStream(field, part);
|
||||||
source.reset();
|
source.reset();
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
throw new RuntimeException("Unable to initialize TokenStream to analyze range part: " + part, e);
|
throw new RuntimeException("Unable to initialize TokenStream to analyze range part: " + part, e);
|
||||||
|
|
|
@ -35,7 +35,6 @@ import java.util.Map;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.StringReader;
|
|
||||||
|
|
||||||
/** <code>TextField</code> is the basic type for configurable text analysis.
|
/** <code>TextField</code> is the basic type for configurable text analysis.
|
||||||
* Analyzers for field types using this implementation should be defined in the schema.
|
* Analyzers for field types using this implementation should be defined in the schema.
|
||||||
|
@ -141,7 +140,7 @@ public class TextField extends FieldType {
|
||||||
|
|
||||||
TokenStream source;
|
TokenStream source;
|
||||||
try {
|
try {
|
||||||
source = analyzerIn.tokenStream(field, new StringReader(part));
|
source = analyzerIn.tokenStream(field, part);
|
||||||
source.reset();
|
source.reset();
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unable to initialize TokenStream to analyze multiTerm term: " + part, e);
|
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unable to initialize TokenStream to analyze multiTerm term: " + part, e);
|
||||||
|
@ -181,7 +180,7 @@ public class TextField extends FieldType {
|
||||||
|
|
||||||
TokenStream source;
|
TokenStream source;
|
||||||
try {
|
try {
|
||||||
source = analyzer.tokenStream(field, new StringReader(queryText));
|
source = analyzer.tokenStream(field, queryText);
|
||||||
source.reset();
|
source.reset();
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
throw new RuntimeException("Unable to initialize TokenStream to analyze query text", e);
|
throw new RuntimeException("Unable to initialize TokenStream to analyze query text", e);
|
||||||
|
|
|
@ -18,8 +18,6 @@
|
||||||
package org.apache.solr.spelling;
|
package org.apache.solr.spelling;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.Reader;
|
|
||||||
import java.io.StringReader;
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
|
@ -28,11 +26,10 @@ import java.util.regex.Pattern;
|
||||||
|
|
||||||
import org.apache.lucene.analysis.Token;
|
import org.apache.lucene.analysis.Token;
|
||||||
import org.apache.lucene.analysis.TokenStream;
|
import org.apache.lucene.analysis.TokenStream;
|
||||||
import org.apache.lucene.analysis.tokenattributes.FlagsAttribute;
|
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||||
import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
|
import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
|
||||||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||||
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
|
||||||
import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
|
import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
|
||||||
|
|
||||||
|
|
||||||
|
@ -160,7 +157,7 @@ public class SpellingQueryConverter extends QueryConverter {
|
||||||
flagValue = TERM_PRECEDES_NEW_BOOLEAN_OPERATOR_FLAG;
|
flagValue = TERM_PRECEDES_NEW_BOOLEAN_OPERATOR_FLAG;
|
||||||
}
|
}
|
||||||
try {
|
try {
|
||||||
analyze(result, new StringReader(word), startIndex, flagValue);
|
analyze(result, word, startIndex, flagValue);
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
// TODO: shouldn't we log something?
|
// TODO: shouldn't we log something?
|
||||||
}
|
}
|
||||||
|
@ -174,7 +171,7 @@ public class SpellingQueryConverter extends QueryConverter {
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
protected void analyze(Collection<Token> result, Reader text, int offset, int flagsAttValue) throws IOException {
|
protected void analyze(Collection<Token> result, String text, int offset, int flagsAttValue) throws IOException {
|
||||||
TokenStream stream = analyzer.tokenStream("", text);
|
TokenStream stream = analyzer.tokenStream("", text);
|
||||||
// TODO: support custom attributes
|
// TODO: support custom attributes
|
||||||
CharTermAttribute termAtt = stream.addAttribute(CharTermAttribute.class);
|
CharTermAttribute termAtt = stream.addAttribute(CharTermAttribute.class);
|
||||||
|
|
|
@ -18,7 +18,6 @@ package org.apache.solr.spelling;
|
||||||
*/
|
*/
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.StringReader;
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
|
@ -38,7 +37,7 @@ public class SuggestQueryConverter extends SpellingQueryConverter {
|
||||||
|
|
||||||
Collection<Token> result = new ArrayList<Token>();
|
Collection<Token> result = new ArrayList<Token>();
|
||||||
try {
|
try {
|
||||||
analyze(result, new StringReader(original), 0, 0);
|
analyze(result, original, 0, 0);
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
throw new RuntimeException(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
|
|
|
@ -27,7 +27,6 @@ import org.junit.After;
|
||||||
import org.junit.BeforeClass;
|
import org.junit.BeforeClass;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
import java.io.StringReader;
|
|
||||||
import java.text.SimpleDateFormat;
|
import java.text.SimpleDateFormat;
|
||||||
import java.util.Locale;
|
import java.util.Locale;
|
||||||
import java.util.TimeZone;
|
import java.util.TimeZone;
|
||||||
|
@ -57,7 +56,7 @@ public class TestTrie extends SolrTestCaseJ4 {
|
||||||
assertTrue(type instanceof TrieField);
|
assertTrue(type instanceof TrieField);
|
||||||
|
|
||||||
String value = String.valueOf(random().nextInt());
|
String value = String.valueOf(random().nextInt());
|
||||||
TokenStream ts = type.getAnalyzer().tokenStream("dummy", new StringReader(value));
|
TokenStream ts = type.getAnalyzer().tokenStream("dummy", value);
|
||||||
OffsetAttribute ofsAtt = ts.addAttribute(OffsetAttribute.class);
|
OffsetAttribute ofsAtt = ts.addAttribute(OffsetAttribute.class);
|
||||||
ts.reset();
|
ts.reset();
|
||||||
int count = 0;
|
int count = 0;
|
||||||
|
@ -74,7 +73,7 @@ public class TestTrie extends SolrTestCaseJ4 {
|
||||||
ts.close();
|
ts.close();
|
||||||
|
|
||||||
// Test empty one:
|
// Test empty one:
|
||||||
ts = type.getAnalyzer().tokenStream("dummy", new StringReader(""));
|
ts = type.getAnalyzer().tokenStream("dummy", "");
|
||||||
ts.reset();
|
ts.reset();
|
||||||
assertFalse(ts.incrementToken());
|
assertFalse(ts.incrementToken());
|
||||||
ts.end();
|
ts.end();
|
||||||
|
|
|
@ -86,7 +86,7 @@ public class TestReversedWildcardFilterFactory extends SolrTestCaseJ4 {
|
||||||
String text = "one two three si\uD834\uDD1Ex";
|
String text = "one two three si\uD834\uDD1Ex";
|
||||||
|
|
||||||
// field one
|
// field one
|
||||||
TokenStream input = a.tokenStream("one", new StringReader(text));
|
TokenStream input = a.tokenStream("one", text);
|
||||||
assertTokenStreamContents(input,
|
assertTokenStreamContents(input,
|
||||||
new String[] { "\u0001eno", "one", "\u0001owt", "two",
|
new String[] { "\u0001eno", "one", "\u0001owt", "two",
|
||||||
"\u0001eerht", "three", "\u0001x\uD834\uDD1Eis", "si\uD834\uDD1Ex" },
|
"\u0001eerht", "three", "\u0001x\uD834\uDD1Eis", "si\uD834\uDD1Ex" },
|
||||||
|
@ -95,7 +95,7 @@ public class TestReversedWildcardFilterFactory extends SolrTestCaseJ4 {
|
||||||
new int[] { 1, 0, 1, 0, 1, 0, 1, 0 }
|
new int[] { 1, 0, 1, 0, 1, 0, 1, 0 }
|
||||||
);
|
);
|
||||||
// field two
|
// field two
|
||||||
input = a.tokenStream("two", new StringReader(text));
|
input = a.tokenStream("two", text);
|
||||||
assertTokenStreamContents(input,
|
assertTokenStreamContents(input,
|
||||||
new String[] { "\u0001eno", "\u0001owt",
|
new String[] { "\u0001eno", "\u0001owt",
|
||||||
"\u0001eerht", "\u0001x\uD834\uDD1Eis" },
|
"\u0001eerht", "\u0001x\uD834\uDD1Eis" },
|
||||||
|
@ -104,7 +104,7 @@ public class TestReversedWildcardFilterFactory extends SolrTestCaseJ4 {
|
||||||
new int[] { 1, 1, 1, 1 }
|
new int[] { 1, 1, 1, 1 }
|
||||||
);
|
);
|
||||||
// field three
|
// field three
|
||||||
input = a.tokenStream("three", new StringReader(text));
|
input = a.tokenStream("three", text);
|
||||||
assertTokenStreamContents(input,
|
assertTokenStreamContents(input,
|
||||||
new String[] { "one", "two", "three", "si\uD834\uDD1Ex" },
|
new String[] { "one", "two", "three", "si\uD834\uDD1Ex" },
|
||||||
new int[] { 0, 4, 8, 14 },
|
new int[] { 0, 4, 8, 14 },
|
||||||
|
|
|
@ -30,7 +30,6 @@ import org.junit.After;
|
||||||
import org.junit.BeforeClass;
|
import org.junit.BeforeClass;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
import java.io.StringReader;
|
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
@ -175,7 +174,7 @@ public class HighlighterTest extends SolrTestCaseJ4 {
|
||||||
public void testTermOffsetsTokenStream() throws Exception {
|
public void testTermOffsetsTokenStream() throws Exception {
|
||||||
String[] multivalued = { "a b c d", "e f g", "h", "i j k l m n" };
|
String[] multivalued = { "a b c d", "e f g", "h", "i j k l m n" };
|
||||||
Analyzer a1 = new WhitespaceAnalyzer(TEST_VERSION_CURRENT);
|
Analyzer a1 = new WhitespaceAnalyzer(TEST_VERSION_CURRENT);
|
||||||
TokenStream tokenStream = a1.tokenStream("", new StringReader("a b c d e f g h i j k l m n"));
|
TokenStream tokenStream = a1.tokenStream("", "a b c d e f g h i j k l m n");
|
||||||
tokenStream.reset();
|
tokenStream.reset();
|
||||||
|
|
||||||
TermOffsetsTokenStream tots = new TermOffsetsTokenStream(
|
TermOffsetsTokenStream tots = new TermOffsetsTokenStream(
|
||||||
|
@ -183,7 +182,7 @@ public class HighlighterTest extends SolrTestCaseJ4 {
|
||||||
for( String v : multivalued ){
|
for( String v : multivalued ){
|
||||||
TokenStream ts1 = tots.getMultiValuedTokenStream( v.length() );
|
TokenStream ts1 = tots.getMultiValuedTokenStream( v.length() );
|
||||||
Analyzer a2 = new WhitespaceAnalyzer(TEST_VERSION_CURRENT);
|
Analyzer a2 = new WhitespaceAnalyzer(TEST_VERSION_CURRENT);
|
||||||
TokenStream ts2 = a2.tokenStream("", new StringReader(v));
|
TokenStream ts2 = a2.tokenStream("", v);
|
||||||
ts2.reset();
|
ts2.reset();
|
||||||
|
|
||||||
while (ts1.incrementToken()) {
|
while (ts1.incrementToken()) {
|
||||||
|
|
|
@ -29,7 +29,6 @@ import org.apache.lucene.util.Version;
|
||||||
|
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.io.StringReader;
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
|
||||||
|
|
||||||
|
@ -44,7 +43,7 @@ class SimpleQueryConverter extends SpellingQueryConverter {
|
||||||
try {
|
try {
|
||||||
Collection<Token> result = new HashSet<Token>();
|
Collection<Token> result = new HashSet<Token>();
|
||||||
WhitespaceAnalyzer analyzer = new WhitespaceAnalyzer(Version.LUCENE_40);
|
WhitespaceAnalyzer analyzer = new WhitespaceAnalyzer(Version.LUCENE_40);
|
||||||
TokenStream ts = analyzer.tokenStream("", new StringReader(origQuery));
|
TokenStream ts = analyzer.tokenStream("", origQuery);
|
||||||
// TODO: support custom attributes
|
// TODO: support custom attributes
|
||||||
CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class);
|
CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class);
|
||||||
OffsetAttribute offsetAtt = ts.addAttribute(OffsetAttribute.class);
|
OffsetAttribute offsetAtt = ts.addAttribute(OffsetAttribute.class);
|
||||||
|
|
Loading…
Reference in New Issue