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/05/17 02:28:55 UTC

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

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



##########
File path: solr/core/src/java/org/apache/solr/api/CustomContainerPlugins.java
##########
@@ -0,0 +1,283 @@
+/*
+ * 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.Closeable;
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Modifier;
+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 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;
+
+import static org.apache.lucene.util.IOUtils.closeWhileHandlingException;
+
+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(null);
+    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 {
+      pluginInfos = ContainerPluginsApi.plugins(coreContainer.zkClientSupplier);
+    } catch (IOException e) {
+      log.error("Could not read plugins data", e);
+      return;
+    }
+    if(pluginInfos.isEmpty()) return;
+
+    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.version, old.apiInfo.info.version)) {
+            //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) {
+        Api old = containerApiBag.unregister(holder.apiInfo.endPoint.method()[0], holder.apiInfo.endPoint.path()[0]);
+        if (old instanceof Closeable) {
+          closeWhileHandlingException((Closeable) old);
+        }
+
+      }
+    }
+  }
+
+  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) {

Review comment:
       The objective is to collect multiple errors and eventually throw one single Exception whihc contains all of them

##########
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 =

Review comment:
       These clusters are created on a per testcase basis. We don't want remnants of other tests to screw up other tests




----------------------------------------------------------------
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