Browse Source

Move pre-configured "keyword" tokenizer to the analysis-common module (#24863)

Moves the keyword tokenizer to the analysis-common module. The keyword tokenizer is special because it is used by CustomNormalizerProvider so I pulled it out into its own PR. To get the move to work I've reworked the lookup from static to one using the AnalysisRegistry. This seems safe enough.

Part of #23658.
Nik Everett 8 years ago
parent
commit
ecc87f613f

+ 3 - 2
core/src/main/java/org/elasticsearch/index/analysis/AnalysisRegistry.java

@@ -466,7 +466,7 @@ public final class AnalysisRegistry implements Closeable {
         }
         for (Map.Entry<String, AnalyzerProvider<?>> entry : normalizerProviders.entrySet()) {
             processNormalizerFactory(deprecationLogger, indexSettings, entry.getKey(), entry.getValue(), normalizers,
-                    tokenFilterFactoryFactories, charFilterFactoryFactories);
+                    tokenizerFactoryFactories.get("keyword"), tokenFilterFactoryFactories, charFilterFactoryFactories);
         }
         for (Map.Entry<String, NamedAnalyzer> entry : analyzerAliases.entrySet()) {
             String key = entry.getKey();
@@ -585,10 +585,11 @@ public final class AnalysisRegistry implements Closeable {
             String name,
             AnalyzerProvider<?> normalizerFactory,
             Map<String, NamedAnalyzer> normalizers,
+            TokenizerFactory keywordTokenizerFactory,
             Map<String, TokenFilterFactory> tokenFilters,
             Map<String, CharFilterFactory> charFilters) {
         if (normalizerFactory instanceof CustomNormalizerProvider) {
-            ((CustomNormalizerProvider) normalizerFactory).build(charFilters, tokenFilters);
+            ((CustomNormalizerProvider) normalizerFactory).build(keywordTokenizerFactory, charFilters, tokenFilters);
         }
         Analyzer normalizerF = normalizerFactory.get();
         if (normalizerF == null) {

+ 3 - 3
core/src/main/java/org/elasticsearch/index/analysis/CustomNormalizerProvider.java

@@ -21,7 +21,6 @@ package org.elasticsearch.index.analysis;
 
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.index.IndexSettings;
-import org.elasticsearch.indices.analysis.PreBuiltTokenizers;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -44,7 +43,8 @@ public final class CustomNormalizerProvider extends AbstractIndexAnalyzerProvide
         this.analyzerSettings = settings;
     }
 
-    public void build(final Map<String, CharFilterFactory> charFilters, final Map<String, TokenFilterFactory> tokenFilters) {
+    public void build(final TokenizerFactory keywordTokenizerFactory, final Map<String, CharFilterFactory> charFilters,
+            final Map<String, TokenFilterFactory> tokenFilters) {
         String tokenizerName = analyzerSettings.get("tokenizer");
         if (tokenizerName != null) {
             throw new IllegalArgumentException("Custom normalizer [" + name() + "] cannot configure a tokenizer");
@@ -83,7 +83,7 @@ public final class CustomNormalizerProvider extends AbstractIndexAnalyzerProvide
 
         this.customAnalyzer = new CustomAnalyzer(
                 "keyword",
-                PreBuiltTokenizers.KEYWORD.getTokenizerFactory(indexSettings.getIndexVersionCreated()),
+                keywordTokenizerFactory,
                 charFiltersList.toArray(new CharFilterFactory[charFiltersList.size()]),
                 tokenFilterList.toArray(new TokenFilterFactory[tokenFilterList.size()])
         );

+ 0 - 48
core/src/main/java/org/elasticsearch/indices/analysis/PreBuiltTokenizers.java

@@ -19,7 +19,6 @@
 package org.elasticsearch.indices.analysis;
 
 import org.apache.lucene.analysis.Tokenizer;
-import org.apache.lucene.analysis.core.KeywordTokenizer;
 import org.apache.lucene.analysis.core.LetterTokenizer;
 import org.apache.lucene.analysis.core.WhitespaceTokenizer;
 import org.apache.lucene.analysis.ngram.EdgeNGramTokenizer;
@@ -32,10 +31,7 @@ import org.apache.lucene.analysis.standard.UAX29URLEmailTokenizer;
 import org.apache.lucene.analysis.th.ThaiTokenizer;
 import org.elasticsearch.Version;
 import org.elasticsearch.common.regex.Regex;
-import org.elasticsearch.index.analysis.CustomNormalizerProvider;
-import org.elasticsearch.index.analysis.MultiTermAwareComponent;
 import org.elasticsearch.index.analysis.TokenFilterFactory;
-import org.elasticsearch.index.analysis.TokenizerFactory;
 import org.elasticsearch.indices.analysis.PreBuiltCacheFactory.CachingStrategy;
 
 public enum PreBuiltTokenizers {
@@ -68,13 +64,6 @@ public enum PreBuiltTokenizers {
         }
     },
 
-    KEYWORD(CachingStrategy.ONE) {
-        @Override
-        protected Tokenizer create(Version version) {
-            return new KeywordTokenizer();
-        }
-    },
-
     LETTER(CachingStrategy.ONE) {
         @Override
         protected Tokenizer create(Version version) {
@@ -125,50 +114,13 @@ public enum PreBuiltTokenizers {
         return null;
     }
 
-    protected final PreBuiltCacheFactory.PreBuiltCache<TokenizerFactory> cache;
     private final CachingStrategy cachingStrategy;
 
     PreBuiltTokenizers(CachingStrategy cachingStrategy) {
         this.cachingStrategy = cachingStrategy;
-        cache = PreBuiltCacheFactory.getCache(cachingStrategy);
     }
 
     public CachingStrategy getCachingStrategy() {
         return cachingStrategy;
     }
-
-    private interface MultiTermAwareTokenizerFactory extends TokenizerFactory, MultiTermAwareComponent {}
-
-    /**
-     * Old style resolution for {@link TokenizerFactory}. Exists entirely to keep
-     * {@link CustomNormalizerProvider#build(java.util.Map, java.util.Map)} working during the migration.
-     */
-    public synchronized TokenizerFactory getTokenizerFactory(final Version version) {
-            TokenizerFactory tokenizerFactory = cache.get(version);
-            if (tokenizerFactory == null) {
-                if (getMultiTermComponent(version) != null) {
-                    tokenizerFactory = new MultiTermAwareTokenizerFactory() {
-                        @Override
-                        public Tokenizer create() {
-                            return PreBuiltTokenizers.this.create(version);
-                        }
-    
-                        @Override
-                        public Object getMultiTermComponent() {
-                            return PreBuiltTokenizers.this.getMultiTermComponent(version);
-                        }
-                    };
-                } else {
-                    tokenizerFactory = new TokenizerFactory() {
-                        @Override
-                        public Tokenizer create() {
-                            return PreBuiltTokenizers.this.create(version);
-                        }
-                    };
-                }
-                cache.put(version, tokenizerFactory);
-            }
-    
-            return tokenizerFactory;
-        }
 }

+ 2 - 0
modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/CommonAnalysisPlugin.java

@@ -32,6 +32,7 @@ import org.apache.lucene.analysis.cjk.CJKWidthFilter;
 import org.apache.lucene.analysis.ckb.SoraniNormalizationFilter;
 import org.apache.lucene.analysis.commongrams.CommonGramsFilter;
 import org.apache.lucene.analysis.core.DecimalDigitFilter;
+import org.apache.lucene.analysis.core.KeywordTokenizer;
 import org.apache.lucene.analysis.core.LowerCaseTokenizer;
 import org.apache.lucene.analysis.core.StopAnalyzer;
 import org.apache.lucene.analysis.core.UpperCaseFilter;
@@ -215,6 +216,7 @@ public class CommonAnalysisPlugin extends Plugin implements AnalysisPlugin {
     @Override
     public List<PreConfiguredTokenizer> getPreConfiguredTokenizers() {
         List<PreConfiguredTokenizer> tokenizers = new ArrayList<>();
+        tokenizers.add(PreConfiguredTokenizer.singleton("keyword", KeywordTokenizer::new, null));
         tokenizers.add(PreConfiguredTokenizer.singleton("lowercase", LowerCaseTokenizer::new, () -> new TokenFilterFactory() {
             @Override
             public String name() {

+ 1 - 0
modules/analysis-common/src/test/java/org/elasticsearch/analysis/common/CommonAnalysisFactoryTests.java

@@ -145,6 +145,7 @@ public class CommonAnalysisFactoryTests extends AnalysisFactoryTestCase {
     @Override
     protected Map<String, Class<?>> getPreConfiguredTokenizers() {
         Map<String, Class<?>> filters = new TreeMap<>(super.getPreConfiguredTokenizers());
+        filters.put("keyword", null);
         filters.put("lowercase", null);
         return filters;
     }