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 2020/04/13 00:06:59 UTC

[lucene-solr] 01/01: initial commit

This is an automated email from the ASF dual-hosted git repository.

noble pushed a commit to branch jira/solr14404
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git

commit 87a57bc680352018da29a6698bda53d7f3f71c82
Author: noble <no...@apache.org>
AuthorDate: Mon Apr 13 10:06:19 2020 +1000

    initial commit
---
 .../src/java/org/apache/solr/api/AnnotatedApi.java |   4 +
 solr/core/src/java/org/apache/solr/api/ApiBag.java |   9 +
 .../apache/solr/api/CustomContainerPlugins.java    | 272 +++++++++++++++++++++
 .../java/org/apache/solr/core/CoreContainer.java   |  21 +-
 .../src/java/org/apache/solr/core/PluginInfo.java  |   2 +-
 .../solr/handler/admin/ContainerPluginsApi.java    | 158 ++++++++++++
 .../java/org/apache/solr/pkg/PackageListeners.java |   4 +-
 .../solr/handler/admin/TestApiFramework.java       |  20 ++
 .../client/solrj/request/beans/PluginMeta.java     |  32 +++
 .../java/org/apache/solr/common/util/PathTrie.java |  35 ++-
 10 files changed, 545 insertions(+), 12 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/api/AnnotatedApi.java b/solr/core/src/java/org/apache/solr/api/AnnotatedApi.java
index 6cd65cd..f15e1d0 100644
--- a/solr/core/src/java/org/apache/solr/api/AnnotatedApi.java
+++ b/solr/core/src/java/org/apache/solr/api/AnnotatedApi.java
@@ -70,6 +70,10 @@ public class AnnotatedApi extends Api implements PermissionNameProvider {
     this(obj, null);
   }
 
+  public EndPoint getEndPoint() {
+    return endPoint;
+  }
+
   public AnnotatedApi(Object obj, Api fallback) {
     super(readSpec(obj.getClass()));
     this.fallback = fallback;
diff --git a/solr/core/src/java/org/apache/solr/api/ApiBag.java b/solr/core/src/java/org/apache/solr/api/ApiBag.java
index 8a3f972..980c115 100644
--- a/solr/core/src/java/org/apache/solr/api/ApiBag.java
+++ b/solr/core/src/java/org/apache/solr/api/ApiBag.java
@@ -31,6 +31,7 @@ import java.util.stream.Collectors;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
+import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SpecProvider;
 import org.apache.solr.common.util.CommandOperation;
@@ -134,6 +135,14 @@ public class ApiBag {
     registry.insert(copy, substitutes, introspect);
   }
 
+  public void unregister(SolrRequest.METHOD method, String path) {
+    List<String> l = PathTrie.getPathSegments(path);
+    List<String> introspectPath = new ArrayList<>(l);
+    introspectPath.add("_introspect");
+    getRegistry(method.toString()).unregister(introspectPath);
+    getRegistry(method.toString()).unregister(l);
+  }
+
   public static class IntrospectApi extends Api {
     Api baseApi;
     final boolean isCoreSpecific;
diff --git a/solr/core/src/java/org/apache/solr/api/CustomContainerPlugins.java b/solr/core/src/java/org/apache/solr/api/CustomContainerPlugins.java
new file mode 100644
index 0000000..1d68846
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/api/CustomContainerPlugins.java
@@ -0,0 +1,272 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.api;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.lang.reflect.Constructor;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableSet;
+import org.apache.solr.client.solrj.request.beans.PluginMeta;
+import org.apache.solr.common.MapWriter;
+import org.apache.solr.common.annotation.JsonProperty;
+import org.apache.solr.common.cloud.ClusterPropertiesListener;
+import org.apache.solr.common.util.Pair;
+import org.apache.solr.common.util.ReflectMapWriter;
+import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.handler.admin.ContainerPluginsApi;
+import org.apache.solr.pkg.PackageLoader;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.util.SolrJacksonAnnotationInspector;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CustomContainerPlugins implements MapWriter, ClusterPropertiesListener {
+  private ObjectMapper mapper = SolrJacksonAnnotationInspector.createObjectMapper();
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  final CoreContainer coreContainer;
+  final ApiBag containerApiBag;
+  private Map<String, ApiHolder> plugins = new HashMap<>();
+  private Map<String, String> pluginNameVsPath = new HashMap<>();
+
+  @Override
+  public boolean onChange(Map<String, Object> properties) {
+    refresh(properties);
+    return false;
+  }
+
+  @Override
+  public void writeMap(EntryWriter ew) {
+    plugins.forEach((s, apiHolder) -> ew.putNoEx(s, apiHolder.apiInfo));
+  }
+
+  public CustomContainerPlugins(CoreContainer coreContainer, ApiBag apiBag) {
+    this.coreContainer = coreContainer;
+    this.containerApiBag = apiBag;
+  }
+
+  public void refresh(Map<String, Object> pluginInfos) {
+    try {
+      if(pluginInfos == null) pluginInfos = ContainerPluginsApi.plugins(coreContainer.zkClientSupplier);
+    } catch (IOException e) {
+      log.error("Could not read plugins data", e);
+    }
+
+    for (Map.Entry<String, Object> e : pluginInfos.entrySet()) {
+      PluginMeta info = null;
+      try {
+        info = mapper.readValue(Utils.toJSON(e.getValue()), PluginMeta.class);
+      } catch (IOException ioException) {
+        log.error("Invalid apiInfo configuration :", ioException);
+      }
+
+      ApiInfo apiInfo = null;
+      try {
+        List<String> errs = new ArrayList<>();
+        apiInfo = new ApiInfo(info, errs);
+        if(!errs.isEmpty()){
+          log.error(StrUtils.join(errs, ','));
+          continue;
+        }
+      } catch (Exception ex) {
+        log.error("unable to instantiate apiInfo ", ex);
+        continue;
+      }
+
+      String path = pluginNameVsPath.get(e.getKey());
+      if (path == null) {
+        // there is a new apiInfo . let's register it
+        try {
+          apiInfo.init();
+          ApiHolder holder = new ApiHolder(apiInfo);
+          plugins.put(holder.key, holder);
+          pluginNameVsPath.put(apiInfo.info.name, holder.key);
+          containerApiBag.register(holder, Collections.EMPTY_MAP);
+        } catch (Exception exp) {
+          log.error("Cannot install apiInfo ", exp);
+        }
+      } else {
+        ApiHolder old = plugins.get(apiInfo.key);
+        if (path.equals(apiInfo.key)) {
+          if (Objects.equals(info.packageVersion, old.apiInfo.info.packageVersion)) {
+            //this plugin uses the same version. No need to update
+            continue;
+          }
+          //this apiInfo existed at the same path but uses a newer version of the package
+          //refresh the existing Api holder
+          try {
+            apiInfo.init();
+          } catch (Exception exception) {
+            log.error("COuld not inititlaize Plugin", exception);
+          }
+          plugins.get(apiInfo.key).refresh(apiInfo);
+        } else {// the path is changed for the same apiInfo. it's not allowed
+          log.error("Cannot register the same apiInfo at a different path old path: " + path + "new path : " + apiInfo.key);
+        }
+      }
+    }
+    Set<String> toBeRemoved = new HashSet<>();
+    for (String s : pluginNameVsPath.keySet()) {
+      if (!pluginInfos.containsKey(s)) {
+        toBeRemoved.add(s);
+      }
+    }
+    for (String s : toBeRemoved) {
+      String pathKey = pluginNameVsPath.remove(s);
+      ApiHolder holder = plugins.remove(pathKey);
+      if(holder != null){
+        containerApiBag.unregister(holder.apiInfo.endPoint.method()[0], holder.apiInfo.endPoint.path()[0]);
+      }
+    }
+  }
+
+  private class ApiHolder extends Api {
+    private final String key;
+    private ApiInfo apiInfo;
+
+    protected ApiHolder(ApiInfo apiInfo) throws Exception {
+      super(apiInfo.delegate);
+      this.apiInfo = apiInfo;
+      this.key = apiInfo.key;
+    }
+
+    @Override
+    public void call(SolrQueryRequest req, SolrQueryResponse rsp) {
+      apiInfo.delegate.call(req, rsp);
+    }
+
+    void refresh(ApiInfo info) {
+      this.apiInfo = info;
+      super.spec = info.delegate;
+    }
+  }
+
+  public class ApiInfo implements ReflectMapWriter {
+    /*This is the path at which this handler is
+     *
+     */
+    @JsonProperty
+    public  String key;
+    @JsonProperty
+    private final PluginMeta info;
+
+    @JsonProperty(value = "package")
+    public final String pkg;
+    private PackageLoader.Package.Version pkgVersion;
+    EndPoint endPoint;
+
+    private  Class klas;
+
+
+    private AnnotatedApi delegate;
+
+
+    public ApiInfo(PluginMeta info, List<String> errs)  {
+      this.info = info;
+      Pair<String, String> klassInfo = org.apache.solr.core.PluginInfo.parseClassName(info.klass);
+      pkg = klassInfo.second();
+      if (klassInfo.first() != null) {
+        PackageLoader.Package p = coreContainer.getPackageLoader().getPackage(klassInfo.first());
+        if(p == null){
+          errs.add("Invalid package "+klassInfo.first());
+          return;
+        }
+        this.pkgVersion = p.getLatest();
+        try {
+          klas = pkgVersion.getLoader().findClass(pkg, Object.class);
+        } catch (Exception e) {
+          errs.add("Error instantiating class "+ e.getMessage());
+          return;
+        }
+      } else {
+        try {
+          klas = Class.forName(klassInfo.second());
+        } catch (ClassNotFoundException e) {
+          errs.add("Error instantiating class "+ e.getMessage());
+          return;
+        }
+        pkgVersion = null;
+      }
+      endPoint = (EndPoint) klas.getAnnotation(EndPoint.class);
+      if (endPoint == null){
+        errs.add("Invalid class, no @EndPoint annotation");
+        return;
+      }
+      if (endPoint.path().length > 1 || endPoint.method().length > 1) {
+        errs.add("Only one HTTP method and url supported for each API");
+        return;
+      }
+      if (endPoint.method().length != 1 || endPoint.path().length != 1) {
+        errs.add("The @EndPint must have exactly one method and path attributes");
+        return;
+      }
+      List<String> pathSegments = StrUtils.splitSmart(endPoint.path()[0], '/', true);
+      if (!supportedPaths.contains(pathSegments.get(0))) {
+        errs.add("path must have a /cluster or /node prefix");
+        return;
+      }
+      this.key = endPoint.method()[0].toString() + " " + endPoint.path()[0];
+      Object alreadyRegistered = containerApiBag.getRegistry(endPoint.method()[0].toString()).lookup(endPoint.path()[0], null);
+      if(alreadyRegistered != null && !pathSegments.contains(key)){
+        errs.add("The path is conflicting with a Solr handler");
+        return;
+      }
+
+
+
+      Constructor constructor = klas.getConstructors()[0];
+      if (constructor.getParameterTypes().length > 1 ||
+          (constructor.getParameterTypes().length == 1 && constructor.getParameterTypes()[0] != CoreContainer.class)) {
+        errs.add("Must have a no-arg constructor or CoreContainer constructor ");
+        return;
+      }
+    }
+
+    void init() throws Exception {
+      Constructor constructor = klas.getConstructors()[0];
+      if (constructor.getParameterTypes().length == 0) {
+        delegate = new AnnotatedApi(constructor.newInstance(null));
+      } else if (constructor.getParameterTypes().length == 1 && constructor.getParameterTypes()[0] == CoreContainer.class) {
+        delegate = new AnnotatedApi(constructor.newInstance(coreContainer));
+      } else {
+        throw new RuntimeException("Must have a no-arg constructor or CoreContainer constructor ");
+      }
+
+    }
+  }
+
+  public ApiInfo createInfo(PluginMeta info, List<String> errs)  {
+    return new ApiInfo(info, errs);
+
+  }
+
+  static final Set<String> supportedPaths = ImmutableSet.of("node", "cluster");
+}
diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
index 4e874d7..3383709 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -41,6 +41,7 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
+import java.util.function.Supplier;
 
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Maps;
@@ -53,6 +54,7 @@ import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.store.Directory;
 import org.apache.solr.api.AnnotatedApi;
+import org.apache.solr.api.CustomContainerPlugins;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.cloud.SolrCloudManager;
 import org.apache.solr.client.solrj.embedded.EmbeddedSolrServer;
@@ -74,6 +76,7 @@ import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Replica.State;
+import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.common.util.IOUtils;
@@ -88,6 +91,7 @@ import org.apache.solr.handler.SnapShooter;
 import org.apache.solr.handler.admin.AutoscalingHistoryHandler;
 import org.apache.solr.handler.admin.CollectionsHandler;
 import org.apache.solr.handler.admin.ConfigSetsHandler;
+import org.apache.solr.handler.admin.ContainerPluginsApi;
 import org.apache.solr.handler.admin.CoreAdminHandler;
 import org.apache.solr.handler.admin.HealthCheckHandler;
 import org.apache.solr.handler.admin.InfoHandler;
@@ -162,6 +166,12 @@ public class CoreContainer {
     }
   }
 
+  private volatile PluginBag<SolrRequestHandler> containerHandlers = new PluginBag<>(SolrRequestHandler.class, null);
+
+  public final Supplier<SolrZkClient> zkClientSupplier = () -> getZkController().getZkClient();
+
+  private final CustomContainerPlugins customContainerPlugins =  new CustomContainerPlugins(this, containerHandlers.getApiBag());
+
   protected final Map<String, CoreLoadFailure> coreInitFailures = new ConcurrentHashMap<>();
 
   protected volatile CoreAdminHandler coreAdminHandler = null;
@@ -201,8 +211,6 @@ public class CoreContainer {
 
   private final BlobRepository blobRepository = new BlobRepository(this);
 
-  private volatile PluginBag<SolrRequestHandler> containerHandlers = new PluginBag<>(SolrRequestHandler.class, null);
-
   private volatile boolean asyncSolrCoreLoad;
 
   protected volatile SecurityConfHandler securityConfHandler;
@@ -819,6 +827,11 @@ public class CoreContainer {
     }
 
     if (isZooKeeperAware()) {
+//      getZkController().zkStateReader.registerClusterPropertiesListener(customContainerPlugins);
+      customContainerPlugins.refresh(null);
+      ContainerPluginsApi containerPluginsApi = new ContainerPluginsApi(this);
+      containerHandlers.getApiBag().register(new AnnotatedApi(containerPluginsApi.readAPI), Collections.EMPTY_MAP);
+      containerHandlers.getApiBag().register(new AnnotatedApi(containerPluginsApi.editAPI), Collections.EMPTY_MAP);
       zkSys.getZkController().checkOverseerDesignate();
       // initialize this handler here when SolrCloudManager is ready
       autoScalingHandler = new AutoScalingHandler(getZkController().getSolrCloudManager(), loader);
@@ -1976,6 +1989,10 @@ public class CoreContainer {
     return false;
   }
 
+  public CustomContainerPlugins getCustomContainerPlugins(){
+    return customContainerPlugins;
+  }
+
   /**
    * @param solrCore the core against which we check if there has been a tragic exception
    * @return whether this Solr core has tragic exception
diff --git a/solr/core/src/java/org/apache/solr/core/PluginInfo.java b/solr/core/src/java/org/apache/solr/core/PluginInfo.java
index bb290e1..9b4a3d9 100644
--- a/solr/core/src/java/org/apache/solr/core/PluginInfo.java
+++ b/solr/core/src/java/org/apache/solr/core/PluginInfo.java
@@ -65,7 +65,7 @@ public class PluginInfo implements MapSerializable {
    * This checks if it is a package name prefixed classname.
    * the return value has first = package name & second = class name
    */
-  static Pair<String,String > parseClassName(String name) {
+  public static Pair<String,String > parseClassName(String name) {
     String pkgName = null;
     String className = name;
     if (name != null) {
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/ContainerPluginsApi.java b/solr/core/src/java/org/apache/solr/handler/admin/ContainerPluginsApi.java
new file mode 100644
index 0000000..86d642d
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/admin/ContainerPluginsApi.java
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.handler.admin;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.function.Supplier;
+
+import org.apache.solr.api.Command;
+import org.apache.solr.api.EndPoint;
+import org.apache.solr.api.PayloadObj;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.request.beans.PluginMeta;
+import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.security.PermissionNameProvider;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.data.Stat;
+
+
+public class ContainerPluginsApi {
+  public static final String PLUGINS = "plugins";
+  private final Supplier<SolrZkClient> zkClientSupplier;
+  private final CoreContainer coreContainer;
+  public final Read readAPI = new Read();
+  public final Edit editAPI = new Edit();
+
+  public ContainerPluginsApi(CoreContainer coreContainer) {
+    this.zkClientSupplier = coreContainer.zkClientSupplier;
+    this.coreContainer = coreContainer;
+  }
+
+  @EndPoint(method = SolrRequest.METHOD.POST,
+      path = "/cluster/plugins",
+      permission = PermissionNameProvider.Name.COLL_READ_PERM)
+  public class Read {
+
+    @Command
+    public void list(SolrQueryRequest req, SolrQueryResponse rsp) throws IOException {
+      rsp.add(PLUGINS, plugins(zkClientSupplier));
+    }
+  }
+
+  @EndPoint(method = SolrRequest.METHOD.POST,
+      path = "/cluster/plugins",
+      permission = PermissionNameProvider.Name.COLL_EDIT_PERM)
+  public class Edit {
+
+    @Command(name = "add")
+    public void add(SolrQueryRequest req, SolrQueryResponse rsp, PayloadObj<PluginMeta> payload) throws IOException {
+      PluginMeta info = payload.get();
+      if (isConfigInvalid(payload, info)) return;
+      persistPlugins(map -> {
+        if (map.containsKey(info.name)) {
+          payload.addError(info.name + " already exists");
+          return null;
+        }
+        map.put(info.name, info);
+        return map;
+      });
+    }
+
+    @Command(name = "remove")
+    public void remove(SolrQueryRequest req, SolrQueryResponse rsp, PayloadObj<String> payload) throws IOException {
+      persistPlugins(map -> {
+        if (map.remove(payload.get()) == null) {
+          payload.addError("No such plugin: " + payload.get());
+          return null;
+        }
+        return map;
+      });
+    }
+
+    @Command(name = "update")
+    public void update(SolrQueryRequest req, SolrQueryResponse rsp, PayloadObj<PluginMeta> payload) throws IOException {
+      PluginMeta info = payload.get();
+      if (isConfigInvalid(payload, info)) return;
+      persistPlugins(map -> {
+        Map existing = (Map) map.get(info.name);
+        if (existing == null) {
+          payload.addError("No such plugin: " + info.name);
+          return null;
+        } else {
+          map.put(info.name, info);
+          return map;
+        }
+      });
+    }
+  }
+
+  private boolean isConfigInvalid(PayloadObj<PluginMeta> payload, PluginMeta info) {
+    if (info.klass.indexOf(':') > 0) {
+      if (info.packageVersion == null) {
+        payload.addError("Using package. must provide a packageVersion");
+        return true;
+      }
+    }
+    List<String> errs = new ArrayList<>();
+    coreContainer.getCustomContainerPlugins().createInfo(info, errs);
+    if (!errs.isEmpty()) {
+      for (String err : errs) payload.addError(err);
+      return true;
+    }
+    return false;
+  }
+
+  public static Map<String, Object> plugins(Supplier<SolrZkClient> zkClientSupplier) throws IOException {
+    SolrZkClient zkClient = zkClientSupplier.get();
+    try {
+      Map<String, Object> clusterPropsJson = (Map<String, Object>) Utils.fromJSON(zkClient.getData(ZkStateReader.CLUSTER_PROPS, null, new Stat(), true));
+      return (Map<String, Object>) clusterPropsJson.computeIfAbsent(PLUGINS, Utils.NEW_LINKED_HASHMAP_FUN);
+    } catch (KeeperException.NoNodeException e) {
+      return new LinkedHashMap<>();
+    } catch (KeeperException | InterruptedException e) {
+      throw new IOException("Error reading cluster property", SolrZkClient.checkInterrupted(e));
+    }
+  }
+
+  private void persistPlugins(Function<Map, Map> modifier) throws IOException {
+    try {
+      zkClientSupplier.get().atomicUpdate(ZkStateReader.CLUSTER_PROPS, bytes -> {
+        Map rawJson = bytes == null ? new LinkedHashMap() :
+            (Map) Utils.fromJSON(bytes);
+        Map pluginsModified = modifier.apply((Map) rawJson.computeIfAbsent(PLUGINS, Utils.NEW_LINKED_HASHMAP_FUN));
+        if (pluginsModified == null) return null;
+        rawJson.put(PLUGINS, pluginsModified);
+        return Utils.toJSON(rawJson);
+      });
+    } catch (KeeperException | InterruptedException e) {
+      throw new IOException("Error reading cluster property", SolrZkClient.checkInterrupted(e));
+    }
+  }
+
+
+}
diff --git a/solr/core/src/java/org/apache/solr/pkg/PackageListeners.java b/solr/core/src/java/org/apache/solr/pkg/PackageListeners.java
index a483514..b54c488 100644
--- a/solr/core/src/java/org/apache/solr/pkg/PackageListeners.java
+++ b/solr/core/src/java/org/apache/solr/pkg/PackageListeners.java
@@ -63,13 +63,13 @@ public class PackageListeners {
   }
 
   synchronized void packagesUpdated(List<PackageLoader.Package> pkgs) {
-    MDCLoggingContext.setCore(core);
+    if(core != null) MDCLoggingContext.setCore(core);
     try {
       for (PackageLoader.Package pkgInfo : pkgs) {
         invokeListeners(pkgInfo);
       }
     } finally {
-      MDCLoggingContext.clear();
+      if(core != null) MDCLoggingContext.clear();
     }
   }
 
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/TestApiFramework.java b/solr/core/src/test/org/apache/solr/handler/admin/TestApiFramework.java
index f979831..38fac62 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/TestApiFramework.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/TestApiFramework.java
@@ -20,6 +20,7 @@ package org.apache.solr.handler.admin;
 import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.io.InputStream;
+import java.lang.reflect.Constructor;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -199,6 +200,25 @@ public class TestApiFramework extends SolrTestCaseJ4 {
 
   }
 
+  public void testApiWrapper() {
+    Class<ApiWithConstructor> klas = ApiWithConstructor.class;
+    for (Constructor<?> constructor : klas.getConstructors()) {
+      constructor.getParameterTypes();
+
+    }
+
+
+
+  }
+
+  public static class ApiWithConstructor{
+    private final CoreContainer cc;
+
+    public ApiWithConstructor(CoreContainer cc) {
+      this.cc = cc;
+    }
+  }
+
   @EndPoint(method = POST, path = "/cluster/package", permission = PermissionNameProvider.Name.ALL)
   public static class ApiTest {
     @Command(name = "add")
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/PluginMeta.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/PluginMeta.java
new file mode 100644
index 0000000..8c20ba2
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/PluginMeta.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.client.solrj.request.beans;
+
+import org.apache.solr.common.annotation.JsonProperty;
+import org.apache.solr.common.util.ReflectMapWriter;
+
+public class PluginMeta implements ReflectMapWriter {
+  @JsonProperty(required = true)
+  public String name;
+
+  @JsonProperty(value = "class", required = true)
+  public String klass;
+
+  @JsonProperty
+  public String packageVersion;
+}
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/PathTrie.java b/solr/solrj/src/java/org/apache/solr/common/util/PathTrie.java
index 742c59d..852dab5 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/PathTrie.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/PathTrie.java
@@ -32,7 +32,7 @@ import static java.util.Collections.emptyList;
  */
 public class PathTrie<T> {
   private final Set<String> reserved = new HashSet<>();
-  Node root = new Node(emptyList(), null);
+  Node root = new Node(emptyList(), null, null);
 
   public PathTrie() {
   }
@@ -82,6 +82,20 @@ public class PathTrie<T> {
     return parts;
   }
 
+  public boolean unregister(List<String> path){
+    Node node = root.lookupNode(path, 0, null, null);
+    if(node != null){
+      node.obj = null;
+      if(node.children.isEmpty()){
+        if(node.parent != null){
+          node.parent.children.remove(node.name);
+        }
+      }
+      return true;
+    }
+    return false;
+
+  }
 
   public T lookup(String path, Map<String, String> templateValues) {
     return root.lookup(getPathSegments(path), 0, templateValues);
@@ -107,8 +121,10 @@ public class PathTrie<T> {
     Map<String, Node> children;
     T obj;
     String templateName;
+    final Node parent;
 
-    Node(List<String> path, T o) {
+    Node(List<String> path, T o, Node parent) {
+      this.parent = parent;
       if (path.isEmpty()) {
         obj = o;
         return;
@@ -133,7 +149,7 @@ public class PathTrie<T> {
 
       matchedChild = children.get(key);
       if (matchedChild == null) {
-        children.put(key, matchedChild = new Node(path, o));
+        children.put(key, matchedChild = new Node(path, o, this));
       }
       if (varName != null) {
         if (!matchedChild.templateName.equals(varName)) {
@@ -179,17 +195,22 @@ public class PathTrie<T> {
      * @param availableSubPaths If not null , available sub paths will be returned in this set
      */
     public T lookup(List<String> pathSegments, int index, Map<String, String> templateVariables, Set<String> availableSubPaths) {
+      Node node = lookupNode(pathSegments, index, templateVariables, availableSubPaths);
+      return node == null?  null: node.obj;
+    }
+
+    Node lookupNode(List<String> pathSegments, int index, Map<String, String> templateVariables, Set<String> availableSubPaths) {
       if (templateName != null) templateVariables.put(templateName, pathSegments.get(index - 1));
       if (pathSegments.size() < index + 1) {
         findAvailableChildren("", availableSubPaths);
         if (obj == null) {//this is not a leaf node
           Node n = children.get("*");
           if (n != null) {
-            return n.obj;
+            return n;
           }
 
         }
-        return obj;
+        return this;
       }
       String piece = pathSegments.get(index);
       if (children == null) {
@@ -205,14 +226,14 @@ public class PathTrie<T> {
             sb.append("/").append(pathSegments.get(i));
           }
           templateVariables.put("*", sb.toString());
-          return n.obj;
+          return n;
 
         }
       }
       if (n == null) {
         return null;
       }
-      return n.lookup(pathSegments, index + 1, templateVariables, availableSubPaths);
+      return n.lookupNode(pathSegments, index + 1, templateVariables, availableSubPaths);
     }
   }