You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2012/07/24 03:48:56 UTC

svn commit: r1364863 - in /lucene/dev/branches/lucene2510/lucene/analysis/common/src/java/org/apache/lucene/analysis/util: AnalysisSPILoader.java CharFilterFactory.java TokenFilterFactory.java TokenizerFactory.java

Author: rmuir
Date: Tue Jul 24 01:48:56 2012
New Revision: 1364863

URL: http://svn.apache.org/viewvc?rev=1364863&view=rev
Log:
LUCENE-4044: add spi support to Tokenizer/CharFilter/TokenFilter factory

Added:
    lucene/dev/branches/lucene2510/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/AnalysisSPILoader.java   (with props)
Modified:
    lucene/dev/branches/lucene2510/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/CharFilterFactory.java
    lucene/dev/branches/lucene2510/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/TokenFilterFactory.java
    lucene/dev/branches/lucene2510/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/TokenizerFactory.java

Added: lucene/dev/branches/lucene2510/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/AnalysisSPILoader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2510/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/AnalysisSPILoader.java?rev=1364863&view=auto
==============================================================================
--- lucene/dev/branches/lucene2510/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/AnalysisSPILoader.java (added)
+++ lucene/dev/branches/lucene2510/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/AnalysisSPILoader.java Tue Jul 24 01:48:56 2012
@@ -0,0 +1,115 @@
+package org.apache.lucene.analysis.util;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.util.Collections;
+import java.util.Locale;
+import java.util.Map;
+import java.util.LinkedHashMap;
+import java.util.Set;
+import java.util.ServiceLoader;
+
+/**
+ * Helper class for loading named SPIs from classpath (e.g. Tokenizers, TokenStreams).
+ * @lucene.internal
+ */
+public final class AnalysisSPILoader<S extends AbstractAnalysisFactory> {
+
+  private final Map<String,Class<S>> services;
+
+  /** This field is a hack for LuceneTestCase to get access
+   * to the modifiable map (to work around bugs in IBM J9) */
+  @SuppressWarnings("unused")
+  @Deprecated
+  // Hackidy-Häck-Hack for bugs in IBM J9 ServiceLoader
+  private final Map<String,Class<S>> modifiableServices;
+  
+  private final Class<S> clazz;
+
+  public AnalysisSPILoader(Class<S> clazz) {
+    this.clazz = clazz;
+    final ServiceLoader<S> loader = ServiceLoader.load(clazz);
+    final LinkedHashMap<String,Class<S>> services = new LinkedHashMap<String,Class<S>>();
+    final String suffix = clazz.getSimpleName();
+    for (final S service : loader) {
+      final String clazzName = service.getClass().getSimpleName();
+      final int suffixIndex = clazzName.lastIndexOf(suffix);
+      final String name = clazzName.substring(0, suffixIndex).toLowerCase(Locale.ROOT);
+      // only add the first one for each name, later services will be ignored
+      // this allows to place services before others in classpath to make 
+      // them used instead of others
+      if (!services.containsKey(name)) {
+        assert checkServiceName(name);
+        services.put(name, (Class<S>) service.getClass());
+      }
+    }
+    this.modifiableServices = services; // hack, remove when IBM J9 is fixed!
+    this.services = Collections.unmodifiableMap(services);
+  }
+  
+  /**
+   * Validates that a service name meets the requirements of {@link NamedSPI}
+   */
+  public static boolean checkServiceName(String name) {
+    // based on harmony charset.java
+    if (name.length() >= 128) {
+      throw new IllegalArgumentException("Illegal service name: '" + name + "' is too long (must be < 128 chars).");
+    }
+    for (int i = 0; i < name.length(); i++) {
+      char c = name.charAt(i);
+      if (!isLetter(c) && !isDigit(c)) {
+        throw new IllegalArgumentException("Illegal service name: '" + name + "' must be simple ascii alphanumeric.");
+      }
+    }
+    return true;
+  }
+  
+  /*
+   * Checks whether a character is a letter (ascii) which are defined in the spec.
+   */
+  private static boolean isLetter(char c) {
+      return ('a' <= c && c <= 'z') || ('A' <= c && c <= 'Z');
+  }
+
+  /*
+   * Checks whether a character is a digit (ascii) which are defined in the spec.
+   */
+  private static boolean isDigit(char c) {
+      return ('0' <= c && c <= '9');
+  }
+  
+  public S newInstance(String name) {
+    final Class<S> service = services.get(name.toLowerCase(Locale.ROOT));
+    if (service != null) {
+      try {
+        return service.newInstance();
+      } catch (Exception e) {
+        throw new IllegalArgumentException("SPI class of type "+clazz.getName()+" with name '"+name+"' cannot be instantiated. " +
+        		"This is likely due to a misconfiguration of the java class '" + service.getName() + "': ", e);
+      }
+    } else {
+      throw new IllegalArgumentException("A SPI class of type "+clazz.getName()+" with name '"+name+"' does not exist. "+
+            "You need to add the corresponding JAR file supporting this SPI to your classpath."+
+            "The current classpath supports the following names: "+availableServices());
+    }
+  }
+
+  public Set<String> availableServices() {
+    return services.keySet();
+  }  
+}

Modified: lucene/dev/branches/lucene2510/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/CharFilterFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2510/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/CharFilterFactory.java?rev=1364863&r1=1364862&r2=1364863&view=diff
==============================================================================
--- lucene/dev/branches/lucene2510/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/CharFilterFactory.java (original)
+++ lucene/dev/branches/lucene2510/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/CharFilterFactory.java Tue Jul 24 01:48:56 2012
@@ -18,6 +18,7 @@ package org.apache.lucene.analysis.util;
  */
 
 import java.io.Reader;
+import java.util.Set;
 
 import org.apache.lucene.analysis.CharFilter;
 
@@ -27,5 +28,18 @@ import org.apache.lucene.analysis.CharFi
  */
 public abstract class CharFilterFactory extends AbstractAnalysisFactory {
 
+  private static final AnalysisSPILoader<CharFilterFactory> loader =
+      new AnalysisSPILoader<CharFilterFactory>(CharFilterFactory.class);
+  
+  /** looks up a charfilter by name */
+  public static CharFilterFactory forName(String name) {
+    return loader.newInstance(name);
+  }
+  
+  /** returns a list of all available charfilter names */
+  public static Set<String> availableCharFilters() {
+    return loader.availableServices();
+  }
+
   public abstract CharFilter create(Reader input);
 }

Modified: lucene/dev/branches/lucene2510/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/TokenFilterFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2510/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/TokenFilterFactory.java?rev=1364863&r1=1364862&r2=1364863&view=diff
==============================================================================
--- lucene/dev/branches/lucene2510/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/TokenFilterFactory.java (original)
+++ lucene/dev/branches/lucene2510/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/TokenFilterFactory.java Tue Jul 24 01:48:56 2012
@@ -17,6 +17,8 @@ package org.apache.lucene.analysis.util;
  * limitations under the License.
  */
 
+import java.util.Set;
+
 import org.apache.lucene.analysis.TokenStream;
 
 /**
@@ -25,6 +27,19 @@ import org.apache.lucene.analysis.TokenS
  */
 public abstract class TokenFilterFactory extends AbstractAnalysisFactory {
 
+  private static final AnalysisSPILoader<TokenFilterFactory> loader =
+      new AnalysisSPILoader<TokenFilterFactory>(TokenFilterFactory.class);
+  
+  /** looks up a tokenfilter by name */
+  public static TokenFilterFactory forName(String name) {
+    return loader.newInstance(name);
+  }
+  
+  /** returns a list of all available tokenfilter names */
+  public static Set<String> availableTokenFilters() {
+    return loader.availableServices();
+  }
+  
   /** Transform the specified input TokenStream */
   public abstract TokenStream create(TokenStream input);
 }

Modified: lucene/dev/branches/lucene2510/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/TokenizerFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2510/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/TokenizerFactory.java?rev=1364863&r1=1364862&r2=1364863&view=diff
==============================================================================
--- lucene/dev/branches/lucene2510/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/TokenizerFactory.java (original)
+++ lucene/dev/branches/lucene2510/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/TokenizerFactory.java Tue Jul 24 01:48:56 2012
@@ -20,6 +20,7 @@ package org.apache.lucene.analysis.util;
 import org.apache.lucene.analysis.Tokenizer;
 
 import java.io.Reader;
+import java.util.Set;
 
 /**
  * Abstract parent class for analysis factories that create {@link Tokenizer}
@@ -27,6 +28,19 @@ import java.io.Reader;
  */
 public abstract class TokenizerFactory extends AbstractAnalysisFactory {
 
+  private static final AnalysisSPILoader<TokenizerFactory> loader =
+      new AnalysisSPILoader<TokenizerFactory>(TokenizerFactory.class);
+  
+  /** looks up a tokenizer by name */
+  public static TokenizerFactory forName(String name) {
+    return loader.newInstance(name);
+  }
+  
+  /** returns a list of all available tokenizer names */
+  public static Set<String> availableTokenizers() {
+    return loader.availableServices();
+  }
+  
   /** Creates a TokenStream of the specified input */
   public abstract Tokenizer create(Reader input);
 }