You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by no...@apache.org on 2015/03/07 06:55:36 UTC

svn commit: r1664797 [1/3] - in /lucene/dev/trunk/solr: ./ core/src/java/org/apache/solr/cloud/ core/src/java/org/apache/solr/core/ core/src/java/org/apache/solr/handler/ core/src/java/org/apache/solr/handler/admin/ core/src/java/org/apache/solr/handle...

Author: noble
Date: Sat Mar  7 05:55:35 2015
New Revision: 1664797

URL: http://svn.apache.org/r1664797
Log:
SOLR-7073: Add an API to add a jar to a collection's classpath

Added:
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ImplicitPlugins.java
      - copied, changed from r1664338, lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/PluginsRegistry.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/MemClassLoader.java   (with props)
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/PluginRegistry.java   (with props)
Removed:
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/PluginsRegistry.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/BlobStoreTestRequestHandlerV2.java
Modified:
    lucene/dev/trunk/solr/CHANGES.txt
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/JarRepository.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/RequestHandlers.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrConfig.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/AdminHandlers.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/SearchComponent.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/QParserPlugin.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/CommandOperation.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/OutputWriterTest.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/BlobStoreTestRequestHandler.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/RequestHandlersTest.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestDynamicLoading.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/TestBlobHandler.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/component/SpellCheckComponentTest.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestStandardQParsers.java

Modified: lucene/dev/trunk/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/CHANGES.txt?rev=1664797&r1=1664796&r2=1664797&view=diff
==============================================================================
--- lucene/dev/trunk/solr/CHANGES.txt (original)
+++ lucene/dev/trunk/solr/CHANGES.txt Sat Mar  7 05:55:35 2015
@@ -84,6 +84,9 @@ Upgrading from Solr 5.0
 * The signature of SolrDispatchFilter.createCoreContainer() has changed to take
   (String,Properties) arguments
 
+* Deprecated the 'lib' option added to create-requesthandler as part of SOLR-6801 in 5.0 release.
+  Please use the add-runtimelib command
+
 Detailed Change List
 ----------------------
 
@@ -136,6 +139,8 @@ New Features
 * SOLR-7155: All SolrClient methods now take an optional 'collection' argument
   (Alan Woodward)
 
+* SOLR-7073: Support adding a jar to a collections classpath (Noble Paul)
+
 Bug Fixes
 ----------------------
 

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java?rev=1664797&r1=1664796&r2=1664797&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java Sat Mar  7 05:55:35 2015
@@ -39,7 +39,6 @@ import org.apache.solr.common.params.Upd
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.core.DirectoryFactory.DirContext;
-import org.apache.solr.core.RequestHandlers.LazyRequestHandlerWrapper;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.handler.ReplicationHandler;
 import org.apache.solr.request.LocalSolrQueryRequest;
@@ -146,9 +145,6 @@ public class RecoveryStrategy extends Th
     
     // use rep handler directly, so we can do this sync rather than async
     SolrRequestHandler handler = core.getRequestHandler(REPLICATION_HANDLER);
-    if (handler instanceof LazyRequestHandlerWrapper) {
-      handler = ((LazyRequestHandlerWrapper) handler).getWrappedHandler();
-    }
     ReplicationHandler replicationHandler = (ReplicationHandler) handler;
     
     if (replicationHandler == null) {

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java?rev=1664797&r1=1664796&r2=1664797&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java Sat Mar  7 05:55:35 2015
@@ -41,7 +41,6 @@ import org.slf4j.LoggerFactory;
 import java.io.File;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
@@ -108,13 +107,13 @@ public class CoreContainer {
   public static final String COLLECTIONS_HANDLER_PATH = "/admin/collections";
   public static final String INFO_HANDLER_PATH = "/admin/info";
 
-  private Map<String, SolrRequestHandler> containerHandlers = new HashMap<>();
+  private PluginRegistry<SolrRequestHandler> containerHandlers = new PluginRegistry<>(SolrRequestHandler.class, null);
 
   public SolrRequestHandler getRequestHandler(String path) {
     return RequestHandlerBase.getRequestHandler(path, containerHandlers);
   }
 
-  public Map<String, SolrRequestHandler> getRequestHandlers(){
+  public PluginRegistry<SolrRequestHandler> getRequestHandlers() {
     return this.containerHandlers;
   }
 

Copied: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ImplicitPlugins.java (from r1664338, lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/PluginsRegistry.java)
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ImplicitPlugins.java?p2=lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ImplicitPlugins.java&p1=lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/PluginsRegistry.java&r1=1664338&r2=1664797&rev=1664797&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/PluginsRegistry.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ImplicitPlugins.java Sat Mar  7 05:55:35 2015
@@ -45,7 +45,7 @@ import static org.apache.solr.common.clo
 import static org.apache.solr.core.PluginInfo.DEFAULTS;
 import static org.apache.solr.core.PluginInfo.INVARIANTS;
 
-public class PluginsRegistry {
+public class ImplicitPlugins {
 
   public static List<PluginInfo> getHandlers(SolrCore solrCore){
     List<PluginInfo> implicits = new ArrayList<>();
@@ -88,4 +88,5 @@ public class PluginsRegistry {
     Map m = makeMap("name", name, "class", clz.getName());
     return new PluginInfo(SolrRequestHandler.TYPE, m, new NamedList<>(singletonMap(DEFAULTS, new NamedList(defaults))),null);
   }
+  public static final String IMPLICIT = "implicit";
 }

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/JarRepository.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/JarRepository.java?rev=1664797&r1=1664796&r2=1664797&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/JarRepository.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/JarRepository.java Sat Mar  7 05:55:35 2015
@@ -24,7 +24,6 @@ import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
@@ -43,7 +42,6 @@ import org.apache.solr.common.cloud.Clus
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
-import org.apache.solr.common.cloud.ZkCoreNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.handler.admin.CollectionsHandler;
 import org.apache.solr.util.SimplePostTool;
@@ -76,37 +74,16 @@ public class JarRepository {
   }
 
   /**
-   * Returns the contents of a jar and increments a reference count. Please return the same object to decerease the refcount
+   * Returns the contents of a jar and increments a reference count. Please return the same object to decrease the refcount
    *
    * @param key it is a combination of blobname and version like blobName/version
    * @return The reference of a jar
    */
-  public JarContentRef getJarIncRef(String key) throws IOException {
+  public JarContentRef getJarIncRef(String key) {
     JarContent jar = jars.get(key);
     if (jar == null) {
       if (this.coreContainer.isZooKeeperAware()) {
-        ZkStateReader zkStateReader = this.coreContainer.getZkController().getZkStateReader();
-        ClusterState cs = zkStateReader.getClusterState();
-        DocCollection coll = cs.getCollectionOrNull(CollectionsHandler.SYSTEM_COLL);
-        if (coll == null) throw new SolrException(SERVICE_UNAVAILABLE, ".system collection not available");
-        ArrayList<Slice> slices = new ArrayList<>(coll.getActiveSlices());
-        if (slices.isEmpty()) throw new SolrException(SERVICE_UNAVAILABLE, "No active slices for .system collection");
-        Collections.shuffle(slices, RANDOM); //do load balancing
-
-        Replica replica = null;
-        for (Slice slice : slices)  {
-          List<Replica> replicas = new ArrayList<>(slice.getReplicasMap().values());
-          Collections.shuffle(replicas, RANDOM);
-          for (Replica r : replicas) {
-            if (ZkStateReader.ACTIVE.equals(r.getStr(ZkStateReader.STATE_PROP))) {
-              replica = r;
-              break;
-            }
-          }
-        }
-        if (replica == null) {
-          throw new SolrException(SERVICE_UNAVAILABLE, ".no active replica available for .system collection");
-        }
+        Replica replica = getSystemCollReplica();
         String url = replica.getStr(BASE_URL_PROP) + "/.system/blob/" + key + "?wt=filestream";
 
         HttpClient httpClient = coreContainer.getUpdateShardHandler().getHttpClient();
@@ -119,6 +96,12 @@ public class JarRepository {
             throw new SolrException(SolrException.ErrorCode.NOT_FOUND, "no such blob or version available: " + key);
           }
           b = SimplePostTool.inputStreamToByteArray(entity.getEntity().getContent());
+        } catch (Exception e) {
+          if (e instanceof SolrException) {
+            throw (SolrException) e;
+          } else {
+            throw new SolrException(SolrException.ErrorCode.NOT_FOUND, "could not load : " + key, e);
+          }
         } finally {
           httpGet.releaseConnection();
         }
@@ -138,6 +121,36 @@ public class JarRepository {
 
   }
 
+  private Replica getSystemCollReplica() {
+    ZkStateReader zkStateReader = this.coreContainer.getZkController().getZkStateReader();
+    ClusterState cs = zkStateReader.getClusterState();
+    DocCollection coll = cs.getCollectionOrNull(CollectionsHandler.SYSTEM_COLL);
+    if (coll == null) throw new SolrException(SERVICE_UNAVAILABLE, ".system collection not available");
+    ArrayList<Slice> slices = new ArrayList<>(coll.getActiveSlices());
+    if (slices.isEmpty()) throw new SolrException(SERVICE_UNAVAILABLE, "No active slices for .system collection");
+    Collections.shuffle(slices, RANDOM); //do load balancing
+
+    Replica replica = null;
+    for (Slice slice : slices) {
+      List<Replica> replicas = new ArrayList<>(slice.getReplicasMap().values());
+      Collections.shuffle(replicas, RANDOM);
+      for (Replica r : replicas) {
+        if (ZkStateReader.ACTIVE.equals(r.getStr(ZkStateReader.STATE_PROP))) {
+          if(zkStateReader.getClusterState().getLiveNodes().contains(r.get(ZkStateReader.NODE_NAME_PROP))){
+            replica = r;
+            break;
+          } else {
+            log.info("replica {} says it is active but not a member of live nodes", r.get(ZkStateReader.NODE_NAME_PROP));
+          }
+        }
+      }
+    }
+    if (replica == null) {
+      throw new SolrException(SERVICE_UNAVAILABLE, ".no active replica available for .system collection");
+    }
+    return replica;
+  }
+
   /**
    * This is to decrement a ref count
    *

Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/MemClassLoader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/MemClassLoader.java?rev=1664797&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/MemClassLoader.java (added)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/MemClassLoader.java Sat Mar  7 05:55:35 2015
@@ -0,0 +1,180 @@
+package org.apache.solr.core;
+
+/*
+ * 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.io.IOException;
+import java.io.InputStream;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.nio.ByteBuffer;
+import java.security.CodeSource;
+import java.security.ProtectionDomain;
+import java.security.cert.Certificate;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.lucene.analysis.util.ResourceLoader;
+import org.apache.solr.common.SolrException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class MemClassLoader extends ClassLoader implements AutoCloseable, ResourceLoader {
+  static final Logger log =  LoggerFactory.getLogger(MemClassLoader.class);
+  private boolean allJarsLoaded = false;
+  private final SolrResourceLoader parentLoader;
+  private List<PluginRegistry.RuntimeLib> libs = new ArrayList<>();
+  private Map<String, Class> classCache = new HashMap<>();
+
+
+  public MemClassLoader(List<PluginRegistry.RuntimeLib> libs, SolrResourceLoader resourceLoader) {
+    this.parentLoader = resourceLoader;
+    this.libs = libs;
+  }
+
+
+  public synchronized void loadJars() {
+    if (allJarsLoaded) return;
+
+    for (PluginRegistry.RuntimeLib lib : libs) {
+      try {
+        lib.loadJar();
+      } catch (Exception exception) {
+        if (exception instanceof SolrException) throw (SolrException) exception;
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Atleast one runtimeLib could not be loaded", exception);
+      }
+    }
+    allJarsLoaded = true;
+  }
+
+
+  @Override
+  protected Class<?> findClass(String name) throws ClassNotFoundException {
+    if(!allJarsLoaded ) loadJars();
+    try {
+      return parentLoader.findClass(name, Object.class);
+    } catch (Exception e) {
+      return loadFromRuntimeLibs(name);
+    }
+  }
+
+  private synchronized  Class<?> loadFromRuntimeLibs(String name) throws ClassNotFoundException {
+    Class result = classCache.get(name);
+    if(result != null)
+      return result;
+    AtomicReference<String> jarName = new AtomicReference<>();
+    ByteBuffer buf = null;
+    try {
+      buf = getByteBuffer(name, jarName);
+    } catch (Exception e) {
+      throw new ClassNotFoundException("class could not be loaded " + name, e);
+    }
+    if (buf == null) throw new ClassNotFoundException("Class not found :" + name);
+    ProtectionDomain defaultDomain = null;
+    //using the default protection domain, with no permissions
+    try {
+      defaultDomain = new ProtectionDomain(new CodeSource(new URL("http://localhost/.system/blob/" + jarName.get()), (Certificate[]) null),
+          null);
+    } catch (MalformedURLException mue) {
+      throw new ClassNotFoundException("Unexpected exception ", mue);
+      //should not happen
+    }
+    log.info("Defining_class {} from runtime jar {} ", name, jarName);
+
+    result = defineClass(name, buf.array(), buf.arrayOffset(), buf.limit(), defaultDomain);
+    classCache.put(name, result);
+    return result;
+  }
+
+  private ByteBuffer getByteBuffer(String name, AtomicReference<String> jarName) throws Exception {
+    if (!allJarsLoaded) {
+      loadJars();
+
+    }
+
+    String path = name.replace('.', '/').concat(".class");
+    ByteBuffer buf = null;
+    for (PluginRegistry.RuntimeLib lib : libs) {
+      try {
+        buf = lib.getFileContent(path);
+        if (buf != null) {
+          jarName.set(lib.name);
+          break;
+        }
+      } catch (Exception exp) {
+        throw new ClassNotFoundException("Unable to load class :" + name, exp);
+      }
+    }
+
+    return buf;
+  }
+
+  @Override
+  public void close() throws Exception {
+    for (PluginRegistry.RuntimeLib lib : libs) {
+      try {
+        lib.close();
+      } catch (Exception e) {
+      }
+    }
+  }
+
+  @Override
+  public InputStream openResource(String resource) throws IOException {
+    AtomicReference<String> jarName = new AtomicReference<>();
+    try {
+      ByteBuffer buf = getByteBuffer(resource, jarName);
+      if (buf == null) throw new IOException("Resource could not be found " + resource);
+    } catch (Exception e) {
+      throw new IOException("Resource could not be found " + resource, e);
+    }
+    return null;
+  }
+
+  @Override
+  public <T> Class<? extends T> findClass(String cname, Class<T> expectedType) {
+    if(!allJarsLoaded ) loadJars();
+    try {
+      return findClass(cname).asSubclass(expectedType);
+    } catch (Exception e) {
+      if (e instanceof SolrException) {
+        throw (SolrException) e;
+      } else {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "error loading class " + cname, e);
+      }
+    }
+
+  }
+
+  @Override
+  public <T> T newInstance(String cname, Class<T> expectedType) {
+    try {
+      return findClass(cname, expectedType).newInstance();
+    } catch (SolrException e) {
+      throw e;
+    } catch (Exception e) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "error instantiating class :" + cname, e);
+    }
+  }
+
+
+}

Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/PluginRegistry.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/PluginRegistry.java?rev=1664797&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/PluginRegistry.java (added)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/PluginRegistry.java Sat Mar  7 05:55:35 2015
@@ -0,0 +1,371 @@
+package org.apache.solr.core;
+
+/*
+ * 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.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.lucene.analysis.util.ResourceLoader;
+import org.apache.lucene.analysis.util.ResourceLoaderAware;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.handler.RequestHandlerBase;
+import org.apache.solr.handler.component.SearchComponent;
+import org.apache.solr.request.SolrRequestHandler;
+import org.apache.solr.util.plugin.NamedListInitializedPlugin;
+import org.apache.solr.util.plugin.PluginInfoInitialized;
+import org.apache.solr.util.plugin.SolrCoreAware;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static java.util.Collections.singletonList;
+
+/**
+ * This manages the lifecycle of a set of plugin of the same type .
+ */
+public class PluginRegistry<T> implements AutoCloseable {
+  public static Logger log = LoggerFactory.getLogger(PluginRegistry.class);
+
+  private Map<String, PluginHolder<T>> registry = new HashMap<>();
+  private Map<String, PluginHolder<T>> immutableRegistry = Collections.unmodifiableMap(registry);
+  private String def;
+  private Class klass;
+  private SolrCore core;
+  private SolrConfig.SolrPluginInfo meta;
+
+  public PluginRegistry(Class<T> klass, SolrCore core) {
+    this.core = core;
+    this.klass = klass;
+    meta = SolrConfig.classVsSolrPluginInfo.get(klass.getName());
+    if (meta == null) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unknown Plugin : " + klass.getName());
+    }
+  }
+
+  static void initInstance(Object inst, PluginInfo info, SolrCore core) {
+    if (inst instanceof PluginInfoInitialized) {
+      ((PluginInfoInitialized) inst).init(info);
+    } else if (inst instanceof NamedListInitializedPlugin) {
+      ((NamedListInitializedPlugin) inst).init(info.initArgs);
+    } else if (inst instanceof SolrRequestHandler) {
+      ((SolrRequestHandler) inst).init(info.initArgs);
+    }
+    if (inst instanceof SearchComponent) {
+      ((SearchComponent) inst).setName(info.name);
+    }
+    if (inst instanceof RequestHandlerBase) {
+      ((RequestHandlerBase) inst).setPluginInfo(info);
+    }
+
+  }
+
+  PluginHolder<T> createPlugin(PluginInfo info, SolrCore core) {
+    if ("true".equals(String.valueOf(info.attributes.get("runtimeLib")))) {
+      log.info(" {} : '{}'  created with runtimeLib=true ", meta.tag, info.name);
+      return new LazyPluginHolder<>(meta, info, core, core.getMemClassLoader());
+    } else if ("lazy".equals(info.attributes.get("startup")) && meta.options.contains(SolrConfig.PluginOpts.LAZY)) {
+      log.info("{} : '{}' created with startup=lazy ", meta.tag, info.name);
+      return new LazyPluginHolder<T>(meta, info, core, core.getResourceLoader());
+    } else {
+      T inst = core.createInstance(info.className, (Class<T>) meta.clazz, meta.tag, null, core.getResourceLoader());
+      initInstance(inst, info, core);
+      return new PluginHolder<>(info, inst);
+    }
+  }
+
+  boolean alias(String src, String target) {
+    PluginHolder<T> a = registry.get(src);
+    if (a == null) return false;
+    PluginHolder<T> b = registry.get(target);
+    if (b != null) return false;
+    registry.put(target, a);
+    return true;
+  }
+
+  /**
+   * Get a plugin by name. If the plugin is not already instantiated, it is
+   * done here
+   */
+  public T get(String name) {
+    PluginHolder<T> result = registry.get(name);
+    return result == null ? null : result.get();
+  }
+
+  /**
+   * Fetches a plugin by name , or the default
+   *
+   * @param name       name using which it is registered
+   * @param useDefault Return the default , if a plugin by that name does not exist
+   */
+  public T get(String name, boolean useDefault) {
+    T result = get(name);
+    if (useDefault && result == null) return get(def);
+    return result;
+  }
+
+  public Set<String> keySet() {
+    return immutableRegistry.keySet();
+  }
+
+  /**
+   * register a plugin by a name
+   */
+  public T put(String name, T plugin) {
+    if (plugin == null) return null;
+    PluginHolder<T> old = put(name, new PluginHolder<T>(null, plugin));
+    return old == null ? null : old.get();
+  }
+
+
+  PluginHolder<T> put(String name, PluginHolder<T> plugin) {
+    PluginHolder<T> old = registry.put(name, plugin);
+    if (plugin.pluginInfo != null && plugin.pluginInfo.isDefault()) {
+      setDefault(name);
+    }
+    if (plugin.isLoaded()) registerMBean(plugin.get(), core, name);
+    return old;
+  }
+
+  void setDefault(String def) {
+    if (!registry.containsKey(def)) return;
+    if (this.def != null) log.warn("Multiple defaults for : " + meta.tag);
+    this.def = def;
+  }
+
+  public Map<String, PluginHolder<T>> getRegistry() {
+    return immutableRegistry;
+  }
+
+  public boolean contains(String name) {
+    return registry.containsKey(name);
+  }
+
+  String getDefault() {
+    return def;
+  }
+
+  T remove(String name) {
+    PluginHolder<T> removed = registry.remove(name);
+    return removed == null ? null : removed.get();
+  }
+
+  void init(Map<String, T> defaults, SolrCore solrCore) {
+    init(defaults, solrCore, solrCore.getSolrConfig().getPluginInfos(klass.getName()));
+  }
+
+  /**
+   * Initializes the plugins after reading the meta data from {@link org.apache.solr.core.SolrConfig}.
+   *
+   * @param defaults These will be registered if not explicitly specified
+   */
+  void init(Map<String, T> defaults, SolrCore solrCore, List<PluginInfo> infos) {
+    core = solrCore;
+    for (PluginInfo info : infos) {
+      PluginHolder<T> o = createPlugin(info, solrCore);
+      String name = info.name;
+      if (meta.clazz.equals(SolrRequestHandler.class)) name = RequestHandlers.normalize(info.name);
+      PluginHolder<T> old = put(name, o);
+      if (old != null) log.warn("Multiple entries of {} with name {}", meta.tag, name);
+    }
+    for (Map.Entry<String, T> e : defaults.entrySet()) {
+      if (!contains(e.getKey())) {
+        put(e.getKey(), new PluginHolder<T>(null, e.getValue()));
+      }
+    }
+  }
+
+  /**
+   * To check if a plugin by a specified name is already loaded
+   */
+  public boolean isLoaded(String name) {
+    PluginHolder<T> result = registry.get(name);
+    if (result == null) return false;
+    return result.isLoaded();
+  }
+
+  private static void registerMBean(Object inst, SolrCore core, String pluginKey) {
+    if (core == null) return;
+    if (inst instanceof SolrInfoMBean) {
+      SolrInfoMBean mBean = (SolrInfoMBean) inst;
+      String name = (inst instanceof SolrRequestHandler) ? pluginKey : mBean.getName();
+      core.registerInfoBean(name, mBean);
+    }
+  }
+
+
+  /**
+   * Close this registry. This will in turn call a close on all the contained plugins
+   */
+  @Override
+  public void close() {
+    for (Map.Entry<String, PluginHolder<T>> e : registry.entrySet()) {
+      try {
+        e.getValue().close();
+      } catch (Exception exp) {
+        log.error("Error closing plugin " + e.getKey() + " of type : " + meta.tag, exp);
+      }
+    }
+  }
+
+  /**
+   * An indirect reference to a plugin. It just wraps a plugin instance.
+   * subclasses may choose to lazily load the plugin
+   */
+  public static class PluginHolder<T> implements AutoCloseable {
+    protected T inst;
+    protected final PluginInfo pluginInfo;
+
+    public PluginHolder(PluginInfo info) {
+      this.pluginInfo = info;
+    }
+
+    public PluginHolder(PluginInfo info, T inst) {
+      this.inst = inst;
+      this.pluginInfo = info;
+    }
+
+    public T get() {
+      return inst;
+    }
+
+    public boolean isLoaded() {
+      return inst != null;
+    }
+
+    @Override
+    public void close() throws Exception {
+      if (inst != null && inst instanceof AutoCloseable) ((AutoCloseable) inst).close();
+
+    }
+  }
+
+  /**
+   * A class that loads plugins Lazily. When the get() method is invoked
+   * the Plugin is initialized and returned.
+   */
+  public static class LazyPluginHolder<T> extends PluginHolder<T> {
+    private final SolrConfig.SolrPluginInfo pluginMeta;
+    protected SolrException solrException;
+    private final SolrCore core;
+    protected ResourceLoader resourceLoader;
+
+
+    LazyPluginHolder(SolrConfig.SolrPluginInfo pluginMeta, PluginInfo pluginInfo, SolrCore core, ResourceLoader loader) {
+      super(pluginInfo);
+      this.pluginMeta = pluginMeta;
+      this.core = core;
+      this.resourceLoader = loader;
+    }
+
+    @Override
+    public T get() {
+      if (inst != null) return inst;
+      if (solrException != null) throw solrException;
+      createInst();
+      registerMBean(inst, core, pluginInfo.name);
+      return inst;
+    }
+
+    protected synchronized void createInst() {
+      if (inst != null) return;
+      log.info("Going to create a new {} with {} ", pluginMeta.tag, pluginInfo.toString());
+      if (resourceLoader instanceof MemClassLoader) {
+        MemClassLoader loader = (MemClassLoader) resourceLoader;
+        loader.loadJars();
+      }
+      Class<T> clazz = (Class<T>) pluginMeta.clazz;
+      inst = core.createInstance(pluginInfo.className, clazz, pluginMeta.tag, null, resourceLoader);
+      initInstance(inst, pluginInfo, core);
+      if (inst instanceof SolrCoreAware) {
+        SolrResourceLoader.assertAwareCompatibility(SolrCoreAware.class, inst);
+        ((SolrCoreAware) inst).inform(core);
+      }
+      if (inst instanceof ResourceLoaderAware) {
+        SolrResourceLoader.assertAwareCompatibility(ResourceLoaderAware.class, inst);
+        try {
+          ((ResourceLoaderAware) inst).inform(core.getResourceLoader());
+        } catch (IOException e) {
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "error initializing component", e);
+        }
+      }
+    }
+
+
+  }
+
+  /**
+   * This represents a Runtime Jar. A jar requires two details , name and version
+   */
+  public static class RuntimeLib implements PluginInfoInitialized, AutoCloseable {
+    String name;
+    String version;
+    private JarRepository.JarContentRef jarContent;
+    private final JarRepository jarRepository;
+
+    @Override
+    public void init(PluginInfo info) {
+      name = info.attributes.get("name");
+      Object v = info.attributes.get("version");
+      if (name == null || v == null) {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "runtimeLib must have name and version");
+      }
+      version = String.valueOf(v);
+    }
+
+    public RuntimeLib(SolrCore core) {
+      jarRepository = core.getCoreDescriptor().getCoreContainer().getJarRepository();
+    }
+
+
+    void loadJar() {
+      if (jarContent != null) return;
+      synchronized (this) {
+        if (jarContent != null) return;
+        jarContent = jarRepository.getJarIncRef(name + "/" + version);
+      }
+    }
+
+    public ByteBuffer getFileContent(String entryName) throws IOException {
+      if (jarContent == null)
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "jar not available: " + name + "/" + version);
+      return jarContent.jar.getFileContent(entryName);
+
+    }
+
+    @Override
+    public void close() throws Exception {
+      if (jarContent != null) jarRepository.decrementJarRefCount(jarContent);
+    }
+
+    public static List<RuntimeLib> getLibObjects(SolrCore core, List<PluginInfo> libs) {
+      List<RuntimeLib> l = new ArrayList<>(libs.size());
+      for (PluginInfo lib : libs) {
+        RuntimeLib rtl = new RuntimeLib(core);
+        rtl.init(lib);
+        l.add(rtl);
+      }
+      return l;
+    }
+  }
+}

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/RequestHandlers.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/RequestHandlers.java?rev=1664797&r1=1664796&r2=1664797&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/RequestHandlers.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/RequestHandlers.java Sat Mar  7 05:55:35 2015
@@ -17,41 +17,14 @@
 
 package org.apache.solr.core;
 
-import java.io.Closeable;
-import java.io.IOException;
-import java.lang.reflect.Constructor;
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.nio.ByteBuffer;
-import java.security.BasicPermission;
-import java.security.CodeSource;
-import java.security.Permissions;
-import java.security.ProtectionDomain;
-import java.security.SecureClassLoader;
-import java.security.cert.Certificate;
-import java.text.MessageFormat;
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.jar.JarFile;
 
-import org.apache.solr.client.solrj.SolrRequest;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.SolrException.ErrorCode;
-import org.apache.solr.common.util.NamedList;
-import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.common.util.StrUtils;
-import org.apache.solr.handler.RequestHandlerBase;
-import org.apache.solr.request.SolrQueryRequest;
-import org.apache.solr.request.SolrQueryRequestBase;
 import org.apache.solr.request.SolrRequestHandler;
-import org.apache.solr.response.SolrQueryResponse;
-import org.apache.solr.util.plugin.PluginInfoInitialized;
-import org.apache.solr.util.plugin.SolrCoreAware;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -61,13 +34,8 @@ public final class RequestHandlers {
   public static Logger log = LoggerFactory.getLogger(RequestHandlers.class);
 
   protected final SolrCore core;
-  // Use a synchronized map - since the handlers can be changed at runtime, 
-  // the map implementation should be thread safe
-  private final Map<String, SolrRequestHandler> handlers =
-      new ConcurrentHashMap<>() ;
-  private final Map<String, SolrRequestHandler> immutableHandlers = Collections.unmodifiableMap(handlers) ;
 
-  public static final boolean disableExternalLib = Boolean.parseBoolean(System.getProperty("disable.external.lib", "false"));
+  final PluginRegistry<SolrRequestHandler> handlers;
 
   /**
    * Trim the trailing '/' if it's there, and convert null to empty string.
@@ -89,6 +57,7 @@ public final class RequestHandlers {
   
   public RequestHandlers(SolrCore core) {
       this.core = core;
+    handlers =  new PluginRegistry<>(SolrRequestHandler.class, core);
   }
 
   /**
@@ -99,17 +68,6 @@ public final class RequestHandlers {
   }
 
   /**
-   * @return a Map of all registered handlers of the specified type.
-   */
-  public <T extends SolrRequestHandler> Map<String,T> getAll(Class<T> clazz) {
-    Map<String,T> result = new HashMap<>(7);
-    for (Map.Entry<String,SolrRequestHandler> e : handlers.entrySet()) {
-      if(clazz.isInstance(e.getValue())) result.put(e.getKey(), clazz.cast(e.getValue()));
-    }
-    return result;
-  }
-
-  /**
    * Handlers must be initialized before calling this function.  As soon as this is
    * called, the handler can immediately accept requests.
    * 
@@ -118,22 +76,20 @@ public final class RequestHandlers {
    * @return the previous handler at the given path or null
    */
   public SolrRequestHandler register( String handlerName, SolrRequestHandler handler ) {
-    String norm = normalize( handlerName );
+    String norm = normalize(handlerName);
     if (handler == null) {
       return handlers.remove(norm);
     }
-    SolrRequestHandler old = handlers.put(norm, handler);
-    if (0 != norm.length() && handler instanceof SolrInfoMBean) {
-      core.getInfoRegistry().put(handlerName, handler);
-    }
-    return old;
+    return handlers.put(norm, handler);
+//    return register(handlerName, new PluginRegistry.PluginHolder<>(null, handler));
   }
 
+
   /**
    * Returns an unmodifiable Map containing the registered handlers
    */
-  public Map<String,SolrRequestHandler> getRequestHandlers() {
-    return immutableHandlers;
+  public PluginRegistry<SolrRequestHandler> getRequestHandlers() {
+    return handlers;
   }
 
 
@@ -157,66 +113,28 @@ public final class RequestHandlers {
    * Handlers will be registered and initialized in the order they appear in solrconfig.xml
    */
 
-  void initHandlersFromConfig(SolrConfig config){
-    List<PluginInfo> implicits = PluginsRegistry.getHandlers(core);
+  void initHandlersFromConfig(SolrConfig config) {
+    List<PluginInfo> implicits = ImplicitPlugins.getHandlers(core);
     // use link map so we iterate in the same order
-    Map<PluginInfo,SolrRequestHandler> handlers = new LinkedHashMap<>();
     Map<String, PluginInfo> infoMap= new LinkedHashMap<>();
     //deduping implicit and explicit requesthandlers
     for (PluginInfo info : implicits) infoMap.put(info.name,info);
     for (PluginInfo info : config.getPluginInfos(SolrRequestHandler.class.getName())) infoMap.put(info.name, info);
     ArrayList<PluginInfo> infos = new ArrayList<>(infoMap.values());
+
+    List<PluginInfo> modifiedInfos = new ArrayList<>();
     for (PluginInfo info : infos) {
-      try {
-        SolrRequestHandler requestHandler;
-        String startup = info.attributes.get("startup");
-        String lib = info.attributes.get("lib");
-        if (lib != null) {
-          requestHandler = new DynamicLazyRequestHandlerWrapper(core);
-        } else if (startup != null) {
-          if ("lazy".equals(startup)) {
-            log.info("adding lazy requestHandler: " + info.className);
-            requestHandler = new LazyRequestHandlerWrapper(core);
-          } else {
-            throw new Exception("Unknown startup value: '" + startup + "' for: " + info.className);
-          }
-        } else {
-          requestHandler = core.createRequestHandler(info.className);
-        }
-        if (requestHandler instanceof RequestHandlerBase) ((RequestHandlerBase) requestHandler).setPluginInfo(info);
-        
-        handlers.put(info, requestHandler);
-        SolrRequestHandler old = register(info.name, requestHandler);
-        if (old != null) {
-          log.warn("Multiple requestHandler registered to the same name: " + info.name + " ignoring: " + old.getClass().getName());
-        }
-        if (info.isDefault()) {
-          old = register("", requestHandler);
-          if (old != null) log.warn("Multiple default requestHandler registered" + " ignoring: " + old.getClass().getName());
-        }
-        log.info("created " + info.name + ": " + info.className);
-      } catch (Exception ex) {
-          throw new SolrException
-            (ErrorCode.SERVER_ERROR, "RequestHandler init failure", ex);
-      }
+      modifiedInfos.add(applyInitParams(config, info));
     }
-
-    // we've now registered all handlers, time to init them in the same order
-    for (Map.Entry<PluginInfo,SolrRequestHandler> entry : handlers.entrySet()) {
-      PluginInfo info = entry.getKey();
-      SolrRequestHandler requestHandler = entry.getValue();
-      info = applyInitParams(config, info);
-      if (requestHandler instanceof PluginInfoInitialized) {
-       ((PluginInfoInitialized) requestHandler).init(info);
-      } else{
-        requestHandler.init(info.initArgs);
+    handlers.init(Collections.emptyMap(),core, modifiedInfos);
+    handlers.alias(handlers.getDefault(), "");
+    log.info("Registered paths: {}" , StrUtils.join(new ArrayList<>(handlers.keySet()) , ',' ));
+    if(!handlers.alias( "/select","")){
+      if(!handlers.alias( "standard","")){
+        log.warn("no default request handler is registered (either '/select' or 'standard')");
       }
     }
 
-    if(get("") == null) register("", get("/select"));//defacto default handler
-    if(get("") == null) register("", get("standard"));//old default handler name; TODO remove?
-    if(get("") == null)
-      log.warn("no default request handler is registered (either '/select' or 'standard')");
   }
 
   private PluginInfo applyInitParams(SolrConfig config, PluginInfo info) {
@@ -239,328 +157,8 @@ public final class RequestHandlers {
     return info;
   }
 
-
-  /**
-   * The <code>LazyRequestHandlerWrapper</code> wraps any {@link SolrRequestHandler}.
-   * Rather then instantiate and initialize the handler on startup, this wrapper waits
-   * until it is actually called.  This should only be used for handlers that are
-   * unlikely to be used in the normal lifecycle.
-   *
-   * You can enable lazy loading in solrconfig.xml using:
-   *
-   * <pre>
-   *  &lt;requestHandler name="..." class="..." startup="lazy"&gt;
-   *    ...
-   *  &lt;/requestHandler&gt;
-   * </pre>
-   *
-   * This is a private class - if there is a real need for it to be public, it could
-   * move
-   *
-   * @since solr 1.2
-   */
-  public static class LazyRequestHandlerWrapper implements SolrRequestHandler, AutoCloseable, PluginInfoInitialized {
-    private final SolrCore core;
-    String _className;
-    SolrRequestHandler _handler;
-    PluginInfo _pluginInfo;
-
-    public LazyRequestHandlerWrapper(SolrCore core) {
-      this.core = core;
-      _handler = null; // don't initialize
-    }
-
-    @Override
-    public void init(NamedList args) {
-    }
-
-    /**
-     * Wait for the first request before initializing the wrapped handler
-     */
-    @Override
-    public void handleRequest(SolrQueryRequest req, SolrQueryResponse rsp) {
-      SolrRequestHandler handler = _handler;
-      if (handler == null) {
-        handler = getWrappedHandler();
-      }
-      handler.handleRequest(req, rsp);
-    }
-
-    public synchronized SolrRequestHandler getWrappedHandler() {
-      if (_handler == null) {
-        try {
-          SolrRequestHandler handler = createRequestHandler();
-          if (handler instanceof PluginInfoInitialized) {
-            ((PluginInfoInitialized) handler).init(_pluginInfo);
-          } else {
-            handler.init(_pluginInfo.initArgs);
-          }
-
-          if (handler instanceof PluginInfoInitialized) {
-            ((PluginInfoInitialized) handler).init(_pluginInfo);
-          } else {
-            handler.init(_pluginInfo.initArgs);
-          }
-
-
-          if (handler instanceof SolrCoreAware) {
-            ((SolrCoreAware) handler).inform(core);
-          }
-          if (handler instanceof RequestHandlerBase) ((RequestHandlerBase) handler).setPluginInfo(_pluginInfo);
-          _handler = handler;
-        } catch (Exception ex) {
-          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "lazy loading error", ex);
-        }
-      }
-      return _handler;
-    }
-
-    protected SolrRequestHandler createRequestHandler() {
-      return core.createRequestHandler(_className);
-    }
-
-    public String getHandlerClass() {
-      return _className;
-    }
-
-    //////////////////////// SolrInfoMBeans methods //////////////////////
-
-    @Override
-    public String getName() {
-      return "Lazy[" + _className + "]";
-    }
-
-    @Override
-    public String getDescription() {
-      if (_handler == null) {
-        return getName();
-      }
-      return _handler.getDescription();
-    }
-
-    @Override
-    public String getVersion() {
-      if (_handler != null) {
-        return _handler.getVersion();
-      }
-      return null;
-    }
-
-    @Override
-    public String getSource() {
-      return null;
-    }
-
-    @Override
-    public URL[] getDocs() {
-      if (_handler == null) {
-        return null;
-      }
-      return _handler.getDocs();
-    }
-
-    @Override
-    public Category getCategory() {
-      return Category.QUERYHANDLER;
-    }
-
-    @Override
-    public NamedList getStatistics() {
-      if (_handler != null) {
-        return _handler.getStatistics();
-      }
-      NamedList<String> lst = new SimpleOrderedMap<>();
-      lst.add("note", "not initialized yet");
-      return lst;
-    }
-
-    @Override
-    public void close() throws Exception {
-      if (_handler == null) return;
-      if (_handler instanceof AutoCloseable && !(_handler instanceof DynamicLazyRequestHandlerWrapper)) {
-        ((AutoCloseable) _handler).close();
-      }
-    }
-
-    @Override
-    public void init(PluginInfo info) {
-      _pluginInfo = info;
-      _className = info.className;
-    }
-  }
-
-  public static class DynamicLazyRequestHandlerWrapper extends LazyRequestHandlerWrapper {
-    private String lib;
-    private String key;
-    private String version;
-    private CoreContainer coreContainer;
-    private SolrResourceLoader solrResourceLoader;
-    private MemClassLoader classLoader;
-    private boolean _closed = false;
-    boolean unrecoverable = false;
-    String errMsg = null;
-    private Exception exception;
-
-
-    public DynamicLazyRequestHandlerWrapper(SolrCore core) {
-      super(core);
-      this.coreContainer = core.getCoreDescriptor().getCoreContainer();
-      this.solrResourceLoader = core.getResourceLoader();
-
-    }
-
-    @Override
-    public void init(PluginInfo info) {
-      super.init(info);
-      this.lib = _pluginInfo.attributes.get("lib");
-
-      if (disableExternalLib) {
-        errMsg = "ERROR external library loading is disabled";
-        unrecoverable = true;
-        _handler = this;
-        log.error(errMsg);
-        return;
-      }
-
-      if (_pluginInfo.attributes.get("version") == null) {
-        errMsg = "ERROR 'lib' attribute must be accompanied with version also";
-        unrecoverable = true;
-        _handler = this;
-        log.error(errMsg);
-        return;
-      }
-      version = String.valueOf(_pluginInfo.attributes.get("version"));
-      classLoader = new MemClassLoader(this);
-    }
-
-    @Override
-    public void handleRequest(SolrQueryRequest req, SolrQueryResponse rsp) {
-      if (unrecoverable) {
-        rsp.add("error", errMsg);
-        if (exception != null) rsp.setException(exception);
-        return;
-      }
-      try {
-        classLoader.checkJarAvailable();
-      } catch (SolrException e) {
-        rsp.add("error", "Jar could not be loaded");
-        rsp.setException(e);
-        return;
-      } catch (IOException e) {
-        unrecoverable = true;
-        errMsg = "Could not load jar";
-        exception = e;
-        handleRequest(req, rsp);
-        return;
-      }
-
-      super.handleRequest(req, rsp);
-    }
-
-    @Override
-    protected SolrRequestHandler createRequestHandler() {
-      try {
-        Class clazz = classLoader.findClass(_className);
-        Constructor<?>[] cons = clazz.getConstructors();
-        for (Constructor<?> con : cons) {
-          Class<?>[] types = con.getParameterTypes();
-          if (types.length == 1 && types[0] == SolrCore.class) {
-            return SolrRequestHandler.class.cast(con.newInstance(this));
-          }
-        }
-        return (SolrRequestHandler) clazz.newInstance();
-      } catch (Exception e) {
-        unrecoverable = true;
-        errMsg = MessageFormat.format("class {0} could not be loaded ", _className);
-        this.exception = e;
-        return this;
-
-      }
-
-    }
-
-    @Override
-    public void close() throws Exception {
-      super.close();
-      if (_closed) return;
-      if (classLoader != null) classLoader.releaseJar();
-      _closed = true;
-    }
-  }
-
-
-  public static class MemClassLoader extends ClassLoader {
-    private JarRepository.JarContentRef jarContent;
-    private final DynamicLazyRequestHandlerWrapper handlerWrapper;
-
-    public MemClassLoader(DynamicLazyRequestHandlerWrapper handlerWrapper) {
-      super(handlerWrapper.solrResourceLoader.classLoader);
-      this.handlerWrapper = handlerWrapper;
-
-    }
-
-    boolean checkJarAvailable() throws IOException {
-      if (jarContent != null) return true;
-
-      try {
-        synchronized (this) {
-          jarContent = handlerWrapper.coreContainer.getJarRepository().getJarIncRef(handlerWrapper.lib + "/" + handlerWrapper.version);
-          return true;
-        }
-      } catch (SolrException se) {
-        throw se;
-      }
-
-    }
-
-    @Override
-    protected Class<?> findClass(String name) throws ClassNotFoundException {
-      try {
-        return super.findClass(name);
-      } catch (ClassNotFoundException e) {
-        String path = name.replace('.', '/').concat(".class");
-        ByteBuffer buf = null;
-        try {
-          if (jarContent == null) checkJarAvailable();
-          buf = jarContent.jar.getFileContent(path);
-          if (buf == null) throw new ClassNotFoundException("class not found in loaded jar" + name);
-        } catch (IOException e1) {
-          throw new ClassNotFoundException("class not found " + name, e1);
-
-        }
-
-        ProtectionDomain defaultDomain = null;
-
-        //using the default protection domain, with no permissions
-        try {
-          defaultDomain = new ProtectionDomain(new CodeSource(new URL("http://localhost/.system/blob/" + handlerWrapper.lib), (Certificate[]) null),
-              null);
-        } catch (MalformedURLException e1) {
-          //should not happen
-        }
-        return defineClass(name, buf.array(), buf.arrayOffset(), buf.limit(), defaultDomain);
-      }
-    }
-
-
-    private void releaseJar() {
-      handlerWrapper.coreContainer.getJarRepository().decrementJarRefCount(jarContent);
-    }
-
-  }
-
   public void close() {
-    for (Map.Entry<String, SolrRequestHandler> e : handlers.entrySet()) {
-      if (e.getValue() instanceof AutoCloseable) {
-        try {
-          ((AutoCloseable) e.getValue()).close();
-        } catch (Exception exp) {
-          log.error("Error closing requestHandler " + e.getKey(), exp);
-        }
-      }
-
-    }
-
+    handlers.close();
   }
 }
 

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrConfig.java?rev=1664797&r1=1664796&r2=1664797&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrConfig.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrConfig.java Sat Mar  7 05:55:35 2015
@@ -26,6 +26,7 @@ import org.apache.solr.cloud.ZkSolrResou
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.util.NamedList;
 import org.apache.solr.handler.component.SearchComponent;
 import org.apache.solr.request.SolrRequestHandler;
 import org.apache.solr.response.QueryResponseWriter;
@@ -45,8 +46,6 @@ import org.apache.solr.update.processor.
 import org.apache.solr.util.DOMUtil;
 import org.apache.solr.util.FileUtils;
 import org.apache.solr.util.RegexFileFilter;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.data.Stat;
 import org.noggit.JSONParser;
 import org.noggit.ObjectBuilder;
 import org.slf4j.Logger;
@@ -77,10 +76,14 @@ import java.util.Set;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+import static java.util.Collections.unmodifiableMap;
+import static org.apache.solr.common.params.CoreAdminParams.NAME;
+import static org.apache.solr.core.SolrConfig.PluginOpts.LAZY;
 import static org.apache.solr.core.SolrConfig.PluginOpts.MULTI_OK;
 import static org.apache.solr.core.SolrConfig.PluginOpts.NOOP;
 import static org.apache.solr.core.SolrConfig.PluginOpts.REQUIRE_CLASS;
 import static org.apache.solr.core.SolrConfig.PluginOpts.REQUIRE_NAME;
+import static org.apache.solr.schema.FieldType.CLASS_NAME;
 
 
 /**
@@ -99,6 +102,7 @@ public class SolrConfig extends Config i
     MULTI_OK, 
     REQUIRE_NAME,
     REQUIRE_CLASS,
+    LAZY,
     // EnumSet.of and/or EnumSet.copyOf(Collection) are anoying
     // because of type determination
     NOOP
@@ -296,9 +300,9 @@ public class SolrConfig extends Config i
   }
 
   public static final  List<SolrPluginInfo> plugins = ImmutableList.<SolrPluginInfo>builder()
-      .add(new SolrPluginInfo(SolrRequestHandler.class, SolrRequestHandler.TYPE, REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK))
+      .add(new SolrPluginInfo(SolrRequestHandler.class, SolrRequestHandler.TYPE, REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK, LAZY))
       .add(new SolrPluginInfo(QParserPlugin.class, "queryParser", REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK))
-      .add(new SolrPluginInfo(QueryResponseWriter.class, "queryResponseWriter", REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK))
+      .add(new SolrPluginInfo(QueryResponseWriter.class, "queryResponseWriter", REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK, LAZY))
       .add(new SolrPluginInfo(ValueSourceParser.class, "valueSourceParser", REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK))
       .add(new SolrPluginInfo(TransformerFactory.class, "transformer", REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK))
       .add(new SolrPluginInfo(SearchComponent.class, "searchComponent", REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK))
@@ -307,6 +311,7 @@ public class SolrConfig extends Config i
       // and even then -- only if there is a single SpellCheckComponent
       // because of queryConverter.setIndexAnalyzer
       .add(new SolrPluginInfo(QueryConverter.class, "queryConverter", REQUIRE_NAME, REQUIRE_CLASS))
+      .add(new SolrPluginInfo(PluginRegistry.RuntimeLib.class, "runtimeLib", REQUIRE_NAME, MULTI_OK))
       // this is hackish, since it picks up all SolrEventListeners,
       // regardless of when/how/why they are used (or even if they are
       // declared outside of the appropriate context) but there's no nice
@@ -323,10 +328,11 @@ public class SolrConfig extends Config i
       .add(new SolrPluginInfo(InitParams.class, InitParams.TYPE, MULTI_OK))
       .add(new SolrPluginInfo(StatsCache.class, "statsCache", REQUIRE_CLASS))
       .build();
-  private static final Map<String, SolrPluginInfo> clsVsInfo = new HashMap<>();
-
+  public static final Map<String, SolrPluginInfo> classVsSolrPluginInfo;
   static {
-    for (SolrPluginInfo plugin : plugins) clsVsInfo.put(plugin.clazz.getName(), plugin);
+    Map<String, SolrPluginInfo> map = new HashMap<>();
+    for (SolrPluginInfo plugin : plugins) map.put(plugin.clazz.getName(), plugin);
+    classVsSolrPluginInfo = Collections.unmodifiableMap(map);
   }
 
   public static class SolrPluginInfo{
@@ -634,7 +640,7 @@ public class SolrConfig extends Config i
    */
   public List<PluginInfo> getPluginInfos(String type) {
     List<PluginInfo> result = pluginStore.get(type);
-    SolrPluginInfo info = clsVsInfo.get(type);
+    SolrPluginInfo info = classVsSolrPluginInfo.get(type);
     if (info != null && info.options.contains(REQUIRE_NAME)) {
       Map<String, Map> infos = overlay.getNamedPlugins(info.tag);
       if (!infos.isEmpty()) {
@@ -664,14 +670,14 @@ public class SolrConfig extends Config i
   private void initLibs() {
     NodeList nodes = (NodeList) evaluate("lib", XPathConstants.NODESET);
     if (nodes == null || nodes.getLength() == 0) return;
-    
+
     log.info("Adding specified lib dirs to ClassLoader");
     SolrResourceLoader loader = getResourceLoader();
-    
+
     try {
       for (int i = 0; i < nodes.getLength(); i++) {
         Node node = nodes.item(i);
-        
+
         String baseDir = DOMUtil.getAttr(node, "dir");
         String path = DOMUtil.getAttr(node, "path");
         if (null != baseDir) {
@@ -696,7 +702,7 @@ public class SolrConfig extends Config i
       loader.reloadLuceneSPI();
     }
   }
-  
+
   public int getMultipartUploadLimitKB() {
     return multipartUploadLimitKB;
   }