LUCENE-4723: Add AnalyzerFactoryTask to benchmark, and enable analyzer creation via the resulting factories using NewAnalyzerTask.

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1439510 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Steven Rowe 2013-01-28 17:18:48 +00:00
parent ce7be4dc65
commit 14ea836c0b
12 changed files with 843 additions and 205 deletions

View File

@ -79,6 +79,9 @@ New Features
near-real-time reader is opened that contains those changes. near-real-time reader is opened that contains those changes.
(Robert Muir, Mike McCandless) (Robert Muir, Mike McCandless)
* LUCENE-4723: Add AnalyzerFactoryTask to benchmark, and enable analyzer
creation via the resulting factories using NewAnalyzerTask. (Steve Rowe)
API Changes API Changes
* LUCENE-4709: FacetResultNode no longer has a residue field. (Shai Erera) * LUCENE-4709: FacetResultNode no longer has a residue field. (Shai Erera)

View File

@ -19,25 +19,43 @@ doc.body.tokenized=true
docs.dir=reuters-out docs.dir=reuters-out
log.step=1000 log.step=1000
-AnalyzerFactory(name:shingle-bigrams-unigrams,
StandardTokenizer,
ShingleFilter(maxShingleSize:2, outputUnigrams:true))
-AnalyzerFactory(name:shingle-bigrams,
StandardTokenizer,
ShingleFilter(maxShingleSize:2, outputUnigrams:false))
-AnalyzerFactory(name:shingle-4grams-unigrams,
StandardTokenizer,
ShingleFilter(maxShingleSize:4, outputUnigrams:true))
-AnalyzerFactory(name:shingle-4grams,
StandardTokenizer,
ShingleFilter(maxShingleSize:4, outputUnigrams:false))
-AnalyzerFactory(name:standard-tokenizer-only, StandardTokenizer)
{ "Rounds" { "Rounds"
-NewShingleAnalyzer(maxShingleSize:2,outputUnigrams:true) -NewAnalyzer(shingle-bigrams-unigrams)
-ResetInputs -ResetInputs
{ "BigramsAndUnigrams" { ReadTokens > : 10000 } { "BigramsAndUnigrams" { ReadTokens > : 10000 }
-NewShingleAnalyzer(maxShingleSize:2,outputUnigrams:false) -NewAnalyzer(shingle-bigrams)
-ResetInputs -ResetInputs
{ "BigramsOnly" { ReadTokens > : 10000 } { "BigramsOnly" { ReadTokens > : 10000 }
-NewShingleAnalyzer(maxShingleSize:4,outputUnigrams:true) -NewAnalyzer(shingle-4grams-unigrams)
-ResetInputs -ResetInputs
{ "FourgramsAndUnigrams" { ReadTokens > : 10000 } { "FourgramsAndUnigrams" { ReadTokens > : 10000 }
-NewShingleAnalyzer(maxShingleSize:4,outputUnigrams:false) -NewAnalyzer(shingle-4grams)
-ResetInputs -ResetInputs
{ "FourgramsOnly" { ReadTokens > : 10000 } { "FourgramsOnly" { ReadTokens > : 10000 }
-NewAnalyzer(standard.StandardAnalyzer) -NewAnalyzer(standard-tokenizer-only)
-ResetInputs -ResetInputs
{ "UnigramsOnly" { ReadTokens > : 10000 } { "UnigramsOnly" { ReadTokens > : 10000 }

View File

@ -51,7 +51,7 @@ while (<>) {
# Print out platform info # Print out platform info
print "JAVA:\n", `java -version 2>&1`, "\nOS:\n"; print "JAVA:\n", `java -version 2>&1`, "\nOS:\n";
if ($^O =~ /win/i) { if ($^O =~ /(?<!dar)win/i) {
print "$^O\n"; print "$^O\n";
eval { eval {
require Win32; require Win32;

View File

@ -23,6 +23,7 @@ import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.HashMap; import java.util.HashMap;
import java.util.Locale; import java.util.Locale;
import java.util.Map;
import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.benchmark.byTask.feeds.ContentSource; import org.apache.lucene.benchmark.byTask.feeds.ContentSource;
@ -34,6 +35,7 @@ import org.apache.lucene.benchmark.byTask.tasks.NewAnalyzerTask;
import org.apache.lucene.benchmark.byTask.tasks.PerfTask; import org.apache.lucene.benchmark.byTask.tasks.PerfTask;
import org.apache.lucene.benchmark.byTask.tasks.ReadTask; import org.apache.lucene.benchmark.byTask.tasks.ReadTask;
import org.apache.lucene.benchmark.byTask.tasks.SearchTask; import org.apache.lucene.benchmark.byTask.tasks.SearchTask;
import org.apache.lucene.benchmark.byTask.utils.AnalyzerFactory;
import org.apache.lucene.benchmark.byTask.utils.Config; import org.apache.lucene.benchmark.byTask.utils.Config;
import org.apache.lucene.benchmark.byTask.utils.FileUtils; import org.apache.lucene.benchmark.byTask.utils.FileUtils;
import org.apache.lucene.facet.taxonomy.TaxonomyReader; import org.apache.lucene.facet.taxonomy.TaxonomyReader;
@ -55,6 +57,7 @@ import org.apache.lucene.util.IOUtils;
* <li>Directory, Writer, Reader. * <li>Directory, Writer, Reader.
* <li>Taxonomy Directory, Writer, Reader. * <li>Taxonomy Directory, Writer, Reader.
* <li>DocMaker, FacetSource and a few instances of QueryMaker. * <li>DocMaker, FacetSource and a few instances of QueryMaker.
* <li>Named AnalysisFactories.
* <li>Analyzer. * <li>Analyzer.
* <li>Statistics data which updated during the run. * <li>Statistics data which updated during the run.
* </ul> * </ul>
@ -78,6 +81,7 @@ public class PerfRunData implements Closeable {
// directory, analyzer, docMaker - created at startup. // directory, analyzer, docMaker - created at startup.
// reader, writer, searcher - maintained by basic tasks. // reader, writer, searcher - maintained by basic tasks.
private Directory directory; private Directory directory;
private Map<String,AnalyzerFactory> analyzerFactories = new HashMap<String,AnalyzerFactory>();
private Analyzer analyzer; private Analyzer analyzer;
private DocMaker docMaker; private DocMaker docMaker;
private ContentSource contentSource; private ContentSource contentSource;
@ -358,7 +362,7 @@ public class PerfRunData implements Closeable {
} }
/** /**
* @return Returns the anlyzer. * @return Returns the analyzer.
*/ */
public Analyzer getAnalyzer() { public Analyzer getAnalyzer() {
return analyzer; return analyzer;
@ -434,4 +438,7 @@ public class PerfRunData implements Closeable {
return qm; return qm;
} }
public Map<String,AnalyzerFactory> getAnalyzerFactories() {
return analyzerFactories;
}
} }

View File

@ -0,0 +1,459 @@
package org.apache.lucene.benchmark.byTask.tasks;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import org.apache.lucene.analysis.util.AbstractAnalysisFactory;
import org.apache.lucene.analysis.util.CharFilterFactory;
import org.apache.lucene.analysis.util.FilesystemResourceLoader;
import org.apache.lucene.analysis.util.ResourceLoaderAware;
import org.apache.lucene.analysis.util.TokenFilterFactory;
import org.apache.lucene.analysis.util.TokenizerFactory;
import org.apache.lucene.benchmark.byTask.PerfRunData;
import org.apache.lucene.benchmark.byTask.utils.AnalyzerFactory;
import org.apache.lucene.util.Version;
import java.io.File;
import java.io.StreamTokenizer;
import java.io.StringReader;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.regex.Pattern;
/**
* Analyzer factory construction task. The name given to the constructed factory may
* be given to NewAnalyzerTask, which will call AnalyzerFactory.create().
*
* Params are in the form argname:argvalue or argname:"argvalue" or argname:'argvalue';
* use backslashes to escape '"' or "'" inside a quoted value when it's used as the enclosing
* quotation mark,
*
* Specify params in a comma separated list of the following, in order:
* <ol>
* <li>Analyzer args:
* <ul>
* <li><b>Required</b>: <code>name:<i>analyzer-factory-name</i></code></li>
* <li>Optional: <tt>positionIncrementGap:<i>int value</i></tt> (default: 0)</li>
* <li>Optional: <tt>offsetGap:<i>int value</i></tt> (default: 1)</li>
* </ul>
* </li>
* <li>zero or more CharFilterFactory's, followed by</li>
* <li>exactly one TokenizerFactory, followed by</li>
* <li>zero or more TokenFilterFactory's</li>
* </ol>
*
* Each component analysis factory map specify <tt>luceneMatchVersion</tt> (defaults to
* {@link Version#LUCENE_CURRENT}) and any of the args understood by the specified
* *Factory class, in the above-describe param format.
* <p/>
* Example:
* <pre>
* -AnalyzerFactory(name:'strip html, fold to ascii, whitespace tokenize, max 10k tokens',
* positionIncrementGap:100,
* HTMLStripCharFilter,
* MappingCharFilter(mapping:'mapping-FoldToASCII.txt'),
* WhitespaceTokenizer(luceneMatchVersion:LUCENE_42),
* TokenLimitFilter(maxTokenCount:10000, consumeAllTokens:false))
* [...]
* -NewAnalyzer('strip html, fold to ascii, whitespace tokenize, max 10k tokens')
* </pre>
* <p/>
* AnalyzerFactory will direct analysis component factories to look for resources
* under the directory specified in the "work.dir" property.
*/
public class AnalyzerFactoryTask extends PerfTask {
private static final String LUCENE_ANALYSIS_PACKAGE_PREFIX = "org.apache.lucene.analysis.";
private static final Pattern ANALYSIS_COMPONENT_SUFFIX_PATTERN
= Pattern.compile("(?s:(?:(?:Token|Char)?Filter|Tokenizer)(?:Factory)?)$");
private static final Pattern TRAILING_DOT_ZERO_PATTERN = Pattern.compile("\\.0$");
private enum ArgType {ANALYZER_ARG, ANALYZER_ARG_OR_CHARFILTER_OR_TOKENIZER, TOKENFILTER }
String factoryName = null;
Integer positionIncrementGap = null;
Integer offsetGap = null;
private List<CharFilterFactory> charFilterFactories = new ArrayList<CharFilterFactory>();
private TokenizerFactory tokenizerFactory = null;
private List<TokenFilterFactory> tokenFilterFactories = new ArrayList<TokenFilterFactory>();
public AnalyzerFactoryTask(PerfRunData runData) {
super(runData);
}
@Override
public int doLogic() {
return 1;
}
/**
* Sets the params.
* Analysis component factory names may optionally include the "Factory" suffix.
*
* @param params analysis pipeline specification: name, (optional) positionIncrementGap,
* (optional) offsetGap, 0+ CharFilterFactory's, 1 TokenizerFactory,
* and 0+ TokenFilterFactory's
*/
@Override
public void setParams(String params) {
super.setParams(params);
ArgType expectedArgType = ArgType.ANALYZER_ARG;
final StreamTokenizer stok = new StreamTokenizer(new StringReader(params));
stok.commentChar('#');
stok.quoteChar('"');
stok.quoteChar('\'');
stok.eolIsSignificant(false);
stok.ordinaryChar('(');
stok.ordinaryChar(')');
stok.ordinaryChar(':');
stok.ordinaryChar(',');
try {
while (stok.nextToken() != StreamTokenizer.TT_EOF) {
switch (stok.ttype) {
case ',': {
// Do nothing
break;
}
case StreamTokenizer.TT_WORD: {
if (expectedArgType.equals(ArgType.ANALYZER_ARG)) {
final String argName = stok.sval;
if ( ! argName.equalsIgnoreCase("name")
&& ! argName.equalsIgnoreCase("positionIncrementGap")
&& ! argName.equalsIgnoreCase("offsetGap")) {
throw new RuntimeException
("Line #" + lineno(stok) + ": Missing 'name' param to AnalyzerFactory: '" + params + "'");
}
stok.nextToken();
if (stok.ttype != ':') {
throw new RuntimeException
("Line #" + lineno(stok) + ": Missing ':' after '" + argName + "' param to AnalyzerFactory");
}
stok.nextToken();
String argValue = stok.sval;
switch (stok.ttype) {
case StreamTokenizer.TT_NUMBER: {
argValue = Double.toString(stok.nval);
// Drop the ".0" from numbers, for integer arguments
argValue = TRAILING_DOT_ZERO_PATTERN.matcher(argValue).replaceFirst("");
// Intentional fallthrough
}
case '"':
case '\'':
case StreamTokenizer.TT_WORD: {
if (argName.equalsIgnoreCase("name")) {
factoryName = argValue;
expectedArgType = ArgType.ANALYZER_ARG_OR_CHARFILTER_OR_TOKENIZER;
} else {
int intArgValue = 0;
try {
intArgValue = Integer.parseInt(argValue);
} catch (NumberFormatException e) {
throw new RuntimeException
("Line #" + lineno(stok) + ": Exception parsing " + argName + " value '" + argValue + "'", e);
}
if (argName.equalsIgnoreCase("positionIncrementGap")) {
positionIncrementGap = intArgValue;
} else if (argName.equalsIgnoreCase("offsetGap")) {
offsetGap = intArgValue;
}
}
break;
}
case StreamTokenizer.TT_EOF: {
throw new RuntimeException("Unexpected EOF: " + stok.toString());
}
default: {
throw new RuntimeException
("Line #" + lineno(stok) + ": Unexpected token: " + stok.toString());
}
}
} else if (expectedArgType.equals(ArgType.ANALYZER_ARG_OR_CHARFILTER_OR_TOKENIZER)) {
final String argName = stok.sval;
if (argName.equalsIgnoreCase("positionIncrementGap")
|| argName.equalsIgnoreCase("offsetGap")) {
stok.nextToken();
if (stok.ttype != ':') {
throw new RuntimeException
("Line #" + lineno(stok) + ": Missing ':' after '" + argName + "' param to AnalyzerFactory");
}
stok.nextToken();
int intArgValue = (int)stok.nval;
switch (stok.ttype) {
case '"':
case '\'':
case StreamTokenizer.TT_WORD: {
intArgValue = 0;
try {
intArgValue = Integer.parseInt(stok.sval.trim());
} catch (NumberFormatException e) {
throw new RuntimeException
("Line #" + lineno(stok) + ": Exception parsing " + argName + " value '" + stok.sval + "'", e);
}
// Intentional fall-through
}
case StreamTokenizer.TT_NUMBER: {
if (argName.equalsIgnoreCase("positionIncrementGap")) {
positionIncrementGap = intArgValue;
} else if (argName.equalsIgnoreCase("offsetGap")) {
offsetGap = intArgValue;
}
break;
}
case StreamTokenizer.TT_EOF: {
throw new RuntimeException("Unexpected EOF: " + stok.toString());
}
default: {
throw new RuntimeException
("Line #" + lineno(stok) + ": Unexpected token: " + stok.toString());
}
}
break;
}
try {
final Class<? extends CharFilterFactory> clazz;
clazz = lookupAnalysisClass(argName, CharFilterFactory.class);
createAnalysisPipelineComponent(stok, clazz);
} catch (IllegalArgumentException e) {
try {
final Class<? extends TokenizerFactory> clazz;
clazz = lookupAnalysisClass(argName, TokenizerFactory.class);
createAnalysisPipelineComponent(stok, clazz);
expectedArgType = ArgType.TOKENFILTER;
} catch (IllegalArgumentException e2) {
throw new RuntimeException("Line #" + lineno(stok) + ": Can't find class '"
+ argName + "' as CharFilterFactory or TokenizerFactory");
}
}
} else { // expectedArgType = ArgType.TOKENFILTER
final String className = stok.sval;
final Class<? extends TokenFilterFactory> clazz;
try {
clazz = lookupAnalysisClass(className, TokenFilterFactory.class);
} catch (IllegalArgumentException e) {
throw new RuntimeException
("Line #" + lineno(stok) + ": Can't find class '" + className + "' as TokenFilterFactory");
}
createAnalysisPipelineComponent(stok, clazz);
}
break;
}
default: {
throw new RuntimeException("Line #" + lineno(stok) + ": Unexpected token: " + stok.toString());
}
}
}
} catch (RuntimeException e) {
if (e.getMessage().startsWith("Line #")) {
throw e;
} else {
throw new RuntimeException("Line #" + lineno(stok) + ": ", e);
}
} catch (Throwable t) {
throw new RuntimeException("Line #" + lineno(stok) + ": ", t);
}
final AnalyzerFactory analyzerFactory = new AnalyzerFactory
(charFilterFactories, tokenizerFactory, tokenFilterFactories);
analyzerFactory.setPositionIncrementGap(positionIncrementGap);
analyzerFactory.setOffsetGap(offsetGap);
getRunData().getAnalyzerFactories().put(factoryName, analyzerFactory);
}
/**
* Instantiates the given analysis factory class after pulling params from
* the given stream tokenizer, then stores the result in the appropriate
* pipeline component list.
*
* @param stok stream tokenizer from which to draw analysis factory params
* @param clazz analysis factory class to instantiate
*/
private void createAnalysisPipelineComponent
(StreamTokenizer stok, Class<? extends AbstractAnalysisFactory> clazz) {
final AbstractAnalysisFactory instance;
try {
instance = clazz.newInstance();
} catch (Exception e) {
throw new RuntimeException("Line #" + lineno(stok) + ": ", e);
}
Version luceneMatchVersion = null;
Map<String,String> argMap = new HashMap<String,String>();
boolean parenthetical = false;
try {
WHILE_LOOP: while (stok.nextToken() != StreamTokenizer.TT_EOF) {
switch (stok.ttype) {
case ',': {
if (parenthetical) {
// Do nothing
break;
} else {
// Finished reading this analysis factory configuration
break WHILE_LOOP;
}
}
case '(': {
if (parenthetical) {
throw new RuntimeException
("Line #" + lineno(stok) + ": Unexpected opening parenthesis.");
}
parenthetical = true;
break;
}
case ')': {
if (parenthetical) {
parenthetical = false;
} else {
throw new RuntimeException
("Line #" + lineno(stok) + ": Unexpected closing parenthesis.");
}
break;
}
case StreamTokenizer.TT_WORD: {
if ( ! parenthetical) {
throw new RuntimeException("Line #" + lineno(stok) + ": Unexpected token '" + stok.sval + "'");
}
String argName = stok.sval;
stok.nextToken();
if (stok.ttype != ':') {
throw new RuntimeException
("Line #" + lineno(stok) + ": Missing ':' after '" + argName + "' param to " + clazz.getSimpleName());
}
stok.nextToken();
String argValue = stok.sval;
switch (stok.ttype) {
case StreamTokenizer.TT_NUMBER: {
argValue = Double.toString(stok.nval);
// Drop the ".0" from numbers, for integer arguments
argValue = TRAILING_DOT_ZERO_PATTERN.matcher(argValue).replaceFirst("");
// Intentional fall-through
}
case '"':
case '\'':
case StreamTokenizer.TT_WORD: {
if (argName.equalsIgnoreCase("luceneMatchVersion")) {
try {
luceneMatchVersion = Version.parseLeniently(argValue);
} catch (IllegalArgumentException e) {
throw new RuntimeException
("Line #" + lineno(stok) + ": Unrecognized luceneMatchVersion '" + argValue + "'", e);
}
} else {
argMap.put(argName, argValue);
}
break;
}
case StreamTokenizer.TT_EOF: {
throw new RuntimeException("Unexpected EOF: " + stok.toString());
}
default: {
throw new RuntimeException
("Line #" + lineno(stok) + ": Unexpected token: " + stok.toString());
}
}
}
}
}
instance.setLuceneMatchVersion
(null == luceneMatchVersion ? Version.LUCENE_CURRENT : luceneMatchVersion);
instance.init(argMap);
if (instance instanceof ResourceLoaderAware) {
File baseDir = new File(getRunData().getConfig().get("work.dir", "work")).getAbsoluteFile();
((ResourceLoaderAware)instance).inform(new FilesystemResourceLoader(baseDir));
}
if (CharFilterFactory.class.isAssignableFrom(clazz)) {
charFilterFactories.add((CharFilterFactory)instance);
} else if (TokenizerFactory.class.isAssignableFrom(clazz)) {
tokenizerFactory = (TokenizerFactory)instance;
} else if (TokenFilterFactory.class.isAssignableFrom(clazz)) {
tokenFilterFactories.add((TokenFilterFactory)instance);
}
} catch (RuntimeException e) {
if (e.getMessage().startsWith("Line #")) {
throw (e);
} else {
throw new RuntimeException("Line #" + lineno(stok) + ": ", e);
}
} catch (Throwable t) {
throw new RuntimeException("Line #" + lineno(stok) + ": ", t);
}
}
/**
* This method looks up a class with its fully qualified name (FQN), or a short-name
* class-simplename, or with a package suffix, assuming "org.apache.lucene.analysis."
* as the package prefix (e.g. "standard.ClassicTokenizerFactory" ->
* "org.apache.lucene.analysis.standard.ClassicTokenizerFactory").
*
* If className contains a period, the class is first looked up as-is, assuming that it
* is an FQN. If this fails, lookup is retried after prepending the Lucene analysis
* package prefix to the class name.
*
* If className does not contain a period, the analysis SPI *Factory.lookupClass()
* methods are used to find the class.
*
* @param className The name or the short name of the class.
* @param expectedType The superclass className is expected to extend
* @return the loaded class.
* @throws ClassNotFoundException if lookup fails
*/
public <T> Class<? extends T> lookupAnalysisClass(String className, Class<T> expectedType)
throws ClassNotFoundException {
if (className.contains(".")) {
try {
// First, try className == FQN
return Class.forName(className).asSubclass(expectedType);
} catch (ClassNotFoundException e) {
try {
// Second, retry lookup after prepending the Lucene analysis package prefix
return Class.forName(LUCENE_ANALYSIS_PACKAGE_PREFIX + className).asSubclass(expectedType);
} catch (ClassNotFoundException e1) {
throw new ClassNotFoundException("Can't find class '" + className
+ "' or '" + LUCENE_ANALYSIS_PACKAGE_PREFIX + className + "'");
}
}
}
// No dot - use analysis SPI lookup
final String analysisComponentName = ANALYSIS_COMPONENT_SUFFIX_PATTERN.matcher(className).replaceFirst("");
if (CharFilterFactory.class.isAssignableFrom(expectedType)) {
return CharFilterFactory.lookupClass(analysisComponentName).asSubclass(expectedType);
} else if (TokenizerFactory.class.isAssignableFrom(expectedType)) {
return TokenizerFactory.lookupClass(analysisComponentName).asSubclass(expectedType);
} else if (TokenFilterFactory.class.isAssignableFrom(expectedType)) {
return TokenFilterFactory.lookupClass(analysisComponentName).asSubclass(expectedType);
}
throw new ClassNotFoundException("Can't find class '" + className + "'");
}
/* (non-Javadoc)
* @see org.apache.lucene.benchmark.byTask.tasks.PerfTask#supportsParams()
*/
@Override
public boolean supportsParams() {
return true;
}
/** Returns the current line in the algorithm file */
public int lineno(StreamTokenizer stok) {
return getAlgLineNum() + stok.lineno();
}
}

View File

@ -16,10 +16,16 @@ package org.apache.lucene.benchmark.byTask.tasks;
*/ */
import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.Analyzer;
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.benchmark.byTask.PerfRunData; import org.apache.lucene.benchmark.byTask.PerfRunData;
import org.apache.lucene.benchmark.byTask.utils.AnalyzerFactory;
import org.apache.lucene.util.Version; import org.apache.lucene.util.Version;
import java.io.IOException; import java.io.IOException;
import java.io.StreamTokenizer;
import java.io.StringReader;
import java.util.*; import java.util.*;
import java.lang.reflect.Constructor; import java.lang.reflect.Constructor;
@ -28,12 +34,12 @@ import java.lang.reflect.Constructor;
* *
*/ */
public class NewAnalyzerTask extends PerfTask { public class NewAnalyzerTask extends PerfTask {
private List<String> analyzerClassNames; private List<String> analyzerNames;
private int current; private int current;
public NewAnalyzerTask(PerfRunData runData) { public NewAnalyzerTask(PerfRunData runData) {
super(runData); super(runData);
analyzerClassNames = new ArrayList<String>(); analyzerNames = new ArrayList<String>();
} }
public static final Analyzer createAnalyzer(String className) throws Exception{ public static final Analyzer createAnalyzer(String className) throws Exception{
@ -50,55 +56,98 @@ public class NewAnalyzerTask extends PerfTask {
@Override @Override
public int doLogic() throws IOException { public int doLogic() throws IOException {
String className = null; String analyzerName = null;
try { try {
if (current >= analyzerClassNames.size()) { if (current >= analyzerNames.size()) {
current = 0; current = 0;
} }
className = analyzerClassNames.get(current++); analyzerName = analyzerNames.get(current++);
Analyzer analyzer = null; Analyzer analyzer = null;
if (null == className || 0 == className.length()) { if (null == analyzerName || 0 == analyzerName.length()) {
className = "org.apache.lucene.analysis.standard.StandardAnalyzer"; analyzerName = "org.apache.lucene.analysis.standard.StandardAnalyzer";
} }
if (-1 == className.indexOf(".")) { // First, lookup analyzerName as a named analyzer factory
AnalyzerFactory factory = getRunData().getAnalyzerFactories().get(analyzerName);
if (null != factory) {
analyzer = factory.create();
} else {
if (analyzerName.contains(".")) {
if (analyzerName.startsWith("standard.")) {
analyzerName = "org.apache.lucene.analysis." + analyzerName;
}
analyzer = createAnalyzer(analyzerName);
} else { // No package
try { try {
// If no package, first attempt to instantiate a core analyzer // Attempt to instantiate a core analyzer
String coreClassName = "org.apache.lucene.analysis.core." + className; String coreClassName = "org.apache.lucene.analysis.core." + analyzerName;
analyzer = createAnalyzer(coreClassName); analyzer = createAnalyzer(coreClassName);
className = coreClassName; analyzerName = coreClassName;
} catch (ClassNotFoundException e) { } catch (ClassNotFoundException e) {
// If not a core analyzer, try the base analysis package // If not a core analyzer, try the base analysis package
className = "org.apache.lucene.analysis." + className; analyzerName = "org.apache.lucene.analysis." + analyzerName;
analyzer = createAnalyzer(className); analyzer = createAnalyzer(analyzerName);
} }
} else {
if (className.startsWith("standard.")) {
className = "org.apache.lucene.analysis." + className;
} }
analyzer = createAnalyzer(className);
} }
getRunData().setAnalyzer(analyzer); getRunData().setAnalyzer(analyzer);
System.out.println("Changed Analyzer to: " + className);
} catch (Exception e) { } catch (Exception e) {
throw new RuntimeException("Error creating Analyzer: " + className, e); throw new RuntimeException("Error creating Analyzer: " + analyzerName, e);
} }
return 1; return 1;
} }
/** /**
* Set the params (analyzerClassName only), Comma-separate list of Analyzer class names. If the Analyzer lives in * Set the params (analyzerName only), Comma-separate list of Analyzer class names. If the Analyzer lives in
* org.apache.lucene.analysis, the name can be shortened by dropping the o.a.l.a part of the Fully Qualified Class Name. * org.apache.lucene.analysis, the name can be shortened by dropping the o.a.l.a part of the Fully Qualified Class Name.
* <p/> * <p/>
* Analyzer names may also refer to previously defined AnalyzerFactory's.
* <p/>
* Example Declaration: {"NewAnalyzer" NewAnalyzer(WhitespaceAnalyzer, SimpleAnalyzer, StopAnalyzer, standard.StandardAnalyzer) > * Example Declaration: {"NewAnalyzer" NewAnalyzer(WhitespaceAnalyzer, SimpleAnalyzer, StopAnalyzer, standard.StandardAnalyzer) >
* <p/>
* Example AnalyzerFactory usage:
* <pre>
* -AnalyzerFactory(name:'whitespace tokenized',WhitespaceTokenizer)
* -NewAnalyzer('whitespace tokenized')
* </pre>
* @param params analyzerClassName, or empty for the StandardAnalyzer * @param params analyzerClassName, or empty for the StandardAnalyzer
*/ */
@Override @Override
public void setParams(String params) { public void setParams(String params) {
super.setParams(params); super.setParams(params);
for (StringTokenizer tokenizer = new StringTokenizer(params, ","); tokenizer.hasMoreTokens();) { final StreamTokenizer stok = new StreamTokenizer(new StringReader(params));
String s = tokenizer.nextToken(); stok.quoteChar('"');
analyzerClassNames.add(s.trim()); stok.quoteChar('\'');
stok.eolIsSignificant(false);
stok.ordinaryChar(',');
try {
while (stok.nextToken() != StreamTokenizer.TT_EOF) {
switch (stok.ttype) {
case ',': {
// Do nothing
break;
} }
case '\'':
case '\"':
case StreamTokenizer.TT_WORD: {
analyzerNames.add(stok.sval);
break;
}
default: {
throw new RuntimeException("Unexpected token: " + stok.toString());
}
}
}
} catch (RuntimeException e) {
if (e.getMessage().startsWith("Line #")) {
throw e;
} else {
throw new RuntimeException("Line #" + (stok.lineno() + getAlgLineNum()) + ": ", e);
}
} catch (Throwable t) {
throw new RuntimeException("Line #" + (stok.lineno() + getAlgLineNum()) + ": ", t);
}
} }
/* (non-Javadoc) /* (non-Javadoc)

View File

@ -1,117 +0,0 @@
package org.apache.lucene.benchmark.byTask.tasks;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.util.StringTokenizer;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.shingle.ShingleAnalyzerWrapper;
import org.apache.lucene.analysis.shingle.ShingleFilter;
import org.apache.lucene.benchmark.byTask.PerfRunData;
/**
* Task to support benchmarking ShingleFilter / ShingleAnalyzerWrapper
* <p>
* <ul>
* <li> <code>NewShingleAnalyzer</code> (constructs with all defaults)
* <li> <code>NewShingleAnalyzer(analyzer:o.a.l.analysis.StandardAnalyzer,maxShingleSize:2,outputUnigrams:true)</code>
* </ul>
* </p>
*/
public class NewShingleAnalyzerTask extends PerfTask {
private String analyzerClassName = "standard.StandardAnalyzer";
private int maxShingleSize = 2;
private boolean outputUnigrams = true;
public NewShingleAnalyzerTask(PerfRunData runData) {
super(runData);
}
private void setAnalyzer() throws Exception {
Analyzer wrappedAnalyzer = null;
if (null == analyzerClassName || 0 == analyzerClassName.length()) {
analyzerClassName = "org.apache.lucene.analysis.standard.StandardAnalyzer";
}
if (-1 == analyzerClassName.indexOf(".")) {
String coreClassName = "org.apache.lucene.analysis.core." + analyzerClassName;
try {
// If there is no package, first attempt to instantiate a core analyzer
wrappedAnalyzer = NewAnalyzerTask.createAnalyzer(coreClassName);
analyzerClassName = coreClassName;
} catch (ClassNotFoundException e) {
// If this is not a core analyzer, try the base analysis package
analyzerClassName = "org.apache.lucene.analysis." + analyzerClassName;
wrappedAnalyzer = NewAnalyzerTask.createAnalyzer(analyzerClassName);
}
} else {
if (analyzerClassName.startsWith("standard.")) {
analyzerClassName = "org.apache.lucene.analysis." + analyzerClassName;
}
wrappedAnalyzer = NewAnalyzerTask.createAnalyzer(analyzerClassName);
}
ShingleAnalyzerWrapper analyzer = new ShingleAnalyzerWrapper(
wrappedAnalyzer,
ShingleFilter.DEFAULT_MIN_SHINGLE_SIZE,
maxShingleSize,
ShingleFilter.TOKEN_SEPARATOR,
outputUnigrams,
false);
getRunData().setAnalyzer(analyzer);
}
@Override
public int doLogic() throws Exception {
try {
setAnalyzer();
System.out.println
("Changed Analyzer to: ShingleAnalyzerWrapper, wrapping ShingleFilter over "
+ analyzerClassName);
} catch (Exception e) {
throw new RuntimeException("Error creating Analyzer", e);
}
return 1;
}
@Override
public void setParams(String params) {
super.setParams(params);
StringTokenizer st = new StringTokenizer(params, ",");
while (st.hasMoreTokens()) {
String param = st.nextToken();
StringTokenizer expr = new StringTokenizer(param, ":");
String key = expr.nextToken();
String value = expr.nextToken();
if (key.equalsIgnoreCase("analyzer")) {
analyzerClassName = value;
} else if (key.equalsIgnoreCase("outputUnigrams")) {
outputUnigrams = Boolean.parseBoolean(value);
} else if (key.equalsIgnoreCase("maxShingleSize")) {
maxShingleSize = (int)Double.parseDouble(value);
} else {
throw new RuntimeException("Unknown parameter " + param);
}
}
}
@Override
public boolean supportsParams() {
return true;
}
}

View File

@ -62,6 +62,9 @@ public abstract class PerfTask implements Cloneable {
private boolean runInBackground; private boolean runInBackground;
private int deltaPri; private int deltaPri;
// The first line of this task's definition in the alg file
private int algLineNum = 0;
protected static final String NEW_LINE = System.getProperty("line.separator"); protected static final String NEW_LINE = System.getProperty("line.separator");
/** Should not be used externally */ /** Should not be used externally */
@ -317,4 +320,11 @@ public abstract class PerfTask implements Cloneable {
this.disableCounting = disableCounting; this.disableCounting = disableCounting;
} }
public void setAlgLineNum(int algLineNum) {
this.algLineNum = algLineNum;
}
public int getAlgLineNum() {
return algLineNum;
}
} }

View File

@ -58,7 +58,8 @@ public class Algorithm {
StreamTokenizer stok = new StreamTokenizer(new StringReader(algTxt)); StreamTokenizer stok = new StreamTokenizer(new StringReader(algTxt));
stok.commentChar('#'); stok.commentChar('#');
stok.eolIsSignificant(false); stok.eolIsSignificant(false);
stok.ordinaryChar('"'); stok.quoteChar('"');
stok.quoteChar('\'');
stok.ordinaryChar('/'); stok.ordinaryChar('/');
stok.ordinaryChar('('); stok.ordinaryChar('(');
stok.ordinaryChar(')'); stok.ordinaryChar(')');
@ -74,6 +75,7 @@ public class Algorithm {
Constructor<? extends PerfTask> cnstr = taskClass(config,s) Constructor<? extends PerfTask> cnstr = taskClass(config,s)
.asSubclass(PerfTask.class).getConstructor(PerfRunData.class); .asSubclass(PerfTask.class).getConstructor(PerfRunData.class);
PerfTask task = cnstr.newInstance(runData); PerfTask task = cnstr.newInstance(runData);
task.setAlgLineNum(stok.lineno());
task.setDisableCounting(isDisableCountNextTask); task.setDisableCounting(isDisableCountNextTask);
isDisableCountNextTask = false; isDisableCountNextTask = false;
currSequence.addTask(task); currSequence.addTask(task);
@ -90,24 +92,54 @@ public class Algorithm {
if (stok.ttype!='(') { if (stok.ttype!='(') {
stok.pushBack(); stok.pushBack();
} else { } else {
// get params, for tasks that supports them, - anything until next ')' // get params, for tasks that supports them - allow recursive parenthetical expressions
stok.eolIsSignificant(true); // Allow params tokenizer to keep track of line number
StringBuilder params = new StringBuilder(); StringBuilder params = new StringBuilder();
stok.nextToken(); stok.nextToken();
while (stok.ttype!=')') { if (stok.ttype != ')') {
int count = 1;
BALANCED_PARENS: while (true) {
switch (stok.ttype) { switch (stok.ttype) {
case StreamTokenizer.TT_NUMBER: case StreamTokenizer.TT_NUMBER: {
params.append(stok.nval); params.append(stok.nval);
break; break;
case StreamTokenizer.TT_WORD: }
case StreamTokenizer.TT_WORD: {
params.append(stok.sval); params.append(stok.sval);
break; break;
case StreamTokenizer.TT_EOF: }
throw new Exception("unexpexted EOF: - "+stok.toString()); case StreamTokenizer.TT_EOF: {
default: throw new RuntimeException("Unexpexted EOF: - "+stok.toString());
}
case '"':
case '\'': {
params.append((char)stok.ttype); params.append((char)stok.ttype);
// re-escape delimiters, if any
params.append(stok.sval.replaceAll("" + (char)stok.ttype, "\\\\" + (char)stok.ttype));
params.append((char)stok.ttype);
break;
}
case '(': {
params.append((char)stok.ttype);
++count;
break;
}
case ')': {
if (--count >= 1) { // exclude final closing parenthesis
params.append((char)stok.ttype);
} else {
break BALANCED_PARENS;
}
break;
}
default: {
params.append((char)stok.ttype);
}
} }
stok.nextToken(); stok.nextToken();
} }
}
stok.eolIsSignificant(false);
String prm = params.toString().trim(); String prm = params.toString().trim();
if (prm.length()>0) { if (prm.length()>0) {
task.setParams(prm); task.setParams(prm);
@ -182,9 +214,7 @@ public class Algorithm {
if (stok.ttype!='"') { if (stok.ttype!='"') {
stok.pushBack(); stok.pushBack();
} else { } else {
stok.nextToken();
name = stok.sval; name = stok.sval;
stok.nextToken();
if (stok.ttype!='"' || name==null || name.length()==0) { if (stok.ttype!='"' || name==null || name.length()==0) {
throw new Exception("sequence name problem - "+stok.toString()); throw new Exception("sequence name problem - "+stok.toString());
} }

View File

@ -0,0 +1,132 @@
package org.apache.lucene.benchmark.byTask.utils;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.Tokenizer;
import org.apache.lucene.analysis.util.CharFilterFactory;
import org.apache.lucene.analysis.util.TokenFilterFactory;
import org.apache.lucene.analysis.util.TokenizerFactory;
import java.io.Reader;
import java.util.List;
/**
* A factory to create an analyzer.
* See {@link org.apache.lucene.benchmark.byTask.tasks.AnalyzerFactoryTask}
*/
public final class AnalyzerFactory {
final private List<CharFilterFactory> charFilterFactories;
final private TokenizerFactory tokenizerFactory;
final private List<TokenFilterFactory> tokenFilterFactories;
private String name = null;
private Integer positionIncrementGap = null;
private Integer offsetGap = null;
public AnalyzerFactory(List<CharFilterFactory> charFilterFactories,
TokenizerFactory tokenizerFactory,
List<TokenFilterFactory> tokenFilterFactories) {
this.charFilterFactories = charFilterFactories;
assert null != tokenizerFactory;
this.tokenizerFactory = tokenizerFactory;
this.tokenFilterFactories = tokenFilterFactories;
}
public void setName(String name) {
this.name = name;
}
public void setPositionIncrementGap(Integer positionIncrementGap) {
this.positionIncrementGap = positionIncrementGap;
}
public void setOffsetGap(Integer offsetGap) {
this.offsetGap = offsetGap;
}
public Analyzer create() {
return new Analyzer() {
private final Integer positionIncrementGap = AnalyzerFactory.this.positionIncrementGap;
private final Integer offsetGap = AnalyzerFactory.this.offsetGap;
@Override
public Reader initReader(String fieldName, Reader reader) {
if (charFilterFactories != null && charFilterFactories.size() > 0) {
Reader wrappedReader = reader;
for (CharFilterFactory charFilterFactory : charFilterFactories) {
wrappedReader = charFilterFactory.create(wrappedReader);
}
reader = wrappedReader;
}
return reader;
}
@Override
protected Analyzer.TokenStreamComponents createComponents(String fieldName, Reader reader) {
final Tokenizer tokenizer = tokenizerFactory.create(reader);
TokenStream tokenStream = tokenizer;
for (TokenFilterFactory filterFactory : tokenFilterFactories) {
tokenStream = filterFactory.create(tokenStream);
}
return new TokenStreamComponents(tokenizer, tokenStream);
}
@Override
public int getPositionIncrementGap(String fieldName) {
return null == positionIncrementGap ? super.getPositionIncrementGap(fieldName) : positionIncrementGap;
}
@Override
public int getOffsetGap(String fieldName) {
return null == offsetGap ? super.getOffsetGap(fieldName) : offsetGap;
}
};
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder("AnalyzerFactory(");
if (null != name) {
sb.append("name:");
sb.append(name);
sb.append(", ");
}
if (null != positionIncrementGap) {
sb.append("positionIncrementGap:");
sb.append(positionIncrementGap);
sb.append(", ");
}
if (null != offsetGap) {
sb.append("offsetGap:");
sb.append(offsetGap);
sb.append(", ");
}
for (CharFilterFactory charFilterFactory: charFilterFactories) {
sb.append(charFilterFactory);
sb.append(", ");
}
sb.append(tokenizerFactory);
for (TokenFilterFactory tokenFilterFactory : tokenFilterFactories) {
sb.append(", ");
sb.append(tokenFilterFactory);
}
sb.append(')');
return sb.toString();
}
}

View File

@ -71,6 +71,7 @@ public class TestPerfTasksLogic extends BenchmarkTestCase {
public void setUp() throws Exception { public void setUp() throws Exception {
super.setUp(); super.setUp();
copyToWorkDir("reuters.first20.lines.txt"); copyToWorkDir("reuters.first20.lines.txt");
copyToWorkDir("test-mapping-ISOLatin1Accent-partial.txt");
} }
/** /**
@ -1020,64 +1021,80 @@ public class TestPerfTasksLogic extends BenchmarkTestCase {
} }
/** /**
* Test that we can create ShingleAnalyzerWrappers. * Test that we can create shingle analyzers using AnalyzerFactory.
*/ */
public void testShingleAnalyzer() throws Exception { public void testShingleAnalyzer() throws Exception {
String text = "one,two,three, four five six"; String text = "one,two,three, four five six";
// Default analyzer, maxShingleSize, and outputUnigrams // StandardTokenizer, maxShingleSize, and outputUnigrams
Benchmark benchmark = execBenchmark(getShingleConfig("")); Benchmark benchmark = execBenchmark(getAnalyzerFactoryConfig
("shingle-analyzer", "StandardTokenizer,ShingleFilter"));
benchmark.getRunData().getAnalyzer().tokenStream benchmark.getRunData().getAnalyzer().tokenStream
("bogus", new StringReader(text)).close(); ("bogus", new StringReader(text)).close();
assertEqualShingle(benchmark.getRunData().getAnalyzer(), text, BaseTokenStreamTestCase.assertAnalyzesTo(benchmark.getRunData().getAnalyzer(), text,
new String[] { "one", "one two", "two", "two three", new String[] { "one", "one two", "two", "two three",
"three", "three four", "four", "four five", "three", "three four", "four", "four five",
"five", "five six", "six" }); "five", "five six", "six" });
// Default analyzer, maxShingleSize = 3, and outputUnigrams = false // StandardTokenizer, maxShingleSize = 3, and outputUnigrams = false
benchmark = execBenchmark benchmark = execBenchmark
(getShingleConfig("maxShingleSize:3,outputUnigrams:false")); (getAnalyzerFactoryConfig
assertEqualShingle(benchmark.getRunData().getAnalyzer(), text, ("shingle-analyzer",
"StandardTokenizer,ShingleFilter(maxShingleSize:3,outputUnigrams:false)"));
BaseTokenStreamTestCase.assertAnalyzesTo(benchmark.getRunData().getAnalyzer(), text,
new String[] { "one two", "one two three", "two three", new String[] { "one two", "one two three", "two three",
"two three four", "three four", "two three four", "three four",
"three four five", "four five", "three four five", "four five",
"four five six", "five six" }); "four five six", "five six" });
// WhitespaceAnalyzer, default maxShingleSize and outputUnigrams // WhitespaceTokenizer, default maxShingleSize and outputUnigrams
benchmark = execBenchmark benchmark = execBenchmark
(getShingleConfig("analyzer:WhitespaceAnalyzer")); (getAnalyzerFactoryConfig("shingle-analyzer", "WhitespaceTokenizer,ShingleFilter"));
assertEqualShingle(benchmark.getRunData().getAnalyzer(), text, BaseTokenStreamTestCase.assertAnalyzesTo(benchmark.getRunData().getAnalyzer(), text,
new String[] { "one,two,three,", "one,two,three, four", new String[] { "one,two,three,", "one,two,three, four",
"four", "four five", "five", "five six", "four", "four five", "five", "five six",
"six" }); "six" });
// WhitespaceAnalyzer, maxShingleSize=3 and outputUnigrams=false // WhitespaceTokenizer, maxShingleSize=3 and outputUnigrams=false
benchmark = execBenchmark benchmark = execBenchmark
(getShingleConfig (getAnalyzerFactoryConfig
("outputUnigrams:false,maxShingleSize:3,analyzer:WhitespaceAnalyzer")); ("shingle-factory",
assertEqualShingle(benchmark.getRunData().getAnalyzer(), text, "WhitespaceTokenizer,ShingleFilter(outputUnigrams:false,maxShingleSize:3)"));
BaseTokenStreamTestCase.assertAnalyzesTo(benchmark.getRunData().getAnalyzer(), text,
new String[] { "one,two,three, four", new String[] { "one,two,three, four",
"one,two,three, four five", "one,two,three, four five",
"four five", "four five six", "four five", "four five six",
"five six" }); "five six" });
} }
private void assertEqualShingle private String[] getAnalyzerFactoryConfig(String name, String params) {
(Analyzer analyzer, String text, String[] expected) throws Exception { final String singleQuoteEscapedName = name.replaceAll("'", "\\\\'");
BaseTokenStreamTestCase.assertAnalyzesTo(analyzer, text, expected);
}
private String[] getShingleConfig(String params) {
String algLines[] = { String algLines[] = {
"content.source=org.apache.lucene.benchmark.byTask.feeds.LineDocSource", "content.source=org.apache.lucene.benchmark.byTask.feeds.LineDocSource",
"docs.file=" + getReuters20LinesFile(), "docs.file=" + getReuters20LinesFile(),
"work.dir=" + getWorkDir().getAbsolutePath(),
"content.source.forever=false", "content.source.forever=false",
"directory=RAMDirectory", "directory=RAMDirectory",
"NewShingleAnalyzer(" + params + ")", "AnalyzerFactory(name:'" + singleQuoteEscapedName + "', " + params + ")",
"NewAnalyzer('" + singleQuoteEscapedName + "')",
"CreateIndex", "CreateIndex",
"{ \"AddDocs\" AddDoc > : * " "{ \"AddDocs\" AddDoc > : * "
}; };
return algLines; return algLines;
} }
public void testAnalyzerFactory() throws Exception {
String text = "Fortieth, Quarantième, Cuadragésimo";
Benchmark benchmark = execBenchmark(getAnalyzerFactoryConfig
("ascii folded, pattern replaced, standard tokenized, downcased, bigrammed.'analyzer'",
"positionIncrementGap:100,offsetGap:1111,"
+"MappingCharFilter(mapping:'test-mapping-ISOLatin1Accent-partial.txt'),"
+"PatternReplaceCharFilterFactory(pattern:'e(\\\\\\\\S*)m',replacement:\"$1xxx$1\"),"
+"StandardTokenizer,LowerCaseFilter,NGramTokenFilter(minGramSize:2,maxGramSize:2)"));
BaseTokenStreamTestCase.assertAnalyzesTo(benchmark.getRunData().getAnalyzer(), text,
new String[] { "fo", "or", "rt", "ti", "ie", "et", "th",
"qu", "ua", "ar", "ra", "an", "nt", "ti", "ix", "xx", "xx", "xe",
"cu", "ua", "ad", "dr", "ra", "ag", "gs", "si", "ix", "xx", "xx", "xs", "si", "io"});
}
private String getReuters20LinesFile() { private String getReuters20LinesFile() {
return getWorkDirResourcePath("reuters.first20.lines.txt"); return getWorkDirResourcePath("reuters.first20.lines.txt");
} }

View File

@ -0,0 +1,30 @@
# 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.
# Syntax:
# "source" => "target"
# "source".length() > 0 (source cannot be empty.)
# "target".length() >= 0 (target can be empty.)
# example:
# "À" => "A"
# "\u00C0" => "A"
# "\u00C0" => "\u0041"
# "ß" => "ss"
# "\t" => " "
# "\n" => ""
# è => e
"\u00E8" => "e"
# é => e
"\u00E9" => "e"