You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by GitBox <gi...@apache.org> on 2020/06/21 21:15:34 UTC

[GitHub] [lucene-solr] dsmiley commented on a change in pull request #1432: SOLR-14404 CoreContainer level custom requesthandlers

dsmiley commented on a change in pull request #1432:
URL: https://github.com/apache/lucene-solr/pull/1432#discussion_r443254655



##########
File path: solr/core/src/java/org/apache/solr/api/AnnotatedApi.java
##########
@@ -48,6 +34,12 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.Closeable;

Review comment:
       Our code style is to put java.* up front.  See https://github.com/apache/lucene-solr/blob/aa5b26f6c40e6f6ac78d4574448ab0241c1b68b0/dev-tools/idea/.idea/codeStyleSettings.xml#L25
   
   Please update your IDE settings accordingly.

##########
File path: solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/PluginMeta.java
##########
@@ -0,0 +1,54 @@
+/*
+ * 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 java.util.Objects;
+
+import org.apache.solr.common.annotation.JsonProperty;
+import org.apache.solr.common.util.ReflectMapWriter;
+
+public class PluginMeta implements ReflectMapWriter {

Review comment:
       Add at least one line of javadoc for every public class

##########
File path: solr/core/src/java/org/apache/solr/api/AnnotatedApi.java
##########
@@ -274,12 +302,21 @@ void invoke(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation cmd) {
       }
 
     }
+
+    private void checkForErrorInPayload(CommandOperation cmd) {
+      if (cmd.hasError()) {
+        throw new ApiBag.ExceptionWithErrObject(SolrException.ErrorCode.BAD_REQUEST, "Error executing command",
+            CommandOperation.captureErrors(Collections.singletonList(cmd)));
+      }
+    }
   }
 
   public static Map<String, Object> createSchema(Method m) {
     Type[] types = m.getGenericParameterTypes();
-    if (types.length == 3) {
-      Type t = types[2];
+    Type t = null;

Review comment:
       Could use a comment here as to what types "looks like"; seems several possibilities.

##########
File path: solr/core/src/java/org/apache/solr/api/PayloadObj.java
##########
@@ -18,18 +18,31 @@
 package org.apache.solr.api;
 
 import org.apache.solr.common.util.CommandOperation;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
 
-public  class PayloadObj<T> extends CommandOperation {
+public class PayloadObj<T> extends CommandOperation {

Review comment:
       Please add at least one sentence javadoc to the class

##########
File path: solr/core/src/java/org/apache/solr/api/CustomContainerPlugins.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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 com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.request.beans.PluginMeta;
+import org.apache.solr.common.annotation.JsonProperty;
+import org.apache.solr.common.cloud.ClusterPropertiesListener;
+import org.apache.solr.common.util.*;
+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;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Modifier;
+import java.util.*;
+
+import static org.apache.lucene.util.IOUtils.closeWhileHandlingException;
+
+public class CustomContainerPlugins implements 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, ApiInfo> currentPlugins = new HashMap();
+
+  @Override
+  public boolean onChange(Map<String, Object> properties) {
+    refresh();
+    return false;
+  }
+  public CustomContainerPlugins(CoreContainer coreContainer, ApiBag apiBag) {
+    this.coreContainer = coreContainer;
+    this.containerApiBag = apiBag;
+  }
+
+  public synchronized void refresh() {
+    Map<String, Object> pluginInfos = null;
+    try {
+      pluginInfos = ContainerPluginsApi.plugins(coreContainer.zkClientSupplier);
+    } catch (IOException e) {
+      log.error("Could not read plugins data", e);
+      return;
+    }
+    Map<String,PluginMeta> newState = new HashMap<>(pluginInfos.size());
+    for (Map.Entry<String, Object> e : pluginInfos.entrySet()) {
+      try {
+        newState.put(e.getKey(),
+            mapper.readValue(Utils.toJSON(e.getValue()), PluginMeta.class));
+      } catch (Exception exp) {
+        log.error("Invalid apiInfo configuration :", exp);
+      }
+    }
+
+    Map<String, PluginMeta> currentState = new HashMap<>();
+    for (Map.Entry<String, ApiInfo> e : currentPlugins.entrySet()) {
+      currentState.put(e.getKey(), e.getValue().info);
+    }
+    Map<String, Diff> diff = compareMaps(currentState, newState);
+    if (diff == null) return;//nothing has changed
+    for (Map.Entry<String, Diff> e : diff.entrySet()) {
+      if (e.getValue() == Diff.UNCHANGED) continue;
+      if (e.getValue() == Diff.REMOVED) {
+        ApiInfo apiInfo = currentPlugins.remove(e.getKey());
+        if (apiInfo == null) continue;
+        for (ApiHolder holder : apiInfo.holders) {
+          Api old = containerApiBag.unregister(holder.api.getEndPoint().method()[0], holder.api.getEndPoint().path()[0]);
+          if (old instanceof Closeable) {
+            closeWhileHandlingException((Closeable) old);
+          }
+        }
+      } else {
+        //ADDED or UPDATED
+        PluginMeta info = newState.get(e.getKey());
+        ApiInfo apiInfo = null;
+        List<String> errs = new ArrayList<>();
+        apiInfo = new ApiInfo(info, errs);
+        if (!errs.isEmpty()) {
+          log.error(StrUtils.join(errs, ','));
+          continue;
+        }
+        try {
+          apiInfo.init();
+        } catch (Exception exp) {
+          log.error("Cannot install apiInfo ", exp);
+          continue;
+        }
+        if (e.getValue() == Diff.ADDED) {
+          for (ApiHolder holder : apiInfo.holders) {
+            containerApiBag.register(holder, Collections.singletonMap("plugin-name", e.getKey()));
+          }
+          currentPlugins.put(e.getKey(), apiInfo);
+        } else {
+          ApiInfo old = currentPlugins.put(e.getKey(), apiInfo);
+          List<ApiHolder> replaced = new ArrayList<>();
+          for (ApiHolder holder : apiInfo.holders) {
+            Api oldApi = containerApiBag.lookup(holder.getPath(),
+                holder.getMethod().toString(), null);
+            if (oldApi instanceof ApiHolder) {
+              replaced.add((ApiHolder) oldApi);
+            }
+            containerApiBag.register(holder, Collections.singletonMap("plugin-name", e.getKey()));
+          }
+          if (old != null) {
+            for (ApiHolder holder : old.holders) {
+              if (replaced.contains(holder)) continue;// this path is present in the new one as well. so it already got replaced
+              containerApiBag.unregister(holder.getMethod(), holder.getPath());
+            }
+            if (old instanceof Closeable) {
+              closeWhileHandlingException((Closeable) old);
+            }
+          }
+        }
+      }
+
+    }
+  }
+
+  private class ApiHolder extends Api {
+    final AnnotatedApi api;
+
+    protected ApiHolder(AnnotatedApi api) {
+      super(api);
+      this.api = api;
+    }
+
+    @Override
+    public void call(SolrQueryRequest req, SolrQueryResponse rsp) {
+      api.call(req, rsp);
+    }
+
+    public String getPath(){
+      return api.getEndPoint().path()[0];
+    }
+
+    public SolrRequest.METHOD getMethod(){
+      return api.getEndPoint().method()[0];
+
+    }
+  }
+
+  public class ApiInfo implements ReflectMapWriter {
+    List<ApiHolder> holders;
+    @JsonProperty
+    private final PluginMeta info;
+
+    @JsonProperty(value = "package")
+    public final String pkg;
+
+    private PackageLoader.Package.Version pkgVersion;
+    private Class klas;
+    Object instance;
+
+
+    public ApiInfo(PluginMeta info, List<String> errs) {
+      this.info = info;
+      Pair<String, String> klassInfo = org.apache.solr.core.PluginInfo.parseClassName(info.klass);
+      pkg = klassInfo.first();
+      if (pkg != null) {
+        PackageLoader.Package p = coreContainer.getPackageLoader().getPackage(pkg);
+        if (p == null) {
+          errs.add("Invalid package " + klassInfo.first());
+          return;
+        }
+        this.pkgVersion = p.getVersion(info.version);
+        if (pkgVersion == null) {
+          errs.add("No such package version:" + pkg + ":" + info.version + " . available versions :" + p.allVersions());
+          return;
+        }
+        try {
+          klas = pkgVersion.getLoader().findClass(klassInfo.second(), Object.class);
+        } catch (Exception e) {
+          log.error("Error loading class", e);
+          errs.add("Error loading class " + e.getMessage());

Review comment:
       FWIW at this line and at some other lines, consider simply doing `e.toString()` thus capturing both the class name & message.  Long ago I once used `e.getMessage()` but I never call it any more because you're throwing away interesting info in the exception type.  For example a FileNotFoundException's message is simply the name of the file, giving no indication of what the problem is.

##########
File path: solr/core/src/java/org/apache/solr/pkg/PackageListeners.java
##########
@@ -69,7 +69,7 @@ synchronized void packagesUpdated(List<PackageLoader.Package> pkgs) {
         invokeListeners(pkgInfo);
       }
     } finally {
-      MDCLoggingContext.clear();
+      if(core != null) MDCLoggingContext.clear();

Review comment:
       No; it was correct before.  Internally, setCore can accept null and doesn't require the caller to care if core is null.  Furthermore, PackageListeners.core will never be null.  It ought to be declared final as well; it's only set in the constructor.

##########
File path: solr/core/src/test/org/apache/solr/handler/TestContainerPlugin.java
##########
@@ -0,0 +1,250 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.Callable;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.solr.api.Command;
+import org.apache.solr.api.EndPoint;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.impl.BaseHttpSolrClient;
+import org.apache.solr.client.solrj.request.V2Request;
+import org.apache.solr.client.solrj.request.beans.Package;
+import org.apache.solr.client.solrj.request.beans.PluginMeta;
+import org.apache.solr.client.solrj.response.V2Response;
+import org.apache.solr.cloud.MiniSolrCloudCluster;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.NavigableObject;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.filestore.PackageStoreAPI;
+import org.apache.solr.filestore.TestDistribPackageStore;
+import org.apache.solr.pkg.TestPackages;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.security.PermissionNameProvider;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import static java.util.Collections.singletonMap;
+import static org.apache.solr.client.solrj.SolrRequest.METHOD.GET;
+import static org.apache.solr.client.solrj.SolrRequest.METHOD.POST;
+import static org.apache.solr.filestore.TestDistribPackageStore.readFile;
+import static org.apache.solr.filestore.TestDistribPackageStore.uploadKey;
+import static org.hamcrest.CoreMatchers.containsString;
+
+public class TestContainerPlugin extends SolrCloudTestCase {
+
+  @Before
+  public void setup() {
+    System.setProperty("enable.packages", "true");
+  }
+
+  @After
+  public void teardown() {
+    System.clearProperty("enable.packages");
+  }
+
+  @Test
+  public void testApi() throws Exception {
+    MiniSolrCloudCluster cluster =
+        configureCluster(4)
+            .withJettyConfig(jetty -> jetty.enableV2(true))
+            .configure();
+    String errPath = "/error/details[0]/errorMessages[0]";
+    try {
+      PluginMeta plugin = new PluginMeta();
+      plugin.name = "testplugin";
+      plugin.klass = C2.class.getName();
+      V2Request req = new V2Request.Builder("/cluster/plugin")
+          .forceV2(true)
+          .withMethod(POST)
+          .withPayload(singletonMap("add", plugin))
+          .build();
+      expectError(req, cluster.getSolrClient(), errPath, "Must have a no-arg constructor or CoreContainer constructor and it must not be a non static inner class");
+
+      plugin.klass = C1.class.getName();
+      expectError(req, cluster.getSolrClient(), errPath, "Invalid class, no @EndPoint annotation");
+
+      plugin.klass = C3.class.getName();
+      req.process(cluster.getSolrClient());
+
+      V2Response rsp = new V2Request.Builder("/cluster/plugin")
+          .forceV2(true)
+          .withMethod(GET)
+          .build()
+          .process(cluster.getSolrClient());
+      assertEquals(C3.class.getName(), rsp._getStr("/plugin/testplugin/class", null));
+
+      TestDistribPackageStore.assertResponseValues(10,
+          () -> new V2Request.Builder("/plugin/my/plugin")
+              .forceV2(true)
+              .withMethod(GET)
+              .build().process(cluster.getSolrClient()),
+          ImmutableMap.of("/testkey", "testval"));
+
+      new V2Request.Builder("/cluster/plugin")
+          .withMethod(POST)
+          .forceV2(true)
+          .withPayload("{remove : testplugin}")
+          .build()
+          .process(cluster.getSolrClient());
+
+      rsp = new V2Request.Builder("/cluster/plugin")
+          .forceV2(true)
+          .withMethod(GET)
+          .build()
+          .process(cluster.getSolrClient());
+      assertEquals(null, rsp._get("/plugin/testplugin/class", null));
+
+    } finally {
+      cluster.shutdown();
+    }
+  }
+  @Test
+  public void testApiFromPackage() throws Exception {
+    MiniSolrCloudCluster cluster =
+        configureCluster(4)
+            .withJettyConfig(jetty -> jetty.enableV2(true))
+            .configure();
+    String FILE1 = "/myplugin/v1.jar";
+    String FILE2 = "/myplugin/v2.jar";
+
+    String errPath = "/error/details[0]/errorMessages[0]";
+    try {
+      byte[] derFile = readFile("cryptokeys/pub_key512.der");
+      uploadKey(derFile, PackageStoreAPI.KEYS_DIR+"/pub_key512.der", cluster);
+      TestPackages.postFileAndWait(cluster, "runtimecode/containerplugin.v.1.jar.bin", FILE1,
+          "pmrmWCDafdNpYle2rueAGnU2J6NYlcAey9mkZYbqh+5RdYo2Ln+llLF9voyRj+DDivK9GV1XdtKvD9rgCxlD7Q==");
+     TestPackages.postFileAndWait(cluster, "runtimecode/containerplugin.v.2.jar.bin", FILE2,
+          "StR3DmqaUSL7qjDOeVEiCqE+ouiZAkW99fsL48F9oWG047o7NGgwwZ36iGgzDC3S2tPaFjRAd9Zg4UK7OZLQzg==");
+
+      Package.AddVersion add = new Package.AddVersion();
+      add.version = "1.0";
+      add.pkg = "mypkg";
+      add.files = List.of(FILE1);
+      V2Request addPkgVersionReq = new V2Request.Builder("/cluster/package")
+          .forceV2(true)
+          .withMethod(POST)
+          .withPayload(singletonMap("add", add))
+          .build();
+      addPkgVersionReq.process(cluster.getSolrClient());
+
+      TestDistribPackageStore.assertResponseValues(10,
+          () -> new V2Request.Builder("/cluster/package").
+              withMethod(GET)
+              .build().process(cluster.getSolrClient()),
+          Utils.makeMap(
+              ":result:packages:mypkg[0]:version", "1.0",
+              ":result:packages:mypkg[0]:files[0]", FILE1
+          ));
+
+      PluginMeta plugin = new PluginMeta();
+      plugin.name = "myplugin";
+      plugin.klass = "mypkg:org.apache.solr.handler.MyPlugin";
+      plugin.version = add.version;
+      V2Request req1 = new V2Request.Builder("/cluster/plugin")
+          .forceV2(true)
+          .withMethod(POST)
+          .withPayload(singletonMap("add", plugin))
+          .build();
+      req1.process(cluster.getSolrClient());
+      TestDistribPackageStore.assertResponseValues(10,
+          () -> new V2Request.Builder("/cluster/plugin").
+              withMethod(GET)
+              .build().process(cluster.getSolrClient()),
+          ImmutableMap.of(
+              "/plugin/myplugin/class", plugin.klass,
+              "/plugin/myplugin/version", plugin.version
+          ));
+      Callable<NavigableObject> invokePlugin = () -> new V2Request.Builder("/plugin/my/path")
+          .forceV2(true)
+          .withMethod(GET)
+          .build().process(cluster.getSolrClient());
+      TestDistribPackageStore.assertResponseValues(10,
+          invokePlugin,
+          ImmutableMap.of("/myplugin.version", "1.0"));
+
+      add.version = "2.0";
+      add.files = List.of(FILE2);
+      addPkgVersionReq.process(cluster.getSolrClient());
+
+      plugin.version = add.version;
+      new V2Request.Builder("/cluster/plugin")
+          .forceV2(true)
+          .withMethod(POST)
+          .withPayload(singletonMap("update", plugin))
+          .build()
+      .process(cluster.getSolrClient());
+
+      TestDistribPackageStore.assertResponseValues(10,
+          () -> new V2Request.Builder("/cluster/plugin").
+              withMethod(GET)
+              .build().process(cluster.getSolrClient()),
+          ImmutableMap.of(
+              "/plugin/myplugin/class", plugin.klass,
+              "/plugin/myplugin/version", "2.0"
+          ));
+      TestDistribPackageStore.assertResponseValues(10,
+          invokePlugin,
+          ImmutableMap.of("/myplugin.version", "2.0"));
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  public static class C1 {
+
+  }
+
+  @EndPoint(
+      method = GET,
+      path = "/plugin/my/plugin",
+      permission = PermissionNameProvider.Name.COLL_READ_PERM)
+  public class C2 {
+
+
+  }
+
+  @EndPoint(
+      method = GET,
+      path = "/plugin/my/plugin",
+      permission = PermissionNameProvider.Name.COLL_READ_PERM)
+  public static class C3 {
+    @Command
+    public void read(SolrQueryRequest req, SolrQueryResponse rsp) {
+      rsp.add("testkey", "testval");
+    }
+
+  }
+
+
+  private void expectError(V2Request req, SolrClient client, String errPath, String expectErrorMsg) throws IOException, SolrServerException {
+    try {

Review comment:
       How so?  It returns the exception of the expected type and then you can inspect it as you are doing now.

##########
File path: solr/core/src/java/org/apache/solr/api/CustomContainerPlugins.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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 com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.request.beans.PluginMeta;
+import org.apache.solr.common.annotation.JsonProperty;
+import org.apache.solr.common.cloud.ClusterPropertiesListener;
+import org.apache.solr.common.util.*;
+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;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Modifier;
+import java.util.*;
+
+import static org.apache.lucene.util.IOUtils.closeWhileHandlingException;
+
+public class CustomContainerPlugins implements 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, ApiInfo> currentPlugins = new HashMap();
+
+  @Override
+  public boolean onChange(Map<String, Object> properties) {
+    refresh();
+    return false;
+  }
+  public CustomContainerPlugins(CoreContainer coreContainer, ApiBag apiBag) {
+    this.coreContainer = coreContainer;
+    this.containerApiBag = apiBag;
+  }
+
+  public synchronized void refresh() {
+    Map<String, Object> pluginInfos = null;
+    try {
+      pluginInfos = ContainerPluginsApi.plugins(coreContainer.zkClientSupplier);
+    } catch (IOException e) {
+      log.error("Could not read plugins data", e);
+      return;
+    }
+    Map<String,PluginMeta> newState = new HashMap<>(pluginInfos.size());
+    for (Map.Entry<String, Object> e : pluginInfos.entrySet()) {
+      try {
+        newState.put(e.getKey(),
+            mapper.readValue(Utils.toJSON(e.getValue()), PluginMeta.class));
+      } catch (Exception exp) {
+        log.error("Invalid apiInfo configuration :", exp);
+      }
+    }
+
+    Map<String, PluginMeta> currentState = new HashMap<>();
+    for (Map.Entry<String, ApiInfo> e : currentPlugins.entrySet()) {
+      currentState.put(e.getKey(), e.getValue().info);
+    }
+    Map<String, Diff> diff = compareMaps(currentState, newState);
+    if (diff == null) return;//nothing has changed
+    for (Map.Entry<String, Diff> e : diff.entrySet()) {
+      if (e.getValue() == Diff.UNCHANGED) continue;
+      if (e.getValue() == Diff.REMOVED) {
+        ApiInfo apiInfo = currentPlugins.remove(e.getKey());
+        if (apiInfo == null) continue;
+        for (ApiHolder holder : apiInfo.holders) {
+          Api old = containerApiBag.unregister(holder.api.getEndPoint().method()[0], holder.api.getEndPoint().path()[0]);
+          if (old instanceof Closeable) {
+            closeWhileHandlingException((Closeable) old);
+          }
+        }
+      } else {
+        //ADDED or UPDATED
+        PluginMeta info = newState.get(e.getKey());
+        ApiInfo apiInfo = null;
+        List<String> errs = new ArrayList<>();
+        apiInfo = new ApiInfo(info, errs);
+        if (!errs.isEmpty()) {
+          log.error(StrUtils.join(errs, ','));
+          continue;
+        }
+        try {
+          apiInfo.init();
+        } catch (Exception exp) {
+          log.error("Cannot install apiInfo ", exp);
+          continue;
+        }
+        if (e.getValue() == Diff.ADDED) {
+          for (ApiHolder holder : apiInfo.holders) {
+            containerApiBag.register(holder, Collections.singletonMap("plugin-name", e.getKey()));
+          }
+          currentPlugins.put(e.getKey(), apiInfo);
+        } else {
+          ApiInfo old = currentPlugins.put(e.getKey(), apiInfo);
+          List<ApiHolder> replaced = new ArrayList<>();
+          for (ApiHolder holder : apiInfo.holders) {
+            Api oldApi = containerApiBag.lookup(holder.getPath(),
+                holder.getMethod().toString(), null);
+            if (oldApi instanceof ApiHolder) {
+              replaced.add((ApiHolder) oldApi);
+            }
+            containerApiBag.register(holder, Collections.singletonMap("plugin-name", e.getKey()));
+          }
+          if (old != null) {
+            for (ApiHolder holder : old.holders) {
+              if (replaced.contains(holder)) continue;// this path is present in the new one as well. so it already got replaced
+              containerApiBag.unregister(holder.getMethod(), holder.getPath());
+            }
+            if (old instanceof Closeable) {
+              closeWhileHandlingException((Closeable) old);
+            }
+          }
+        }
+      }
+
+    }
+  }
+
+  private class ApiHolder extends Api {

Review comment:
       should be static; maybe other inner classes here too?

##########
File path: solr/solrj/src/java/org/apache/solr/common/util/PathTrie.java
##########
@@ -82,6 +80,22 @@ public boolean add(String s) {
     return parts;
   }
 
+  public T unregister(List<String> path) {

Review comment:
       I think the name "unregister" would only make sense if there was a corresponding "register" method.  How about a more generic "remove"?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org