mirror of https://github.com/apache/lucene.git
LUCENE-10352: Convert TestAllAnalyzersHaveFactories and TestRandomChains to a global integration test and discover classes to check from module system (#582)
Co-authored-by: Robert Muir <rmuir@apache.org>
This commit is contained in:
parent
238119224a
commit
475fbd0bdd
|
@ -214,7 +214,7 @@ allprojects {
|
|||
}
|
||||
|
||||
// Configure (tasks.test, sourceSets.test)
|
||||
tasks.matching { it.name == "test" }.all { Test task ->
|
||||
tasks.matching { it.name ==~ /test(_[0-9]+)?/ }.all { Test task ->
|
||||
configureTestTaskForSourceSet(task, task.project.sourceSets.test)
|
||||
}
|
||||
|
||||
|
|
|
@ -102,6 +102,7 @@ allprojects {
|
|||
break
|
||||
|
||||
case ":lucene:analysis:common":
|
||||
case ":lucene:analysis.tests":
|
||||
exclude "src/**/*.aff"
|
||||
exclude "src/**/*.dic"
|
||||
exclude "src/**/*.good"
|
||||
|
|
|
@ -153,6 +153,12 @@ Bug Fixes
|
|||
* LUCENE-10349: Fix all analyzers to behave according to their documentation:
|
||||
getDefaultStopSet() methods now return unmodifiable CharArraySets. (Uwe Schindler)
|
||||
|
||||
* LUCENE-10352: Add missing service provider entries: KoreanNumberFilterFactory,
|
||||
DaitchMokotoffSoundexFilterFactory (Uwe Schindler, Robert Muir)
|
||||
|
||||
* LUCENE-10352: Fixed ctor argument checks: JapaneseKatakanaStemFilter,
|
||||
DoubleMetaphoneFilter (Uwe Schindler, Robert Muir)
|
||||
|
||||
Other
|
||||
---------------------
|
||||
|
||||
|
@ -163,6 +169,13 @@ Other
|
|||
* LUCENE-10310: TestXYDocValuesQueries#doRandomDistanceTest does not produce random circles with radius
|
||||
with '0' value any longer.
|
||||
|
||||
* LUCENE-10352: Removed duplicate instances of StringMockResourceLoader and migrated class to
|
||||
test-framework. (Uwe Schindler, Robert Muir)
|
||||
|
||||
* LUCENE-10352: Convert TestAllAnalyzersHaveFactories and TestRandomChains to a global integration test
|
||||
and discover classes to check from module system. The test now checks all analyzer modules,
|
||||
so it may discover new bugs outside of analysis:common module. (Uwe Schindler, Robert Muir)
|
||||
|
||||
======================= Lucene 9.0.0 =======================
|
||||
|
||||
New Features
|
||||
|
|
|
@ -0,0 +1,33 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
apply plugin: 'java-library'
|
||||
|
||||
description = 'Module integration tests for all :lucene:analysis modules'
|
||||
|
||||
dependencies {
|
||||
moduleTestImplementation project(':lucene:analysis:common')
|
||||
moduleTestImplementation project(':lucene:analysis:icu')
|
||||
moduleTestImplementation project(':lucene:analysis:kuromoji')
|
||||
moduleTestImplementation project(':lucene:analysis:morfologik')
|
||||
moduleTestImplementation project(':lucene:analysis:nori')
|
||||
moduleTestImplementation project(':lucene:analysis:opennlp')
|
||||
moduleTestImplementation project(':lucene:analysis:phonetic')
|
||||
moduleTestImplementation project(':lucene:analysis:smartcn')
|
||||
moduleTestImplementation project(':lucene:analysis:stempel')
|
||||
moduleTestImplementation project(':lucene:test-framework')
|
||||
}
|
|
@ -0,0 +1,39 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
/**
|
||||
* Test module for global integration tests of all {@code org.apache.lucene.analysis}
|
||||
* packages/modules.
|
||||
*/
|
||||
@SuppressWarnings({"requires-automatic"})
|
||||
module org.apache.lucene.analysis.tests {
|
||||
requires java.xml;
|
||||
requires org.apache.lucene.core;
|
||||
requires org.apache.lucene.analysis.common;
|
||||
requires org.apache.lucene.analysis.icu;
|
||||
requires org.apache.lucene.analysis.kuromoji;
|
||||
requires org.apache.lucene.analysis.morfologik;
|
||||
requires org.apache.lucene.analysis.nori;
|
||||
requires org.apache.lucene.analysis.opennlp;
|
||||
requires org.apache.lucene.analysis.phonetic;
|
||||
requires org.apache.lucene.analysis.smartcn;
|
||||
requires org.apache.lucene.analysis.stempel;
|
||||
requires org.apache.lucene.test_framework;
|
||||
requires junit;
|
||||
|
||||
exports org.apache.lucene.analysis.tests;
|
||||
}
|
|
@ -0,0 +1,95 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.analysis.tests;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.lang.module.ResolvedModule;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.SortedMap;
|
||||
import java.util.TreeMap;
|
||||
import java.util.function.Predicate;
|
||||
import org.apache.lucene.tests.util.LuceneTestCase;
|
||||
import org.junit.Assert;
|
||||
|
||||
/** Discovers all classes from the module graph and loads them (without initialization) */
|
||||
abstract class ModuleClassDiscovery {
|
||||
|
||||
private static final Module THIS_MODULE = ModuleClassDiscovery.class.getModule();
|
||||
private static final ModuleLayer LAYER = THIS_MODULE.getLayer();
|
||||
private static final SortedMap<String, ResolvedModule> ALL_ANALYSIS_MODULES;
|
||||
|
||||
private static final Predicate<String> ALLOW_MODULES =
|
||||
name ->
|
||||
name.equals("org.apache.lucene.core") || name.startsWith("org.apache.lucene.analysis.");
|
||||
|
||||
static {
|
||||
Assert.assertTrue(
|
||||
"Analysis integration tests must run in Java Module System as named module",
|
||||
THIS_MODULE.isNamed());
|
||||
Assert.assertNotNull("Module layer is missing", LAYER);
|
||||
|
||||
var mods = new TreeMap<String, ResolvedModule>();
|
||||
discoverAnalysisModules(LAYER, mods);
|
||||
ALL_ANALYSIS_MODULES = Collections.unmodifiableSortedMap(mods);
|
||||
if (LuceneTestCase.VERBOSE) {
|
||||
System.out.println(
|
||||
"Discovered the following analysis modules: " + ALL_ANALYSIS_MODULES.keySet());
|
||||
}
|
||||
}
|
||||
|
||||
private static void discoverAnalysisModules(
|
||||
ModuleLayer layer, Map<String, ResolvedModule> result) {
|
||||
for (var mod : layer.configuration().modules()) {
|
||||
String name = mod.name();
|
||||
if (ALLOW_MODULES.test(name) && !Objects.equals(name, THIS_MODULE.getName())) {
|
||||
result.put(name, mod);
|
||||
}
|
||||
}
|
||||
for (var parent : layer.parents()) {
|
||||
discoverAnalysisModules(parent, result);
|
||||
}
|
||||
}
|
||||
|
||||
/** Finds all classes in package across all analysis modules */
|
||||
public static List<Class<?>> getClassesForPackage(String pkgname) throws IOException {
|
||||
final var prefix = pkgname.concat(".");
|
||||
final var classes = new ArrayList<Class<?>>();
|
||||
for (var resolvedModule : ALL_ANALYSIS_MODULES.values()) {
|
||||
final var module = LAYER.findModule(resolvedModule.name()).orElseThrow();
|
||||
try (var reader = resolvedModule.reference().open()) {
|
||||
reader
|
||||
.list()
|
||||
.filter(entry -> entry.endsWith(".class"))
|
||||
.map(entry -> entry.substring(0, entry.length() - 6).replace('/', '.'))
|
||||
.filter(clazzname -> clazzname.startsWith(prefix))
|
||||
.sorted()
|
||||
.map(
|
||||
clazzname ->
|
||||
Objects.requireNonNull(
|
||||
Class.forName(module, clazzname),
|
||||
"Class '" + clazzname + "' not found in module '" + module.getName() + "'"))
|
||||
.forEach(classes::add);
|
||||
}
|
||||
}
|
||||
Assert.assertFalse("No classes found in package:" + pkgname, classes.isEmpty());
|
||||
return classes;
|
||||
}
|
||||
}
|
|
@ -14,15 +14,12 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.lucene.analysis.core;
|
||||
package org.apache.lucene.analysis.tests;
|
||||
|
||||
import java.io.Reader;
|
||||
import java.io.StringReader;
|
||||
import java.lang.reflect.Modifier;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.IdentityHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
@ -34,27 +31,17 @@ import org.apache.lucene.analysis.TokenFilterFactory;
|
|||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.Tokenizer;
|
||||
import org.apache.lucene.analysis.TokenizerFactory;
|
||||
import org.apache.lucene.analysis.core.KeywordTokenizer;
|
||||
import org.apache.lucene.analysis.core.UnicodeWhitespaceTokenizer;
|
||||
import org.apache.lucene.analysis.miscellaneous.PatternKeywordMarkerFilter;
|
||||
import org.apache.lucene.analysis.miscellaneous.SetKeywordMarkerFilter;
|
||||
import org.apache.lucene.analysis.path.ReversePathHierarchyTokenizer;
|
||||
import org.apache.lucene.analysis.sinks.TeeSinkTokenFilter;
|
||||
import org.apache.lucene.analysis.snowball.SnowballFilter;
|
||||
import org.apache.lucene.analysis.sr.SerbianNormalizationRegularFilter;
|
||||
import org.apache.lucene.analysis.util.StringMockResourceLoader;
|
||||
import org.apache.lucene.tests.analysis.CrankyTokenFilter;
|
||||
import org.apache.lucene.tests.analysis.MockCharFilter;
|
||||
import org.apache.lucene.tests.analysis.MockFixedLengthPayloadFilter;
|
||||
import org.apache.lucene.tests.analysis.MockGraphTokenFilter;
|
||||
import org.apache.lucene.tests.analysis.MockHoleInjectingTokenFilter;
|
||||
import org.apache.lucene.tests.analysis.MockLowerCaseFilter;
|
||||
import org.apache.lucene.tests.analysis.MockRandomLookaheadTokenFilter;
|
||||
import org.apache.lucene.tests.analysis.MockSynonymFilter;
|
||||
import org.apache.lucene.tests.analysis.MockTokenFilter;
|
||||
import org.apache.lucene.tests.analysis.MockTokenizer;
|
||||
import org.apache.lucene.tests.analysis.MockVariableLengthPayloadFilter;
|
||||
import org.apache.lucene.tests.analysis.SimplePayloadFilter;
|
||||
import org.apache.lucene.tests.analysis.ValidatingTokenFilter;
|
||||
import org.apache.lucene.analysis.stempel.StempelFilter;
|
||||
import org.apache.lucene.tests.util.LuceneTestCase;
|
||||
import org.apache.lucene.tests.util.StringMockResourceLoader;
|
||||
import org.apache.lucene.util.ResourceLoader;
|
||||
import org.apache.lucene.util.ResourceLoaderAware;
|
||||
import org.apache.lucene.util.Version;
|
||||
|
@ -65,71 +52,37 @@ import org.apache.lucene.util.Version;
|
|||
*/
|
||||
public class TestAllAnalyzersHaveFactories extends LuceneTestCase {
|
||||
|
||||
// these are test-only components (e.g. test-framework)
|
||||
private static final Set<Class<?>> testComponents =
|
||||
Collections.newSetFromMap(new IdentityHashMap<Class<?>, Boolean>());
|
||||
|
||||
static {
|
||||
Collections.<Class<?>>addAll(
|
||||
testComponents,
|
||||
MockTokenizer.class,
|
||||
MockCharFilter.class,
|
||||
MockFixedLengthPayloadFilter.class,
|
||||
MockGraphTokenFilter.class,
|
||||
MockHoleInjectingTokenFilter.class,
|
||||
MockLowerCaseFilter.class,
|
||||
MockRandomLookaheadTokenFilter.class,
|
||||
MockSynonymFilter.class,
|
||||
MockTokenFilter.class,
|
||||
MockVariableLengthPayloadFilter.class,
|
||||
ValidatingTokenFilter.class,
|
||||
CrankyTokenFilter.class,
|
||||
SimplePayloadFilter.class);
|
||||
}
|
||||
|
||||
// these are 'crazy' components like cachingtokenfilter. does it make sense to add factories for
|
||||
// these?
|
||||
private static final Set<Class<?>> crazyComponents =
|
||||
Collections.newSetFromMap(new IdentityHashMap<Class<?>, Boolean>());
|
||||
|
||||
static {
|
||||
Collections.<Class<?>>addAll(
|
||||
crazyComponents, CachingTokenFilter.class, TeeSinkTokenFilter.class);
|
||||
}
|
||||
Set.of(CachingTokenFilter.class, TeeSinkTokenFilter.class);
|
||||
|
||||
// these are oddly-named (either the actual analyzer, or its factory)
|
||||
// they do actually have factories.
|
||||
// TODO: clean this up!
|
||||
private static final Set<Class<?>> oddlyNamedComponents =
|
||||
Collections.newSetFromMap(new IdentityHashMap<Class<?>, Boolean>());
|
||||
|
||||
static {
|
||||
Collections.<Class<?>>addAll(
|
||||
oddlyNamedComponents,
|
||||
// this is supported via an option to PathHierarchyTokenizer's factory
|
||||
ReversePathHierarchyTokenizer.class,
|
||||
SnowballFilter.class, // this is called SnowballPorterFilterFactory
|
||||
PatternKeywordMarkerFilter.class,
|
||||
SetKeywordMarkerFilter.class,
|
||||
UnicodeWhitespaceTokenizer.class, // a supported option via WhitespaceTokenizerFactory
|
||||
// class from core, but StopFilterFactory creates one from this module
|
||||
org.apache.lucene.analysis.StopFilter.class,
|
||||
// class from core, but LowerCaseFilterFactory creates one from this module
|
||||
org.apache.lucene.analysis.LowerCaseFilter.class);
|
||||
}
|
||||
Set.of(
|
||||
// this is supported via an option to PathHierarchyTokenizer's factory
|
||||
ReversePathHierarchyTokenizer.class,
|
||||
SnowballFilter.class, // this is called SnowballPorterFilterFactory
|
||||
StempelFilter.class, // this is called StempelPolishStemFilterFactory
|
||||
PatternKeywordMarkerFilter.class,
|
||||
SetKeywordMarkerFilter.class,
|
||||
UnicodeWhitespaceTokenizer.class, // a supported option via WhitespaceTokenizerFactory
|
||||
// class from core, but StopFilterFactory creates one from this module
|
||||
org.apache.lucene.analysis.StopFilter.class,
|
||||
// class from core, but LowerCaseFilterFactory creates one from this module
|
||||
org.apache.lucene.analysis.LowerCaseFilter.class);
|
||||
|
||||
// The following token filters are excused from having their factory.
|
||||
private static final Set<Class<?>> tokenFiltersWithoutFactory = new HashSet<>();
|
||||
|
||||
static {
|
||||
tokenFiltersWithoutFactory.add(SerbianNormalizationRegularFilter.class);
|
||||
}
|
||||
private static final Set<Class<?>> tokenFiltersWithoutFactory =
|
||||
Set.of(SerbianNormalizationRegularFilter.class);
|
||||
|
||||
private static final ResourceLoader loader = new StringMockResourceLoader("");
|
||||
|
||||
public void test() throws Exception {
|
||||
List<Class<?>> analysisClasses =
|
||||
TestRandomChains.getClassesForPackage("org.apache.lucene.analysis");
|
||||
ModuleClassDiscovery.getClassesForPackage("org.apache.lucene.analysis");
|
||||
|
||||
for (final Class<?> c : analysisClasses) {
|
||||
final int modifiers = c.getModifiers();
|
||||
|
@ -141,7 +94,6 @@ public class TestAllAnalyzersHaveFactories extends LuceneTestCase {
|
|||
|| c.isAnonymousClass()
|
||||
|| c.isMemberClass()
|
||||
|| c.isInterface()
|
||||
|| testComponents.contains(c)
|
||||
|| crazyComponents.contains(c)
|
||||
|| oddlyNamedComponents.contains(c)
|
||||
|| tokenFiltersWithoutFactory.contains(c)
|
|
@ -0,0 +1,961 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.analysis.tests;
|
||||
|
||||
import com.ibm.icu.text.Normalizer2;
|
||||
import com.ibm.icu.text.Transliterator;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.Reader;
|
||||
import java.io.StringReader;
|
||||
import java.lang.reflect.Constructor;
|
||||
import java.lang.reflect.InvocationTargetException;
|
||||
import java.lang.reflect.Modifier;
|
||||
import java.text.DateFormat;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.IdentityHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Random;
|
||||
import java.util.Set;
|
||||
import java.util.function.Function;
|
||||
import java.util.function.Predicate;
|
||||
import java.util.regex.Pattern;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.commons.codec.Encoder;
|
||||
import org.apache.commons.codec.language.Caverphone2;
|
||||
import org.apache.commons.codec.language.ColognePhonetic;
|
||||
import org.apache.commons.codec.language.DoubleMetaphone;
|
||||
import org.apache.commons.codec.language.Metaphone;
|
||||
import org.apache.commons.codec.language.Nysiis;
|
||||
import org.apache.commons.codec.language.RefinedSoundex;
|
||||
import org.apache.commons.codec.language.Soundex;
|
||||
import org.apache.commons.codec.language.bm.PhoneticEngine;
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.CharArrayMap;
|
||||
import org.apache.lucene.analysis.CharArraySet;
|
||||
import org.apache.lucene.analysis.CharFilter;
|
||||
import org.apache.lucene.analysis.TokenFilter;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
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.hyphenation.HyphenationTree;
|
||||
import org.apache.lucene.analysis.core.FlattenGraphFilter;
|
||||
import org.apache.lucene.analysis.hunspell.Dictionary;
|
||||
import org.apache.lucene.analysis.icu.segmentation.DefaultICUTokenizerConfig;
|
||||
import org.apache.lucene.analysis.icu.segmentation.ICUTokenizerConfig;
|
||||
import org.apache.lucene.analysis.ja.JapaneseCompletionFilter;
|
||||
import org.apache.lucene.analysis.ja.JapaneseTokenizer;
|
||||
import org.apache.lucene.analysis.ko.KoreanTokenizer;
|
||||
import org.apache.lucene.analysis.minhash.MinHashFilter;
|
||||
import org.apache.lucene.analysis.miscellaneous.ConcatenateGraphFilter;
|
||||
import org.apache.lucene.analysis.miscellaneous.ConditionalTokenFilter;
|
||||
import org.apache.lucene.analysis.miscellaneous.FingerprintFilter;
|
||||
import org.apache.lucene.analysis.miscellaneous.LimitTokenCountFilter;
|
||||
import org.apache.lucene.analysis.miscellaneous.LimitTokenOffsetFilter;
|
||||
import org.apache.lucene.analysis.miscellaneous.LimitTokenPositionFilter;
|
||||
import org.apache.lucene.analysis.miscellaneous.StemmerOverrideFilter;
|
||||
import org.apache.lucene.analysis.miscellaneous.StemmerOverrideFilter.StemmerOverrideMap;
|
||||
import org.apache.lucene.analysis.pattern.PatternTypingFilter;
|
||||
import org.apache.lucene.analysis.payloads.IdentityEncoder;
|
||||
import org.apache.lucene.analysis.payloads.PayloadEncoder;
|
||||
import org.apache.lucene.analysis.pl.PolishAnalyzer;
|
||||
import org.apache.lucene.analysis.shingle.FixedShingleFilter;
|
||||
import org.apache.lucene.analysis.shingle.ShingleFilter;
|
||||
import org.apache.lucene.analysis.standard.StandardTokenizer;
|
||||
import org.apache.lucene.analysis.stempel.StempelStemmer;
|
||||
import org.apache.lucene.analysis.synonym.SynonymMap;
|
||||
import org.apache.lucene.store.ByteBuffersDirectory;
|
||||
import org.apache.lucene.tests.analysis.BaseTokenStreamTestCase;
|
||||
import org.apache.lucene.tests.analysis.MockTokenFilter;
|
||||
import org.apache.lucene.tests.analysis.MockTokenizer;
|
||||
import org.apache.lucene.tests.analysis.ValidatingTokenFilter;
|
||||
import org.apache.lucene.tests.util.Rethrow;
|
||||
import org.apache.lucene.tests.util.TestUtil;
|
||||
import org.apache.lucene.tests.util.automaton.AutomatonTestUtil;
|
||||
import org.apache.lucene.util.AttributeFactory;
|
||||
import org.apache.lucene.util.AttributeSource;
|
||||
import org.apache.lucene.util.CharsRef;
|
||||
import org.apache.lucene.util.IgnoreRandomChains;
|
||||
import org.apache.lucene.util.Version;
|
||||
import org.apache.lucene.util.automaton.Automaton;
|
||||
import org.apache.lucene.util.automaton.CharacterRunAutomaton;
|
||||
import org.apache.lucene.util.automaton.Operations;
|
||||
import org.apache.lucene.util.automaton.RegExp;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.tartarus.snowball.SnowballStemmer;
|
||||
import org.xml.sax.InputSource;
|
||||
|
||||
/** tests random analysis chains */
|
||||
public class TestRandomChains extends BaseTokenStreamTestCase {
|
||||
|
||||
static List<Constructor<? extends Tokenizer>> tokenizers;
|
||||
static List<Constructor<? extends TokenFilter>> tokenfilters;
|
||||
static List<Constructor<? extends CharFilter>> charfilters;
|
||||
|
||||
static List<Class<? extends SnowballStemmer>> snowballStemmers;
|
||||
|
||||
private static final Set<Class<?>> avoidConditionals =
|
||||
Set.of(
|
||||
FingerprintFilter.class,
|
||||
MinHashFilter.class,
|
||||
ConcatenateGraphFilter.class,
|
||||
// ShingleFilter doesn't handle input graphs correctly, so wrapping it in a condition can
|
||||
// expose inconsistent offsets
|
||||
// https://issues.apache.org/jira/browse/LUCENE-4170
|
||||
ShingleFilter.class,
|
||||
FixedShingleFilter.class,
|
||||
// FlattenGraphFilter changes the output graph entirely, so wrapping it in a condition
|
||||
// can break position lengths
|
||||
FlattenGraphFilter.class,
|
||||
// LimitToken*Filters don't set end offsets correctly
|
||||
LimitTokenOffsetFilter.class,
|
||||
LimitTokenCountFilter.class,
|
||||
LimitTokenPositionFilter.class);
|
||||
|
||||
private static final Map<Constructor<?>, Predicate<Object[]>> brokenConstructors;
|
||||
|
||||
static {
|
||||
try {
|
||||
final Map<Constructor<?>, Predicate<Object[]>> map = new HashMap<>();
|
||||
// LimitToken*Filter can only use special ctor when last arg is true
|
||||
for (final var c :
|
||||
List.of(
|
||||
LimitTokenCountFilter.class,
|
||||
LimitTokenOffsetFilter.class,
|
||||
LimitTokenPositionFilter.class)) {
|
||||
map.put(
|
||||
c.getConstructor(TokenStream.class, int.class, boolean.class),
|
||||
args -> {
|
||||
assert args.length == 3;
|
||||
return false == ((Boolean) args[2]); // args are broken if consumeAllTokens is false
|
||||
});
|
||||
}
|
||||
brokenConstructors = Collections.unmodifiableMap(map);
|
||||
} catch (Exception e) {
|
||||
throw new Error(e);
|
||||
}
|
||||
}
|
||||
|
||||
private static final Map<Class<?>, Function<Random, Object>> argProducers =
|
||||
Collections.unmodifiableMap(
|
||||
new IdentityHashMap<Class<?>, Function<Random, Object>>() {
|
||||
{
|
||||
put(
|
||||
int.class,
|
||||
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, -50, 50));
|
||||
});
|
||||
put(
|
||||
char.class,
|
||||
random -> {
|
||||
// TODO: fix any filters that care to throw IAE instead.
|
||||
// also add a unicode validating filter to validate termAtt?
|
||||
// return Character.valueOf((char)random.nextInt(65536));
|
||||
while (true) {
|
||||
char c = (char) random.nextInt(65536);
|
||||
if (c < '\uD800' || c > '\uDFFF') {
|
||||
return Character.valueOf(c);
|
||||
}
|
||||
}
|
||||
});
|
||||
put(float.class, Random::nextFloat);
|
||||
put(boolean.class, Random::nextBoolean);
|
||||
put(byte.class, random -> (byte) random.nextInt(256));
|
||||
put(
|
||||
byte[].class,
|
||||
random -> {
|
||||
byte[] bytes = new byte[random.nextInt(256)];
|
||||
random.nextBytes(bytes);
|
||||
return bytes;
|
||||
});
|
||||
put(Random.class, random -> new Random(random.nextLong()));
|
||||
put(Version.class, random -> Version.LATEST);
|
||||
put(AttributeFactory.class, BaseTokenStreamTestCase::newAttributeFactory);
|
||||
put(AttributeSource.class, random -> null); // force IAE/NPE
|
||||
put(
|
||||
Set.class,
|
||||
random -> {
|
||||
// TypeTokenFilter
|
||||
Set<String> set = new HashSet<>();
|
||||
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,
|
||||
random -> {
|
||||
// CapitalizationFilter
|
||||
Collection<char[]> col = new ArrayList<>();
|
||||
int num = random.nextInt(5);
|
||||
for (int i = 0; i < num; i++) {
|
||||
col.add(TestUtil.randomSimpleString(random).toCharArray());
|
||||
}
|
||||
return col;
|
||||
});
|
||||
put(
|
||||
CharArraySet.class,
|
||||
random -> {
|
||||
int num = random.nextInt(10);
|
||||
CharArraySet set = new CharArraySet(num, random.nextBoolean());
|
||||
for (int i = 0; i < num; i++) {
|
||||
// TODO: make nastier
|
||||
set.add(TestUtil.randomSimpleString(random));
|
||||
}
|
||||
return set;
|
||||
});
|
||||
// TODO: don't want to make the exponentially slow ones Dawid documents
|
||||
// in TestPatternReplaceFilter, so dont use truly random patterns (for now)
|
||||
put(Pattern.class, random -> Pattern.compile("a"));
|
||||
put(
|
||||
Pattern[].class,
|
||||
random ->
|
||||
new Pattern[] {Pattern.compile("([a-z]+)"), Pattern.compile("([0-9]+)")});
|
||||
put(
|
||||
PayloadEncoder.class,
|
||||
random ->
|
||||
new IdentityEncoder()); // the other encoders will throw exceptions if tokens
|
||||
// arent numbers?
|
||||
put(
|
||||
Dictionary.class,
|
||||
random -> {
|
||||
// TODO: make nastier
|
||||
InputStream affixStream =
|
||||
TestRandomChains.class.getResourceAsStream("simple.aff");
|
||||
InputStream dictStream =
|
||||
TestRandomChains.class.getResourceAsStream("simple.dic");
|
||||
try {
|
||||
return new Dictionary(
|
||||
new ByteBuffersDirectory(), "dictionary", affixStream, dictStream);
|
||||
} catch (Exception ex) {
|
||||
Rethrow.rethrow(ex);
|
||||
return null; // unreachable code
|
||||
}
|
||||
});
|
||||
put(
|
||||
HyphenationTree.class,
|
||||
random -> {
|
||||
// TODO: make nastier
|
||||
try {
|
||||
InputSource is =
|
||||
new InputSource(
|
||||
TestRandomChains.class.getResource("da_UTF8.xml").toExternalForm());
|
||||
HyphenationTree hyphenator =
|
||||
HyphenationCompoundWordTokenFilter.getHyphenationTree(is);
|
||||
return hyphenator;
|
||||
} catch (Exception ex) {
|
||||
Rethrow.rethrow(ex);
|
||||
return null; // unreachable code
|
||||
}
|
||||
});
|
||||
put(
|
||||
SnowballStemmer.class,
|
||||
random -> {
|
||||
try {
|
||||
var clazz = snowballStemmers.get(random.nextInt(snowballStemmers.size()));
|
||||
return clazz.getConstructor().newInstance();
|
||||
} catch (Exception ex) {
|
||||
Rethrow.rethrow(ex);
|
||||
return null; // unreachable code
|
||||
}
|
||||
});
|
||||
put(
|
||||
String.class,
|
||||
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,
|
||||
random -> {
|
||||
NormalizeCharMap.Builder builder = new NormalizeCharMap.Builder();
|
||||
// we can't add duplicate keys, or NormalizeCharMap gets angry
|
||||
Set<String> keys = new HashSet<>();
|
||||
int num = random.nextInt(5);
|
||||
// System.out.println("NormalizeCharMap=");
|
||||
for (int i = 0; i < num; i++) {
|
||||
String key = TestUtil.randomSimpleString(random);
|
||||
if (!keys.contains(key) && key.length() > 0) {
|
||||
String value = TestUtil.randomSimpleString(random);
|
||||
builder.add(key, value);
|
||||
keys.add(key);
|
||||
// System.out.println("mapping: '" + key + "' => '" + value + "'");
|
||||
}
|
||||
}
|
||||
return builder.build();
|
||||
});
|
||||
put(
|
||||
CharacterRunAutomaton.class,
|
||||
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,
|
||||
random -> {
|
||||
int num = random.nextInt(10);
|
||||
CharArrayMap<String> map = new CharArrayMap<>(num, random.nextBoolean());
|
||||
for (int i = 0; i < num; i++) {
|
||||
// TODO: make nastier
|
||||
map.put(
|
||||
TestUtil.randomSimpleString(random), TestUtil.randomSimpleString(random));
|
||||
}
|
||||
return map;
|
||||
});
|
||||
put(
|
||||
StemmerOverrideMap.class,
|
||||
random -> {
|
||||
int num = random.nextInt(10);
|
||||
StemmerOverrideFilter.Builder builder =
|
||||
new StemmerOverrideFilter.Builder(random.nextBoolean());
|
||||
for (int i = 0; i < num; i++) {
|
||||
String input = "";
|
||||
do {
|
||||
input = TestUtil.randomRealisticUnicodeString(random);
|
||||
} while (input.isEmpty());
|
||||
String out = "";
|
||||
TestUtil.randomSimpleString(random);
|
||||
do {
|
||||
out = TestUtil.randomRealisticUnicodeString(random);
|
||||
} while (out.isEmpty());
|
||||
builder.add(input, out);
|
||||
}
|
||||
try {
|
||||
return builder.build();
|
||||
} catch (Exception ex) {
|
||||
Rethrow.rethrow(ex);
|
||||
return null; // unreachable code
|
||||
}
|
||||
});
|
||||
put(
|
||||
SynonymMap.class,
|
||||
new Function<Random, Object>() {
|
||||
@Override
|
||||
public Object apply(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 ex) {
|
||||
Rethrow.rethrow(ex);
|
||||
return null; // unreachable code
|
||||
}
|
||||
}
|
||||
|
||||
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;
|
||||
}
|
||||
}
|
||||
}
|
||||
});
|
||||
put(
|
||||
DateFormat.class,
|
||||
random -> {
|
||||
if (random.nextBoolean()) return null;
|
||||
return DateFormat.getDateInstance(DateFormat.DEFAULT, randomLocale(random));
|
||||
});
|
||||
put(
|
||||
Automaton.class,
|
||||
random -> {
|
||||
return Operations.determinize(
|
||||
new RegExp(AutomatonTestUtil.randomRegexp(random), RegExp.NONE)
|
||||
.toAutomaton(),
|
||||
Operations.DEFAULT_DETERMINIZE_WORK_LIMIT);
|
||||
});
|
||||
put(
|
||||
PatternTypingFilter.PatternTypingRule[].class,
|
||||
random -> {
|
||||
int numRules = TestUtil.nextInt(random, 1, 3);
|
||||
PatternTypingFilter.PatternTypingRule[] patternTypingRules =
|
||||
new PatternTypingFilter.PatternTypingRule[numRules];
|
||||
for (int i = 0; i < patternTypingRules.length; i++) {
|
||||
String s = TestUtil.randomSimpleString(random, 1, 2);
|
||||
// random regex with one group
|
||||
String regex = s + "(.*)";
|
||||
// pattern rule with a template that accepts one group.
|
||||
patternTypingRules[i] =
|
||||
new PatternTypingFilter.PatternTypingRule(
|
||||
Pattern.compile(regex), TestUtil.nextInt(random, 1, 8), s + "_$1");
|
||||
}
|
||||
return patternTypingRules;
|
||||
});
|
||||
|
||||
// ICU:
|
||||
put(
|
||||
Normalizer2.class,
|
||||
random -> {
|
||||
switch (random.nextInt(5)) {
|
||||
case 0:
|
||||
return Normalizer2.getNFCInstance();
|
||||
case 1:
|
||||
return Normalizer2.getNFDInstance();
|
||||
case 2:
|
||||
return Normalizer2.getNFKCInstance();
|
||||
case 3:
|
||||
return Normalizer2.getNFKDInstance();
|
||||
default:
|
||||
return Normalizer2.getNFKCCasefoldInstance();
|
||||
}
|
||||
});
|
||||
final var icuTransliterators = Collections.list(Transliterator.getAvailableIDs());
|
||||
Collections.sort(icuTransliterators);
|
||||
put(
|
||||
Transliterator.class,
|
||||
random ->
|
||||
Transliterator.getInstance(
|
||||
icuTransliterators.get(random.nextInt(icuTransliterators.size()))));
|
||||
put(
|
||||
ICUTokenizerConfig.class,
|
||||
random ->
|
||||
new DefaultICUTokenizerConfig(random.nextBoolean(), random.nextBoolean()));
|
||||
|
||||
// Kuromoji:
|
||||
final var jaComplFilterModes = JapaneseCompletionFilter.Mode.values();
|
||||
put(
|
||||
JapaneseCompletionFilter.Mode.class,
|
||||
random -> jaComplFilterModes[random.nextInt(jaComplFilterModes.length)]);
|
||||
final var jaTokModes = JapaneseTokenizer.Mode.values();
|
||||
put(
|
||||
JapaneseTokenizer.Mode.class,
|
||||
random -> jaTokModes[random.nextInt(jaTokModes.length)]);
|
||||
put(org.apache.lucene.analysis.ja.dict.UserDictionary.class, random -> null);
|
||||
|
||||
// Nori:
|
||||
final var koComplFilterModes = KoreanTokenizer.DecompoundMode.values();
|
||||
put(
|
||||
KoreanTokenizer.DecompoundMode.class,
|
||||
random -> koComplFilterModes[random.nextInt(koComplFilterModes.length)]);
|
||||
put(org.apache.lucene.analysis.ko.dict.UserDictionary.class, random -> null);
|
||||
|
||||
// Phonetic:
|
||||
final var bmNameTypes = org.apache.commons.codec.language.bm.NameType.values();
|
||||
final var bmRuleTypes =
|
||||
Stream.of(org.apache.commons.codec.language.bm.RuleType.values())
|
||||
.filter(e -> e != org.apache.commons.codec.language.bm.RuleType.RULES)
|
||||
.toArray(org.apache.commons.codec.language.bm.RuleType[]::new);
|
||||
put(
|
||||
PhoneticEngine.class,
|
||||
random ->
|
||||
new PhoneticEngine(
|
||||
bmNameTypes[random.nextInt(bmNameTypes.length)],
|
||||
bmRuleTypes[random.nextInt(bmRuleTypes.length)],
|
||||
random.nextBoolean()));
|
||||
put(
|
||||
Encoder.class,
|
||||
random -> {
|
||||
switch (random.nextInt(7)) {
|
||||
case 0:
|
||||
return new DoubleMetaphone();
|
||||
case 1:
|
||||
return new Metaphone();
|
||||
case 2:
|
||||
return new Soundex();
|
||||
case 3:
|
||||
return new RefinedSoundex();
|
||||
case 4:
|
||||
return new Caverphone2();
|
||||
case 5:
|
||||
return new ColognePhonetic();
|
||||
default:
|
||||
return new Nysiis();
|
||||
}
|
||||
});
|
||||
|
||||
// Stempel
|
||||
put(
|
||||
StempelStemmer.class,
|
||||
random -> new StempelStemmer(PolishAnalyzer.getDefaultTable()));
|
||||
}
|
||||
});
|
||||
|
||||
static final Set<Class<?>> allowedTokenizerArgs = argProducers.keySet(),
|
||||
allowedTokenFilterArgs =
|
||||
union(argProducers.keySet(), List.of(TokenStream.class, CommonGramsFilter.class)),
|
||||
allowedCharFilterArgs = union(argProducers.keySet(), List.of(Reader.class));
|
||||
|
||||
@BeforeClass
|
||||
public static void beforeClass() throws Exception {
|
||||
List<Class<?>> analysisClasses =
|
||||
ModuleClassDiscovery.getClassesForPackage("org.apache.lucene.analysis");
|
||||
tokenizers = new ArrayList<>();
|
||||
tokenfilters = new ArrayList<>();
|
||||
charfilters = new ArrayList<>();
|
||||
for (final Class<?> c : analysisClasses) {
|
||||
final int modifiers = c.getModifiers();
|
||||
if (
|
||||
// don't waste time with abstract classes, deprecated, or @IgnoreRandomChains annotated
|
||||
// classes:
|
||||
Modifier.isAbstract(modifiers)
|
||||
|| !Modifier.isPublic(modifiers)
|
||||
|| c.isSynthetic()
|
||||
|| c.isAnonymousClass()
|
||||
|| c.isMemberClass()
|
||||
|| c.isInterface()
|
||||
|| c.isAnnotationPresent(Deprecated.class)
|
||||
|| c.isAnnotationPresent(IgnoreRandomChains.class)
|
||||
|| !(Tokenizer.class.isAssignableFrom(c)
|
||||
|| TokenFilter.class.isAssignableFrom(c)
|
||||
|| CharFilter.class.isAssignableFrom(c))) {
|
||||
continue;
|
||||
}
|
||||
|
||||
for (final Constructor<?> ctor : c.getConstructors()) {
|
||||
// don't test synthetic, deprecated, or @IgnoreRandomChains annotated ctors, they likely
|
||||
// have known bugs:
|
||||
if (ctor.isSynthetic()
|
||||
|| ctor.isAnnotationPresent(Deprecated.class)
|
||||
|| ctor.isAnnotationPresent(IgnoreRandomChains.class)) {
|
||||
continue;
|
||||
}
|
||||
// conditional filters are tested elsewhere
|
||||
if (ConditionalTokenFilter.class.isAssignableFrom(c)) {
|
||||
continue;
|
||||
}
|
||||
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 (CharFilter.class.isAssignableFrom(c)) {
|
||||
assertTrue(
|
||||
ctor.toGenericString() + " has unsupported parameter types",
|
||||
allowedCharFilterArgs.containsAll(Arrays.asList(ctor.getParameterTypes())));
|
||||
charfilters.add(castConstructor(CharFilter.class, ctor));
|
||||
} else {
|
||||
fail("Cannot get here");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
final Comparator<Constructor<?>> ctorComp = Comparator.comparing(Constructor::toGenericString);
|
||||
Collections.sort(tokenizers, ctorComp);
|
||||
Collections.sort(tokenfilters, ctorComp);
|
||||
Collections.sort(charfilters, ctorComp);
|
||||
if (VERBOSE) {
|
||||
System.out.println("tokenizers = " + tokenizers);
|
||||
System.out.println("tokenfilters = " + tokenfilters);
|
||||
System.out.println("charfilters = " + charfilters);
|
||||
}
|
||||
|
||||
// TODO: Eclipse does not get that cast right, so make explicit:
|
||||
final Function<Class<?>, Class<? extends SnowballStemmer>> stemmerCast =
|
||||
c -> c.asSubclass(SnowballStemmer.class);
|
||||
snowballStemmers =
|
||||
ModuleClassDiscovery.getClassesForPackage("org.tartarus.snowball.ext").stream()
|
||||
.filter(c -> c.getName().endsWith("Stemmer"))
|
||||
.map(stemmerCast)
|
||||
.sorted(Comparator.comparing(Class::getName))
|
||||
.collect(Collectors.toList());
|
||||
if (VERBOSE) {
|
||||
System.out.println("snowballStemmers = " + snowballStemmers);
|
||||
}
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void afterClass() {
|
||||
tokenizers = null;
|
||||
tokenfilters = null;
|
||||
charfilters = null;
|
||||
snowballStemmers = null;
|
||||
}
|
||||
|
||||
/** Creates a static/unmodifiable set from 2 collections as union. */
|
||||
private static <T> Set<T> union(Collection<T> c1, Collection<T> c2) {
|
||||
return Stream.concat(c1.stream(), c2.stream()).collect(Collectors.toUnmodifiableSet());
|
||||
}
|
||||
|
||||
/**
|
||||
* 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<T> castConstructor(Class<T> instanceClazz, Constructor<?> ctor) {
|
||||
return (Constructor<T>) ctor;
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
static <T> T newRandomArg(Random random, Class<T> paramType) {
|
||||
final Function<Random, Object> producer = argProducers.get(paramType);
|
||||
assertNotNull("No producer for arguments of type " + paramType.getName() + " found", producer);
|
||||
return (T) producer.apply(random);
|
||||
}
|
||||
|
||||
static Object[] newTokenizerArgs(Random random, Class<?>[] paramTypes) {
|
||||
Object[] args = new Object[paramTypes.length];
|
||||
for (int i = 0; i < args.length; i++) {
|
||||
Class<?> paramType = paramTypes[i];
|
||||
args[i] = newRandomArg(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 {
|
||||
args[i] = newRandomArg(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) {
|
||||
// TODO: fix this one, thats broken: CommonGramsQueryFilter takes this one explicitly
|
||||
args[i] = new CommonGramsFilter(stream, newRandomArg(random, CharArraySet.class));
|
||||
} else {
|
||||
args[i] = newRandomArg(random, paramType);
|
||||
}
|
||||
}
|
||||
return args;
|
||||
}
|
||||
|
||||
static class MockRandomAnalyzer extends Analyzer {
|
||||
final long seed;
|
||||
|
||||
MockRandomAnalyzer(long seed) {
|
||||
this.seed = seed;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected TokenStreamComponents createComponents(String fieldName) {
|
||||
Random random = new Random(seed);
|
||||
TokenizerSpec tokenizerSpec = newTokenizer(random);
|
||||
// System.out.println("seed=" + seed + ",create tokenizer=" + tokenizerSpec.toString);
|
||||
TokenFilterSpec filterSpec = newFilterChain(random, tokenizerSpec.tokenizer);
|
||||
// System.out.println("seed=" + seed + ",create filter=" + filterSpec.toString);
|
||||
return new TokenStreamComponents(tokenizerSpec.tokenizer, filterSpec.stream);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Reader initReader(String fieldName, Reader reader) {
|
||||
Random random = new Random(seed);
|
||||
CharFilterSpec charfilterspec = newCharFilterChain(random, reader);
|
||||
return charfilterspec.reader;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
Random random = new Random(seed);
|
||||
StringBuilder sb = new StringBuilder();
|
||||
CharFilterSpec charFilterSpec = newCharFilterChain(random, new StringReader(""));
|
||||
sb.append("\ncharfilters=");
|
||||
sb.append(charFilterSpec.toString);
|
||||
// intentional: initReader gets its own separate random
|
||||
random = new Random(seed);
|
||||
TokenizerSpec tokenizerSpec = newTokenizer(random);
|
||||
sb.append("\n");
|
||||
sb.append("tokenizer=");
|
||||
sb.append(tokenizerSpec.toString);
|
||||
TokenFilterSpec tokenFilterSpec = newFilterChain(random, tokenizerSpec.tokenizer);
|
||||
sb.append("\n");
|
||||
sb.append("filters=");
|
||||
sb.append(tokenFilterSpec.toString);
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
private <T> T createComponent(
|
||||
Constructor<T> ctor, Object[] args, StringBuilder descr, boolean isConditional) {
|
||||
try {
|
||||
final T instance = ctor.newInstance(args);
|
||||
/*
|
||||
if (descr.length() > 0) {
|
||||
descr.append(",");
|
||||
}
|
||||
*/
|
||||
descr.append("\n ");
|
||||
if (isConditional) {
|
||||
descr.append("Conditional:");
|
||||
}
|
||||
descr.append(ctor.getDeclaringClass().getName());
|
||||
String params = Arrays.deepToString(args);
|
||||
params = params.substring(1, params.length() - 1);
|
||||
descr.append("(").append(params).append(")");
|
||||
return instance;
|
||||
} catch (InvocationTargetException ite) {
|
||||
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 | InstantiationException iae) {
|
||||
Rethrow.rethrow(iae);
|
||||
}
|
||||
return null; // no success
|
||||
}
|
||||
|
||||
private boolean broken(Constructor<?> ctor, Object[] args) {
|
||||
final Predicate<Object[]> pred = brokenConstructors.get(ctor);
|
||||
return pred != null && pred.test(args);
|
||||
}
|
||||
|
||||
// create a new random tokenizer from classpath
|
||||
private TokenizerSpec newTokenizer(Random random) {
|
||||
TokenizerSpec spec = new TokenizerSpec();
|
||||
while (spec.tokenizer == null) {
|
||||
final Constructor<? extends Tokenizer> ctor =
|
||||
tokenizers.get(random.nextInt(tokenizers.size()));
|
||||
final StringBuilder descr = new StringBuilder();
|
||||
final Object[] args = newTokenizerArgs(random, ctor.getParameterTypes());
|
||||
if (broken(ctor, args)) {
|
||||
continue;
|
||||
}
|
||||
spec.tokenizer = createComponent(ctor, args, descr, false);
|
||||
if (spec.tokenizer != null) {
|
||||
spec.toString = descr.toString();
|
||||
}
|
||||
}
|
||||
return spec;
|
||||
}
|
||||
|
||||
private CharFilterSpec newCharFilterChain(Random random, Reader reader) {
|
||||
CharFilterSpec spec = new CharFilterSpec();
|
||||
spec.reader = reader;
|
||||
StringBuilder descr = new StringBuilder();
|
||||
int numFilters = random.nextInt(3);
|
||||
for (int i = 0; i < numFilters; i++) {
|
||||
while (true) {
|
||||
final Constructor<? extends CharFilter> ctor =
|
||||
charfilters.get(random.nextInt(charfilters.size()));
|
||||
final Object[] args = newCharFilterArgs(random, spec.reader, ctor.getParameterTypes());
|
||||
if (broken(ctor, args)) {
|
||||
continue;
|
||||
}
|
||||
reader = createComponent(ctor, args, descr, false);
|
||||
if (reader != null) {
|
||||
spec.reader = reader;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
spec.toString = descr.toString();
|
||||
return spec;
|
||||
}
|
||||
|
||||
private TokenFilterSpec newFilterChain(Random random, Tokenizer tokenizer) {
|
||||
TokenFilterSpec spec = new TokenFilterSpec();
|
||||
spec.stream = tokenizer;
|
||||
StringBuilder descr = new StringBuilder();
|
||||
int numFilters = random.nextInt(5);
|
||||
for (int i = 0; i < numFilters; i++) {
|
||||
|
||||
// Insert ValidatingTF after each stage so we can
|
||||
// catch problems right after the TF that "caused"
|
||||
// them:
|
||||
spec.stream = new ValidatingTokenFilter(spec.stream, "stage " + i);
|
||||
|
||||
while (true) {
|
||||
final Constructor<? extends TokenFilter> ctor =
|
||||
tokenfilters.get(random.nextInt(tokenfilters.size()));
|
||||
if (random.nextBoolean()
|
||||
&& avoidConditionals.contains(ctor.getDeclaringClass()) == false) {
|
||||
long seed = random.nextLong();
|
||||
spec.stream =
|
||||
new ConditionalTokenFilter(
|
||||
spec.stream,
|
||||
in -> {
|
||||
final Object[] args = newFilterArgs(random, in, ctor.getParameterTypes());
|
||||
if (broken(ctor, args)) {
|
||||
return in;
|
||||
}
|
||||
TokenStream ts = createComponent(ctor, args, descr, true);
|
||||
if (ts == null) {
|
||||
return in;
|
||||
}
|
||||
return ts;
|
||||
}) {
|
||||
Random random = new Random(seed);
|
||||
|
||||
@Override
|
||||
public void reset() throws IOException {
|
||||
super.reset();
|
||||
random = new Random(seed);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean shouldFilter() throws IOException {
|
||||
return random.nextBoolean();
|
||||
}
|
||||
};
|
||||
break;
|
||||
} else {
|
||||
final Object[] args = newFilterArgs(random, spec.stream, ctor.getParameterTypes());
|
||||
if (broken(ctor, args)) {
|
||||
continue;
|
||||
}
|
||||
final TokenFilter flt = createComponent(ctor, args, descr, false);
|
||||
if (flt != null) {
|
||||
spec.stream = flt;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Insert ValidatingTF after each stage so we can
|
||||
// catch problems right after the TF that "caused"
|
||||
// them:
|
||||
spec.stream = new ValidatingTokenFilter(spec.stream, "last stage");
|
||||
|
||||
spec.toString = descr.toString();
|
||||
return spec;
|
||||
}
|
||||
}
|
||||
|
||||
static class TokenizerSpec {
|
||||
Tokenizer tokenizer;
|
||||
String toString;
|
||||
}
|
||||
|
||||
static class TokenFilterSpec {
|
||||
TokenStream stream;
|
||||
String toString;
|
||||
}
|
||||
|
||||
static class CharFilterSpec {
|
||||
Reader reader;
|
||||
String toString;
|
||||
}
|
||||
|
||||
public void testRandomChains() throws Throwable {
|
||||
int numIterations = TEST_NIGHTLY ? atLeast(20) : 3;
|
||||
Random random = random();
|
||||
for (int i = 0; i < numIterations; i++) {
|
||||
try (MockRandomAnalyzer a = new MockRandomAnalyzer(random.nextLong())) {
|
||||
if (VERBOSE) {
|
||||
System.out.println("Creating random analyzer:" + a);
|
||||
}
|
||||
try {
|
||||
checkNormalize(a);
|
||||
checkRandomData(
|
||||
random,
|
||||
a,
|
||||
500 * RANDOM_MULTIPLIER,
|
||||
20,
|
||||
false,
|
||||
false /* We already validate our own offsets... */);
|
||||
} catch (Throwable e) {
|
||||
System.err.println("Exception from random analyzer: " + a);
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void checkNormalize(Analyzer a) {
|
||||
// normalization should not modify characters that may be used for wildcards
|
||||
// or regular expressions
|
||||
String s = "([0-9]+)?*";
|
||||
assertEquals(s, a.normalize("dummy", s).utf8ToString());
|
||||
}
|
||||
|
||||
// we might regret this decision...
|
||||
public void testRandomChainsWithLargeStrings() throws Throwable {
|
||||
int numIterations = TEST_NIGHTLY ? atLeast(20) : 3;
|
||||
Random random = random();
|
||||
for (int i = 0; i < numIterations; i++) {
|
||||
try (MockRandomAnalyzer a = new MockRandomAnalyzer(random.nextLong())) {
|
||||
if (VERBOSE) {
|
||||
System.out.println("Creating random analyzer:" + a);
|
||||
}
|
||||
try {
|
||||
checkRandomData(
|
||||
random,
|
||||
a,
|
||||
50 * RANDOM_MULTIPLIER,
|
||||
80,
|
||||
false,
|
||||
false /* We already validate our own offsets... */);
|
||||
} catch (Throwable e) {
|
||||
System.err.println("Exception from random analyzer: " + a);
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -0,0 +1,20 @@
|
|||
SET UTF-8
|
||||
TRY abcdefghijklmopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ
|
||||
|
||||
SFX A Y 3
|
||||
SFX A 0 e n
|
||||
SFX A 0 e t
|
||||
SFX A 0 e h
|
||||
|
||||
SFX C Y 2
|
||||
SFX C 0 d/C c
|
||||
SFX C 0 c b
|
||||
|
||||
SFX D Y 1
|
||||
SFX D 0 s o
|
||||
|
||||
SFX E Y 1
|
||||
SFX E 0 d o
|
||||
|
||||
PFX B Y 1
|
||||
PFX B 0 s o
|
|
@ -0,0 +1,11 @@
|
|||
9
|
||||
ab/C
|
||||
apach/A
|
||||
foo/D
|
||||
foo/E
|
||||
lucen/A
|
||||
lucene
|
||||
mahout/A
|
||||
moo/E
|
||||
olr/B
|
||||
db
|
|
@ -21,6 +21,7 @@ import org.apache.lucene.analysis.TokenFilter;
|
|||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||
import org.apache.lucene.search.BoostAttribute;
|
||||
import org.apache.lucene.util.IgnoreRandomChains;
|
||||
|
||||
/**
|
||||
* Characters before the delimiter are the "token", those after are the boost.
|
||||
|
@ -30,6 +31,8 @@ import org.apache.lucene.search.BoostAttribute;
|
|||
*
|
||||
* <p>Note make sure your Tokenizer doesn't split on the delimiter, or this won't work
|
||||
*/
|
||||
@IgnoreRandomChains(
|
||||
reason = "requires a special encoded token value, so it may fail with random data")
|
||||
public final class DelimitedBoostTokenFilter extends TokenFilter {
|
||||
private final char delimiter;
|
||||
private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
|
||||
|
|
|
@ -26,6 +26,7 @@ import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
|||
import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.IgnoreRandomChains;
|
||||
|
||||
/**
|
||||
* Forms bigrams of CJK terms that are generated from StandardTokenizer or ICUTokenizer.
|
||||
|
@ -47,6 +48,7 @@ import org.apache.lucene.util.ArrayUtil;
|
|||
*
|
||||
* <p>In all cases, all non-CJK input is passed thru unmodified.
|
||||
*/
|
||||
@IgnoreRandomChains(reason = "LUCENE-8092: doesn't handle graph inputs")
|
||||
public final class CJKBigramFilter extends TokenFilter {
|
||||
// configuration
|
||||
/** bigram flag for Han Ideographs */
|
||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
|||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
|
||||
import org.apache.lucene.util.IgnoreRandomChains;
|
||||
|
||||
/*
|
||||
* TODO: Consider implementing https://issues.apache.org/jira/browse/LUCENE-1688 changes to stop list and associated constructors
|
||||
|
@ -43,10 +44,7 @@ import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
|
|||
* "the-quick" has a term.type() of "gram"
|
||||
* </ul>
|
||||
*/
|
||||
|
||||
/*
|
||||
* Constructors and makeCommonSet based on similar code in StopFilter
|
||||
*/
|
||||
@IgnoreRandomChains(reason = "LUCENE-4983")
|
||||
public final class CommonGramsFilter extends TokenFilter {
|
||||
|
||||
public static final String GRAM_TYPE = "gram";
|
||||
|
|
|
@ -23,6 +23,7 @@ import org.apache.lucene.analysis.TokenFilter;
|
|||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
|
||||
import org.apache.lucene.util.IgnoreRandomChains;
|
||||
|
||||
/**
|
||||
* Wrap a CommonGramsFilter optimizing phrase queries by only returning single words when they are
|
||||
|
@ -42,6 +43,7 @@ import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
|
|||
* See:http://hudson.zones.apache.org/hudson/job/Lucene-trunk/javadoc//all/org/apache/lucene/analysis/TokenStream.html and
|
||||
* http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/analysis/package.html?revision=718798
|
||||
*/
|
||||
@IgnoreRandomChains(reason = "TODO: doesn't handle graph inputs")
|
||||
public final class CommonGramsQueryFilter extends TokenFilter {
|
||||
|
||||
private final TypeAttribute typeAttribute = addAttribute(TypeAttribute.class);
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
package org.apache.lucene.analysis.core;
|
||||
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.util.IgnoreRandomChains;
|
||||
|
||||
/**
|
||||
* Normalizes token text to lower case.
|
||||
|
@ -27,6 +28,7 @@ import org.apache.lucene.analysis.TokenStream;
|
|||
* @see org.apache.lucene.analysis.LowerCaseFilter
|
||||
* @see LowerCaseFilterFactory
|
||||
*/
|
||||
@IgnoreRandomChains(reason = "clones of core's filters")
|
||||
public final class LowerCaseFilter extends org.apache.lucene.analysis.LowerCaseFilter {
|
||||
|
||||
/**
|
||||
|
|
|
@ -18,6 +18,7 @@ package org.apache.lucene.analysis.core;
|
|||
|
||||
import org.apache.lucene.analysis.CharArraySet;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.util.IgnoreRandomChains;
|
||||
|
||||
/**
|
||||
* Removes stop words from a token stream.
|
||||
|
@ -28,6 +29,7 @@ import org.apache.lucene.analysis.TokenStream;
|
|||
* @see org.apache.lucene.analysis.StopFilter
|
||||
* @see StopFilterFactory
|
||||
*/
|
||||
@IgnoreRandomChains(reason = "clones of core's filters")
|
||||
public final class StopFilter extends org.apache.lucene.analysis.StopFilter {
|
||||
|
||||
/**
|
||||
|
|
|
@ -24,6 +24,7 @@ import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
|||
import org.apache.lucene.analysis.tokenattributes.TermFrequencyAttribute;
|
||||
import org.apache.lucene.index.IndexOptions;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.IgnoreRandomChains;
|
||||
|
||||
/**
|
||||
* Characters before the delimiter are the "token", the textual integer after is the term frequency.
|
||||
|
@ -36,6 +37,8 @@ import org.apache.lucene.util.ArrayUtil;
|
|||
*
|
||||
* <p>Note make sure your Tokenizer doesn't split on the delimiter, or this won't work
|
||||
*/
|
||||
@IgnoreRandomChains(
|
||||
reason = "requires a special encoded token value, so it may fail with random data")
|
||||
public final class DelimitedTermFrequencyTokenFilter extends TokenFilter {
|
||||
public static final char DEFAULT_DELIMITER = '|';
|
||||
|
||||
|
|
|
@ -20,6 +20,7 @@ import java.io.IOException;
|
|||
import org.apache.lucene.analysis.*;
|
||||
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.util.IgnoreRandomChains;
|
||||
|
||||
/**
|
||||
* When the plain text is extracted from documents, we will often have many words hyphenated and
|
||||
|
@ -50,6 +51,8 @@ import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
|||
* </fieldtype>
|
||||
* </pre>
|
||||
*/
|
||||
@IgnoreRandomChains(
|
||||
reason = "TODO: doesn't handle graph inputs (or even look at positionIncrement)")
|
||||
public final class HyphenatedWordsFilter extends TokenFilter {
|
||||
|
||||
private final CharTermAttribute termAttribute = addAttribute(CharTermAttribute.class);
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.lucene.analysis.miscellaneous;
|
|||
import java.io.IOException;
|
||||
import org.apache.lucene.analysis.TokenFilter;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.util.IgnoreRandomChains;
|
||||
|
||||
/**
|
||||
* This TokenFilter limits the number of tokens while indexing. It is a replacement for the maximum
|
||||
|
@ -45,6 +46,7 @@ public final class LimitTokenCountFilter extends TokenFilter {
|
|||
*
|
||||
* @see #LimitTokenCountFilter(TokenStream,int,boolean)
|
||||
*/
|
||||
@IgnoreRandomChains(reason = "all tokens must be consumed")
|
||||
public LimitTokenCountFilter(TokenStream in, int maxTokenCount) {
|
||||
this(in, maxTokenCount, false);
|
||||
}
|
||||
|
|
|
@ -20,6 +20,7 @@ import java.io.IOException;
|
|||
import org.apache.lucene.analysis.TokenFilter;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.util.IgnoreRandomChains;
|
||||
|
||||
/**
|
||||
* Lets all tokens pass through until it sees one with a start offset <= a configured limit,
|
||||
|
@ -46,6 +47,7 @@ public final class LimitTokenOffsetFilter extends TokenFilter {
|
|||
*
|
||||
* @param maxStartOffset the maximum start offset allowed
|
||||
*/
|
||||
@IgnoreRandomChains(reason = "all tokens must be consumed")
|
||||
public LimitTokenOffsetFilter(TokenStream input, int maxStartOffset) {
|
||||
this(input, maxStartOffset, false);
|
||||
}
|
||||
|
|
|
@ -20,6 +20,7 @@ import java.io.IOException;
|
|||
import org.apache.lucene.analysis.TokenFilter;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||
import org.apache.lucene.util.IgnoreRandomChains;
|
||||
|
||||
/**
|
||||
* This TokenFilter limits its emitted tokens to those with positions that are not greater than the
|
||||
|
@ -50,6 +51,7 @@ public final class LimitTokenPositionFilter extends TokenFilter {
|
|||
* @param maxTokenPosition max position of tokens to produce (1st token always has position 1)
|
||||
* @see #LimitTokenPositionFilter(TokenStream,int,boolean)
|
||||
*/
|
||||
@IgnoreRandomChains(reason = "all tokens must be consumed")
|
||||
public LimitTokenPositionFilter(TokenStream in, int maxTokenPosition) {
|
||||
this(in, maxTokenPosition, false);
|
||||
}
|
||||
|
|
|
@ -30,6 +30,7 @@ import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
|
|||
import org.apache.lucene.search.PhraseQuery;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.AttributeSource;
|
||||
import org.apache.lucene.util.IgnoreRandomChains;
|
||||
import org.apache.lucene.util.InPlaceMergeSorter;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
|
@ -83,6 +84,7 @@ import org.apache.lucene.util.RamUsageEstimator;
|
|||
* StandardTokenizer} immediately removes many intra-word delimiters, it is recommended that this
|
||||
* filter be used after a tokenizer that does not do this (such as {@link WhitespaceTokenizer}).
|
||||
*/
|
||||
@IgnoreRandomChains(reason = "Cannot correct offsets when a char filter had changed them")
|
||||
public final class WordDelimiterGraphFilter extends TokenFilter {
|
||||
|
||||
/**
|
||||
|
|
|
@ -22,6 +22,7 @@ import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
|||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||
import org.apache.lucene.util.AttributeFactory;
|
||||
import org.apache.lucene.util.IgnoreRandomChains;
|
||||
|
||||
/**
|
||||
* Tokenizer for path-like hierarchies.
|
||||
|
@ -40,6 +41,7 @@ import org.apache.lucene.util.AttributeFactory;
|
|||
* /something/something/else
|
||||
* </pre>
|
||||
*/
|
||||
@IgnoreRandomChains(reason = "broken offsets")
|
||||
public class PathHierarchyTokenizer extends Tokenizer {
|
||||
|
||||
public PathHierarchyTokenizer() {
|
||||
|
|
|
@ -24,6 +24,7 @@ import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
|||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||
import org.apache.lucene.util.AttributeFactory;
|
||||
import org.apache.lucene.util.IgnoreRandomChains;
|
||||
|
||||
/**
|
||||
* Tokenizer for domain-like hierarchies.
|
||||
|
@ -43,6 +44,7 @@ import org.apache.lucene.util.AttributeFactory;
|
|||
* uk
|
||||
* </pre>
|
||||
*/
|
||||
@IgnoreRandomChains(reason = "broken offsets")
|
||||
public class ReversePathHierarchyTokenizer extends Tokenizer {
|
||||
|
||||
public ReversePathHierarchyTokenizer() {
|
||||
|
|
|
@ -26,6 +26,7 @@ import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
|||
import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
|
||||
import org.apache.lucene.util.AttributeFactory;
|
||||
import org.apache.lucene.util.AttributeSource;
|
||||
import org.apache.lucene.util.IgnoreRandomChains;
|
||||
|
||||
/**
|
||||
* Extension of StandardTokenizer that is aware of Wikipedia syntax. It is based off of the
|
||||
|
@ -34,6 +35,7 @@ import org.apache.lucene.util.AttributeSource;
|
|||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
@IgnoreRandomChains(reason = "TODO: it seems to mess up offsets!?")
|
||||
public final class WikipediaTokenizer extends Tokenizer {
|
||||
public static final String INTERNAL_LINK = "il";
|
||||
public static final String EXTERNAL_LINK = "el";
|
||||
|
|
|
@ -69,7 +69,7 @@ public class TestBugInSomething extends BaseTokenStreamTestCase {
|
|||
protected Reader initReader(String fieldName, Reader reader) {
|
||||
reader = new MockCharFilter(reader, 0);
|
||||
reader = new MappingCharFilter(map, reader);
|
||||
reader = new TestRandomChains.CheckThatYouDidntReadAnythingReaderWrapper(reader);
|
||||
reader = new CheckThatYouDidntReadAnythingReaderWrapper(reader);
|
||||
return reader;
|
||||
}
|
||||
};
|
||||
|
@ -137,7 +137,7 @@ public class TestBugInSomething extends BaseTokenStreamTestCase {
|
|||
};
|
||||
|
||||
public void testWrapping() throws Exception {
|
||||
CharFilter cs = new TestRandomChains.CheckThatYouDidntReadAnythingReaderWrapper(wrappedStream);
|
||||
CharFilter cs = new CheckThatYouDidntReadAnythingReaderWrapper(wrappedStream);
|
||||
Exception expected =
|
||||
expectThrows(
|
||||
Exception.class,
|
||||
|
@ -221,6 +221,69 @@ public class TestBugInSomething extends BaseTokenStreamTestCase {
|
|||
|
||||
// todo: test framework?
|
||||
|
||||
static class CheckThatYouDidntReadAnythingReaderWrapper extends CharFilter {
|
||||
boolean readSomething;
|
||||
|
||||
CheckThatYouDidntReadAnythingReaderWrapper(Reader in) {
|
||||
super(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int correct(int currentOff) {
|
||||
return currentOff; // we don't change any offsets
|
||||
}
|
||||
|
||||
@Override
|
||||
public int read(char[] cbuf, int off, int len) throws IOException {
|
||||
readSomething = true;
|
||||
return input.read(cbuf, off, len);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int read() throws IOException {
|
||||
readSomething = true;
|
||||
return input.read();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int read(CharBuffer target) throws IOException {
|
||||
readSomething = true;
|
||||
return input.read(target);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int read(char[] cbuf) throws IOException {
|
||||
readSomething = true;
|
||||
return input.read(cbuf);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long skip(long n) throws IOException {
|
||||
readSomething = true;
|
||||
return input.skip(n);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void mark(int readAheadLimit) throws IOException {
|
||||
input.mark(readAheadLimit);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean markSupported() {
|
||||
return input.markSupported();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean ready() throws IOException {
|
||||
return input.ready();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void reset() throws IOException {
|
||||
input.reset();
|
||||
}
|
||||
}
|
||||
|
||||
static final class SopTokenFilter extends TokenFilter {
|
||||
|
||||
SopTokenFilter(TokenStream input) {
|
||||
|
|
|
@ -33,10 +33,10 @@ import org.apache.lucene.analysis.Tokenizer;
|
|||
import org.apache.lucene.analysis.TokenizerFactory;
|
||||
import org.apache.lucene.analysis.boost.DelimitedBoostTokenFilterFactory;
|
||||
import org.apache.lucene.analysis.miscellaneous.DelimitedTermFrequencyTokenFilterFactory;
|
||||
import org.apache.lucene.analysis.util.StringMockResourceLoader;
|
||||
import org.apache.lucene.tests.analysis.BaseTokenStreamTestCase;
|
||||
import org.apache.lucene.tests.analysis.MockTokenizer;
|
||||
import org.apache.lucene.tests.util.LuceneTestCase.Nightly;
|
||||
import org.apache.lucene.tests.util.StringMockResourceLoader;
|
||||
import org.apache.lucene.util.AttributeFactory;
|
||||
import org.apache.lucene.util.ResourceLoaderAware;
|
||||
import org.apache.lucene.util.Version;
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -19,8 +19,8 @@ package org.apache.lucene.analysis.miscellaneous;
|
|||
import java.io.Reader;
|
||||
import java.io.StringReader;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.util.StringMockResourceLoader;
|
||||
import org.apache.lucene.tests.analysis.BaseTokenStreamFactoryTestCase;
|
||||
import org.apache.lucene.tests.util.StringMockResourceLoader;
|
||||
import org.apache.lucene.util.Version;
|
||||
|
||||
/** Simple tests to ensure the keyword marker filter factory is working. */
|
||||
|
|
|
@ -19,8 +19,8 @@ package org.apache.lucene.analysis.miscellaneous;
|
|||
import java.io.Reader;
|
||||
import java.io.StringReader;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.util.StringMockResourceLoader;
|
||||
import org.apache.lucene.tests.analysis.BaseTokenStreamFactoryTestCase;
|
||||
import org.apache.lucene.tests.util.StringMockResourceLoader;
|
||||
import org.apache.lucene.util.Version;
|
||||
|
||||
/** Simple tests to ensure the stemmer override filter factory is working. */
|
||||
|
|
|
@ -19,10 +19,10 @@ package org.apache.lucene.analysis.pattern;
|
|||
|
||||
import org.apache.lucene.analysis.TokenFilterFactory;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.util.StringMockResourceLoader;
|
||||
import org.apache.lucene.tests.analysis.BaseTokenStreamFactoryTestCase;
|
||||
import org.apache.lucene.tests.analysis.CannedTokenStream;
|
||||
import org.apache.lucene.tests.analysis.Token;
|
||||
import org.apache.lucene.tests.util.StringMockResourceLoader;
|
||||
import org.apache.lucene.util.Version;
|
||||
|
||||
/** This test just ensures the factory works */
|
||||
|
|
|
@ -19,8 +19,8 @@ package org.apache.lucene.analysis.snowball;
|
|||
import java.io.Reader;
|
||||
import java.io.StringReader;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.util.StringMockResourceLoader;
|
||||
import org.apache.lucene.tests.analysis.BaseTokenStreamFactoryTestCase;
|
||||
import org.apache.lucene.tests.util.StringMockResourceLoader;
|
||||
import org.apache.lucene.util.Version;
|
||||
import org.tartarus.snowball.ext.EnglishStemmer;
|
||||
|
||||
|
|
|
@ -19,8 +19,8 @@ package org.apache.lucene.analysis.synonym;
|
|||
import java.io.Reader;
|
||||
import java.io.StringReader;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.util.StringMockResourceLoader;
|
||||
import org.apache.lucene.tests.analysis.BaseTokenStreamFactoryTestCase;
|
||||
import org.apache.lucene.tests.util.StringMockResourceLoader;
|
||||
import org.apache.lucene.util.Version;
|
||||
|
||||
/** @since solr 1.4 */
|
||||
|
|
|
@ -22,8 +22,8 @@ import org.apache.lucene.analysis.TokenFilterFactory;
|
|||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.cjk.CJKAnalyzer;
|
||||
import org.apache.lucene.analysis.pattern.PatternTokenizerFactory;
|
||||
import org.apache.lucene.analysis.util.StringMockResourceLoader;
|
||||
import org.apache.lucene.tests.analysis.BaseTokenStreamFactoryTestCase;
|
||||
import org.apache.lucene.tests.util.StringMockResourceLoader;
|
||||
import org.apache.lucene.util.Version;
|
||||
|
||||
@Deprecated
|
||||
|
|
|
@ -26,6 +26,7 @@ import org.apache.lucene.analysis.CharArraySet;
|
|||
import org.apache.lucene.analysis.TokenFilterFactory;
|
||||
import org.apache.lucene.analysis.WordlistLoader;
|
||||
import org.apache.lucene.tests.util.LuceneTestCase;
|
||||
import org.apache.lucene.tests.util.StringMockResourceLoader;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.ResourceLoader;
|
||||
|
||||
|
|
|
@ -30,6 +30,7 @@ import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
|||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||
import org.apache.lucene.util.CharsRef;
|
||||
import org.apache.lucene.util.CharsRefBuilder;
|
||||
import org.apache.lucene.util.IgnoreRandomChains;
|
||||
|
||||
/**
|
||||
* A {@link org.apache.lucene.analysis.TokenFilter} that adds Japanese romanized tokens to the term
|
||||
|
@ -54,6 +55,7 @@ import org.apache.lucene.util.CharsRefBuilder;
|
|||
* WIDTH NORMALIZATION IS NOT PERFORMED, THIS DOES NOT WORK AS EXPECTED. See also: {@link
|
||||
* JapaneseCompletionAnalyzer}.
|
||||
*/
|
||||
@IgnoreRandomChains(reason = "LUCENE-10363: fails with incorrect offsets")
|
||||
public final class JapaneseCompletionFilter extends TokenFilter {
|
||||
public static final Mode DEFAULT_MODE = Mode.INDEX;
|
||||
|
||||
|
|
|
@ -20,6 +20,7 @@ import java.io.IOException;
|
|||
import java.io.Reader;
|
||||
import org.apache.lucene.analysis.CharFilter;
|
||||
import org.apache.lucene.analysis.util.RollingCharBuffer;
|
||||
import org.apache.lucene.util.IgnoreRandomChains;
|
||||
|
||||
/**
|
||||
* Normalizes Japanese horizontal iteration marks (odoriji) to their expanded form.
|
||||
|
@ -36,6 +37,8 @@ import org.apache.lucene.analysis.util.RollingCharBuffer;
|
|||
* reached in order to not keep a copy of the character stream in memory. Vertical iteration marks,
|
||||
* which are even rarer than horizontal iteration marks in contemporary Japanese, are unsupported.
|
||||
*/
|
||||
@IgnoreRandomChains(
|
||||
reason = "LUCENE-10358: fails with incorrect offsets or causes IndexOutOfBounds")
|
||||
public class JapaneseIterationMarkCharFilter extends CharFilter {
|
||||
|
||||
/** Normalize kanji iteration marks by default */
|
||||
|
|
|
@ -45,6 +45,9 @@ public final class JapaneseKatakanaStemFilter extends TokenFilter {
|
|||
|
||||
public JapaneseKatakanaStemFilter(TokenStream input, int minimumLength) {
|
||||
super(input);
|
||||
if (minimumLength < 1) {
|
||||
throw new IllegalArgumentException("minimumLength must be >=1");
|
||||
}
|
||||
this.minimumKatakanaLength = minimumLength;
|
||||
}
|
||||
|
||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.lucene.analysis.tokenattributes.KeywordAttribute;
|
|||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
|
||||
import org.apache.lucene.util.IgnoreRandomChains;
|
||||
|
||||
/**
|
||||
* A {@link TokenFilter} that normalizes Japanese numbers (kansūji) to regular Arabic decimal
|
||||
|
@ -82,6 +83,7 @@ import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
|
|||
* <p>Japanese formal numbers (daiji), accounting numbers and decimal fractions are currently not
|
||||
* supported.
|
||||
*/
|
||||
@IgnoreRandomChains(reason = "LUCENE-10362: fails with incorrect offsets")
|
||||
public class JapaneseNumberFilter extends TokenFilter {
|
||||
|
||||
private final CharTermAttribute termAttr = addAttribute(CharTermAttribute.class);
|
||||
|
|
|
@ -41,6 +41,7 @@ import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
|
|||
import org.apache.lucene.analysis.util.RollingCharBuffer;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.AttributeFactory;
|
||||
import org.apache.lucene.util.IgnoreRandomChains;
|
||||
import org.apache.lucene.util.IntsRef;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.fst.FST;
|
||||
|
@ -275,6 +276,7 @@ public final class JapaneseTokenizer extends Tokenizer {
|
|||
* @param mode tokenization mode.
|
||||
* @lucene.experimental
|
||||
*/
|
||||
@IgnoreRandomChains(reason = "Parameters are too complex to be tested")
|
||||
public JapaneseTokenizer(
|
||||
AttributeFactory factory,
|
||||
TokenInfoDictionary systemDictionary,
|
||||
|
|
|
@ -1,46 +0,0 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.analysis.ja;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import org.apache.lucene.util.ResourceLoader;
|
||||
|
||||
/** Fake resource loader for tests: works if you want to fake reading a single file */
|
||||
class StringMockResourceLoader implements ResourceLoader {
|
||||
String text;
|
||||
|
||||
public StringMockResourceLoader(String text) {
|
||||
this.text = text;
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> Class<? extends T> findClass(String cname, Class<T> expectedType) {
|
||||
try {
|
||||
return Class.forName(cname).asSubclass(expectedType);
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException("Cannot load class: " + cname, e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public InputStream openResource(String resource) throws IOException {
|
||||
return new ByteArrayInputStream(text.getBytes(StandardCharsets.UTF_8));
|
||||
}
|
||||
}
|
|
@ -36,6 +36,7 @@ import org.apache.lucene.analysis.miscellaneous.DelimitedTermFrequencyTokenFilte
|
|||
import org.apache.lucene.tests.analysis.BaseTokenStreamTestCase;
|
||||
import org.apache.lucene.tests.analysis.MockTokenizer;
|
||||
import org.apache.lucene.tests.util.LuceneTestCase.Nightly;
|
||||
import org.apache.lucene.tests.util.StringMockResourceLoader;
|
||||
import org.apache.lucene.util.AttributeFactory;
|
||||
import org.apache.lucene.util.ResourceLoaderAware;
|
||||
import org.apache.lucene.util.Version;
|
||||
|
|
|
@ -22,6 +22,7 @@ import java.util.HashMap;
|
|||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.Tokenizer;
|
||||
import org.apache.lucene.tests.analysis.BaseTokenStreamTestCase;
|
||||
import org.apache.lucene.tests.util.StringMockResourceLoader;
|
||||
|
||||
/** Simple tests for {@link JapaneseBaseFormFilterFactory} */
|
||||
public class TestJapaneseBaseFormFilterFactory extends BaseTokenStreamTestCase {
|
||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.lucene.analysis.TokenStream;
|
|||
import org.apache.lucene.analysis.Tokenizer;
|
||||
import org.apache.lucene.tests.analysis.BaseTokenStreamTestCase;
|
||||
import org.apache.lucene.tests.analysis.MockTokenizer;
|
||||
import org.apache.lucene.tests.util.StringMockResourceLoader;
|
||||
|
||||
/** Simple tests for {@link JapaneseIterationMarkCharFilterFactory} */
|
||||
public class TestJapaneseIterationMarkCharFilterFactory extends BaseTokenStreamTestCase {
|
||||
|
|
|
@ -22,6 +22,7 @@ import java.util.HashMap;
|
|||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.Tokenizer;
|
||||
import org.apache.lucene.tests.analysis.BaseTokenStreamTestCase;
|
||||
import org.apache.lucene.tests.util.StringMockResourceLoader;
|
||||
|
||||
/** Simple tests for {@link JapaneseKatakanaStemFilterFactory} */
|
||||
public class TestJapaneseKatakanaStemFilterFactory extends BaseTokenStreamTestCase {
|
||||
|
|
|
@ -23,6 +23,7 @@ import java.util.Map;
|
|||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.Tokenizer;
|
||||
import org.apache.lucene.tests.analysis.BaseTokenStreamTestCase;
|
||||
import org.apache.lucene.tests.util.StringMockResourceLoader;
|
||||
|
||||
/** Simple tests for {@link org.apache.lucene.analysis.ja.JapaneseNumberFilterFactory} */
|
||||
public class TestJapaneseNumberFilterFactory extends BaseTokenStreamTestCase {
|
||||
|
|
|
@ -23,6 +23,7 @@ import java.util.Map;
|
|||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.Tokenizer;
|
||||
import org.apache.lucene.tests.analysis.BaseTokenStreamTestCase;
|
||||
import org.apache.lucene.tests.util.StringMockResourceLoader;
|
||||
import org.apache.lucene.util.ClasspathResourceLoader;
|
||||
import org.apache.lucene.util.Version;
|
||||
|
||||
|
|
|
@ -22,6 +22,7 @@ import java.util.HashMap;
|
|||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.Tokenizer;
|
||||
import org.apache.lucene.tests.analysis.BaseTokenStreamTestCase;
|
||||
import org.apache.lucene.tests.util.StringMockResourceLoader;
|
||||
|
||||
/** Simple tests for {@link JapaneseReadingFormFilterFactory} */
|
||||
public class TestJapaneseReadingFormFilterFactory extends BaseTokenStreamTestCase {
|
||||
|
|
|
@ -23,6 +23,7 @@ import java.util.Map;
|
|||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.Tokenizer;
|
||||
import org.apache.lucene.tests.analysis.BaseTokenStreamTestCase;
|
||||
import org.apache.lucene.tests.util.StringMockResourceLoader;
|
||||
|
||||
/** Simple tests for {@link JapaneseTokenizerFactory} */
|
||||
public class TestJapaneseTokenizerFactory extends BaseTokenStreamTestCase {
|
||||
|
|
|
@ -32,6 +32,7 @@ import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
|||
import org.apache.lucene.analysis.tokenattributes.KeywordAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||
import org.apache.lucene.util.CharsRefBuilder;
|
||||
import org.apache.lucene.util.IgnoreRandomChains;
|
||||
|
||||
/**
|
||||
* {@link TokenFilter} using Morfologik library to transform input tokens into lemma and
|
||||
|
@ -73,6 +74,7 @@ public class MorfologikFilter extends TokenFilter {
|
|||
* @param in input token stream.
|
||||
* @param dict Dictionary to use for stemming.
|
||||
*/
|
||||
@IgnoreRandomChains(reason = "No dictionary support yet")
|
||||
public MorfologikFilter(final TokenStream in, final Dictionary dict) {
|
||||
super(in);
|
||||
this.input = in;
|
||||
|
|
|
@ -28,6 +28,7 @@ module org.apache.lucene.analysis.nori {
|
|||
provides org.apache.lucene.analysis.TokenizerFactory with
|
||||
org.apache.lucene.analysis.ko.KoreanTokenizerFactory;
|
||||
provides org.apache.lucene.analysis.TokenFilterFactory with
|
||||
org.apache.lucene.analysis.ko.KoreanNumberFilterFactory,
|
||||
org.apache.lucene.analysis.ko.KoreanPartOfSpeechStopFilterFactory,
|
||||
org.apache.lucene.analysis.ko.KoreanReadingFormFilterFactory;
|
||||
}
|
||||
|
|
|
@ -26,6 +26,7 @@ import org.apache.lucene.analysis.tokenattributes.KeywordAttribute;
|
|||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
|
||||
import org.apache.lucene.util.IgnoreRandomChains;
|
||||
|
||||
/**
|
||||
* A {@link TokenFilter} that normalizes Korean numbers to regular Arabic decimal numbers in
|
||||
|
@ -72,6 +73,7 @@ import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
|
|||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
@IgnoreRandomChains(reason = "LUCENE-10361: KoreanNumberFilter messes up offsets")
|
||||
public class KoreanNumberFilter extends TokenFilter {
|
||||
|
||||
private final CharTermAttribute termAttr = addAttribute(CharTermAttribute.class);
|
||||
|
|
|
@ -40,6 +40,7 @@ import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
|
|||
import org.apache.lucene.analysis.util.RollingCharBuffer;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.AttributeFactory;
|
||||
import org.apache.lucene.util.IgnoreRandomChains;
|
||||
import org.apache.lucene.util.IntsRef;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.fst.FST;
|
||||
|
@ -59,6 +60,7 @@ import org.apache.lucene.util.fst.FST;
|
|||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
@IgnoreRandomChains(reason = "LUCENE-10359: fails with incorrect offsets")
|
||||
public final class KoreanTokenizer extends Tokenizer {
|
||||
|
||||
/** Token type reflecting the original source of this token */
|
||||
|
@ -205,6 +207,7 @@ public final class KoreanTokenizer extends Tokenizer {
|
|||
* @param discardPunctuation true if punctuation tokens should be dropped from the output.
|
||||
* @lucene.experimental
|
||||
*/
|
||||
@IgnoreRandomChains(reason = "Parameters are too complex to be tested")
|
||||
public KoreanTokenizer(
|
||||
AttributeFactory factory,
|
||||
TokenInfoDictionary systemDictionary,
|
||||
|
|
|
@ -13,5 +13,6 @@
|
|||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
|
||||
org.apache.lucene.analysis.ko.KoreanNumberFilterFactory
|
||||
org.apache.lucene.analysis.ko.KoreanPartOfSpeechStopFilterFactory
|
||||
org.apache.lucene.analysis.ko.KoreanReadingFormFilterFactory
|
|
@ -1,46 +0,0 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.analysis.ko;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import org.apache.lucene.util.ResourceLoader;
|
||||
|
||||
/** Fake resource loader for tests: works if you want to fake reading a single file */
|
||||
class StringMockResourceLoader implements ResourceLoader {
|
||||
private String text;
|
||||
|
||||
public StringMockResourceLoader(String text) {
|
||||
this.text = text;
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> Class<? extends T> findClass(String cname, Class<T> expectedType) {
|
||||
try {
|
||||
return Class.forName(cname).asSubclass(expectedType);
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException("Cannot load class: " + cname, e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public InputStream openResource(String resource) throws IOException {
|
||||
return new ByteArrayInputStream(text.getBytes(StandardCharsets.UTF_8));
|
||||
}
|
||||
}
|
|
@ -23,6 +23,7 @@ import java.util.Map;
|
|||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.Tokenizer;
|
||||
import org.apache.lucene.tests.analysis.BaseTokenStreamTestCase;
|
||||
import org.apache.lucene.tests.util.StringMockResourceLoader;
|
||||
|
||||
/** Simple tests for {@link org.apache.lucene.analysis.ko.KoreanNumberFilterFactory} */
|
||||
public class TestKoreanNumberFilterFactory extends BaseTokenStreamTestCase {
|
||||
|
|
|
@ -23,6 +23,7 @@ import java.util.Map;
|
|||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.Tokenizer;
|
||||
import org.apache.lucene.tests.analysis.BaseTokenStreamTestCase;
|
||||
import org.apache.lucene.tests.util.StringMockResourceLoader;
|
||||
import org.apache.lucene.util.Version;
|
||||
|
||||
/** Simple tests for {@link KoreanPartOfSpeechStopFilterFactory} */
|
||||
|
|
|
@ -21,6 +21,7 @@ import java.io.StringReader;
|
|||
import java.util.HashMap;
|
||||
import org.apache.lucene.analysis.Tokenizer;
|
||||
import org.apache.lucene.tests.analysis.BaseTokenStreamTestCase;
|
||||
import org.apache.lucene.tests.util.StringMockResourceLoader;
|
||||
|
||||
/** Simple tests for {@link KoreanReadingFormFilterFactory} */
|
||||
public class TestKoreanReadingFormFilterFactory extends BaseTokenStreamTestCase {
|
||||
|
|
|
@ -23,6 +23,7 @@ import java.util.HashMap;
|
|||
import java.util.Map;
|
||||
import org.apache.lucene.analysis.Tokenizer;
|
||||
import org.apache.lucene.tests.analysis.BaseTokenStreamTestCase;
|
||||
import org.apache.lucene.tests.util.StringMockResourceLoader;
|
||||
|
||||
/** Simple tests for {@link KoreanTokenizerFactory} */
|
||||
public class TestKoreanTokenizerFactory extends BaseTokenStreamTestCase {
|
||||
|
|
|
@ -27,12 +27,14 @@ import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
|||
import org.apache.lucene.analysis.tokenattributes.FlagsAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
|
||||
import org.apache.lucene.util.AttributeSource;
|
||||
import org.apache.lucene.util.IgnoreRandomChains;
|
||||
|
||||
/**
|
||||
* Run OpenNLP chunker. Prerequisite: the OpenNLPTokenizer and OpenNLPPOSFilter must precede this
|
||||
* filter. Tags terms in the TypeAttribute, replacing the POS tags previously put there by
|
||||
* OpenNLPPOSFilter.
|
||||
*/
|
||||
@IgnoreRandomChains(reason = "other filters must precede this one (see docs)")
|
||||
public final class OpenNLPChunkerFilter extends TokenFilter {
|
||||
|
||||
private List<AttributeSource> sentenceTokenAttrs = new ArrayList<>();
|
||||
|
|
|
@ -29,6 +29,7 @@ import org.apache.lucene.analysis.tokenattributes.FlagsAttribute;
|
|||
import org.apache.lucene.analysis.tokenattributes.KeywordAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
|
||||
import org.apache.lucene.util.AttributeSource;
|
||||
import org.apache.lucene.util.IgnoreRandomChains;
|
||||
|
||||
/**
|
||||
* Runs OpenNLP dictionary-based and/or MaxEnt lemmatizers.
|
||||
|
@ -41,6 +42,7 @@ import org.apache.lucene.util.AttributeSource;
|
|||
* <p>The dictionary file must be encoded as UTF-8, with one entry per line, in the form <code>
|
||||
* word[tab]lemma[tab]part-of-speech</code>
|
||||
*/
|
||||
@IgnoreRandomChains(reason = "LUCENE-10352: no dictionary support yet")
|
||||
public class OpenNLPLemmatizerFilter extends TokenFilter {
|
||||
private final NLPLemmatizerOp lemmatizerOp;
|
||||
private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
|
||||
|
|
|
@ -27,8 +27,10 @@ import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
|||
import org.apache.lucene.analysis.tokenattributes.FlagsAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
|
||||
import org.apache.lucene.util.AttributeSource;
|
||||
import org.apache.lucene.util.IgnoreRandomChains;
|
||||
|
||||
/** Run OpenNLP POS tagger. Tags all terms in the TypeAttribute. */
|
||||
@IgnoreRandomChains(reason = "LUCENE-10352: add argument providers for this one")
|
||||
public final class OpenNLPPOSFilter extends TokenFilter {
|
||||
|
||||
private List<AttributeSource> sentenceTokenAttrs = new ArrayList<>();
|
||||
|
|
|
@ -26,12 +26,14 @@ import org.apache.lucene.analysis.tokenattributes.FlagsAttribute;
|
|||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.analysis.util.SegmentingTokenizerBase;
|
||||
import org.apache.lucene.util.AttributeFactory;
|
||||
import org.apache.lucene.util.IgnoreRandomChains;
|
||||
|
||||
/**
|
||||
* Run OpenNLP SentenceDetector and Tokenizer. The last token in each sentence is marked by setting
|
||||
* the {@link #EOS_FLAG_BIT} in the FlagsAttribute; following filters can use this information to
|
||||
* apply operations to tokens one sentence at a time.
|
||||
*/
|
||||
@IgnoreRandomChains(reason = "LUCENE-10352: add argument providers for this one")
|
||||
public final class OpenNLPTokenizer extends SegmentingTokenizerBase {
|
||||
public static int EOS_FLAG_BIT = 1;
|
||||
|
||||
|
|
|
@ -23,7 +23,7 @@ dependencies {
|
|||
moduleApi project(':lucene:core')
|
||||
moduleApi project(':lucene:analysis:common')
|
||||
|
||||
moduleImplementation 'commons-codec:commons-codec'
|
||||
moduleApi 'commons-codec:commons-codec'
|
||||
|
||||
testImplementation project(':lucene:test-framework')
|
||||
}
|
||||
|
|
|
@ -26,6 +26,7 @@ module org.apache.lucene.analysis.phonetic {
|
|||
|
||||
provides org.apache.lucene.analysis.TokenFilterFactory with
|
||||
org.apache.lucene.analysis.phonetic.BeiderMorseFilterFactory,
|
||||
org.apache.lucene.analysis.phonetic.DaitchMokotoffSoundexFilterFactory,
|
||||
org.apache.lucene.analysis.phonetic.DoubleMetaphoneFilterFactory,
|
||||
org.apache.lucene.analysis.phonetic.PhoneticFilterFactory;
|
||||
}
|
||||
|
|
|
@ -26,6 +26,7 @@ import org.apache.lucene.analysis.TokenFilter;
|
|||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||
import org.apache.lucene.util.IgnoreRandomChains;
|
||||
|
||||
/**
|
||||
* TokenFilter for Beider-Morse phonetic encoding.
|
||||
|
@ -33,6 +34,8 @@ import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
|||
* @see BeiderMorseEncoder
|
||||
* @lucene.experimental
|
||||
*/
|
||||
@IgnoreRandomChains(
|
||||
reason = "LUCENE-10360: cannot handle empty tokens (or those only dashes and whitespace)")
|
||||
public final class BeiderMorseFilter extends TokenFilter {
|
||||
private final PhoneticEngine engine;
|
||||
private final LanguageSet languages;
|
||||
|
@ -72,6 +75,7 @@ public final class BeiderMorseFilter extends TokenFilter {
|
|||
* @param languages optional Set of original languages. Can be null (which means it will be
|
||||
* guessed).
|
||||
*/
|
||||
@IgnoreRandomChains(reason = "LUCENE-10352: Add support for LanguageSet randomization")
|
||||
public BeiderMorseFilter(TokenStream input, PhoneticEngine engine, LanguageSet languages) {
|
||||
super(input);
|
||||
this.engine = engine;
|
||||
|
|
|
@ -39,6 +39,9 @@ public final class DoubleMetaphoneFilter extends TokenFilter {
|
|||
*/
|
||||
public DoubleMetaphoneFilter(TokenStream input, int maxCodeLength, boolean inject) {
|
||||
super(input);
|
||||
if (maxCodeLength < 1) {
|
||||
throw new IllegalArgumentException("maxCodeLength must be >=1");
|
||||
}
|
||||
this.encoder.setMaxCodeLen(maxCodeLength);
|
||||
this.inject = inject;
|
||||
}
|
||||
|
|
|
@ -14,5 +14,6 @@
|
|||
# limitations under the License.
|
||||
|
||||
org.apache.lucene.analysis.phonetic.BeiderMorseFilterFactory
|
||||
org.apache.lucene.analysis.phonetic.DaitchMokotoffSoundexFilterFactory
|
||||
org.apache.lucene.analysis.phonetic.DoubleMetaphoneFilterFactory
|
||||
org.apache.lucene.analysis.phonetic.PhoneticFilterFactory
|
||||
|
|
|
@ -21,6 +21,7 @@ import java.util.ArrayList;
|
|||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import org.apache.lucene.util.AttributeSource;
|
||||
import org.apache.lucene.util.IgnoreRandomChains;
|
||||
|
||||
/**
|
||||
* This class can be used if the token attributes of a TokenStream are intended to be consumed more
|
||||
|
@ -31,6 +32,9 @@ import org.apache.lucene.util.AttributeSource;
|
|||
* although only before {@link #incrementToken()} is called the first time. Prior to Lucene 5, it
|
||||
* was never propagated.
|
||||
*/
|
||||
@IgnoreRandomChains(
|
||||
reason =
|
||||
"doesn't actual reset itself! TODO: this statement is probably obsolete as of LUCENE-6121")
|
||||
public final class CachingTokenFilter extends TokenFilter {
|
||||
private List<AttributeSource.State> cache = null;
|
||||
private Iterator<AttributeSource.State> iterator = null;
|
||||
|
|
|
@ -0,0 +1,34 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.util;
|
||||
|
||||
import java.lang.annotation.ElementType;
|
||||
import java.lang.annotation.Retention;
|
||||
import java.lang.annotation.RetentionPolicy;
|
||||
import java.lang.annotation.Target;
|
||||
|
||||
/**
|
||||
* Annotation to not test a class or constructor with {@code TestRandomChains} integration test.
|
||||
*
|
||||
* @lucene.internal
|
||||
*/
|
||||
@Retention(RetentionPolicy.RUNTIME)
|
||||
@Target({ElementType.CONSTRUCTOR, ElementType.TYPE})
|
||||
public @interface IgnoreRandomChains {
|
||||
/** A reason for ignoring should always be given. */
|
||||
String reason();
|
||||
}
|
|
@ -14,7 +14,7 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.lucene.analysis.util;
|
||||
package org.apache.lucene.tests.util;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.IOException;
|
|
@ -36,6 +36,7 @@ include "lucene:analysis:opennlp"
|
|||
include "lucene:analysis:phonetic"
|
||||
include "lucene:analysis:smartcn"
|
||||
include "lucene:analysis:stempel"
|
||||
include "lucene:analysis.tests"
|
||||
include "lucene:backward-codecs"
|
||||
include "lucene:benchmark"
|
||||
include "lucene:classification"
|
||||
|
|
Loading…
Reference in New Issue