diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 28d3163d8a3..5340b6c126b 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -300,6 +300,11 @@ API Changes an overhead parameter, so you can easily pass a different value other than 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 * LUCENE-4987: Upgrade randomized testing to version 2.0.10: diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymMap.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymMap.java index 0a72ca4a65e..a30463b4762 100644 --- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymMap.java +++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymMap.java @@ -18,7 +18,6 @@ package org.apache.lucene.analysis.synonym; */ import java.io.IOException; -import java.io.StringReader; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; @@ -112,7 +111,7 @@ public class SynonymMap { * separates by {@link SynonymMap#WORD_SEPARATOR}. * reuse and its chars must not be null. */ 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); PositionIncrementAttribute posIncAtt = ts.addAttribute(PositionIncrementAttribute.class); ts.reset(); diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestStopAnalyzer.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestStopAnalyzer.java index 0c4fd19fb46..9f9c2714547 100644 --- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestStopAnalyzer.java +++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestStopAnalyzer.java @@ -24,7 +24,6 @@ import org.apache.lucene.analysis.tokenattributes.CharTermAttribute; import org.apache.lucene.analysis.util.CharArraySet; import org.apache.lucene.util.Version; -import java.io.StringReader; import java.io.IOException; import java.util.Iterator; import java.util.Set; @@ -47,8 +46,7 @@ public class TestStopAnalyzer extends BaseTokenStreamTestCase { public void testDefaults() throws IOException { assertTrue(stop != null); - StringReader reader = new StringReader("This is a test of the english stop analyzer"); - TokenStream stream = stop.tokenStream("test", reader); + TokenStream stream = stop.tokenStream("test", "This is a test of the english stop analyzer"); assertTrue(stream != null); CharTermAttribute termAtt = stream.getAttribute(CharTermAttribute.class); stream.reset(); @@ -61,8 +59,7 @@ public class TestStopAnalyzer extends BaseTokenStreamTestCase { public void testStopList() throws IOException { CharArraySet stopWordsSet = new CharArraySet(TEST_VERSION_CURRENT, asSet("good", "test", "analyzer"), false); 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", reader); + TokenStream stream = newStop.tokenStream("test", "This is a good test of the english stop analyzer"); assertNotNull(stream); CharTermAttribute termAtt = stream.getAttribute(CharTermAttribute.class); @@ -76,9 +73,9 @@ public class TestStopAnalyzer extends BaseTokenStreamTestCase { public void testStopListPositions() throws IOException { CharArraySet stopWordsSet = new CharArraySet(TEST_VERSION_CURRENT, asSet("good", "test", "analyzer"), false); StopAnalyzer newStop = new StopAnalyzer(TEST_VERSION_CURRENT, stopWordsSet); - StringReader reader = new StringReader("This is a good test of the english stop analyzer with positions"); - int expectedIncr[] = { 1, 1, 1, 3, 1, 1, 1, 2, 1}; - TokenStream stream = newStop.tokenStream("test", reader); + 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}; + TokenStream stream = newStop.tokenStream("test", s); assertNotNull(stream); int i = 0; CharTermAttribute termAtt = stream.getAttribute(CharTermAttribute.class); diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLimitTokenCountAnalyzer.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLimitTokenCountAnalyzer.java index 7998732d2f3..b55482a37c5 100644 --- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLimitTokenCountAnalyzer.java +++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLimitTokenCountAnalyzer.java @@ -18,13 +18,10 @@ package org.apache.lucene.analysis.miscellaneous; */ import java.io.IOException; -import java.io.StringReader; import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.BaseTokenStreamTestCase; 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.Field; import org.apache.lucene.index.DirectoryReader; @@ -47,14 +44,14 @@ public class TestLimitTokenCountAnalyzer extends BaseTokenStreamTestCase { 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)! - 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", 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, 3 }, new int[] { 1, 4 }, consumeAll ? 16 : 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 - 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 - 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); } } diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLimitTokenPositionFilter.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLimitTokenPositionFilter.java index 0593e1ae009..35521853516 100644 --- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLimitTokenPositionFilter.java +++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLimitTokenPositionFilter.java @@ -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)! - 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); 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); // 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); // 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); } } diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestPerFieldAnalyzerWrapper.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestPerFieldAnalyzerWrapper.java index 2407c1c8ce8..ce0cd745095 100644 --- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestPerFieldAnalyzerWrapper.java +++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestPerFieldAnalyzerWrapper.java @@ -1,7 +1,6 @@ package org.apache.lucene.analysis.miscellaneous; import java.io.Reader; -import java.io.StringReader; import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -38,8 +37,7 @@ public class TestPerFieldAnalyzerWrapper extends BaseTokenStreamTestCase { PerFieldAnalyzerWrapper analyzer = new PerFieldAnalyzerWrapper(new WhitespaceAnalyzer(TEST_VERSION_CURRENT), analyzerPerField); - TokenStream tokenStream = analyzer.tokenStream("field", - new StringReader(text)); + TokenStream tokenStream = analyzer.tokenStream("field", text); CharTermAttribute termAtt = tokenStream.getAttribute(CharTermAttribute.class); tokenStream.reset(); @@ -48,8 +46,7 @@ public class TestPerFieldAnalyzerWrapper extends BaseTokenStreamTestCase { "Qwerty", termAtt.toString()); - tokenStream = analyzer.tokenStream("special", - new StringReader(text)); + tokenStream = analyzer.tokenStream("special", text); termAtt = tokenStream.getAttribute(CharTermAttribute.class); tokenStream.reset(); diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/query/QueryAutoStopWordAnalyzerTest.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/query/QueryAutoStopWordAnalyzerTest.java index 8291f0236d9..670c1db9f2c 100644 --- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/query/QueryAutoStopWordAnalyzerTest.java +++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/query/QueryAutoStopWordAnalyzerTest.java @@ -26,7 +26,6 @@ import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriterConfig; import org.apache.lucene.store.RAMDirectory; -import java.io.StringReader; import java.util.Arrays; import java.util.Collections; @@ -66,44 +65,44 @@ public class QueryAutoStopWordAnalyzerTest extends BaseTokenStreamTestCase { public void testNoStopwords() throws Exception { // Note: an empty list of fields passed in protectedAnalyzer = new QueryAutoStopWordAnalyzer(TEST_VERSION_CURRENT, appAnalyzer, reader, Collections.emptyList(), 1); - TokenStream protectedTokenStream = protectedAnalyzer.tokenStream("variedField", new StringReader("quick")); + TokenStream protectedTokenStream = protectedAnalyzer.tokenStream("variedField", "quick"); assertTokenStreamContents(protectedTokenStream, new String[]{"quick"}); - protectedTokenStream = protectedAnalyzer.tokenStream("repetitiveField", new StringReader("boring")); + protectedTokenStream = protectedAnalyzer.tokenStream("repetitiveField", "boring"); assertTokenStreamContents(protectedTokenStream, new String[]{"boring"}); } public void testDefaultStopwordsAllFields() throws Exception { 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 } public void testStopwordsAllFieldsMaxPercentDocs() throws Exception { 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 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 assertTokenStreamContents(protectedTokenStream, new String[]{"vaguelyboring"}); 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 assertTokenStreamContents(protectedTokenStream, new String[0]); } public void testStopwordsPerFieldMaxPercentDocs() throws Exception { 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 assertTokenStreamContents(protectedTokenStream, new String[]{"boring"}); 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 assertTokenStreamContents(protectedTokenStream, new String[0]); } @@ -121,11 +120,11 @@ public class QueryAutoStopWordAnalyzerTest extends BaseTokenStreamTestCase { public void testNoFieldNamePollution() throws Exception { 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 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 assertTokenStreamContents(protectedTokenStream, new String[]{"boring"}); } @@ -134,7 +133,7 @@ public class QueryAutoStopWordAnalyzerTest extends BaseTokenStreamTestCase { QueryAutoStopWordAnalyzer a = new QueryAutoStopWordAnalyzer( TEST_VERSION_CURRENT, 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" }); } } diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/shingle/ShingleAnalyzerWrapperTest.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/shingle/ShingleAnalyzerWrapperTest.java index 5850ea59f76..6a578d2352f 100644 --- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/shingle/ShingleAnalyzerWrapperTest.java +++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/shingle/ShingleAnalyzerWrapperTest.java @@ -17,8 +17,6 @@ package org.apache.lucene.analysis.shingle; * limitations under the License. */ -import java.io.StringReader; - import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.BaseTokenStreamTestCase; import org.apache.lucene.analysis.MockAnalyzer; @@ -97,7 +95,7 @@ public class ShingleAnalyzerWrapperTest extends BaseTokenStreamTestCase { public void testShingleAnalyzerWrapperPhraseQuery() throws Exception { PhraseQuery q = new PhraseQuery(); - TokenStream ts = analyzer.tokenStream("content", new StringReader("this sentence")); + TokenStream ts = analyzer.tokenStream("content", "this sentence"); int j = -1; PositionIncrementAttribute posIncrAtt = ts.addAttribute(PositionIncrementAttribute.class); @@ -123,7 +121,7 @@ public class ShingleAnalyzerWrapperTest extends BaseTokenStreamTestCase { public void testShingleAnalyzerWrapperBooleanQuery() throws Exception { 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); diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/sinks/TestTeeSinkTokenFilter.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/sinks/TestTeeSinkTokenFilter.java index 21ebf307488..b49e1224ff7 100644 --- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/sinks/TestTeeSinkTokenFilter.java +++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/sinks/TestTeeSinkTokenFilter.java @@ -91,7 +91,7 @@ public class TestTeeSinkTokenFilter extends BaseTokenStreamTestCase { Analyzer analyzer = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false); IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer)); Document doc = new Document(); - TokenStream tokenStream = analyzer.tokenStream("field", new StringReader("abcd ")); + TokenStream tokenStream = analyzer.tokenStream("field", "abcd "); TeeSinkTokenFilter tee = new TeeSinkTokenFilter(tokenStream); TokenStream sink = tee.newSinkTokenStream(); FieldType ft = new FieldType(TextField.TYPE_NOT_STORED); diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/th/TestThaiAnalyzer.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/th/TestThaiAnalyzer.java index ab4a4d9e3e5..76a7d6480ff 100644 --- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/th/TestThaiAnalyzer.java +++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/th/TestThaiAnalyzer.java @@ -19,7 +19,6 @@ package org.apache.lucene.analysis.th; import java.io.IOException; import java.io.Reader; -import java.io.StringReader; import java.util.Random; 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.tokenattributes.FlagsAttribute; import org.apache.lucene.analysis.util.CharArraySet; -import org.apache.lucene.util.Version; /** * Test case for ThaiAnalyzer, modified from TestFrenchAnalyzer @@ -122,10 +120,10 @@ public class TestThaiAnalyzer extends BaseTokenStreamTestCase { public void testAttributeReuse() throws Exception { ThaiAnalyzer analyzer = new ThaiAnalyzer(TEST_VERSION_CURRENT); // just consume - TokenStream ts = analyzer.tokenStream("dummy", new StringReader("ภาษาไทย")); + TokenStream ts = analyzer.tokenStream("dummy", "ภาษาไทย"); assertTokenStreamContents(ts, new String[] { "ภาษา", "ไทย" }); // this consumer adds flagsAtt, which this analyzer does not use. - ts = analyzer.tokenStream("dummy", new StringReader("ภาษาไทย")); + ts = analyzer.tokenStream("dummy", "ภาษาไทย"); ts.addAttribute(FlagsAttribute.class); assertTokenStreamContents(ts, new String[] { "ภาษา", "ไทย" }); } diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/util/TestCharTokenizers.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/util/TestCharTokenizers.java index 66e4b096757..e8880dfeaf8 100644 --- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/util/TestCharTokenizers.java +++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/util/TestCharTokenizers.java @@ -123,7 +123,7 @@ public class TestCharTokenizers extends BaseTokenStreamTestCase { int num = 1000 * RANDOM_MULTIPLIER; for (int i = 0; i < num; i++) { String s = _TestUtil.randomUnicodeString(random()); - TokenStream ts = analyzer.tokenStream("foo", new StringReader(s)); + TokenStream ts = analyzer.tokenStream("foo", s); ts.reset(); OffsetAttribute offsetAtt = ts.addAttribute(OffsetAttribute.class); while (ts.incrementToken()) { @@ -161,7 +161,7 @@ public class TestCharTokenizers extends BaseTokenStreamTestCase { int num = 1000 * RANDOM_MULTIPLIER; for (int i = 0; i < num; i++) { String s = _TestUtil.randomUnicodeString(random()); - TokenStream ts = analyzer.tokenStream("foo", new StringReader(s)); + TokenStream ts = analyzer.tokenStream("foo", s); ts.reset(); OffsetAttribute offsetAtt = ts.addAttribute(OffsetAttribute.class); while (ts.incrementToken()) { diff --git a/lucene/analysis/icu/src/test/org/apache/lucene/analysis/icu/segmentation/TestICUTokenizer.java b/lucene/analysis/icu/src/test/org/apache/lucene/analysis/icu/segmentation/TestICUTokenizer.java index c80ffab8dad..f11200853e7 100644 --- a/lucene/analysis/icu/src/test/org/apache/lucene/analysis/icu/segmentation/TestICUTokenizer.java +++ b/lucene/analysis/icu/src/test/org/apache/lucene/analysis/icu/segmentation/TestICUTokenizer.java @@ -249,7 +249,7 @@ public class TestICUTokenizer extends BaseTokenStreamTestCase { } 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); ts.reset(); while (ts.incrementToken()) { diff --git a/lucene/analysis/kuromoji/src/test/org/apache/lucene/analysis/ja/TestExtendedMode.java b/lucene/analysis/kuromoji/src/test/org/apache/lucene/analysis/ja/TestExtendedMode.java index 73f7d3a8551..051ca3af237 100644 --- a/lucene/analysis/kuromoji/src/test/org/apache/lucene/analysis/ja/TestExtendedMode.java +++ b/lucene/analysis/kuromoji/src/test/org/apache/lucene/analysis/ja/TestExtendedMode.java @@ -19,7 +19,6 @@ package org.apache.lucene.analysis.ja; import java.io.IOException; import java.io.Reader; -import java.io.StringReader; import java.util.Random; import org.apache.lucene.analysis.Analyzer; @@ -54,7 +53,7 @@ public class TestExtendedMode extends BaseTokenStreamTestCase { int numIterations = atLeast(1000); for (int i = 0; i < numIterations; i++) { 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); ts.reset(); while (ts.incrementToken()) { diff --git a/lucene/analysis/kuromoji/src/test/org/apache/lucene/analysis/ja/TestJapaneseAnalyzer.java b/lucene/analysis/kuromoji/src/test/org/apache/lucene/analysis/ja/TestJapaneseAnalyzer.java index 670f11b792b..65e55aac50e 100644 --- a/lucene/analysis/kuromoji/src/test/org/apache/lucene/analysis/ja/TestJapaneseAnalyzer.java +++ b/lucene/analysis/kuromoji/src/test/org/apache/lucene/analysis/ja/TestJapaneseAnalyzer.java @@ -18,7 +18,6 @@ package org.apache.lucene.analysis.ja; */ import java.io.IOException; -import java.io.StringReader; import java.util.Random; import org.apache.lucene.analysis.Analyzer; @@ -151,7 +150,7 @@ public class TestJapaneseAnalyzer extends BaseTokenStreamTestCase { Mode.SEARCH, JapaneseAnalyzer.getDefaultStopSet(), JapaneseAnalyzer.getDefaultStopTags()); - assertTokenStreamContents(a.tokenStream("foo", new StringReader("abcd")), + assertTokenStreamContents(a.tokenStream("foo", "abcd"), new String[] { "a", "b", "cd" }, new int[] { 0, 1, 2 }, new int[] { 1, 2, 4 }, diff --git a/lucene/analysis/kuromoji/src/test/org/apache/lucene/analysis/ja/TestJapaneseTokenizer.java b/lucene/analysis/kuromoji/src/test/org/apache/lucene/analysis/ja/TestJapaneseTokenizer.java index 5dce63ebfb4..4ba79af0dc2 100644 --- a/lucene/analysis/kuromoji/src/test/org/apache/lucene/analysis/ja/TestJapaneseTokenizer.java +++ b/lucene/analysis/kuromoji/src/test/org/apache/lucene/analysis/ja/TestJapaneseTokenizer.java @@ -22,7 +22,6 @@ import java.io.InputStream; import java.io.InputStreamReader; import java.io.LineNumberReader; import java.io.Reader; -import java.io.StringReader; import java.util.Random; import org.apache.lucene.analysis.Analyzer; @@ -142,7 +141,7 @@ public class TestJapaneseTokenizer extends BaseTokenStreamTestCase { * ideally the test would actually fail instead of hanging... */ public void testDecomposition5() throws Exception { - TokenStream ts = analyzer.tokenStream("bogus", new StringReader("くよくよくよくよくよくよくよくよくよくよくよくよくよくよくよくよくよくよくよくよ")); + TokenStream ts = analyzer.tokenStream("bogus", "くよくよくよくよくよくよくよくよくよくよくよくよくよくよくよくよくよくよくよくよ"); ts.reset(); while (ts.incrementToken()) { @@ -166,8 +165,8 @@ public class TestJapaneseTokenizer extends BaseTokenStreamTestCase { /** Tests that sentence offset is incorporated into the resulting offsets */ public void testTwoSentences() throws Exception { /* - //TokenStream ts = a.tokenStream("foo", new StringReader("妹の咲子です。俺と年子で、今受験生です。")); - TokenStream ts = analyzer.tokenStream("foo", new StringReader("�?>-->;")); + //TokenStream ts = a.tokenStream("foo", "妹の咲子です。俺と年子で、今受験生です。"); + TokenStream ts = analyzer.tokenStream("foo", "�?>-->;"); ts.reset(); CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class); while(ts.incrementToken()) { @@ -214,7 +213,7 @@ public class TestJapaneseTokenizer extends BaseTokenStreamTestCase { public void testLargeDocReliability() throws Exception { for (int i = 0; i < 100; i++) { String s = _TestUtil.randomUnicodeString(random(), 10000); - TokenStream ts = analyzer.tokenStream("foo", new StringReader(s)); + TokenStream ts = analyzer.tokenStream("foo", s); ts.reset(); while (ts.incrementToken()) { } @@ -235,7 +234,7 @@ public class TestJapaneseTokenizer extends BaseTokenStreamTestCase { System.out.println("\nTEST: iter=" + i); } 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); ts.reset(); while (ts.incrementToken()) { @@ -245,14 +244,14 @@ public class TestJapaneseTokenizer extends BaseTokenStreamTestCase { } public void testOnlyPunctuation() throws IOException { - TokenStream ts = analyzerNoPunct.tokenStream("foo", new StringReader("。、。。")); + TokenStream ts = analyzerNoPunct.tokenStream("foo", "。、。。"); ts.reset(); assertFalse(ts.incrementToken()); ts.end(); } public void testOnlyPunctuationExtended() throws IOException { - TokenStream ts = extendedModeAnalyzerNoPunct.tokenStream("foo", new StringReader("......")); + TokenStream ts = extendedModeAnalyzerNoPunct.tokenStream("foo", "......"); ts.reset(); assertFalse(ts.incrementToken()); ts.end(); @@ -261,14 +260,14 @@ public class TestJapaneseTokenizer extends BaseTokenStreamTestCase { // note: test is kinda silly since kuromoji emits punctuation tokens. // but, when/if we filter these out it will be useful. public void testEnd() throws Exception { - assertTokenStreamContents(analyzerNoPunct.tokenStream("foo", new StringReader("これは本ではない")), + assertTokenStreamContents(analyzerNoPunct.tokenStream("foo", "これは本ではない"), new String[] { "これ", "は", "本", "で", "は", "ない" }, new int[] { 0, 2, 3, 4, 5, 6 }, new int[] { 2, 3, 4, 5, 6, 8 }, new Integer(8) ); - assertTokenStreamContents(analyzerNoPunct.tokenStream("foo", new StringReader("これは本ではない ")), + assertTokenStreamContents(analyzerNoPunct.tokenStream("foo", "これは本ではない "), new String[] { "これ", "は", "本", "で", "は", "ない" }, new int[] { 0, 2, 3, 4, 5, 6, 8 }, new int[] { 2, 3, 4, 5, 6, 8, 9 }, @@ -279,7 +278,7 @@ public class TestJapaneseTokenizer extends BaseTokenStreamTestCase { public void testUserDict() throws Exception { // Not a great test because w/o userdict.txt the // segmentation is the same: - assertTokenStreamContents(analyzer.tokenStream("foo", new StringReader("関西国際空港に行った")), + assertTokenStreamContents(analyzer.tokenStream("foo", "関西国際空港に行った"), new String[] { "関西", "国際", "空港", "に", "行っ", "た" }, new int[] { 0, 2, 4, 6, 7, 9 }, new int[] { 2, 4, 6, 7, 9, 10 }, @@ -289,7 +288,7 @@ public class TestJapaneseTokenizer extends BaseTokenStreamTestCase { public void testUserDict2() throws Exception { // Better test: w/o userdict the segmentation is different: - assertTokenStreamContents(analyzer.tokenStream("foo", new StringReader("朝青龍")), + assertTokenStreamContents(analyzer.tokenStream("foo", "朝青龍"), new String[] { "朝青龍" }, new int[] { 0 }, new int[] { 3 }, @@ -299,7 +298,7 @@ public class TestJapaneseTokenizer extends BaseTokenStreamTestCase { public void testUserDict3() throws Exception { // 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 int[] { 0, 1, 2 }, new int[] { 1, 2, 4 }, @@ -315,7 +314,7 @@ public class TestJapaneseTokenizer extends BaseTokenStreamTestCase { /* public void testUserDict4() throws Exception { // 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 int[] { 0, 2, 4, 7 }, new int[] { 2, 4, 7, 10 }, @@ -366,7 +365,7 @@ public class TestJapaneseTokenizer extends BaseTokenStreamTestCase { } 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); ts.reset(); for(String reading : readings) { @@ -378,7 +377,7 @@ public class TestJapaneseTokenizer extends BaseTokenStreamTestCase { } 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); ts.reset(); for(String pronunciation : pronunciations) { @@ -390,7 +389,7 @@ public class TestJapaneseTokenizer extends BaseTokenStreamTestCase { } 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); ts.reset(); for(String baseForm : baseForms) { @@ -402,7 +401,7 @@ public class TestJapaneseTokenizer extends BaseTokenStreamTestCase { } 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); ts.reset(); for(String inflectionType : inflectionTypes) { @@ -414,7 +413,7 @@ public class TestJapaneseTokenizer extends BaseTokenStreamTestCase { } 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); ts.reset(); for(String inflectionForm : inflectionForms) { @@ -426,7 +425,7 @@ public class TestJapaneseTokenizer extends BaseTokenStreamTestCase { } 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); ts.reset(); for(String partOfSpeech : partsOfSpeech) { @@ -619,7 +618,7 @@ public class TestJapaneseTokenizer extends BaseTokenStreamTestCase { if (numIterations > 1) { // warmup 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(); while(ts.incrementToken()); } @@ -628,7 +627,7 @@ public class TestJapaneseTokenizer extends BaseTokenStreamTestCase { long totalStart = System.currentTimeMillis(); 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(); while(ts.incrementToken()); } @@ -640,7 +639,7 @@ public class TestJapaneseTokenizer extends BaseTokenStreamTestCase { totalStart = System.currentTimeMillis(); for (int i = 0; i < numIterations; i++) { for (String sentence: sentences) { - final TokenStream ts = analyzer.tokenStream("ignored", new StringReader(sentence)); + final TokenStream ts = analyzer.tokenStream("ignored", sentence); ts.reset(); while(ts.incrementToken()); } diff --git a/lucene/analysis/morfologik/src/test/org/apache/lucene/analysis/morfologik/TestMorfologikAnalyzer.java b/lucene/analysis/morfologik/src/test/org/apache/lucene/analysis/morfologik/TestMorfologikAnalyzer.java index 7490caa927d..93bbe3c48a6 100644 --- a/lucene/analysis/morfologik/src/test/org/apache/lucene/analysis/morfologik/TestMorfologikAnalyzer.java +++ b/lucene/analysis/morfologik/src/test/org/apache/lucene/analysis/morfologik/TestMorfologikAnalyzer.java @@ -19,7 +19,6 @@ package org.apache.lucene.analysis.morfologik; import java.io.IOException; import java.io.Reader; -import java.io.StringReader; import java.util.TreeSet; import org.apache.lucene.analysis.Analyzer; @@ -73,7 +72,7 @@ public class TestMorfologikAnalyzer extends BaseTokenStreamTestCase { @SuppressWarnings("unused") private void dumpTokens(String input) throws IOException { - TokenStream ts = getTestAnalyzer().tokenStream("dummy", new StringReader(input)); + TokenStream ts = getTestAnalyzer().tokenStream("dummy", input); ts.reset(); MorphosyntacticTagsAttribute attribute = ts.getAttribute(MorphosyntacticTagsAttribute.class); @@ -86,7 +85,7 @@ public class TestMorfologikAnalyzer extends BaseTokenStreamTestCase { /** Test reuse of MorfologikFilter with leftover stems. */ public final void testLeftoverStems() throws IOException { 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); ts_1.reset(); ts_1.incrementToken(); @@ -94,7 +93,7 @@ public class TestMorfologikAnalyzer extends BaseTokenStreamTestCase { ts_1.end(); 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); ts_2.reset(); ts_2.incrementToken(); @@ -141,7 +140,7 @@ public class TestMorfologikAnalyzer extends BaseTokenStreamTestCase { /** Test morphosyntactic annotations. */ public final void testPOSAttribute() throws IOException { - TokenStream ts = getTestAnalyzer().tokenStream("dummy", new StringReader("liście")); + TokenStream ts = getTestAnalyzer().tokenStream("dummy", "liście"); ts.reset(); assertPOSToken(ts, "liście", diff --git a/lucene/analysis/smartcn/src/test/org/apache/lucene/analysis/cn/smart/TestSmartChineseAnalyzer.java b/lucene/analysis/smartcn/src/test/org/apache/lucene/analysis/cn/smart/TestSmartChineseAnalyzer.java index 98435307e2d..899bfbfdf85 100644 --- a/lucene/analysis/smartcn/src/test/org/apache/lucene/analysis/cn/smart/TestSmartChineseAnalyzer.java +++ b/lucene/analysis/smartcn/src/test/org/apache/lucene/analysis/cn/smart/TestSmartChineseAnalyzer.java @@ -19,7 +19,6 @@ package org.apache.lucene.analysis.cn.smart; import java.io.IOException; import java.io.Reader; -import java.io.StringReader; import java.util.Random; import org.apache.lucene.analysis.BaseTokenStreamTestCase; @@ -185,7 +184,7 @@ public class TestSmartChineseAnalyzer extends BaseTokenStreamTestCase { sb.append("我购买了道具和服装。"); } Analyzer analyzer = new SmartChineseAnalyzer(TEST_VERSION_CURRENT); - TokenStream stream = analyzer.tokenStream("", new StringReader(sb.toString())); + TokenStream stream = analyzer.tokenStream("", sb.toString()); stream.reset(); while (stream.incrementToken()) { } @@ -198,7 +197,7 @@ public class TestSmartChineseAnalyzer extends BaseTokenStreamTestCase { sb.append("我购买了道具和服装"); } Analyzer analyzer = new SmartChineseAnalyzer(TEST_VERSION_CURRENT); - TokenStream stream = analyzer.tokenStream("", new StringReader(sb.toString())); + TokenStream stream = analyzer.tokenStream("", sb.toString()); stream.reset(); while (stream.incrementToken()) { } diff --git a/lucene/analysis/uima/src/test/org/apache/lucene/analysis/uima/UIMABaseAnalyzerTest.java b/lucene/analysis/uima/src/test/org/apache/lucene/analysis/uima/UIMABaseAnalyzerTest.java index 24f3e5a1b4f..702fff495e3 100644 --- a/lucene/analysis/uima/src/test/org/apache/lucene/analysis/uima/UIMABaseAnalyzerTest.java +++ b/lucene/analysis/uima/src/test/org/apache/lucene/analysis/uima/UIMABaseAnalyzerTest.java @@ -35,7 +35,6 @@ import org.junit.After; import org.junit.Before; import org.junit.Test; -import java.io.StringReader; import java.util.HashMap; import java.util.Map; @@ -62,7 +61,7 @@ public class UIMABaseAnalyzerTest extends BaseTokenStreamTestCase { @Test 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"}); } diff --git a/lucene/analysis/uima/src/test/org/apache/lucene/analysis/uima/UIMATypeAwareAnalyzerTest.java b/lucene/analysis/uima/src/test/org/apache/lucene/analysis/uima/UIMATypeAwareAnalyzerTest.java index d595c9a2250..8036b2fa4d0 100644 --- a/lucene/analysis/uima/src/test/org/apache/lucene/analysis/uima/UIMATypeAwareAnalyzerTest.java +++ b/lucene/analysis/uima/src/test/org/apache/lucene/analysis/uima/UIMATypeAwareAnalyzerTest.java @@ -23,8 +23,6 @@ import org.junit.After; import org.junit.Before; import org.junit.Test; -import java.io.StringReader; - /** * Testcase for {@link UIMATypeAwareAnalyzer} */ @@ -51,7 +49,7 @@ public class UIMATypeAwareAnalyzerTest extends BaseTokenStreamTestCase { public void baseUIMATypeAwareAnalyzerStreamTest() throws Exception { // 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 assertTokenStreamContents(ts, diff --git a/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java b/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java index 6b11ffdb5de..75259bb702f 100755 --- a/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java +++ b/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java @@ -21,7 +21,6 @@ import java.io.BufferedReader; import java.io.File; import java.io.FileInputStream; import java.io.InputStreamReader; -import java.io.StringReader; import java.text.Collator; import java.util.List; import java.util.Locale; @@ -979,8 +978,8 @@ public class TestPerfTasksLogic extends BenchmarkTestCase { private void assertEqualCollation(Analyzer a1, Analyzer a2, String text) throws Exception { - TokenStream ts1 = a1.tokenStream("bogus", new StringReader(text)); - TokenStream ts2 = a2.tokenStream("bogus", new StringReader(text)); + TokenStream ts1 = a1.tokenStream("bogus", text); + TokenStream ts2 = a2.tokenStream("bogus", text); ts1.reset(); ts2.reset(); TermToBytesRefAttribute termAtt1 = ts1.addAttribute(TermToBytesRefAttribute.class); @@ -1030,7 +1029,7 @@ public class TestPerfTasksLogic extends BenchmarkTestCase { Benchmark benchmark = execBenchmark(getAnalyzerFactoryConfig ("shingle-analyzer", "StandardTokenizer,ShingleFilter")); benchmark.getRunData().getAnalyzer().tokenStream - ("bogus", new StringReader(text)).close(); + ("bogus", text).close(); BaseTokenStreamTestCase.assertAnalyzesTo(benchmark.getRunData().getAnalyzer(), text, new String[] { "one", "one two", "two", "two three", "three", "three four", "four", "four five", diff --git a/lucene/classification/src/java/org/apache/lucene/classification/SimpleNaiveBayesClassifier.java b/lucene/classification/src/java/org/apache/lucene/classification/SimpleNaiveBayesClassifier.java index 692fe4f8492..2fe5c832b18 100644 --- a/lucene/classification/src/java/org/apache/lucene/classification/SimpleNaiveBayesClassifier.java +++ b/lucene/classification/src/java/org/apache/lucene/classification/SimpleNaiveBayesClassifier.java @@ -33,7 +33,6 @@ import org.apache.lucene.search.WildcardQuery; import org.apache.lucene.util.BytesRef; import java.io.IOException; -import java.io.StringReader; import java.util.Collection; import java.util.LinkedList; @@ -86,7 +85,7 @@ public class SimpleNaiveBayesClassifier implements Classifier { private String[] tokenizeDoc(String doc) throws IOException { Collection result = new LinkedList(); - TokenStream tokenStream = analyzer.tokenStream(textFieldName, new StringReader(doc)); + TokenStream tokenStream = analyzer.tokenStream(textFieldName, doc); CharTermAttribute charTermAttribute = tokenStream.addAttribute(CharTermAttribute.class); tokenStream.reset(); while (tokenStream.incrementToken()) { diff --git a/lucene/core/src/java/org/apache/lucene/analysis/Analyzer.java b/lucene/core/src/java/org/apache/lucene/analysis/Analyzer.java index dd48520c709..01bd18b2d26 100644 --- a/lucene/core/src/java/org/apache/lucene/analysis/Analyzer.java +++ b/lucene/core/src/java/org/apache/lucene/analysis/Analyzer.java @@ -120,11 +120,16 @@ public abstract class Analyzer implements Closeable { * See the {@link org.apache.lucene.analysis Analysis package documentation} for * some examples demonstrating this. * + * NOTE: 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 reader the reader the streams source reads from * @return TokenStream for iterating the analyzed content of reader * @throws AlreadyClosedException if the Analyzer is closed. * @throws IOException if an i/o error occurs. + * @see #tokenStream(String, String) */ public final TokenStream tokenStream(final String fieldName, final Reader reader) throws IOException { @@ -139,6 +144,45 @@ public abstract class Analyzer implements Closeable { return components.getTokenStream(); } + /** + * Returns a TokenStream suitable for fieldName, tokenizing + * the contents of text. + *

+ * 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)}. + *

+ * NOTE: 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 reader + * @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. *

@@ -208,6 +252,9 @@ public abstract class Analyzer implements Closeable { * the chain. This can be the source if there are no filters. */ protected final TokenStream sink; + + /** Internal cache only used by {@link Analyzer#tokenStream(String, String)}. */ + transient ReusableStringReader reusableStringReader; /** * Creates a new {@link TokenStreamComponents} instance. diff --git a/lucene/core/src/java/org/apache/lucene/analysis/ReusableStringReader.java b/lucene/core/src/java/org/apache/lucene/analysis/ReusableStringReader.java new file mode 100644 index 00000000000..a629c0d871e --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/analysis/ReusableStringReader.java @@ -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; + } +} diff --git a/lucene/core/src/java/org/apache/lucene/document/Field.java b/lucene/core/src/java/org/apache/lucene/document/Field.java index ad6bf941e96..4be105ad09e 100644 --- a/lucene/core/src/java/org/apache/lucene/document/Field.java +++ b/lucene/core/src/java/org/apache/lucene/document/Field.java @@ -75,7 +75,6 @@ public class Field implements IndexableField, StorableField { protected TokenStream tokenStream; private transient TokenStream internalTokenStream; - private transient ReusableStringReader internalReader; /** * Field's boost @@ -552,56 +551,12 @@ public class Field implements IndexableField, StorableField { } else if (readerValue() != null) { return analyzer.tokenStream(name(), readerValue()); } else if (stringValue() != null) { - if (internalReader == null) { - internalReader = new ReusableStringReader(); - } - internalReader.setValue(stringValue()); - return analyzer.tokenStream(name(), internalReader); + return analyzer.tokenStream(name(), stringValue()); } 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 { private final CharTermAttribute termAttribute = addAttribute(CharTermAttribute.class); private final OffsetAttribute offsetAttribute = addAttribute(OffsetAttribute.class); diff --git a/lucene/core/src/test/org/apache/lucene/analysis/TestMockAnalyzer.java b/lucene/core/src/test/org/apache/lucene/analysis/TestMockAnalyzer.java index a8cdbf08a6c..1bac429fcb3 100644 --- a/lucene/core/src/test/org/apache/lucene/analysis/TestMockAnalyzer.java +++ b/lucene/core/src/test/org/apache/lucene/analysis/TestMockAnalyzer.java @@ -96,7 +96,7 @@ public class TestMockAnalyzer extends BaseTokenStreamTestCase { String testString = "t"; Analyzer analyzer = new MockAnalyzer(random()); - TokenStream stream = analyzer.tokenStream("dummy", new StringReader(testString)); + TokenStream stream = analyzer.tokenStream("dummy", testString); stream.reset(); while (stream.incrementToken()) { // consume diff --git a/lucene/core/src/test/org/apache/lucene/analysis/TestReusableStringReader.java b/lucene/core/src/test/org/apache/lucene/analysis/TestReusableStringReader.java new file mode 100644 index 00000000000..45f8c424958 --- /dev/null +++ b/lucene/core/src/test/org/apache/lucene/analysis/TestReusableStringReader.java @@ -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()); + } + +} diff --git a/lucene/core/src/test/org/apache/lucene/document/TestField.java b/lucene/core/src/test/org/apache/lucene/document/TestField.java index 41aac6116e3..2417c65069b 100644 --- a/lucene/core/src/test/org/apache/lucene/document/TestField.java +++ b/lucene/core/src/test/org/apache/lucene/document/TestField.java @@ -18,11 +18,8 @@ package org.apache.lucene.document; */ import java.io.StringReader; -import java.nio.CharBuffer; - import org.apache.lucene.analysis.CannedTokenStream; import org.apache.lucene.analysis.Token; -import org.apache.lucene.document.Field.ReusableStringReader; import org.apache.lucene.util.BytesRef; 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()); - } } diff --git a/lucene/core/src/test/org/apache/lucene/index/TestLongPostings.java b/lucene/core/src/test/org/apache/lucene/index/TestLongPostings.java index ca9b8c3af3b..34f0a590fa1 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestLongPostings.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestLongPostings.java @@ -18,8 +18,6 @@ package org.apache.lucene.index; */ import java.io.IOException; -import java.io.StringReader; - import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.MockAnalyzer; import org.apache.lucene.analysis.TokenStream; @@ -49,7 +47,7 @@ public class TestLongPostings extends LuceneTestCase { if (other != null && s.equals(other)) { 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 BytesRef termBytes = termAtt.getBytesRef(); ts.reset(); diff --git a/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsWriter.java b/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsWriter.java index 67a5790ab31..37589097ac9 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsWriter.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsWriter.java @@ -18,8 +18,6 @@ package org.apache.lucene.index; */ import java.io.IOException; -import java.io.StringReader; - import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.CachingTokenFilter; import org.apache.lucene.analysis.MockAnalyzer; @@ -176,7 +174,7 @@ public class TestTermVectorsWriter extends LuceneTestCase { Analyzer analyzer = new MockAnalyzer(random()); IndexWriter w = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, analyzer)); 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 = new CachingTokenFilter(stream); FieldType customType = new FieldType(TextField.TYPE_NOT_STORED); diff --git a/lucene/core/src/test/org/apache/lucene/search/TestPhraseQuery.java b/lucene/core/src/test/org/apache/lucene/search/TestPhraseQuery.java index 29d05f1eea7..44ab8cf276b 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestPhraseQuery.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestPhraseQuery.java @@ -617,7 +617,7 @@ public class TestPhraseQuery extends LuceneTestCase { break; } } - TokenStream ts = analyzer.tokenStream("ignore", new StringReader(term)); + TokenStream ts = analyzer.tokenStream("ignore", term); CharTermAttribute termAttr = ts.addAttribute(CharTermAttribute.class); ts.reset(); while(ts.incrementToken()) { diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/highlight/Highlighter.java b/lucene/highlighter/src/java/org/apache/lucene/search/highlight/Highlighter.java index af0e54cd12a..e59d43080af 100644 --- a/lucene/highlighter/src/java/org/apache/lucene/search/highlight/Highlighter.java +++ b/lucene/highlighter/src/java/org/apache/lucene/search/highlight/Highlighter.java @@ -17,7 +17,6 @@ package org.apache.lucene.search.highlight; */ import java.io.IOException; -import java.io.StringReader; import java.util.ArrayList; 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.tokenattributes.CharTermAttribute; import org.apache.lucene.analysis.tokenattributes.OffsetAttribute; -import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute; import org.apache.lucene.util.PriorityQueue; /** @@ -78,7 +76,7 @@ public class Highlighter public final String getBestFragment(Analyzer analyzer, String fieldName,String text) throws IOException, InvalidTokenOffsetsException { - TokenStream tokenStream = analyzer.tokenStream(fieldName, new StringReader(text)); + TokenStream tokenStream = analyzer.tokenStream(fieldName, text); return getBestFragment(tokenStream, text); } @@ -130,7 +128,7 @@ public class Highlighter int maxNumFragments) throws IOException, InvalidTokenOffsetsException { - TokenStream tokenStream = analyzer.tokenStream(fieldName, new StringReader(text)); + TokenStream tokenStream = analyzer.tokenStream(fieldName, text); return getBestFragments(tokenStream, text, maxNumFragments); } diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/highlight/TokenSources.java b/lucene/highlighter/src/java/org/apache/lucene/search/highlight/TokenSources.java index db45d8bdd76..e0b3c8c95ea 100644 --- a/lucene/highlighter/src/java/org/apache/lucene/search/highlight/TokenSources.java +++ b/lucene/highlighter/src/java/org/apache/lucene/search/highlight/TokenSources.java @@ -36,7 +36,6 @@ import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.BytesRef; import java.io.IOException; -import java.io.StringReader; import java.util.ArrayList; import java.util.Comparator; @@ -314,7 +313,7 @@ public class TokenSources { public static TokenStream getTokenStream(String field, String contents, Analyzer analyzer) { try { - return analyzer.tokenStream(field, new StringReader(contents)); + return analyzer.tokenStream(field, contents); } catch (IOException ex) { throw new RuntimeException(ex); } diff --git a/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java b/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java index 907d079e66f..7813b17e42b 100644 --- a/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java +++ b/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java @@ -20,7 +20,6 @@ package org.apache.lucene.search.highlight; import java.io.ByteArrayInputStream; import java.io.IOException; import java.io.Reader; -import java.io.StringReader; import java.util.ArrayList; import java.util.HashMap; import java.util.Iterator; @@ -248,7 +247,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte private String highlightField(Query query, String fieldName, String text) throws IOException, InvalidTokenOffsetsException { TokenStream tokenStream = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true, MockTokenFilter.ENGLISH_STOPSET) - .tokenStream(fieldName, new StringReader(text)); + .tokenStream(fieldName, text); // Assuming "", "" used to highlight SimpleHTMLFormatter formatter = new SimpleHTMLFormatter(); 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++) { 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)); 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++) { 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)); @@ -377,7 +375,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte for (int i = 0; i < hits.totalHits; i++) { 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)); @@ -406,7 +404,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte for (int i = 0; i < hits.totalHits; i++) { 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)); @@ -431,7 +429,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte for (int i = 0; i < hits.totalHits; i++) { 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)); @@ -455,7 +453,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte for (int i = 0; i < hits.totalHits; i++) { 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)); @@ -479,7 +477,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte for (int i = 0; i < hits.totalHits; i++) { 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)); @@ -505,7 +503,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte for (int i = 0; i < hits.totalHits; i++) { 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)); @@ -533,7 +531,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte for (int i = 0; i < hits.totalHits; i++) { 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, "..."); @@ -555,7 +553,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte for (int i = 0; i < hits.totalHits; i++) { 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); 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++) { 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)); @@ -608,7 +606,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte for (int i = 0; i < hits.totalHits; i++) { 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)); @@ -639,7 +637,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte for (int i = 0; i < hits.totalHits; i++) { 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)); @@ -710,7 +708,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte int maxNumFragmentsRequired = 2; for (int i = 0; i < hits.totalHits; i++) { 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, "..."); @@ -907,7 +905,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte int maxNumFragmentsRequired = 2; String fragmentSeparator = "..."; 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); @@ -931,7 +929,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte int maxNumFragmentsRequired = 2; String fragmentSeparator = "..."; 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); @@ -955,7 +953,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte int maxNumFragmentsRequired = 2; String fragmentSeparator = "..."; 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); @@ -1126,7 +1124,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte numHighlights = 0; for (int i = 0; i < hits.totalHits; i++) { 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); @@ -1187,7 +1185,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte Highlighter highlighter = getHighlighter(wTerms, HighlighterTest.this);// new // Highlighter(new // 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)); String result = highlighter.getBestFragment(tokenStream, texts[0]).trim(); @@ -1196,7 +1194,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte // readjust weights 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.setTextFragmenter(new SimpleFragmenter(2)); @@ -1232,7 +1230,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte Highlighter highlighter = getHighlighter(query, null, HighlighterTest.this); // 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 expectedResult = "football-soccer in the euro 2004 footie competition"; @@ -1257,7 +1255,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte for (int i = 0; i < hits.totalHits; i++) { 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); 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++) { 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);// new Highlighter(this, new @@ -1288,7 +1286,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte highlighter.setTextFragmenter(new SimpleFragmenter(20)); 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, true, 10); @@ -1318,7 +1316,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte public void run() throws Exception { numHighlights = 0; 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, HighlighterTest.this);// new Highlighter(this, new // QueryTermScorer(query)); @@ -1432,7 +1430,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte for (int i = 0; i < hits.totalHits; i++) { 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.setTextFragmenter(new SimpleFragmenter(40)); @@ -1461,7 +1459,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte doSearching(new TermQuery(new Term(FIELD_NAME, "aninvalidquerywhichshouldyieldnoresults"))); 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, HighlighterTest.this); String result = highlighter.getBestFragment(tokenStream, text); @@ -1503,7 +1501,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte } }); 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, ""); // 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 = * 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); * System.out.println(highlightedText); } } */ @@ -1855,7 +1853,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte final int expectedHighlights) throws Exception { for (int i = 0; i < hits.totalHits; i++) { 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); Highlighter highlighter = new Highlighter(this, scorer); @@ -2104,7 +2102,7 @@ final class SynonymTokenizer extends TokenStream { int maxNumFragmentsRequired = 2; String fragmentSeparator = "..."; Scorer scorer = null; - TokenStream tokenStream = analyzer.tokenStream(HighlighterTest.FIELD_NAME, new StringReader(text)); + TokenStream tokenStream = analyzer.tokenStream(HighlighterTest.FIELD_NAME, text); if (mode == QUERY) { scorer = new QueryScorer(query); } else if (mode == QUERY_TERM) { diff --git a/lucene/highlighter/src/test/org/apache/lucene/search/highlight/custom/HighlightCustomQueryTest.java b/lucene/highlighter/src/test/org/apache/lucene/search/highlight/custom/HighlightCustomQueryTest.java index c57210a60c3..536259ade85 100644 --- a/lucene/highlighter/src/test/org/apache/lucene/search/highlight/custom/HighlightCustomQueryTest.java +++ b/lucene/highlighter/src/test/org/apache/lucene/search/highlight/custom/HighlightCustomQueryTest.java @@ -17,7 +17,6 @@ package org.apache.lucene.search.highlight.custom; * limitations under the License. */ import java.io.IOException; -import java.io.StringReader; import java.util.Map; import org.apache.lucene.analysis.MockAnalyzer; @@ -89,8 +88,7 @@ public class HighlightCustomQueryTest extends LuceneTestCase { private String highlightField(Query query, String fieldName, String text) throws IOException, InvalidTokenOffsetsException { TokenStream tokenStream = new MockAnalyzer(random(), MockTokenizer.SIMPLE, - true, MockTokenFilter.ENGLISH_STOPSET).tokenStream(fieldName, - new StringReader(text)); + true, MockTokenFilter.ENGLISH_STOPSET).tokenStream(fieldName, text); // Assuming "", "" used to highlight SimpleHTMLFormatter formatter = new SimpleHTMLFormatter(); MyQueryScorer scorer = new MyQueryScorer(query, fieldName, FIELD_NAME); diff --git a/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/AbstractTestCase.java b/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/AbstractTestCase.java index 87273812afe..43ea21f5004 100644 --- a/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/AbstractTestCase.java +++ b/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/AbstractTestCase.java @@ -19,7 +19,6 @@ package org.apache.lucene.search.vectorhighlight; import java.io.IOException; import java.io.Reader; -import java.io.StringReader; import java.util.ArrayList; import java.util.Collection; import java.util.List; @@ -171,7 +170,7 @@ public abstract class AbstractTestCase extends LuceneTestCase { protected List analyze(String text, String field, Analyzer analyzer) throws IOException { List bytesRefs = new ArrayList(); - TokenStream tokenStream = analyzer.tokenStream(field, new StringReader(text)); + TokenStream tokenStream = analyzer.tokenStream(field, text); TermToBytesRefAttribute termAttribute = tokenStream.getAttribute(TermToBytesRefAttribute.class); BytesRef bytesRef = termAttribute.getBytesRef(); diff --git a/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java b/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java index 43feb828fb5..c8918523f0b 100644 --- a/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java +++ b/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java @@ -18,7 +18,6 @@ package org.apache.lucene.index.memory; */ import java.io.IOException; -import java.io.StringReader; import java.util.Arrays; import java.util.Collection; import java.util.Comparator; @@ -291,7 +290,7 @@ public class MemoryIndex { TokenStream stream; try { - stream = analyzer.tokenStream(fieldName, new StringReader(text)); + stream = analyzer.tokenStream(fieldName, text); } catch (IOException ex) { throw new RuntimeException(ex); } diff --git a/lucene/memory/src/test/org/apache/lucene/index/memory/MemoryIndexTest.java b/lucene/memory/src/test/org/apache/lucene/index/memory/MemoryIndexTest.java index a0c6abcea3d..5d9b1d654d8 100644 --- a/lucene/memory/src/test/org/apache/lucene/index/memory/MemoryIndexTest.java +++ b/lucene/memory/src/test/org/apache/lucene/index/memory/MemoryIndexTest.java @@ -22,7 +22,6 @@ import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; import java.io.Reader; -import java.io.StringReader; import java.util.HashSet; import java.util.Set; @@ -378,7 +377,7 @@ public class MemoryIndexTest extends BaseTokenStreamTestCase { SpanQuery wrappedquery = new SpanMultiTermQueryWrapper(regex); 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 assertEquals(0, mindex.search(wrappedquery), 0.00001f); @@ -390,7 +389,7 @@ public class MemoryIndexTest extends BaseTokenStreamTestCase { SpanQuery wrappedquery = new SpanOrQuery(new SpanMultiTermQueryWrapper(regex)); 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 assertEquals(0, mindex.search(wrappedquery), 0.00001f); diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/analyzing/AnalyzingQueryParser.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/analyzing/AnalyzingQueryParser.java index dfe57935a48..3ce2ae34723 100644 --- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/analyzing/AnalyzingQueryParser.java +++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/analyzing/AnalyzingQueryParser.java @@ -18,7 +18,6 @@ package org.apache.lucene.queryparser.analyzing; */ import java.io.IOException; -import java.io.StringReader; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -165,7 +164,7 @@ public class AnalyzingQueryParser extends org.apache.lucene.queryparser.classic. String analyzed = null; TokenStream stream = null; try{ - stream = getAnalyzer().tokenStream(field, new StringReader(chunk)); + stream = getAnalyzer().tokenStream(field, chunk); stream.reset(); CharTermAttribute termAtt = stream.getAttribute(CharTermAttribute.class); // get first and hopefully only output token diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/QueryParserBase.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/QueryParserBase.java index b1b92739e05..2bf91203e0a 100644 --- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/QueryParserBase.java +++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/QueryParserBase.java @@ -500,7 +500,7 @@ public abstract class QueryParserBase implements CommonQueryParserConfiguration TokenStream source; try { - source = analyzer.tokenStream(field, new StringReader(queryText)); + source = analyzer.tokenStream(field, queryText); source.reset(); } catch (IOException e) { 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; try { - source = analyzerIn.tokenStream(field, new StringReader(part)); + source = analyzerIn.tokenStream(field, part); source.reset(); } catch (IOException e) { throw new RuntimeException("Unable to initialize TokenStream to analyze multiTerm term: " + part, e); diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/AnalyzerQueryNodeProcessor.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/AnalyzerQueryNodeProcessor.java index 3f993559e82..78610af393b 100644 --- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/AnalyzerQueryNodeProcessor.java +++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/AnalyzerQueryNodeProcessor.java @@ -18,7 +18,6 @@ package org.apache.lucene.queryparser.flexible.standard.processors; */ import java.io.IOException; -import java.io.StringReader; import java.util.ArrayList; import java.util.LinkedList; import java.util.List; @@ -116,7 +115,7 @@ public class AnalyzerQueryNodeProcessor extends QueryNodeProcessorImpl { TokenStream source; try { - source = this.analyzer.tokenStream(field, new StringReader(text)); + source = this.analyzer.tokenStream(field, text); source.reset(); } catch (IOException e1) { throw new RuntimeException(e1); diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/LikeThisQueryBuilder.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/LikeThisQueryBuilder.java index 3781c2808a5..56cc66ef73c 100644 --- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/LikeThisQueryBuilder.java +++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/LikeThisQueryBuilder.java @@ -4,7 +4,6 @@ package org.apache.lucene.queryparser.xml.builders; import java.io.IOException; -import java.io.StringReader; import java.util.HashSet; import java.util.Set; @@ -75,7 +74,7 @@ public class LikeThisQueryBuilder implements QueryBuilder { stopWordsSet = new HashSet(); for (String field : fields) { try { - TokenStream ts = analyzer.tokenStream(field, new StringReader(stopWords)); + TokenStream ts = analyzer.tokenStream(field, stopWords); CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class); ts.reset(); while (ts.incrementToken()) { diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/SpanOrTermsBuilder.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/SpanOrTermsBuilder.java index 99853660b69..ecda31135a9 100644 --- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/SpanOrTermsBuilder.java +++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/SpanOrTermsBuilder.java @@ -13,9 +13,9 @@ import org.apache.lucene.queryparser.xml.ParserException; import org.w3c.dom.Element; import java.io.IOException; -import java.io.StringReader; import java.util.ArrayList; import java.util.List; + /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -51,7 +51,7 @@ public class SpanOrTermsBuilder extends SpanBuilderBase { try { List clausesList = new ArrayList(); - TokenStream ts = analyzer.tokenStream(fieldName, new StringReader(value)); + TokenStream ts = analyzer.tokenStream(fieldName, value); TermToBytesRefAttribute termAtt = ts.addAttribute(TermToBytesRefAttribute.class); BytesRef bytes = termAtt.getBytesRef(); ts.reset(); diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/TermsFilterBuilder.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/TermsFilterBuilder.java index e614608d64b..65b13014f24 100644 --- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/TermsFilterBuilder.java +++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/TermsFilterBuilder.java @@ -3,7 +3,6 @@ package org.apache.lucene.queryparser.xml.builders; import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.TokenStream; import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute; -import org.apache.lucene.index.Term; import org.apache.lucene.search.Filter; import org.apache.lucene.queries.TermsFilter; import org.apache.lucene.util.BytesRef; @@ -13,7 +12,6 @@ import org.apache.lucene.queryparser.xml.ParserException; import org.w3c.dom.Element; import java.io.IOException; -import java.io.StringReader; import java.util.ArrayList; import java.util.List; @@ -57,9 +55,8 @@ public class TermsFilterBuilder implements FilterBuilder { String fieldName = DOMUtils.getAttributeWithInheritanceOrFail(e, "fieldName"); try { - TokenStream ts = analyzer.tokenStream(fieldName, new StringReader(text)); + TokenStream ts = analyzer.tokenStream(fieldName, text); TermToBytesRefAttribute termAtt = ts.addAttribute(TermToBytesRefAttribute.class); - Term term = null; BytesRef bytes = termAtt.getBytesRef(); ts.reset(); while (ts.incrementToken()) { diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/TermsQueryBuilder.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/TermsQueryBuilder.java index 29f03e854a2..ed06d091ba4 100644 --- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/TermsQueryBuilder.java +++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/TermsQueryBuilder.java @@ -15,7 +15,6 @@ import org.apache.lucene.queryparser.xml.QueryBuilder; import org.w3c.dom.Element; import java.io.IOException; -import java.io.StringReader; /* * 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)); bq.setMinimumNumberShouldMatch(DOMUtils.getAttribute(e, "minimumNumberShouldMatch", 0)); try { - TokenStream ts = analyzer.tokenStream(fieldName, new StringReader(text)); + TokenStream ts = analyzer.tokenStream(fieldName, text); TermToBytesRefAttribute termAtt = ts.addAttribute(TermToBytesRefAttribute.class); Term term = null; BytesRef bytes = termAtt.getBytesRef(); diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/FuzzyLikeThisQuery.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/FuzzyLikeThisQuery.java index 5ee775ad677..6d205c78a74 100644 --- a/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/FuzzyLikeThisQuery.java +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/FuzzyLikeThisQuery.java @@ -18,7 +18,6 @@ package org.apache.lucene.sandbox.queries; */ import java.io.IOException; -import java.io.StringReader; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; @@ -194,7 +193,7 @@ public class FuzzyLikeThisQuery extends Query private void addTerms(IndexReader reader, FieldVals f) throws IOException { 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); int corpusNumDocs = reader.numDocs(); diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java index 6e797ad1edc..b1b9b79b4ec 100644 --- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java +++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java @@ -21,7 +21,6 @@ import java.io.File; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; -import java.io.StringReader; import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; @@ -856,7 +855,7 @@ public class AnalyzingSuggester extends Lookup { final Set toFiniteStrings(final BytesRef surfaceForm, final TokenStreamToAutomaton ts2a) throws IOException { // Analyze surface form: - TokenStream ts = indexAnalyzer.tokenStream("", new StringReader(surfaceForm.utf8ToString())); + TokenStream ts = indexAnalyzer.tokenStream("", surfaceForm.utf8ToString()); // Create corresponding automaton: labels are bytes // 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 { // TODO: is there a Reader from a CharSequence? // Turn tokenstream into automaton: - TokenStream ts = queryAnalyzer.tokenStream("", new StringReader(key.toString())); + TokenStream ts = queryAnalyzer.tokenStream("", key.toString()); Automaton automaton = (getTokenStreamToAutomaton()).toAutomaton(ts); ts.close(); diff --git a/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java index 0dcb5bfcced..687da2f8c4e 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java +++ b/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java @@ -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 { - 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 { - 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 { - 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 { @@ -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 { - 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 { @@ -891,7 +891,7 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase { protected String toDot(Analyzer a, String inputText) throws IOException { final StringWriter sw = new StringWriter(); - final TokenStream ts = a.tokenStream("field", new StringReader(inputText)); + final TokenStream ts = a.tokenStream("field", inputText); ts.reset(); new TokenStreamToDot(inputText, ts, new PrintWriter(sw)).toDot(); return sw.toString(); @@ -899,7 +899,7 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase { protected void toDotFile(Analyzer a, String inputText, String localFileName) throws IOException { 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(); new TokenStreamToDot(inputText, ts, new PrintWriter(w)).toDot(); w.close(); diff --git a/lucene/test-framework/src/java/org/apache/lucene/analysis/CollationTestBase.java b/lucene/test-framework/src/java/org/apache/lucene/analysis/CollationTestBase.java index 716303e8d0b..38f006becf0 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/analysis/CollationTestBase.java +++ b/lucene/test-framework/src/java/org/apache/lucene/analysis/CollationTestBase.java @@ -19,7 +19,6 @@ package org.apache.lucene.analysis; import java.io.IOException; -import java.io.StringReader; import java.util.HashMap; 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.IndexWriter; import org.apache.lucene.index.IndexWriterConfig; -import org.apache.lucene.index.IndexableField; import org.apache.lucene.index.StorableField; import org.apache.lucene.index.StoredDocument; 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 TextField("contents", sortData[i][1], Field.Store.NO)); 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) - 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) - 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) - 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.forceMerge(1); @@ -260,7 +258,7 @@ public abstract class CollationTestBase extends LuceneTestCase { for (int i = 0; i < numTestPoints; i++) { 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); BytesRef bytes = termAtt.getBytesRef(); ts.reset(); @@ -279,7 +277,7 @@ public abstract class CollationTestBase extends LuceneTestCase { for (Map.Entry mapping : map.entrySet()) { String term = mapping.getKey(); BytesRef expected = mapping.getValue(); - TokenStream ts = analyzer.tokenStream("fake", new StringReader(term)); + TokenStream ts = analyzer.tokenStream("fake", term); TermToBytesRefAttribute termAtt = ts.addAttribute(TermToBytesRefAttribute.class); BytesRef bytes = termAtt.getBytesRef(); ts.reset(); diff --git a/solr/contrib/analysis-extras/src/java/org/apache/solr/schema/ICUCollationField.java b/solr/contrib/analysis-extras/src/java/org/apache/solr/schema/ICUCollationField.java index b8ed1957a47..2cacb63e793 100644 --- a/solr/contrib/analysis-extras/src/java/org/apache/solr/schema/ICUCollationField.java +++ b/solr/contrib/analysis-extras/src/java/org/apache/solr/schema/ICUCollationField.java @@ -19,7 +19,6 @@ package org.apache.solr.schema; import java.io.IOException; import java.io.InputStream; -import java.io.StringReader; import java.util.Map; 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.tokenattributes.TermToBytesRefAttribute; 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.search.Query; import org.apache.lucene.search.SortField; @@ -240,7 +237,7 @@ public class ICUCollationField extends FieldType { TokenStream source; try { - source = analyzer.tokenStream(field, new StringReader(part)); + source = analyzer.tokenStream(field, part); source.reset(); } catch (IOException e) { throw new RuntimeException("Unable to initialize TokenStream to analyze range part: " + part, e); diff --git a/solr/core/src/java/org/apache/solr/handler/AnalysisRequestHandlerBase.java b/solr/core/src/java/org/apache/solr/handler/AnalysisRequestHandlerBase.java index 7cf5f830fa1..20d9425e960 100644 --- a/solr/core/src/java/org/apache/solr/handler/AnalysisRequestHandlerBase.java +++ b/solr/core/src/java/org/apache/solr/handler/AnalysisRequestHandlerBase.java @@ -86,7 +86,7 @@ public abstract class AnalysisRequestHandlerBase extends RequestHandlerBase { TokenStream tokenStream = null; try { - tokenStream = analyzer.tokenStream(context.getFieldName(), new StringReader(value)); + tokenStream = analyzer.tokenStream(context.getFieldName(), value); } catch (IOException e) { throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e); } @@ -140,7 +140,7 @@ public abstract class AnalysisRequestHandlerBase extends RequestHandlerBase { protected Set getQueryTokenSet(String query, Analyzer analyzer) { try { final Set tokens = new HashSet(); - final TokenStream tokenStream = analyzer.tokenStream("", new StringReader(query)); + final TokenStream tokenStream = analyzer.tokenStream("", query); final TermToBytesRefAttribute bytesAtt = tokenStream.getAttribute(TermToBytesRefAttribute.class); final BytesRef bytes = bytesAtt.getBytesRef(); diff --git a/solr/core/src/java/org/apache/solr/handler/component/QueryElevationComponent.java b/solr/core/src/java/org/apache/solr/handler/component/QueryElevationComponent.java index 28fa1f3f0d8..5aa353ec57e 100644 --- a/solr/core/src/java/org/apache/solr/handler/component/QueryElevationComponent.java +++ b/solr/core/src/java/org/apache/solr/handler/component/QueryElevationComponent.java @@ -72,7 +72,6 @@ import javax.xml.xpath.XPathFactory; import java.io.File; import java.io.IOException; import java.io.InputStream; -import java.io.StringReader; import java.net.MalformedURLException; import java.net.URL; import java.util.ArrayList; @@ -344,7 +343,7 @@ public class QueryElevationComponent extends SearchComponent implements SolrCore return query; } StringBuilder norm = new StringBuilder(); - TokenStream tokens = analyzer.tokenStream("", new StringReader(query)); + TokenStream tokens = analyzer.tokenStream("", query); tokens.reset(); CharTermAttribute termAtt = tokens.addAttribute(CharTermAttribute.class); diff --git a/solr/core/src/java/org/apache/solr/handler/component/SpellCheckComponent.java b/solr/core/src/java/org/apache/solr/handler/component/SpellCheckComponent.java index dc99ba02980..1deec69da6a 100644 --- a/solr/core/src/java/org/apache/solr/handler/component/SpellCheckComponent.java +++ b/solr/core/src/java/org/apache/solr/handler/component/SpellCheckComponent.java @@ -18,7 +18,6 @@ package org.apache.solr.handler.component; import java.io.IOException; -import java.io.StringReader; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -464,7 +463,7 @@ public class SpellCheckComponent extends SearchComponent implements SolrCoreAwar private Collection getTokens(String q, Analyzer analyzer) throws IOException { Collection result = new ArrayList(); assert analyzer != null; - TokenStream ts = analyzer.tokenStream("", new StringReader(q)); + TokenStream ts = analyzer.tokenStream("", q); ts.reset(); // TODO: support custom attributes CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class); diff --git a/solr/core/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java b/solr/core/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java index f53c5472907..3de873fca35 100644 --- a/solr/core/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java +++ b/solr/core/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java @@ -48,7 +48,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; -import java.io.StringReader; 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 { TokenStream tstream; - TokenStream ts = schema.getAnalyzer().tokenStream(fieldName, new StringReader(docText)); + TokenStream ts = schema.getAnalyzer().tokenStream(fieldName, docText); ts.reset(); tstream = new TokenOrderingFilter(ts, 10); return tstream; diff --git a/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java b/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java index f92febc0c5d..863b03ee79c 100644 --- a/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java +++ b/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java @@ -405,7 +405,7 @@ public abstract class SolrQueryParserBase { TokenStream source; try { - source = analyzer.tokenStream(field, new StringReader(queryText)); + source = analyzer.tokenStream(field, queryText); source.reset(); } catch (IOException e) { throw new SyntaxError("Unable to initialize TokenStream to analyze query text", e); diff --git a/solr/core/src/java/org/apache/solr/schema/CollationField.java b/solr/core/src/java/org/apache/solr/schema/CollationField.java index 77c72f0ce3d..4fc8b16edac 100644 --- a/solr/core/src/java/org/apache/solr/schema/CollationField.java +++ b/solr/core/src/java/org/apache/solr/schema/CollationField.java @@ -19,7 +19,6 @@ package org.apache.solr.schema; import java.io.IOException; import java.io.InputStream; -import java.io.StringReader; import java.text.Collator; import java.text.ParseException; 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.tokenattributes.TermToBytesRefAttribute; 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.search.Query; import org.apache.lucene.search.SortField; @@ -216,7 +213,7 @@ public class CollationField extends FieldType { TokenStream source; try { - source = analyzer.tokenStream(field, new StringReader(part)); + source = analyzer.tokenStream(field, part); source.reset(); } catch (IOException e) { throw new RuntimeException("Unable to initialize TokenStream to analyze range part: " + part, e); diff --git a/solr/core/src/java/org/apache/solr/schema/TextField.java b/solr/core/src/java/org/apache/solr/schema/TextField.java index f599d9df3f5..c651259fb0f 100644 --- a/solr/core/src/java/org/apache/solr/schema/TextField.java +++ b/solr/core/src/java/org/apache/solr/schema/TextField.java @@ -35,7 +35,6 @@ import java.util.Map; import java.util.List; import java.util.ArrayList; import java.io.IOException; -import java.io.StringReader; /** TextField is the basic type for configurable text analysis. * Analyzers for field types using this implementation should be defined in the schema. @@ -141,7 +140,7 @@ public class TextField extends FieldType { TokenStream source; try { - source = analyzerIn.tokenStream(field, new StringReader(part)); + source = analyzerIn.tokenStream(field, part); source.reset(); } catch (IOException 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; try { - source = analyzer.tokenStream(field, new StringReader(queryText)); + source = analyzer.tokenStream(field, queryText); source.reset(); } catch (IOException e) { throw new RuntimeException("Unable to initialize TokenStream to analyze query text", e); diff --git a/solr/core/src/java/org/apache/solr/spelling/SpellingQueryConverter.java b/solr/core/src/java/org/apache/solr/spelling/SpellingQueryConverter.java index bb7c10ca173..359cc69d63e 100644 --- a/solr/core/src/java/org/apache/solr/spelling/SpellingQueryConverter.java +++ b/solr/core/src/java/org/apache/solr/spelling/SpellingQueryConverter.java @@ -18,8 +18,6 @@ package org.apache.solr.spelling; import java.io.IOException; -import java.io.Reader; -import java.io.StringReader; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -28,11 +26,10 @@ import java.util.regex.Pattern; import org.apache.lucene.analysis.Token; 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.PayloadAttribute; import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute; -import org.apache.lucene.analysis.tokenattributes.CharTermAttribute; import org.apache.lucene.analysis.tokenattributes.TypeAttribute; @@ -160,7 +157,7 @@ public class SpellingQueryConverter extends QueryConverter { flagValue = TERM_PRECEDES_NEW_BOOLEAN_OPERATOR_FLAG; } try { - analyze(result, new StringReader(word), startIndex, flagValue); + analyze(result, word, startIndex, flagValue); } catch (IOException e) { // TODO: shouldn't we log something? } @@ -174,7 +171,7 @@ public class SpellingQueryConverter extends QueryConverter { return result; } - protected void analyze(Collection result, Reader text, int offset, int flagsAttValue) throws IOException { + protected void analyze(Collection result, String text, int offset, int flagsAttValue) throws IOException { TokenStream stream = analyzer.tokenStream("", text); // TODO: support custom attributes CharTermAttribute termAtt = stream.addAttribute(CharTermAttribute.class); diff --git a/solr/core/src/java/org/apache/solr/spelling/SuggestQueryConverter.java b/solr/core/src/java/org/apache/solr/spelling/SuggestQueryConverter.java index 58a2e3fafc0..c16f6c65a0d 100644 --- a/solr/core/src/java/org/apache/solr/spelling/SuggestQueryConverter.java +++ b/solr/core/src/java/org/apache/solr/spelling/SuggestQueryConverter.java @@ -18,7 +18,6 @@ package org.apache.solr.spelling; */ import java.io.IOException; -import java.io.StringReader; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -38,7 +37,7 @@ public class SuggestQueryConverter extends SpellingQueryConverter { Collection result = new ArrayList(); try { - analyze(result, new StringReader(original), 0, 0); + analyze(result, original, 0, 0); } catch (IOException e) { throw new RuntimeException(e); } diff --git a/solr/core/src/test/org/apache/solr/TestTrie.java b/solr/core/src/test/org/apache/solr/TestTrie.java index 0e69d70ea86..82cd5259181 100644 --- a/solr/core/src/test/org/apache/solr/TestTrie.java +++ b/solr/core/src/test/org/apache/solr/TestTrie.java @@ -27,7 +27,6 @@ import org.junit.After; import org.junit.BeforeClass; import org.junit.Test; -import java.io.StringReader; import java.text.SimpleDateFormat; import java.util.Locale; import java.util.TimeZone; @@ -57,7 +56,7 @@ public class TestTrie extends SolrTestCaseJ4 { assertTrue(type instanceof TrieField); 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); ts.reset(); int count = 0; @@ -74,7 +73,7 @@ public class TestTrie extends SolrTestCaseJ4 { ts.close(); // Test empty one: - ts = type.getAnalyzer().tokenStream("dummy", new StringReader("")); + ts = type.getAnalyzer().tokenStream("dummy", ""); ts.reset(); assertFalse(ts.incrementToken()); ts.end(); diff --git a/solr/core/src/test/org/apache/solr/analysis/TestReversedWildcardFilterFactory.java b/solr/core/src/test/org/apache/solr/analysis/TestReversedWildcardFilterFactory.java index 448ef8c3af0..c019465daf9 100644 --- a/solr/core/src/test/org/apache/solr/analysis/TestReversedWildcardFilterFactory.java +++ b/solr/core/src/test/org/apache/solr/analysis/TestReversedWildcardFilterFactory.java @@ -86,7 +86,7 @@ public class TestReversedWildcardFilterFactory extends SolrTestCaseJ4 { String text = "one two three si\uD834\uDD1Ex"; // field one - TokenStream input = a.tokenStream("one", new StringReader(text)); + TokenStream input = a.tokenStream("one", text); assertTokenStreamContents(input, new String[] { "\u0001eno", "one", "\u0001owt", "two", "\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 } ); // field two - input = a.tokenStream("two", new StringReader(text)); + input = a.tokenStream("two", text); assertTokenStreamContents(input, new String[] { "\u0001eno", "\u0001owt", "\u0001eerht", "\u0001x\uD834\uDD1Eis" }, @@ -104,7 +104,7 @@ public class TestReversedWildcardFilterFactory extends SolrTestCaseJ4 { new int[] { 1, 1, 1, 1 } ); // field three - input = a.tokenStream("three", new StringReader(text)); + input = a.tokenStream("three", text); assertTokenStreamContents(input, new String[] { "one", "two", "three", "si\uD834\uDD1Ex" }, new int[] { 0, 4, 8, 14 }, diff --git a/solr/core/src/test/org/apache/solr/highlight/HighlighterTest.java b/solr/core/src/test/org/apache/solr/highlight/HighlighterTest.java index 7861a5c5922..de4edbfe565 100755 --- a/solr/core/src/test/org/apache/solr/highlight/HighlighterTest.java +++ b/solr/core/src/test/org/apache/solr/highlight/HighlighterTest.java @@ -30,7 +30,6 @@ import org.junit.After; import org.junit.BeforeClass; import org.junit.Test; -import java.io.StringReader; import java.util.Arrays; import java.util.HashMap; import java.util.List; @@ -175,7 +174,7 @@ public class HighlighterTest extends SolrTestCaseJ4 { public void testTermOffsetsTokenStream() throws Exception { String[] multivalued = { "a b c d", "e f g", "h", "i j k l m n" }; 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(); TermOffsetsTokenStream tots = new TermOffsetsTokenStream( @@ -183,7 +182,7 @@ public class HighlighterTest extends SolrTestCaseJ4 { for( String v : multivalued ){ TokenStream ts1 = tots.getMultiValuedTokenStream( v.length() ); Analyzer a2 = new WhitespaceAnalyzer(TEST_VERSION_CURRENT); - TokenStream ts2 = a2.tokenStream("", new StringReader(v)); + TokenStream ts2 = a2.tokenStream("", v); ts2.reset(); while (ts1.incrementToken()) { diff --git a/solr/core/src/test/org/apache/solr/spelling/SimpleQueryConverter.java b/solr/core/src/test/org/apache/solr/spelling/SimpleQueryConverter.java index 63ede26e004..a9ad3b9b627 100644 --- a/solr/core/src/test/org/apache/solr/spelling/SimpleQueryConverter.java +++ b/solr/core/src/test/org/apache/solr/spelling/SimpleQueryConverter.java @@ -29,7 +29,6 @@ import org.apache.lucene.util.Version; import java.util.Collection; import java.util.HashSet; -import java.io.StringReader; import java.io.IOException; @@ -44,7 +43,7 @@ class SimpleQueryConverter extends SpellingQueryConverter { try { Collection result = new HashSet(); WhitespaceAnalyzer analyzer = new WhitespaceAnalyzer(Version.LUCENE_40); - TokenStream ts = analyzer.tokenStream("", new StringReader(origQuery)); + TokenStream ts = analyzer.tokenStream("", origQuery); // TODO: support custom attributes CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class); OffsetAttribute offsetAtt = ts.addAttribute(OffsetAttribute.class);