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 2014/12/23 11:35:17 UTC

svn commit: r1647539 - in /lucene/dev/branches/branch_5x/solr: ./ core/ core/src/java/org/apache/solr/core/ core/src/java/org/apache/solr/handler/ core/src/java/org/apache/solr/servlet/ core/src/test/org/apache/solr/core/ core/src/test/org/apache/solr/...

Author: noble
Date: Tue Dec 23 10:35:17 2014
New Revision: 1647539

URL: http://svn.apache.org/r1647539
Log:
SOLR-6801 Load RequestHandler from blob store

Added:
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/JarRepository.java
      - copied, changed from r1646964, lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/JarRepository.java
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/BlobStoreTestRequestHandler.java
      - copied, changed from r1646964, lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/BlobStoreTestRequestHandler.java
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestDynamicLoading.java
      - copied, changed from r1646964, lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestDynamicLoading.java
Modified:
    lucene/dev/branches/branch_5x/solr/   (props changed)
    lucene/dev/branches/branch_5x/solr/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_5x/solr/core/   (props changed)
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/CoreContainer.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/RequestHandlers.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrCore.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/handler/TestBlobHandler.java

Modified: lucene/dev/branches/branch_5x/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/CHANGES.txt?rev=1647539&r1=1647538&r2=1647539&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/CHANGES.txt (original)
+++ lucene/dev/branches/branch_5x/solr/CHANGES.txt Tue Dec 23 10:35:17 2014
@@ -208,6 +208,8 @@ New Features
 
 * SOLR-6787: API to manage blobs in Solr (Noble Paul)
 
+* SOLR-6801:  Load RequestHandler from blob store (Noble Paul)
+
 Bug Fixes
 ----------------------
 

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/CoreContainer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/CoreContainer.java?rev=1647539&r1=1647538&r2=1647539&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/CoreContainer.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/CoreContainer.java Tue Dec 23 10:35:17 2014
@@ -102,6 +102,7 @@ public class CoreContainer {
   protected final CoresLocator coresLocator;
   
   private String hostName;
+  private final JarRepository jarRepository = new JarRepository(this);
   
   private Map<String ,SolrRequestHandler> containerHandlers = new HashMap<>();
 
@@ -790,6 +791,10 @@ public class CoreContainer {
     return core;
   }
 
+  public JarRepository getJarRepository(){
+    return jarRepository;
+  }
+
   // ---------------- CoreContainer request handlers --------------
 
   protected <T> T createHandler(String handlerClass, Class<T> clazz) {

Copied: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/JarRepository.java (from r1646964, lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/JarRepository.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/JarRepository.java?p2=lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/JarRepository.java&p1=lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/JarRepository.java&r1=1646964&r2=1647539&rev=1647539&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/JarRepository.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/JarRepository.java Tue Dec 23 10:35:17 2014
@@ -17,6 +17,8 @@ package org.apache.solr.core;
  * limitations under the License.
  */
 
+import static org.apache.solr.common.SolrException.ErrorCode.SERVICE_UNAVAILABLE;
+import static org.apache.solr.common.cloud.ZkStateReader.BASE_URL_PROP;
 
 import java.io.ByteArrayInputStream;
 import java.io.IOException;
@@ -36,87 +38,84 @@ 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.handler.BlobHandler;
 import org.apache.solr.handler.admin.CollectionsHandler;
 import org.apache.solr.util.SimplePostTool;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.solr.common.SolrException.ErrorCode.SERVICE_UNAVAILABLE;
-import static org.apache.solr.common.cloud.ZkStateReader.BASE_URL_PROP;
-
-/**The purpose of this class is to store the Jars loaded in memory and to keep
- * only one copy of the Jar in a single node.
+/**
+ * The purpose of this class is to store the Jars loaded in memory and to keep only one copy of the Jar in a single node.
  */
 public class JarRepository {
   public static Logger log = LoggerFactory.getLogger(JarRepository.class);
-
+  
   private final CoreContainer coreContainer;
-
-  private Map<String, JarContent> jars = new ConcurrentHashMap<>();
-
+  
+  private Map<String,JarContent> jars = new ConcurrentHashMap<>();
+  
   public JarRepository(CoreContainer coreContainer) {
     this.coreContainer = coreContainer;
   }
-
-  /**Returns the contents of a jar and increments a reference count. Please return the same
-   * object to decerease the refcount
-   * @param key it is a combination of blobname and version like blobName/version
+  
+  /**
+   * Returns the contents of a jar and increments a reference count. Please return the same object to decerease 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 {
     JarContent jar = jars.get(key);
-    if(jar ==null){
-      if(this.coreContainer.isZooKeeperAware()){
+    if (jar == null) {
+      if (this.coreContainer.isZooKeeperAware()) {
         ClusterState cs = this.coreContainer.getZkController().getZkStateReader().getClusterState();
         DocCollection coll = cs.getCollectionOrNull(CollectionsHandler.SYSTEM_COLL);
-        if(coll == null) throw new SolrException(SERVICE_UNAVAILABLE,
-            ".system collection not available");
+        if (coll == null) throw new SolrException(SERVICE_UNAVAILABLE, ".system collection not available");
         Slice slice = coll.getActiveSlices().iterator().next();
-        if(slice == null) throw new SolrException(SERVICE_UNAVAILABLE,
-            ".no active slices for .system collection");
+        if (slice == null) throw new SolrException(SERVICE_UNAVAILABLE, ".no active slices for .system collection");
         Replica replica = slice.getReplicas().iterator().next();
-        if(replica == null) throw new SolrException(SERVICE_UNAVAILABLE,
-            ".no active replica available for .system collection");
-        String url = replica.getStr(BASE_URL_PROP) + "/.system/blob/"+ key+"?wt=filestream";
-
+        if (replica == null) throw new SolrException(SERVICE_UNAVAILABLE, ".no active replica available for .system collection");
+        String url = replica.getStr(BASE_URL_PROP) + "/.system/blob/" + key + "?wt=filestream";
+        
         HttpClient httpClient = coreContainer.getUpdateShardHandler().getHttpClient();
         HttpGet httpGet = new HttpGet(url);
         ByteBuffer b;
         try {
-          HttpResponse entity =  httpClient.execute(httpGet);
+          HttpResponse entity = httpClient.execute(httpGet);
           int statusCode = entity.getStatusLine().getStatusCode();
-          if(statusCode != 200){
-            throw new SolrException(SolrException.ErrorCode.NOT_FOUND,"no such blob or version available: "+ key);
+          if (statusCode != 200) {
+            throw new SolrException(SolrException.ErrorCode.NOT_FOUND, "no such blob or version available: " + key);
           }
           b = SimplePostTool.inputStreamToByteArray(entity.getEntity().getContent());
         } finally {
           httpGet.releaseConnection();
         }
-        jars.put(key,jar = new JarContent(key,b));
+        jars.put(key, jar = new JarContent(key, b));
       } else {
-
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,"Jar loading is not supported in non-cloud mode");
-        //todo
-
+        
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Jar loading is not supported in non-cloud mode");
+        // todo
+        
       }
-
+      
     }
-
+    
     JarContentRef ref = new JarContentRef(jar);
     synchronized (jar.references) {
       jar.references.add(ref);
     }
     return ref;
-
+    
   }
-
-
-  /**This is to decrement a ref count
-   * @param ref The reference that is already there. Doing multiple calls with same ref will not matter
+  
+  /**
+   * This is to decrement a ref count
+   * 
+   * @param ref
+   *          The reference that is already there. Doing multiple calls with same ref will not matter
    */
-  public void decrementJarRefCount(JarContentRef ref){
-    if(ref == null) return;
+  public void decrementJarRefCount(JarContentRef ref) {
+    if (ref == null) return;
     synchronized (ref.jar.references) {
       if (!ref.jar.references.remove(ref)) {
         log.error("Multiple releases for the same reference");
@@ -125,34 +124,33 @@ public class JarRepository {
         jars.remove(ref.jar.key);
       }
     }
-
+    
   }
-
-
+  
   public static class JarContent {
     private final String key;
-    //TODO move this off-heap
+    // TODO move this off-heap
     private final ByteBuffer buffer;
     // ref counting mechanism
     private final Set<JarContentRef> references = new HashSet<>();
-
+    
     public JarContent(String key, ByteBuffer buffer) {
       this.key = key;
       this.buffer = buffer;
     }
-
+    
     public ByteBuffer getFileContent(String entryName) throws IOException {
-      ByteArrayInputStream zipContents=new ByteArrayInputStream(buffer.array(),buffer.arrayOffset(),buffer.limit());
-      ZipInputStream zis=new ZipInputStream(zipContents);
+      ByteArrayInputStream zipContents = new ByteArrayInputStream(buffer.array(), buffer.arrayOffset(), buffer.limit());
+      ZipInputStream zis = new ZipInputStream(zipContents);
       try {
         ZipEntry entry;
-        while ((entry=zis.getNextEntry()) != null) {
+        while ((entry = zis.getNextEntry()) != null) {
           if (entryName == null || entryName.equals(entry.getName())) {
-            SimplePostTool.BAOS out=new SimplePostTool.BAOS();
-            byte[] buffer=new byte[2048];
+            SimplePostTool.BAOS out = new SimplePostTool.BAOS();
+            byte[] buffer = new byte[2048];
             int size;
-            while ((size=zis.read(buffer,0,buffer.length)) != -1) {
-              out.write(buffer,0,size);
+            while ((size = zis.read(buffer, 0, buffer.length)) != -1) {
+              out.write(buffer, 0, size);
             }
             out.close();
             return out.getByteBuffer();
@@ -163,14 +161,15 @@ public class JarRepository {
       }
       return null;
     }
-
+    
   }
-
-  public static class JarContentRef  {
+  
+  public static class JarContentRef {
     public final JarContent jar;
+    
     private JarContentRef(JarContent jar) {
       this.jar = jar;
     }
   }
-
+  
 }

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/RequestHandlers.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/RequestHandlers.java?rev=1647539&r1=1647538&r2=1647539&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/RequestHandlers.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/RequestHandlers.java Tue Dec 23 10:35:17 2014
@@ -17,7 +17,19 @@
 
 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;
@@ -25,13 +37,17 @@ 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;
@@ -51,6 +67,8 @@ public final class RequestHandlers {
       new ConcurrentHashMap<>() ;
   private final Map<String, SolrRequestHandler> immutableHandlers = Collections.unmodifiableMap(handlers) ;
 
+  public static final boolean disableExternalLib = Boolean.parseBoolean(System.getProperty("disable.external.lib", "false"));
+
   /**
    * Trim the trailing '/' if its there, and convert null to empty string.
    * 
@@ -101,8 +119,8 @@ public final class RequestHandlers {
    */
   public SolrRequestHandler register( String handlerName, SolrRequestHandler handler ) {
     String norm = normalize( handlerName );
-    if( handler == null ) {
-      return handlers.remove( norm );
+    if (handler == null) {
+      return handlers.remove(norm);
     }
     SolrRequestHandler old = handlers.put(norm, handler);
     if (0 != norm.length() && handler instanceof SolrInfoMBean) {
@@ -147,7 +165,7 @@ public final class RequestHandlers {
     //deduping implicit and explicit requesthandlers
     for (PluginInfo info : implicits) infoMap.put(info.name,info);
     for (PluginInfo info : config.getPluginInfos(SolrRequestHandler.class.getName()))
-      if(infoMap.containsKey(info.name)) infoMap.remove(info.name);
+      if (infoMap.containsKey(info.name)) infoMap.remove(info.name);
     for (Map.Entry e : core.getSolrConfig().getOverlay().getReqHandlers().entrySet())
       infoMap.put((String)e.getKey(), new PluginInfo(SolrRequestHandler.TYPE, (Map)e.getValue()));
 
@@ -156,28 +174,32 @@ public final class RequestHandlers {
     for (PluginInfo info : infos) {
       try {
         SolrRequestHandler requestHandler;
-        String startup = info.attributes.get("startup") ;
-        if( startup != null ) {
-          if( "lazy".equals(startup) ) {
+        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, info.className);
+            requestHandler = new LazyRequestHandlerWrapper(core);
           } else {
-            throw new Exception( "Unknown startup value: '"+startup+"' for: "+info.className );
+            throw new Exception("Unknown startup value: '" + startup + "' for: " + info.className);
           }
         } else {
           requestHandler = core.createRequestHandler(info.className);
         }
-        handlers.put(info,requestHandler);
+        if (requestHandler instanceof RequestHandlerBase) ((RequestHandlerBase) requestHandler).setPluginInfo(info);
+        
+        handlers.put(info, requestHandler);
         SolrRequestHandler old = register(info.name, requestHandler);
-        if(old != null) {
+        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()); 
+        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);
+        log.info("created " + info.name + ": " + info.className);
       } catch (Exception ex) {
           throw new SolrException
             (ErrorCode.SERVER_ERROR, "RequestHandler init failure", ex);
@@ -242,25 +264,22 @@ public final class RequestHandlers {
    * 
    * @since solr 1.2
    */
-  public static final class LazyRequestHandlerWrapper implements SolrRequestHandler , PluginInfoInitialized
+  public static class LazyRequestHandlerWrapper implements SolrRequestHandler , AutoCloseable, PluginInfoInitialized
   {
     private final SolrCore core;
-    private String _className;
-    private SolrRequestHandler _handler;
-    private PluginInfo _pluginInfo;
+    String _className;
+    SolrRequestHandler _handler;
+    PluginInfo _pluginInfo;
     
-    public LazyRequestHandlerWrapper( SolrCore core, String className )
+    public LazyRequestHandlerWrapper( SolrCore core)
     {
       this.core = core;
-      _className = className;
       _handler = null; // don't initialize
     }
 
     @Override
     public void init(NamedList args) { }
 
-
-
     /**
      * Wait for the first request before initializing the wrapped handler 
      */
@@ -277,7 +296,12 @@ public final class RequestHandlers {
     {
       if( _handler == null ) {
         try {
-          SolrRequestHandler handler = core.createRequestHandler(_className);
+          SolrRequestHandler handler = createRequestHandler();
+          if (handler instanceof PluginInfoInitialized) {
+            ((PluginInfoInitialized) handler).init(_pluginInfo);
+          } else {
+            handler.init( _pluginInfo.initArgs );
+          }
 
           if (handler instanceof PluginInfoInitialized) {
             ((PluginInfoInitialized) handler).init(_pluginInfo);
@@ -298,6 +322,10 @@ public final class RequestHandlers {
       return _handler;
     }
 
+    protected SolrRequestHandler createRequestHandler() {
+      return core.createRequestHandler(_className);
+    }
+
     public String getHandlerClass()
     {
       return _className;
@@ -357,10 +385,191 @@ public final class RequestHandlers {
     }
 
     @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;
+      }
+
+      version = _pluginInfo.attributes.get("version");
+      if (version == null) {
+        errMsg = "ERROR 'lib' attribute must be accompanied with version also";
+        unrecoverable = true;
+        _handler = this;
+        log.error(errMsg);
+        return;
+      }
+      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;
+      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);
+        }
+      }
 
     }
+
   }
 }
 

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrCore.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrCore.java?rev=1647539&r1=1647538&r2=1647539&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrCore.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrCore.java Tue Dec 23 10:35:17 2014
@@ -1069,6 +1069,7 @@ public final class SolrCore implements S
       }
     }
 
+    if(reqHandlers != null) reqHandlers.close();
 
 
     try {

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java?rev=1647539&r1=1647538&r2=1647539&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java Tue Dec 23 10:35:17 2014
@@ -21,6 +21,7 @@ import org.apache.solr.common.SolrExcept
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.core.PluginInfo;
 import org.apache.solr.core.RequestHandlers;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrInfoMBean;
@@ -54,6 +55,7 @@ public abstract class RequestHandlerBase
   private final AtomicLong numTimeouts = new AtomicLong();
   private final Timer requestTimes = new Timer();
   private final long handlerStart = System.currentTimeMillis();
+  private PluginInfo pluginInfo;
 
   /**
    * Initializes the {@link org.apache.solr.request.SolrRequestHandler} by creating three {@link org.apache.solr.common.params.SolrParams} named.
@@ -233,6 +235,18 @@ public abstract class RequestHandlerBase
     return handler;
   }
 
+  /**
+   *
+   * @param pluginInfo
+   */
+  public void setPluginInfo(PluginInfo pluginInfo){
+    if(pluginInfo==null) this.pluginInfo = pluginInfo;
+  }
+
+  public PluginInfo getPluginInfo(){
+    return  pluginInfo;
+  }
+
 
   @Override
   public NamedList<Object> getStatistics() {

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java?rev=1647539&r1=1647538&r2=1647539&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java Tue Dec 23 10:35:17 2014
@@ -121,7 +121,7 @@ public class SolrDispatchFilter extends
   @Override
   public void init(FilterConfig config) throws ServletException
   {
-    log.info("SolrDispatchFilter.init()");
+    log.info("SolrDispatchFilter.init()" + this.getClass().getClassLoader());
 
     try {
       // web.xml configuration

Copied: lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/BlobStoreTestRequestHandler.java (from r1646964, lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/BlobStoreTestRequestHandler.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/BlobStoreTestRequestHandler.java?p2=lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/BlobStoreTestRequestHandler.java&p1=lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/BlobStoreTestRequestHandler.java&r1=1646964&r2=1647539&rev=1647539&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/BlobStoreTestRequestHandler.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/BlobStoreTestRequestHandler.java Tue Dec 23 10:35:17 2014
@@ -28,6 +28,6 @@ public class BlobStoreTestRequestHandler
   @Override
   public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws IOException {
     super.handleRequestBody(req, rsp);
-    rsp.add("classloader", this.getClass().getClassLoader().getClass().getName());
+    rsp.add("class", BlobStoreTestRequestHandler.class.getName());
   }
 }

Copied: lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestDynamicLoading.java (from r1646964, lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestDynamicLoading.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestDynamicLoading.java?p2=lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestDynamicLoading.java&p1=lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestDynamicLoading.java&r1=1646964&r2=1647539&rev=1647539&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestDynamicLoading.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestDynamicLoading.java Tue Dec 23 10:35:17 2014
@@ -20,6 +20,7 @@ package org.apache.solr.core;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
@@ -30,6 +31,7 @@ import java.util.zip.ZipOutputStream;
 import org.apache.solr.client.solrj.SolrServer;
 import org.apache.solr.client.solrj.impl.HttpSolrServer;
 import org.apache.solr.cloud.AbstractFullDistribZkTestBase;
+import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.handler.TestBlobHandler;
 import org.apache.solr.util.RESTfulServerProvider;
 import org.apache.solr.util.RestTestHarness;
@@ -94,11 +96,16 @@ public class TestDynamicLoading extends
     ByteBuffer jar = generateZip( TestDynamicLoading.class,BlobStoreTestRequestHandler.class);
     TestBlobHandler.postAndCheck(cloudClient, baseURL, jar,1);
 
-//    Thread.sleep(100);
-    map = TestSolrConfigHandler.getRespMap("/test1?wt=json", client);
-
-    assertEquals(RequestHandlers.MemClassLoader.class.getName(), map.get("classloader"));
-
+    boolean success= false;
+    for(int i=0;i<50;i++) {
+      map = TestSolrConfigHandler.getRespMap("/test1?wt=json", client);
+      if(BlobStoreTestRequestHandler.class.getName().equals(map.get("class"))){
+        success = true;
+        break;
+      }
+      Thread.sleep(100);
+    }
+    assertTrue(new String( ZkStateReader.toJSON(map) , StandardCharsets.UTF_8), success );
 
   }
 

Modified: lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java?rev=1647539&r1=1647538&r2=1647539&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java Tue Dec 23 10:35:17 2014
@@ -21,22 +21,19 @@ package org.apache.solr.core;
 import java.io.File;
 import java.io.IOException;
 import java.io.StringReader;
-import java.nio.charset.StandardCharsets;
 import java.util.Arrays;
 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.concurrent.TimeUnit;
 
 import com.google.common.collect.ImmutableList;
 import org.apache.commons.io.FileUtils;
-import org.apache.http.HttpEntity;
-import org.apache.http.client.methods.HttpGet;
-import org.apache.http.util.EntityUtils;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.client.solrj.impl.CloudSolrServer;
-import org.apache.solr.client.solrj.impl.HttpSolrServer;
 import org.apache.solr.handler.TestSolrConfigHandlerConcurrent;
 import org.apache.solr.util.RestTestBase;
 import org.apache.solr.util.RestTestHarness;
@@ -142,7 +139,7 @@ public class TestSolrConfigHandler exten
     reqhandlertests(restTestHarness, null,null);
   }
 
-  private static void runConfigCommand(RestTestHarness harness, String uri,  String payload) throws IOException {
+  public static void runConfigCommand(RestTestHarness harness, String uri,  String payload) throws IOException {
     String response = harness.post(uri, SolrTestCaseJ4.json(payload));
     Map map = (Map) ObjectBuilder.getVal(new JSONParser(new StringReader(response)));
     assertNull(response,  map.get("errors"));
@@ -155,53 +152,34 @@ public class TestSolrConfigHandler exten
         "}";
     runConfigCommand(writeHarness,"/config?wt=json", payload);
 
-    boolean success = false;
-    long startTime = System.nanoTime();
-    long maxTimeoutSeconds = 10;
-    while ( TimeUnit.SECONDS.convert(System.nanoTime() - startTime, TimeUnit.NANOSECONDS) < maxTimeoutSeconds) {
-      String uri = "/config/overlay?wt=json";
-      Map m = testServerBaseUrl ==null?  getRespMap(uri,writeHarness) : TestSolrConfigHandlerConcurrent.getAsMap(testServerBaseUrl+uri ,cloudSolrServer) ;
-      if("lazy".equals( ConfigOverlay.getObjectByPath(m,  true, Arrays.asList("overlay", "requestHandler", "/x","startup")))) {
-        Map map = getRespMap("/x?wt=json",writeHarness);
-        if(map.containsKey("params")) {
-          success = true;
-          break;
-        }
-      }
-      Thread.sleep(100);
-
-    }
-
-    assertTrue( "Could not register requestHandler  ", success);
+    testForResponseElement(writeHarness,
+        testServerBaseUrl,
+        "/config/overlay?wt=json",
+        cloudSolrServer,
+        Arrays.asList("overlay", "requestHandler", "/x", "startup"),
+        "lazy",
+        10);
 
     payload = "{\n" +
         "'update-requesthandler' : { 'name' : '/x', 'class': 'org.apache.solr.handler.DumpRequestHandler' , 'startup' : 'lazy' , 'a':'b'}\n" +
         "}";
     runConfigCommand(writeHarness,"/config?wt=json", payload);
 
-    success = false;
-    startTime = System.nanoTime();
-    maxTimeoutSeconds = 10;
-    while ( TimeUnit.SECONDS.convert(System.nanoTime() - startTime, TimeUnit.NANOSECONDS) < maxTimeoutSeconds) {
-      String uri = "/config/overlay?wt=json";
-      Map m = testServerBaseUrl ==null?  getRespMap(uri,writeHarness) : TestSolrConfigHandlerConcurrent.getAsMap(testServerBaseUrl+uri ,cloudSolrServer) ;
-      if("b".equals( ConfigOverlay.getObjectByPath(m,  true, Arrays.asList("overlay", "requestHandler", "/x","a")))) {
-          success = true;
-          break;
-      }
-      Thread.sleep(100);
-
-    }
-
-    assertTrue( "Could not update requestHandler  ", success);
+    testForResponseElement(writeHarness,
+        testServerBaseUrl,
+        "/config/overlay?wt=json",
+        cloudSolrServer,
+        Arrays.asList("overlay", "requestHandler", "/x", "a"),
+        "b",
+        10);
 
     payload = "{\n" +
         "'delete-requesthandler' : '/x'" +
         "}";
     runConfigCommand(writeHarness,"/config?wt=json", payload);
-    success = false;
-    startTime = System.nanoTime();
-    maxTimeoutSeconds = 10;
+    boolean success = false;
+    long startTime = System.nanoTime();
+    int maxTimeoutSeconds = 10;
     while ( TimeUnit.SECONDS.convert(System.nanoTime() - startTime, TimeUnit.NANOSECONDS) < maxTimeoutSeconds) {
       String uri = "/config/overlay?wt=json";
       Map m = testServerBaseUrl ==null?  getRespMap(uri,writeHarness) : TestSolrConfigHandlerConcurrent.getAsMap(testServerBaseUrl+uri ,cloudSolrServer) ;
@@ -216,6 +194,33 @@ public class TestSolrConfigHandler exten
 
   }
 
+  public static void testForResponseElement(RestTestHarness harness,
+                                            String testServerBaseUrl,
+                                            String uri,
+                                            CloudSolrServer cloudSolrServer,List<String> jsonPath,
+                                            String expected,
+                                            long maxTimeoutSeconds ) throws Exception {
+
+    boolean success = false;
+    long startTime = System.nanoTime();
+    while ( TimeUnit.SECONDS.convert(System.nanoTime() - startTime, TimeUnit.NANOSECONDS) < maxTimeoutSeconds) {
+      Map m = testServerBaseUrl ==null?  getRespMap(uri,harness) : TestSolrConfigHandlerConcurrent.getAsMap(testServerBaseUrl + uri, cloudSolrServer) ;
+      if(Objects.equals(expected,ConfigOverlay.getObjectByPath(m, true, jsonPath))) {
+        success = true;
+        break;
+        /*Map map = getRespMap("/x?wt=json",harness);
+        if(map.containsKey("params")) {
+          success = true;
+          break;
+        }*/
+      }
+      Thread.sleep(100);
+
+    }
+
+    assertTrue( "Could not add/change requestHandler  ", success);
+  }
+
 
   public static Map getRespMap(String path, RestTestHarness restHarness) throws Exception {
     String response = restHarness.query(path);

Modified: lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/handler/TestBlobHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/handler/TestBlobHandler.java?rev=1647539&r1=1647538&r2=1647539&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/handler/TestBlobHandler.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/handler/TestBlobHandler.java Tue Dec 23 10:35:17 2014
@@ -17,13 +17,7 @@ package org.apache.solr.handler;
  * limitations under the License.
  */
 
-import java.io.ByteArrayOutputStream;
-import java.io.DataOutputStream;
 import java.io.IOException;
-import java.io.InputStream;
-import java.net.HttpURLConnection;
-import java.net.URL;
-import java.net.URLConnection;
 import java.nio.ByteBuffer;
 import java.nio.charset.StandardCharsets;
 import java.util.Arrays;
@@ -31,7 +25,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
-import com.google.common.io.Closeables;
 import org.apache.http.HttpEntity;
 import org.apache.http.HttpResponse;
 import org.apache.http.client.HttpClient;
@@ -41,6 +34,7 @@ import org.apache.http.entity.ByteArrayE
 import org.apache.http.util.EntityUtils;
 import org.apache.solr.client.solrj.SolrServer;
 import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.impl.CloudSolrServer;
 import org.apache.solr.client.solrj.impl.HttpSolrServer;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.response.CollectionAdminResponse;
@@ -57,7 +51,7 @@ import org.slf4j.LoggerFactory;
 import static org.apache.solr.core.ConfigOverlay.getObjectByPath;
 
 public class TestBlobHandler extends AbstractFullDistribZkTestBase {
-  static final Logger log =  LoggerFactory.getLogger(TestSolrConfigHandlerConcurrent.class);
+  static final Logger log =  LoggerFactory.getLogger(TestBlobHandler.class);
 
   private void doBlobHandlerTest() throws Exception {
     SolrServer server = createNewSolrServer("", getBaseUrl((HttpSolrServer) clients.get(0)));
@@ -87,8 +81,8 @@ public class TestBlobHandler extends Abs
     for (int i = 0; i < bytarr.length; i++) bytarr[i]= (byte) (i % 127);
     byte[] bytarr2  = new byte[2048];
     for (int i = 0; i < bytarr2.length; i++) bytarr2[i]= (byte) (i % 127);
-    postAndCheck(baseUrl, bytarr, 1);
-    postAndCheck(baseUrl, bytarr2, 2);
+    postAndCheck(cloudClient, baseUrl, ByteBuffer.wrap( bytarr), 1);
+    postAndCheck(cloudClient, baseUrl, ByteBuffer.wrap( bytarr2), 2);
 
     url = baseUrl + "/.system/blob/test/1";
     map = TestSolrConfigHandlerConcurrent.getAsMap(url,cloudClient);
@@ -102,6 +96,17 @@ public class TestBlobHandler extends Abs
 
   }
 
+  public static  void createSysColl(SolrServer server) throws SolrServerException, IOException {
+    CollectionAdminResponse response1;
+    CollectionAdminRequest.Create createCollectionRequest = new CollectionAdminRequest.Create();
+    createCollectionRequest.setCollectionName(".system");
+    createCollectionRequest.setNumShards(1);
+    createCollectionRequest.setReplicationFactor(2);
+    response1 = createCollectionRequest.process(server);
+    assertEquals(0, response1.getStatus());
+    assertTrue(response1.isSuccess());
+  }
+
   @Override
   public void tearDown() throws Exception {
     super.tearDown();
@@ -112,8 +117,8 @@ public class TestBlobHandler extends Abs
     DirectUpdateHandler2.commitOnClose = true;
   }
 
-  private void postAndCheck(String baseUrl, byte[] bytes, int count) throws Exception {
-    postData(baseUrl, bytes);
+  public static void postAndCheck(CloudSolrServer cloudClient, String baseUrl, ByteBuffer bytes, int count) throws Exception {
+    postData(cloudClient, baseUrl, bytes);
     String url;
     Map map;
     List l;
@@ -132,7 +137,7 @@ public class TestBlobHandler extends Abs
       l = (List) ConfigOverlay.getObjectByPath(map, false, Arrays.asList("response", "docs"));
       assertNotNull(l);
       map = (Map) l.get(0);
-      assertEquals("" + bytes.length, String.valueOf(map.get("size")));
+      assertEquals("" + bytes.limit(), String.valueOf(map.get("size")));
       break;
     }
   }
@@ -155,14 +160,14 @@ public class TestBlobHandler extends Abs
 
   }
 
-  private String postData(String baseUrl, byte[] bytarr) throws IOException {
+  public static String postData(CloudSolrServer cloudClient, String baseUrl, ByteBuffer bytarr) throws IOException {
     HttpPost httpPost = null;
     HttpEntity entity;
     String response;
     try {
       httpPost = new HttpPost(baseUrl+"/.system/blob/test");
       httpPost.setHeader("Content-Type","application/octet-stream");
-      httpPost.setEntity(new ByteArrayEntity(bytarr));
+      httpPost.setEntity(new ByteArrayEntity(bytarr.array(), bytarr.arrayOffset(), bytarr.limit()));
       entity = cloudClient.getLbServer().getHttpClient().execute(httpPost).getEntity();
       return EntityUtils.toString(entity, StandardCharsets.UTF_8);
     } finally {