LUCENE-3969: commit current state

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene3969@1311220 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Robert Muir 2012-04-09 13:25:28 +00:00
parent d55447b25e
commit 6311f71de6
12 changed files with 526 additions and 100 deletions

View File

@ -76,7 +76,7 @@ public final class MockAnalyzer extends Analyzer {
* MockAnalyzer(random, runAutomaton, lowerCase, MockTokenFilter.EMPTY_STOPSET, false}). * MockAnalyzer(random, runAutomaton, lowerCase, MockTokenFilter.EMPTY_STOPSET, false}).
*/ */
public MockAnalyzer(Random random, CharacterRunAutomaton runAutomaton, boolean lowerCase) { public MockAnalyzer(Random random, CharacterRunAutomaton runAutomaton, boolean lowerCase) {
this(random, runAutomaton, lowerCase, MockTokenFilter.EMPTY_STOPSET, false); this(random, runAutomaton, lowerCase, MockTokenFilter.EMPTY_STOPSET, true);
} }
/** /**
@ -93,7 +93,8 @@ public final class MockAnalyzer extends Analyzer {
public TokenStreamComponents createComponents(String fieldName, Reader reader) { public TokenStreamComponents createComponents(String fieldName, Reader reader) {
MockTokenizer tokenizer = new MockTokenizer(reader, runAutomaton, lowerCase, maxTokenLength); MockTokenizer tokenizer = new MockTokenizer(reader, runAutomaton, lowerCase, maxTokenLength);
tokenizer.setEnableChecks(enableChecks); tokenizer.setEnableChecks(enableChecks);
TokenFilter filt = new MockTokenFilter(tokenizer, filter, enablePositionIncrements); MockTokenFilter filt = new MockTokenFilter(tokenizer, filter);
filt.setEnablePositionIncrements(enablePositionIncrements);
return new TokenStreamComponents(tokenizer, maybePayload(filt, fieldName)); return new TokenStreamComponents(tokenizer, maybePayload(filt, fieldName));
} }

View File

@ -34,7 +34,9 @@ public class MockCharFilter extends CharStream {
// TODO: instead of fixed remainder... maybe a fixed // TODO: instead of fixed remainder... maybe a fixed
// random seed? // random seed?
this.remainder = remainder; this.remainder = remainder;
assert remainder >= 0 && remainder < 10 : "invalid parameter"; if (remainder < 0 || remainder >= 10) {
throw new IllegalArgumentException("invalid remainder parameter (must be 0..10): " + remainder);
}
} }
// for testing only, uses a remainder of 0 // for testing only, uses a remainder of 0

View File

@ -34,6 +34,9 @@ public final class MockFixedLengthPayloadFilter extends TokenFilter {
public MockFixedLengthPayloadFilter(Random random, TokenStream in, int length) { public MockFixedLengthPayloadFilter(Random random, TokenStream in, int length) {
super(in); super(in);
if (length < 0) {
throw new IllegalArgumentException("length must be >= 0");
}
this.random = random; this.random = random;
this.bytes = new byte[length]; this.bytes = new byte[length];
this.payload = new Payload(bytes); this.payload = new Payload(bytes);

View File

@ -55,7 +55,7 @@ public final class MockTokenFilter extends TokenFilter {
makeString("with")))); makeString("with"))));
private final CharacterRunAutomaton filter; private final CharacterRunAutomaton filter;
private boolean enablePositionIncrements = false; private boolean enablePositionIncrements = true;
private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class); private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
private final PositionIncrementAttribute posIncrAtt = addAttribute(PositionIncrementAttribute.class); private final PositionIncrementAttribute posIncrAtt = addAttribute(PositionIncrementAttribute.class);
@ -67,14 +67,16 @@ public final class MockTokenFilter extends TokenFilter {
* @param filter DFA representing the terms that should be removed. * @param filter DFA representing the terms that should be removed.
* @param enablePositionIncrements true if the removal should accumulate position increments. * @param enablePositionIncrements true if the removal should accumulate position increments.
*/ */
public MockTokenFilter(TokenStream input, CharacterRunAutomaton filter, boolean enablePositionIncrements) { public MockTokenFilter(TokenStream input, CharacterRunAutomaton filter) {
super(input); super(input);
this.filter = filter; this.filter = filter;
this.enablePositionIncrements = enablePositionIncrements;
} }
@Override @Override
public boolean incrementToken() throws IOException { public boolean incrementToken() throws IOException {
// TODO: fix me when posInc=false, to work like FilteringTokenFilter in that case and not return
// initial token with posInc=0 ever
// return the first non-stop word found // return the first non-stop word found
int skippedPositions = 0; int skippedPositions = 0;
while (input.incrementToken()) { while (input.incrementToken()) {

View File

@ -43,16 +43,25 @@ public final class KeywordTokenizer extends Tokenizer {
public KeywordTokenizer(Reader input, int bufferSize) { public KeywordTokenizer(Reader input, int bufferSize) {
super(input); super(input);
if (bufferSize <= 0) {
throw new IllegalArgumentException("bufferSize must be > 0");
}
termAtt.resizeBuffer(bufferSize); termAtt.resizeBuffer(bufferSize);
} }
public KeywordTokenizer(AttributeSource source, Reader input, int bufferSize) { public KeywordTokenizer(AttributeSource source, Reader input, int bufferSize) {
super(source, input); super(source, input);
if (bufferSize <= 0) {
throw new IllegalArgumentException("bufferSize must be > 0");
}
termAtt.resizeBuffer(bufferSize); termAtt.resizeBuffer(bufferSize);
} }
public KeywordTokenizer(AttributeFactory factory, Reader input, int bufferSize) { public KeywordTokenizer(AttributeFactory factory, Reader input, int bufferSize) {
super(factory, input); super(factory, input);
if (bufferSize <= 0) {
throw new IllegalArgumentException("bufferSize must be > 0");
}
termAtt.resizeBuffer(bufferSize); termAtt.resizeBuffer(bufferSize);
} }

View File

@ -65,6 +65,12 @@ public class PathHierarchyTokenizer extends Tokenizer {
public PathHierarchyTokenizer(Reader input, int bufferSize, char delimiter, char replacement, int skip) { public PathHierarchyTokenizer(Reader input, int bufferSize, char delimiter, char replacement, int skip) {
super(input); super(input);
if (bufferSize < 0) {
throw new IllegalArgumentException("bufferSize cannot be negative");
}
if (skip < 0) {
throw new IllegalArgumentException("skip cannot be negative");
}
termAtt.resizeBuffer(bufferSize); termAtt.resizeBuffer(bufferSize);
this.delimiter = delimiter; this.delimiter = delimiter;
@ -85,10 +91,11 @@ public class PathHierarchyTokenizer extends Tokenizer {
private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class); private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
private final PositionIncrementAttribute posAtt = addAttribute(PositionIncrementAttribute.class); private final PositionIncrementAttribute posAtt = addAttribute(PositionIncrementAttribute.class);
private int startPosition = 0; private int startPosition = 0;
private int finalOffset = 0;
private int skipped = 0; private int skipped = 0;
private boolean endDelimiter = false; private boolean endDelimiter = false;
private StringBuilder resultToken; private StringBuilder resultToken;
private int charsRead = 0;
@Override @Override
@ -112,12 +119,13 @@ public class PathHierarchyTokenizer extends Tokenizer {
while (true) { while (true) {
int c = input.read(); int c = input.read();
if( c < 0 ){ if (c >= 0) {
charsRead++;
} else {
if( skipped > skip ) { if( skipped > skip ) {
length += resultToken.length(); length += resultToken.length();
termAtt.setLength(length); termAtt.setLength(length);
finalOffset = correctOffset(startPosition + length); offsetAtt.setOffset(correctOffset(startPosition), correctOffset(startPosition + length));
offsetAtt.setOffset(correctOffset(startPosition), finalOffset);
if( added ){ if( added ){
resultToken.setLength(0); resultToken.setLength(0);
resultToken.append(termAtt.buffer(), 0, length); resultToken.append(termAtt.buffer(), 0, length);
@ -125,7 +133,6 @@ public class PathHierarchyTokenizer extends Tokenizer {
return added; return added;
} }
else{ else{
finalOffset = correctOffset(startPosition + length);
return false; return false;
} }
} }
@ -168,8 +175,7 @@ public class PathHierarchyTokenizer extends Tokenizer {
} }
length += resultToken.length(); length += resultToken.length();
termAtt.setLength(length); termAtt.setLength(length);
finalOffset = correctOffset(startPosition + length); offsetAtt.setOffset(correctOffset(startPosition), correctOffset(startPosition+length));
offsetAtt.setOffset(correctOffset(startPosition), finalOffset);
resultToken.setLength(0); resultToken.setLength(0);
resultToken.append(termAtt.buffer(), 0, length); resultToken.append(termAtt.buffer(), 0, length);
return true; return true;
@ -178,14 +184,15 @@ public class PathHierarchyTokenizer extends Tokenizer {
@Override @Override
public final void end() { public final void end() {
// set final offset // set final offset
int finalOffset = correctOffset(charsRead);
offsetAtt.setOffset(finalOffset, finalOffset); offsetAtt.setOffset(finalOffset, finalOffset);
} }
@Override @Override
public void reset(Reader input) throws IOException { public void reset() throws IOException {
super.reset(input); super.reset();
resultToken.setLength(0); resultToken.setLength(0);
finalOffset = 0; charsRead = 0;
endDelimiter = false; endDelimiter = false;
skipped = 0; skipped = 0;
} }

View File

@ -77,6 +77,13 @@ public class ReversePathHierarchyTokenizer extends Tokenizer {
public ReversePathHierarchyTokenizer(Reader input, int bufferSize, char delimiter, char replacement, int skip) { public ReversePathHierarchyTokenizer(Reader input, int bufferSize, char delimiter, char replacement, int skip) {
super(input); super(input);
if (bufferSize < 0) {
throw new IllegalArgumentException("bufferSize cannot be negative");
}
if (skip < 0) {
// nocommit: not quite right right here: see line 84... if skip > numTokensFound we always get a NegativeArrayException? needs fixing!
throw new IllegalArgumentException("skip cannot be negative");
}
termAtt.resizeBuffer(bufferSize); termAtt.resizeBuffer(bufferSize);
this.delimiter = delimiter; this.delimiter = delimiter;
this.replacement = replacement; this.replacement = replacement;
@ -137,7 +144,11 @@ public class ReversePathHierarchyTokenizer extends Tokenizer {
} }
resultToken.getChars(0, resultToken.length(), resultTokenBuffer, 0); resultToken.getChars(0, resultToken.length(), resultTokenBuffer, 0);
resultToken.setLength(0); resultToken.setLength(0);
endPosition = delimiterPositions.get(delimitersCount-1 - skip); int idx = delimitersCount-1 - skip;
if (idx >= 0) {
// otherwise its ok, because we will skip and return false
endPosition = delimiterPositions.get(idx);
}
finalOffset = correctOffset(length); finalOffset = correctOffset(length);
posAtt.setPositionIncrement(1); posAtt.setPositionIncrement(1);
} }
@ -163,10 +174,11 @@ public class ReversePathHierarchyTokenizer extends Tokenizer {
} }
@Override @Override
public void reset(Reader input) throws IOException { public void reset() throws IOException {
super.reset(input); super.reset();
resultToken.setLength(0); resultToken.setLength(0);
finalOffset = 0; finalOffset = 0;
endPosition = 0;
skipped = 0; skipped = 0;
delimitersCount = -1; delimitersCount = -1;
delimiterPositions.clear(); delimiterPositions.clear();

View File

@ -71,6 +71,10 @@ public final class PatternTokenizer extends Tokenizer {
this.group = group; this.group = group;
fillBuffer(str, input); fillBuffer(str, input);
matcher = pattern.matcher(str); matcher = pattern.matcher(str);
// confusingly group count depends ENTIRELY on the pattern but is only accessible via matcher
if (group >= 0 && group > matcher.groupCount()) {
throw new IllegalArgumentException("invalid group specified: pattern only has: " + matcher.groupCount() + " capturing groups");
}
index = 0; index = 0;
} }

View File

@ -57,6 +57,9 @@ public final class PositionFilter extends TokenFilter {
*/ */
public PositionFilter(final TokenStream input, final int positionIncrement) { public PositionFilter(final TokenStream input, final int positionIncrement) {
super(input); super(input);
if (positionIncrement < 0) {
throw new IllegalArgumentException("positionIncrement may not be negative");
}
this.positionIncrement = positionIncrement; this.positionIncrement = positionIncrement;
} }

View File

@ -67,7 +67,7 @@ public final class SnowballFilter extends TokenFilter {
Class.forName("org.tartarus.snowball.ext." + name + "Stemmer").asSubclass(SnowballProgram.class); Class.forName("org.tartarus.snowball.ext." + name + "Stemmer").asSubclass(SnowballProgram.class);
stemmer = stemClass.newInstance(); stemmer = stemClass.newInstance();
} catch (Exception e) { } catch (Exception e) {
throw new RuntimeException(e.toString()); throw new IllegalArgumentException("Invalid stemmer class specified: " + name, e);
} }
} }

View File

@ -18,17 +18,26 @@ package org.apache.lucene.analysis.core;
*/ */
import java.io.File; import java.io.File;
import java.io.InputStream;
import java.io.Reader; import java.io.Reader;
import java.io.StringReader; import java.io.StringReader;
import java.lang.reflect.Constructor; import java.lang.reflect.Constructor;
import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Modifier; import java.lang.reflect.Modifier;
import java.net.URL; import java.net.URL;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections; import java.util.Collections;
import java.util.Comparator; import java.util.Comparator;
import java.util.Enumeration; import java.util.Enumeration;
import java.util.HashSet;
import java.util.List; import java.util.List;
import java.util.Random; import java.util.Random;
import java.util.Set;
import java.util.Map;
import java.util.IdentityHashMap;
import java.util.regex.Pattern;
import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.BaseTokenStreamTestCase; import org.apache.lucene.analysis.BaseTokenStreamTestCase;
@ -36,67 +45,113 @@ import org.apache.lucene.analysis.CachingTokenFilter;
import org.apache.lucene.analysis.CharReader; import org.apache.lucene.analysis.CharReader;
import org.apache.lucene.analysis.CharStream; import org.apache.lucene.analysis.CharStream;
import org.apache.lucene.analysis.EmptyTokenizer; import org.apache.lucene.analysis.EmptyTokenizer;
import org.apache.lucene.analysis.MockTokenFilter;
import org.apache.lucene.analysis.MockTokenizer;
import org.apache.lucene.analysis.TokenFilter; import org.apache.lucene.analysis.TokenFilter;
import org.apache.lucene.analysis.TokenStream; import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.Tokenizer; import org.apache.lucene.analysis.Tokenizer;
import org.apache.lucene.analysis.charfilter.NormalizeCharMap;
import org.apache.lucene.analysis.commongrams.CommonGramsFilter;
import org.apache.lucene.analysis.compound.HyphenationCompoundWordTokenFilter;
import org.apache.lucene.analysis.compound.TestCompoundWordTokenFilter;
import org.apache.lucene.analysis.compound.hyphenation.HyphenationTree;
import org.apache.lucene.analysis.hunspell.HunspellDictionary;
import org.apache.lucene.analysis.hunspell.HunspellDictionaryTest;
import org.apache.lucene.analysis.miscellaneous.LimitTokenCountFilter;
import org.apache.lucene.analysis.ngram.EdgeNGramTokenFilter; import org.apache.lucene.analysis.ngram.EdgeNGramTokenFilter;
import org.apache.lucene.analysis.ngram.EdgeNGramTokenizer; import org.apache.lucene.analysis.ngram.EdgeNGramTokenizer;
import org.apache.lucene.analysis.ngram.NGramTokenFilter; import org.apache.lucene.analysis.ngram.NGramTokenFilter;
import org.apache.lucene.analysis.ngram.NGramTokenizer; import org.apache.lucene.analysis.ngram.NGramTokenizer;
import org.apache.lucene.analysis.payloads.IdentityEncoder;
import org.apache.lucene.analysis.payloads.PayloadEncoder;
import org.apache.lucene.analysis.snowball.TestSnowball;
import org.apache.lucene.analysis.standard.StandardTokenizer;
import org.apache.lucene.analysis.synonym.SynonymMap;
import org.apache.lucene.analysis.util.CharArrayMap;
import org.apache.lucene.analysis.util.CharArraySet;
import org.apache.lucene.util.AttributeSource;
import org.apache.lucene.util.AttributeSource.AttributeFactory;
import org.apache.lucene.util.CharsRef;
import org.apache.lucene.util.Rethrow;
import org.apache.lucene.util.Version; import org.apache.lucene.util.Version;
import org.apache.lucene.util._TestUtil;
import org.apache.lucene.util.automaton.CharacterRunAutomaton;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.BeforeClass; import org.junit.BeforeClass;
import org.tartarus.snowball.SnowballProgram;
import org.xml.sax.InputSource;
/** tests random analysis chains */ /** tests random analysis chains */
public class TestRandomChains extends BaseTokenStreamTestCase { public class TestRandomChains extends BaseTokenStreamTestCase {
static List<Class<? extends Tokenizer>> tokenizers; static List<Constructor<? extends Tokenizer>> tokenizers;
static List<Class<? extends TokenFilter>> tokenfilters; static List<Constructor<? extends TokenFilter>> tokenfilters;
static List<Class<? extends CharStream>> charfilters; static List<Constructor<? extends CharStream>> charfilters;
@BeforeClass @BeforeClass
public static void beforeClass() throws Exception { public static void beforeClass() throws Exception {
List<Class<?>> analysisClasses = new ArrayList<Class<?>>(); List<Class<?>> analysisClasses = new ArrayList<Class<?>>();
getClassesForPackage("org.apache.lucene.analysis", analysisClasses); getClassesForPackage("org.apache.lucene.analysis", analysisClasses);
tokenizers = new ArrayList<Class<? extends Tokenizer>>(); tokenizers = new ArrayList<Constructor<? extends Tokenizer>>();
tokenfilters = new ArrayList<Class<? extends TokenFilter>>(); tokenfilters = new ArrayList<Constructor<? extends TokenFilter>>();
charfilters = new ArrayList<Class<? extends CharStream>>(); charfilters = new ArrayList<Constructor<? extends CharStream>>();
for (Class<?> c : analysisClasses) { for (final Class<?> c : analysisClasses) {
// don't waste time with abstract classes or deprecated known-buggy ones
final int modifiers = c.getModifiers(); final int modifiers = c.getModifiers();
if (Modifier.isAbstract(modifiers) || !Modifier.isPublic(modifiers) if (
|| c.getAnnotation(Deprecated.class) != null // don't waste time with abstract classes or deprecated known-buggy ones
|| c.isSynthetic() || c.isAnonymousClass() || c.isMemberClass() || c.isInterface() Modifier.isAbstract(modifiers) || !Modifier.isPublic(modifiers)
// TODO: fix basetokenstreamtestcase not to trip because this one has no CharTermAtt || c.isAnnotationPresent(Deprecated.class)
|| c.equals(EmptyTokenizer.class) || c.isSynthetic() || c.isAnonymousClass() || c.isMemberClass() || c.isInterface()
// doesn't actual reset itself! || !(Tokenizer.class.isAssignableFrom(c) || TokenFilter.class.isAssignableFrom(c) || CharStream.class.isAssignableFrom(c))
|| c.equals(CachingTokenFilter.class) // TODO: fix basetokenstreamtestcase not to trip because this one has no CharTermAtt
// broken! || c == EmptyTokenizer.class
|| c.equals(NGramTokenizer.class) // doesn't actual reset itself!
// broken! || c == CachingTokenFilter.class
|| c.equals(NGramTokenFilter.class) // doesn't consume whole stream!
// broken! || c == LimitTokenCountFilter.class
|| c.equals(EdgeNGramTokenizer.class) // broken!
// broken! || c == NGramTokenizer.class
|| c.equals(EdgeNGramTokenFilter.class)) { // broken!
|| c == NGramTokenFilter.class
// broken!
|| c == EdgeNGramTokenizer.class
// broken!
|| c == EdgeNGramTokenFilter.class
) {
continue; continue;
} }
if (Tokenizer.class.isAssignableFrom(c)) { for (final Constructor<?> ctor : c.getConstructors()) {
tokenizers.add(c.asSubclass(Tokenizer.class)); // don't test deprecated ctors, they likely have known bugs:
} else if (TokenFilter.class.isAssignableFrom(c)) { if (ctor.isAnnotationPresent(Deprecated.class) || ctor.isSynthetic()) {
tokenfilters.add(c.asSubclass(TokenFilter.class)); continue;
} else if (CharStream.class.isAssignableFrom(c)) { }
charfilters.add(c.asSubclass(CharStream.class)); if (Tokenizer.class.isAssignableFrom(c)) {
assertTrue(ctor.toGenericString() + " has unsupported parameter types",
allowedTokenizerArgs.containsAll(Arrays.asList(ctor.getParameterTypes())));
tokenizers.add(castConstructor(Tokenizer.class, ctor));
} else if (TokenFilter.class.isAssignableFrom(c)) {
assertTrue(ctor.toGenericString() + " has unsupported parameter types",
allowedTokenFilterArgs.containsAll(Arrays.asList(ctor.getParameterTypes())));
tokenfilters.add(castConstructor(TokenFilter.class, ctor));
} else if (CharStream.class.isAssignableFrom(c)) {
assertTrue(ctor.toGenericString() + " has unsupported parameter types",
allowedCharFilterArgs.containsAll(Arrays.asList(ctor.getParameterTypes())));
charfilters.add(castConstructor(CharStream.class, ctor));
} else {
fail("Cannot get here");
}
} }
} }
final Comparator<Class<?>> classComp = new Comparator<Class<?>>() {
final Comparator<Constructor<?>> ctorComp = new Comparator<Constructor<?>>() {
@Override @Override
public int compare(Class<?> arg0, Class<?> arg1) { public int compare(Constructor<?> arg0, Constructor<?> arg1) {
return arg0.getName().compareTo(arg1.getName()); return arg0.toGenericString().compareTo(arg1.toGenericString());
} }
}; };
Collections.sort(tokenizers, classComp); Collections.sort(tokenizers, ctorComp);
Collections.sort(tokenfilters, classComp); Collections.sort(tokenfilters, ctorComp);
Collections.sort(charfilters, classComp); Collections.sort(charfilters, ctorComp);
if (VERBOSE) { if (VERBOSE) {
System.out.println("tokenizers = " + tokenizers); System.out.println("tokenizers = " + tokenizers);
System.out.println("tokenfilters = " + tokenfilters); System.out.println("tokenfilters = " + tokenfilters);
@ -111,6 +166,304 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
charfilters = null; charfilters = null;
} }
/** Hack to work around the stupidness of Oracle's strict Java backwards compatibility.
* {@code Class<T>#getConstructors()} should return unmodifiable {@code List<Constructor<T>>} not array! */
@SuppressWarnings("unchecked")
private static <T> Constructor<? extends T> castConstructor(Class<T> instanceClazz, Constructor<?> ctor) {
return (Constructor<? extends T>) ctor;
}
private static interface ArgProducer {
Object create(Random random);
}
private static final Map<Class<?>,ArgProducer> argProducers = new IdentityHashMap<Class<?>,ArgProducer>() {{
put(int.class, new ArgProducer() {
@Override public Object create(Random random) {
// TODO: could cause huge ram usage to use full int range for some filters
// (e.g. allocate enormous arrays)
// return Integer.valueOf(random.nextInt());
return Integer.valueOf(_TestUtil.nextInt(random, -100, 100));
}
});
put(char.class, new ArgProducer() {
@Override public Object create(Random random) {
return Character.valueOf((char)random.nextInt(65536));
}
});
put(float.class, new ArgProducer() {
@Override public Object create(Random random) {
return Float.valueOf(random.nextFloat());
}
});
put(boolean.class, new ArgProducer() {
@Override public Object create(Random random) {
return Boolean.valueOf(random.nextBoolean());
}
});
put(byte.class, new ArgProducer() {
@Override public Object create(Random random) {
byte bytes[] = new byte[1];
random.nextBytes(bytes);
return Byte.valueOf(bytes[0]);
}
});
put(byte[].class, new ArgProducer() {
@Override public Object create(Random random) {
byte bytes[] = new byte[random.nextInt(256)];
random.nextBytes(bytes);
return bytes;
}
});
put(Random.class, new ArgProducer() {
@Override public Object create(Random random) {
return new Random(random.nextLong());
}
});
put(Version.class, new ArgProducer() {
@Override public Object create(Random random) {
// we expect bugs in emulating old versions
return TEST_VERSION_CURRENT;
}
});
put(Set.class, new ArgProducer() {
@Override public Object create(Random random) {
// TypeTokenFilter
Set<String> set = new HashSet<String>();
int num = random.nextInt(5);
for (int i = 0; i < num; i++) {
set.add(StandardTokenizer.TOKEN_TYPES[random.nextInt(StandardTokenizer.TOKEN_TYPES.length)]);
}
return set;
}
});
put(Collection.class, new ArgProducer() {
@Override public Object create(Random random) {
// CapitalizationFilter
Collection<char[]> col = new ArrayList<char[]>();
int num = random.nextInt(5);
for (int i = 0; i < num; i++) {
col.add(_TestUtil.randomSimpleString(random).toCharArray());
}
return col;
}
});
put(CharArraySet.class, new ArgProducer() {
@Override public Object create(Random random) {
int num = random.nextInt(10);
CharArraySet set = new CharArraySet(TEST_VERSION_CURRENT, num, random.nextBoolean());
for (int i = 0; i < num; i++) {
// TODO: make nastier
set.add(_TestUtil.randomSimpleString(random));
}
return set;
}
});
put(Pattern.class, new ArgProducer() {
@Override public Object create(Random random) {
// TODO: don't want to make the exponentially slow ones Dawid documents
// in TestPatternReplaceFilter, so dont use truly random patterns (for now)
return Pattern.compile("a");
}
});
put(PayloadEncoder.class, new ArgProducer() {
@Override public Object create(Random random) {
return new IdentityEncoder(); // the other encoders will throw exceptions if tokens arent numbers?
}
});
put(HunspellDictionary.class, new ArgProducer() {
@Override public Object create(Random random) {
// TODO: make nastier
InputStream affixStream = HunspellDictionaryTest.class.getResourceAsStream("test.aff");
InputStream dictStream = HunspellDictionaryTest.class.getResourceAsStream("test.dic");
try {
return new HunspellDictionary(affixStream, dictStream, TEST_VERSION_CURRENT);
} catch (Exception ex) {
throw new RuntimeException(ex);
}
}
});
put(HyphenationTree.class, new ArgProducer() {
@Override public Object create(Random random) {
// TODO: make nastier
try {
InputSource is = new InputSource(TestCompoundWordTokenFilter.class.getResource("da_UTF8.xml").toExternalForm());
HyphenationTree hyphenator = HyphenationCompoundWordTokenFilter.getHyphenationTree(is);
return hyphenator;
} catch (Exception ex) {
Rethrow.rethrow(ex);
return null; // unreachable code
}
}
});
put(SnowballProgram.class, new ArgProducer() {
@Override public Object create(Random random) {
try {
String lang = TestSnowball.SNOWBALL_LANGS[random.nextInt(TestSnowball.SNOWBALL_LANGS.length)];
Class<? extends SnowballProgram> clazz = Class.forName("org.tartarus.snowball.ext." + lang + "Stemmer").asSubclass(SnowballProgram.class);
return clazz.newInstance();
} catch (Exception ex) {
Rethrow.rethrow(ex);
return null; // unreachable code
}
}
});
put(String.class, new ArgProducer() {
@Override public Object create(Random random) {
// TODO: make nastier
if (random.nextBoolean()) {
// a token type
return StandardTokenizer.TOKEN_TYPES[random.nextInt(StandardTokenizer.TOKEN_TYPES.length)];
} else {
return _TestUtil.randomSimpleString(random);
}
}
});
put(NormalizeCharMap.class, new ArgProducer() {
@Override public Object create(Random random) {
NormalizeCharMap map = new NormalizeCharMap();
// we can't add duplicate keys, or NormalizeCharMap gets angry
Set<String> keys = new HashSet<String>();
int num = random.nextInt(5);
for (int i = 0; i < num; i++) {
String key = _TestUtil.randomSimpleString(random);
if (!keys.contains(key)) {
map.add(key,_TestUtil.randomSimpleString(random));
keys.add(key);
}
}
return map;
}
});
put(CharacterRunAutomaton.class, new ArgProducer() {
@Override public Object create(Random random) {
// TODO: could probably use a purely random automaton
switch(random.nextInt(5)) {
case 0: return MockTokenizer.KEYWORD;
case 1: return MockTokenizer.SIMPLE;
case 2: return MockTokenizer.WHITESPACE;
case 3: return MockTokenFilter.EMPTY_STOPSET;
default: return MockTokenFilter.ENGLISH_STOPSET;
}
}
});
put(CharArrayMap.class, new ArgProducer() {
@Override public Object create(Random random) {
int num = random.nextInt(10);
CharArrayMap<String> map = new CharArrayMap<String>(TEST_VERSION_CURRENT, num, random.nextBoolean());
for (int i = 0; i < num; i++) {
// TODO: make nastier
map.put(_TestUtil.randomSimpleString(random), _TestUtil.randomSimpleString(random));
}
return map;
}
});
put(SynonymMap.class, new ArgProducer() {
@Override public Object create(Random random) {
SynonymMap.Builder b = new SynonymMap.Builder(random.nextBoolean());
final int numEntries = atLeast(10);
for (int j = 0; j < numEntries; j++) {
addSyn(b, randomNonEmptyString(random), randomNonEmptyString(random), random.nextBoolean());
}
try {
return b.build();
} catch (Exception e) {
throw new RuntimeException(e);
}
}
private void addSyn(SynonymMap.Builder b, String input, String output, boolean keepOrig) {
b.add(new CharsRef(input.replaceAll(" +", "\u0000")),
new CharsRef(output.replaceAll(" +", "\u0000")),
keepOrig);
}
private String randomNonEmptyString(Random random) {
while(true) {
final String s = _TestUtil.randomUnicodeString(random).trim();
if (s.length() != 0 && s.indexOf('\u0000') == -1) {
return s;
}
}
}
});
}};
static final Set<Class<?>> allowedTokenizerArgs, allowedTokenFilterArgs, allowedCharFilterArgs;
static {
allowedTokenizerArgs = Collections.newSetFromMap(new IdentityHashMap<Class<?>,Boolean>());
allowedTokenizerArgs.addAll(argProducers.keySet());
allowedTokenizerArgs.add(Reader.class);
allowedTokenizerArgs.add(AttributeFactory.class);
allowedTokenizerArgs.add(AttributeSource.class);
allowedTokenFilterArgs = Collections.newSetFromMap(new IdentityHashMap<Class<?>,Boolean>());
allowedTokenFilterArgs.addAll(argProducers.keySet());
allowedTokenFilterArgs.add(TokenStream.class);
allowedTokenFilterArgs.add(CommonGramsFilter.class);
allowedCharFilterArgs = Collections.newSetFromMap(new IdentityHashMap<Class<?>,Boolean>());
allowedCharFilterArgs.addAll(argProducers.keySet());
allowedCharFilterArgs.add(Reader.class);
allowedCharFilterArgs.add(CharStream.class);
}
@SuppressWarnings("unchecked")
static <T> T createRandomArg(Random random, Class<T> paramType) {
final ArgProducer producer = argProducers.get(paramType);
assertNotNull("No producer for arguments of type " + paramType.getName() + " found", producer);
return (T) producer.create(random);
}
static Object[] newTokenizerArgs(Random random, Reader reader, Class<?>[] paramTypes) {
Object[] args = new Object[paramTypes.length];
for (int i = 0; i < args.length; i++) {
Class<?> paramType = paramTypes[i];
if (paramType == Reader.class) {
args[i] = reader;
} else if (paramType == AttributeFactory.class) {
// TODO: maybe the collator one...???
args[i] = AttributeFactory.DEFAULT_ATTRIBUTE_FACTORY;
} else if (paramType == AttributeSource.class) {
args[i] = null; // this always gives IAE: fine
} else {
args[i] = createRandomArg(random, paramType);
}
}
return args;
}
static Object[] newCharFilterArgs(Random random, Reader reader, Class<?>[] paramTypes) {
Object[] args = new Object[paramTypes.length];
for (int i = 0; i < args.length; i++) {
Class<?> paramType = paramTypes[i];
if (paramType == Reader.class) {
args[i] = reader;
} else if (paramType == CharStream.class) {
args[i] = CharReader.get(reader);
} else {
args[i] = createRandomArg(random, paramType);
}
}
return args;
}
static Object[] newFilterArgs(Random random, TokenStream stream, Class<?>[] paramTypes) {
Object[] args = new Object[paramTypes.length];
for (int i = 0; i < args.length; i++) {
Class<?> paramType = paramTypes[i];
if (paramType == TokenStream.class) {
args[i] = stream;
} else if (paramType == CommonGramsFilter.class) {
// CommonGramsQueryFilter takes this one explicitly
args[i] = new CommonGramsFilter(TEST_VERSION_CURRENT, stream, createRandomArg(random, CharArraySet.class));
} else {
args[i] = createRandomArg(random, paramType);
}
}
return args;
}
static class MockRandomAnalyzer extends Analyzer { static class MockRandomAnalyzer extends Analyzer {
final long seed; final long seed;
@ -123,6 +476,8 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
Random random = new Random(seed); Random random = new Random(seed);
TokenizerSpec tokenizerspec = newTokenizer(random, reader); TokenizerSpec tokenizerspec = newTokenizer(random, reader);
TokenFilterSpec filterspec = newFilterChain(random, tokenizerspec.tokenizer); TokenFilterSpec filterspec = newFilterChain(random, tokenizerspec.tokenizer);
//System.out.println("seed=" + seed + ",tokenizerSpec=" + tokenizerspec.toString);
//System.out.println("seed=" + seed + ",tokenfilterSpec=" + filterspec.toString);
return new TokenStreamComponents(tokenizerspec.tokenizer, filterspec.stream); return new TokenStreamComponents(tokenizerspec.tokenizer, filterspec.stream);
} }
@ -130,6 +485,7 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
protected Reader initReader(Reader reader) { protected Reader initReader(Reader reader) {
Random random = new Random(seed); Random random = new Random(seed);
CharFilterSpec charfilterspec = newCharFilterChain(random, reader); CharFilterSpec charfilterspec = newCharFilterChain(random, reader);
//System.out.println("seed=" + seed + ",charFilterSpec=" + charfilterspec.toString);
return charfilterspec.reader; return charfilterspec.reader;
} }
@ -159,20 +515,27 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
boolean success = false; boolean success = false;
while (!success) { while (!success) {
try { try {
// TODO: check Reader+Version,Version+Reader too final Constructor<? extends Tokenizer> ctor = tokenizers.get(random.nextInt(tokenizers.size()));
// also look for other variants and handle them special final Object args[] = newTokenizerArgs(random, reader, ctor.getParameterTypes());
int idx = random.nextInt(tokenizers.size()); spec.tokenizer = ctor.newInstance(args);
try { spec.toString = ctor.getDeclaringClass().getName() + ("(" + Arrays.toString(args) + ")");
Constructor<? extends Tokenizer> c = tokenizers.get(idx).getConstructor(Version.class, Reader.class);
spec.tokenizer = c.newInstance(TEST_VERSION_CURRENT, reader);
} catch (NoSuchMethodException e) {
Constructor<? extends Tokenizer> c = tokenizers.get(idx).getConstructor(Reader.class);
spec.tokenizer = c.newInstance(reader);
}
spec.toString = tokenizers.get(idx).toString();
success = true; success = true;
} catch (Exception e) { } catch (InvocationTargetException ite) {
// ignore final Throwable cause = ite.getCause();
if (cause instanceof IllegalArgumentException ||
cause instanceof UnsupportedOperationException) {
// thats ok, ignore
if (VERBOSE) {
System.err.println("Ignoring IAE/UOE from ctor:");
cause.printStackTrace(System.err);
}
} else {
Rethrow.rethrow(cause);
}
} catch (IllegalAccessException iae) {
Rethrow.rethrow(iae);
} catch (InstantiationException ie) {
Rethrow.rethrow(ie);
} }
} }
return spec; return spec;
@ -187,23 +550,32 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
boolean success = false; boolean success = false;
while (!success) { while (!success) {
try { try {
// TODO: also look for other variants and handle them special final Constructor<? extends CharStream> ctor = charfilters.get(random.nextInt(charfilters.size()));
int idx = random.nextInt(charfilters.size()); final Object args[] = newCharFilterArgs(random, spec.reader, ctor.getParameterTypes());
try { spec.reader = ctor.newInstance(args);
Constructor<? extends CharStream> c = charfilters.get(idx).getConstructor(Reader.class);
spec.reader = c.newInstance(spec.reader);
} catch (NoSuchMethodException e) {
Constructor<? extends CharStream> c = charfilters.get(idx).getConstructor(CharStream.class);
spec.reader = c.newInstance(CharReader.get(spec.reader));
}
if (descr.length() > 0) { if (descr.length() > 0) {
descr.append(","); descr.append(",");
} }
descr.append(charfilters.get(idx).toString()); descr.append(ctor.getDeclaringClass().getName());
descr.append("(" + Arrays.toString(args) + ")");
success = true; success = true;
} catch (Exception e) { } catch (InvocationTargetException ite) {
// ignore final Throwable cause = ite.getCause();
if (cause instanceof IllegalArgumentException ||
cause instanceof UnsupportedOperationException) {
// thats ok, ignore
if (VERBOSE) {
System.err.println("Ignoring IAE/UOE from ctor:");
cause.printStackTrace(System.err);
}
} else {
Rethrow.rethrow(cause);
}
} catch (IllegalAccessException iae) {
Rethrow.rethrow(iae);
} catch (InstantiationException ie) {
Rethrow.rethrow(ie);
} }
} }
} }
@ -220,22 +592,31 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
boolean success = false; boolean success = false;
while (!success) { while (!success) {
try { try {
// TODO: also look for other variants and handle them special final Constructor<? extends TokenFilter> ctor = tokenfilters.get(random.nextInt(tokenfilters.size()));
int idx = random.nextInt(tokenfilters.size()); final Object args[] = newFilterArgs(random, spec.stream, ctor.getParameterTypes());
try { spec.stream = ctor.newInstance(args);
Constructor<? extends TokenFilter> c = tokenfilters.get(idx).getConstructor(Version.class, TokenStream.class);
spec.stream = c.newInstance(TEST_VERSION_CURRENT, spec.stream);
} catch (NoSuchMethodException e) {
Constructor<? extends TokenFilter> c = tokenfilters.get(idx).getConstructor(TokenStream.class);
spec.stream = c.newInstance(spec.stream);
}
if (descr.length() > 0) { if (descr.length() > 0) {
descr.append(","); descr.append(",");
} }
descr.append(tokenfilters.get(idx).toString()); descr.append(ctor.getDeclaringClass().getName());
descr.append("(" + Arrays.toString(args) + ")");
success = true; success = true;
} catch (Exception e) { } catch (InvocationTargetException ite) {
// ignore final Throwable cause = ite.getCause();
if (cause instanceof IllegalArgumentException ||
cause instanceof UnsupportedOperationException) {
// thats ok, ignore
if (VERBOSE) {
System.err.println("Ignoring IAE/UOE from ctor:");
cause.printStackTrace(System.err);
}
} else {
Rethrow.rethrow(cause);
}
} catch (IllegalAccessException iae) {
Rethrow.rethrow(iae);
} catch (InstantiationException ie) {
Rethrow.rethrow(ie);
} }
} }
} }
@ -263,7 +644,7 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
int numIterations = atLeast(20); int numIterations = atLeast(20);
for (int i = 0; i < numIterations; i++) { for (int i = 0; i < numIterations; i++) {
MockRandomAnalyzer a = new MockRandomAnalyzer(random.nextLong()); MockRandomAnalyzer a = new MockRandomAnalyzer(random.nextLong());
if (VERBOSE) { if (true || VERBOSE) {
System.out.println("Creating random analyzer:" + a); System.out.println("Creating random analyzer:" + a);
} }
try { try {

View File

@ -142,14 +142,16 @@ public class TestSnowball extends BaseTokenStreamTestCase {
} }
} }
/** for testing purposes ONLY */
public static String SNOWBALL_LANGS[] = {
"Armenian", "Basque", "Catalan", "Danish", "Dutch", "English",
"Finnish", "French", "German2", "German", "Hungarian", "Irish",
"Italian", "Kp", "Lovins", "Norwegian", "Porter", "Portuguese",
"Romanian", "Russian", "Spanish", "Swedish", "Turkish"
};
public void testEmptyTerm() throws IOException { public void testEmptyTerm() throws IOException {
String langs[] = { for (final String lang : SNOWBALL_LANGS) {
"Armenian", "Basque", "Catalan", "Danish", "Dutch", "English",
"Finnish", "French", "German2", "German", "Hungarian", "Irish",
"Italian", "Kp", "Lovins", "Norwegian", "Porter", "Portuguese",
"Romanian", "Russian", "Spanish", "Swedish", "Turkish"
};
for (final String lang : langs) {
Analyzer a = new Analyzer() { Analyzer a = new Analyzer() {
@Override @Override
protected TokenStreamComponents createComponents(String fieldName, Reader reader) { protected TokenStreamComponents createComponents(String fieldName, Reader reader) {