LUCENE-6958: Improve CustomAnalyzer to take class references to factories as alternative to their SPI name. This enables compile-time safety when defining analyzer's components

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1723027 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Uwe Schindler 2016-01-05 09:44:27 +00:00
parent 214f48c9bd
commit d3d524428b
4 changed files with 166 additions and 11 deletions

View File

@ -124,6 +124,10 @@ New Features
BlendedInfixSuggester, to even more strongly favor suggestions that
match closer to the beginning (Arcadius Ahouansou via Mike McCandless)
* LUCENE-6958: Improved CustomAnalyzer to take class references to factories
as alternative to their SPI name. This enables compile-time safety when
defining analyzer's components. (Uwe Schindler, Shai Erera)
API Changes
* LUCENE-6908: GeoUtils static relational methods have been refactored to new

View File

@ -17,6 +17,8 @@ package org.apache.lucene.analysis.custom;
* limitations under the License.
*/
import static org.apache.lucene.analysis.util.AnalysisSPILoader.newFactoryClassInstance;
import java.io.IOException;
import java.io.Reader;
import java.nio.file.Path;
@ -49,15 +51,24 @@ import org.apache.lucene.util.Version;
* <p>You can create an instance of this Analyzer using the builder:
* <pre class="prettyprint">
* Analyzer ana = CustomAnalyzer.builder(Paths.get(&quot;/path/to/config/dir&quot;))
* .withTokenizer(StandardTokenizerFactory.class)
* .addTokenFilter(StandardFilterFactory.class)
* .addTokenFilter(LowerCaseFilterFactory.class)
* .addTokenFilter(StopFilterFactory.class, &quot;ignoreCase&quot;, &quot;false&quot;, &quot;words&quot;, &quot;stopwords.txt&quot;, &quot;format&quot;, &quot;wordset&quot;)
* .build();
* </pre>
* The parameters passed to components are also used by Apache Solr and are documented
* on their corresponding factory classes. Refer to documentation of subclasses
* of {@link TokenizerFactory}, {@link TokenFilterFactory}, and {@link CharFilterFactory}.
* <p>You can also use the SPI names (as defined by {@link java.util.ServiceLoader} interface):
* <pre class="prettyprint">
* Analyzer ana = CustomAnalyzer.builder(Paths.get(&quot;/path/to/config/dir&quot;))
* .withTokenizer(&quot;standard&quot;)
* .addTokenFilter(&quot;standard&quot;)
* .addTokenFilter(&quot;lowercase&quot;)
* .addTokenFilter(&quot;stop&quot;, &quot;ignoreCase&quot;, &quot;false&quot;, &quot;words&quot;, &quot;stopwords.txt&quot;, &quot;format&quot;, &quot;wordset&quot;)
* .build();
* </pre>
* The parameters passed to components are also used by Apache Solr and are documented
* on their corresponding factory classes. Refer to documentation of subclasses
* of {@link TokenizerFactory}, {@link TokenFilterFactory}, and {@link CharFilterFactory}.
* <p>The list of names to be used for components can be looked up through:
* {@link TokenizerFactory#availableTokenizers()}, {@link TokenFilterFactory#availableTokenFilters()},
* and {@link CharFilterFactory#availableCharFilters()}.
@ -213,6 +224,26 @@ public final class CustomAnalyzer extends Analyzer {
return this;
}
/** Uses the given tokenizer.
* @param factory class that is used to create the tokenizer.
* @param params a list of factory string params as key/value pairs.
* The number of parameters must be an even number, as they are pairs.
*/
public Builder withTokenizer(Class<? extends TokenizerFactory> factory, String... params) throws IOException {
return withTokenizer(factory, paramsToMap(params));
}
/** Uses the given tokenizer.
* @param factory class that is used to create the tokenizer.
* @param params the map of parameters to be passed to factory. The map must be modifiable.
*/
public Builder withTokenizer(Class<? extends TokenizerFactory> factory, Map<String,String> params) throws IOException {
Objects.requireNonNull(factory, "Tokenizer factory may not be null");
tokenizer.set(applyResourceLoader(newFactoryClassInstance(factory, applyDefaultParams(params))));
componentsAdded = true;
return this;
}
/** Uses the given tokenizer.
* @param name is used to look up the factory with {@link TokenizerFactory#forName(String, Map)}.
* The list of possible names can be looked up with {@link TokenizerFactory#availableTokenizers()}.
@ -235,6 +266,26 @@ public final class CustomAnalyzer extends Analyzer {
return this;
}
/** Adds the given token filter.
* @param factory class that is used to create the token filter.
* @param params a list of factory string params as key/value pairs.
* The number of parameters must be an even number, as they are pairs.
*/
public Builder addTokenFilter(Class<? extends TokenFilterFactory> factory, String... params) throws IOException {
return addTokenFilter(factory, paramsToMap(params));
}
/** Adds the given token filter.
* @param factory class that is used to create the token filter.
* @param params the map of parameters to be passed to factory. The map must be modifiable.
*/
public Builder addTokenFilter(Class<? extends TokenFilterFactory> factory, Map<String,String> params) throws IOException {
Objects.requireNonNull(factory, "TokenFilter name may not be null");
tokenFilters.add(applyResourceLoader(newFactoryClassInstance(factory, applyDefaultParams(params))));
componentsAdded = true;
return this;
}
/** Adds the given token filter.
* @param name is used to look up the factory with {@link TokenFilterFactory#forName(String, Map)}.
* The list of possible names can be looked up with {@link TokenFilterFactory#availableTokenFilters()}.
@ -257,6 +308,26 @@ public final class CustomAnalyzer extends Analyzer {
return this;
}
/** Adds the given char filter.
* @param factory class that is used to create the char filter.
* @param params a list of factory string params as key/value pairs.
* The number of parameters must be an even number, as they are pairs.
*/
public Builder addCharFilter(Class<? extends CharFilterFactory> factory, String... params) throws IOException {
return addCharFilter(factory, paramsToMap(params));
}
/** Adds the given char filter.
* @param factory class that is used to create the char filter.
* @param params the map of parameters to be passed to factory. The map must be modifiable.
*/
public Builder addCharFilter(Class<? extends CharFilterFactory> factory, Map<String,String> params) throws IOException {
Objects.requireNonNull(factory, "CharFilter name may not be null");
charFilters.add(applyResourceLoader(newFactoryClassInstance(factory, applyDefaultParams(params))));
componentsAdded = true;
return this;
}
/** Adds the given char filter.
* @param name is used to look up the factory with {@link CharFilterFactory#forName(String, Map)}.
* The list of possible names can be looked up with {@link CharFilterFactory#availableCharFilters()}.

View File

@ -33,7 +33,7 @@ import org.apache.lucene.util.SPIClassIterator;
* Helper class for loading named SPIs from classpath (e.g. Tokenizers, TokenStreams).
* @lucene.internal
*/
final class AnalysisSPILoader<S extends AbstractAnalysisFactory> {
public final class AnalysisSPILoader<S extends AbstractAnalysisFactory> {
private volatile Map<String,Class<? extends S>> services = Collections.emptyMap();
private final Class<S> clazz;
@ -130,7 +130,8 @@ final class AnalysisSPILoader<S extends AbstractAnalysisFactory> {
return services.keySet();
}
private static <T extends AbstractAnalysisFactory> T newFactoryClassInstance(Class<T> clazz, Map<String,String> args) {
/** Creates a new instance of the given {@link AbstractAnalysisFactory} by invoking the constructor, passing the given argument map. */
public static <T extends AbstractAnalysisFactory> T newFactoryClassInstance(Class<T> clazz, Map<String,String> args) {
try {
return clazz.getConstructor(Map.class).newInstance(args);
} catch (InvocationTargetException ite) {

View File

@ -30,8 +30,10 @@ import org.apache.lucene.analysis.core.StopFilterFactory;
import org.apache.lucene.analysis.core.WhitespaceTokenizerFactory;
import org.apache.lucene.analysis.miscellaneous.ASCIIFoldingFilterFactory;
import org.apache.lucene.analysis.standard.ClassicTokenizerFactory;
import org.apache.lucene.analysis.standard.StandardTokenizerFactory;
import org.apache.lucene.analysis.util.CharFilterFactory;
import org.apache.lucene.analysis.util.TokenFilterFactory;
import org.apache.lucene.analysis.util.TokenizerFactory;
import org.apache.lucene.util.SetOnce.AlreadySetException;
import org.apache.lucene.util.Version;
@ -39,6 +41,32 @@ public class TestCustomAnalyzer extends BaseTokenStreamTestCase {
// Test some examples (TODO: we only check behavior, we may need something like TestRandomChains...)
public void testWhitespaceFactoryWithFolding() throws Exception {
CustomAnalyzer a = CustomAnalyzer.builder()
.withTokenizer(WhitespaceTokenizerFactory.class)
.addTokenFilter(ASCIIFoldingFilterFactory.class, "preserveOriginal", "true")
.addTokenFilter(LowerCaseFilterFactory.class)
.build();
assertSame(WhitespaceTokenizerFactory.class, a.getTokenizerFactory().getClass());
assertEquals(Collections.emptyList(), a.getCharFilterFactories());
List<TokenFilterFactory> tokenFilters = a.getTokenFilterFactories();
assertEquals(2, tokenFilters.size());
assertSame(ASCIIFoldingFilterFactory.class, tokenFilters.get(0).getClass());
assertSame(LowerCaseFilterFactory.class, tokenFilters.get(1).getClass());
assertEquals(0, a.getPositionIncrementGap("dummy"));
assertEquals(1, a.getOffsetGap("dummy"));
assertSame(Version.LATEST, a.getVersion());
assertAnalyzesTo(a, "foo bar FOO BAR",
new String[] { "foo", "bar", "foo", "bar" },
new int[] { 1, 1, 1, 1});
assertAnalyzesTo(a, "föó bär FÖÖ BAR",
new String[] { "foo", "föó", "bar", "bär", "foo", "föö", "bar" },
new int[] { 1, 0, 1, 0, 1, 0, 1});
a.close();
}
public void testWhitespaceWithFolding() throws Exception {
CustomAnalyzer a = CustomAnalyzer.builder()
.withTokenizer("whitespace")
@ -65,6 +93,38 @@ public class TestCustomAnalyzer extends BaseTokenStreamTestCase {
a.close();
}
public void testFactoryHtmlStripClassicFolding() throws Exception {
CustomAnalyzer a = CustomAnalyzer.builder()
.withDefaultMatchVersion(Version.LUCENE_5_0_0)
.addCharFilter(HTMLStripCharFilterFactory.class)
.withTokenizer(ClassicTokenizerFactory.class)
.addTokenFilter(ASCIIFoldingFilterFactory.class, "preserveOriginal", "true")
.addTokenFilter(LowerCaseFilterFactory.class)
.withPositionIncrementGap(100)
.withOffsetGap(1000)
.build();
assertSame(ClassicTokenizerFactory.class, a.getTokenizerFactory().getClass());
List<CharFilterFactory> charFilters = a.getCharFilterFactories();
assertEquals(1, charFilters.size());
assertEquals(HTMLStripCharFilterFactory.class, charFilters.get(0).getClass());
List<TokenFilterFactory> tokenFilters = a.getTokenFilterFactories();
assertEquals(2, tokenFilters.size());
assertSame(ASCIIFoldingFilterFactory.class, tokenFilters.get(0).getClass());
assertSame(LowerCaseFilterFactory.class, tokenFilters.get(1).getClass());
assertEquals(100, a.getPositionIncrementGap("dummy"));
assertEquals(1000, a.getOffsetGap("dummy"));
assertSame(Version.LUCENE_5_0_0, a.getVersion());
assertAnalyzesTo(a, "<p>foo bar</p> FOO BAR",
new String[] { "foo", "bar", "foo", "bar" },
new int[] { 1, 1, 1, 1});
assertAnalyzesTo(a, "<p><b>föó</b> bär FÖÖ BAR</p>",
new String[] { "foo", "föó", "bar", "bär", "foo", "föö", "bar" },
new int[] { 1, 0, 1, 0, 1, 0, 1});
a.close();
}
public void testHtmlStripClassicFolding() throws Exception {
CustomAnalyzer a = CustomAnalyzer.builder()
.withDefaultMatchVersion(Version.LUCENE_5_0_0)
@ -99,7 +159,7 @@ public class TestCustomAnalyzer extends BaseTokenStreamTestCase {
public void testStopWordsFromClasspath() throws Exception {
CustomAnalyzer a = CustomAnalyzer.builder()
.withTokenizer("whitespace")
.withTokenizer(WhitespaceTokenizerFactory.class)
.addTokenFilter("stop",
"ignoreCase", "true",
"words", "org/apache/lucene/analysis/custom/teststop.txt",
@ -125,7 +185,8 @@ public class TestCustomAnalyzer extends BaseTokenStreamTestCase {
stopConfig1.put("words", "org/apache/lucene/analysis/custom/teststop.txt");
stopConfig1.put("format", "wordset");
Map<String,String> stopConfig2 = Collections.unmodifiableMap(new HashMap<>(stopConfig1));
Map<String,String> stopConfig2 = new HashMap<>(stopConfig1);
Map<String,String> stopConfigImmutable = Collections.unmodifiableMap(new HashMap<>(stopConfig1));
CustomAnalyzer a = CustomAnalyzer.builder()
.withTokenizer("whitespace")
@ -134,14 +195,21 @@ public class TestCustomAnalyzer extends BaseTokenStreamTestCase {
assertTrue(stopConfig1.isEmpty());
assertAnalyzesTo(a, "foo Foo Bar", new String[0]);
a = CustomAnalyzer.builder()
.withTokenizer(WhitespaceTokenizerFactory.class)
.addTokenFilter(StopFilterFactory.class, stopConfig2)
.build();
assertTrue(stopConfig2.isEmpty());
assertAnalyzesTo(a, "foo Foo Bar", new String[0]);
// try with unmodifiableMap, should fail
try {
CustomAnalyzer.builder()
.withTokenizer("whitespace")
.addTokenFilter("stop", stopConfig2)
.addTokenFilter("stop", stopConfigImmutable)
.build();
fail();
} catch (IllegalArgumentException | UnsupportedOperationException e) {
} catch (UnsupportedOperationException e) {
// pass
}
a.close();
@ -202,7 +270,7 @@ public class TestCustomAnalyzer extends BaseTokenStreamTestCase {
try {
CustomAnalyzer.builder()
.withTokenizer("whitespace")
.withTokenizer("standard")
.withTokenizer(StandardTokenizerFactory.class)
.build();
fail();
} catch (AlreadySetException e) {
@ -261,7 +329,18 @@ public class TestCustomAnalyzer extends BaseTokenStreamTestCase {
public void testNullTokenizer() throws Exception {
try {
CustomAnalyzer.builder()
.withTokenizer(null)
.withTokenizer((String) null)
.build();
fail();
} catch (NullPointerException e) {
// pass
}
}
public void testNullTokenizerFactory() throws Exception {
try {
CustomAnalyzer.builder()
.withTokenizer((Class<TokenizerFactory>) null)
.build();
fail();
} catch (NullPointerException e) {