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/11/06 04:17:28 UTC

[lucene-solr] 01/01: SOLR-14977 : ContainerPlugins should be configurable

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

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

commit b59e29d90f585b8a8f2ee8d3dfd2a36ca968d075
Author: Noble Paul <no...@gmail.com>
AuthorDate: Fri Nov 6 15:16:47 2020 +1100

    SOLR-14977 :  ContainerPlugins should be configurable
---
 .../src/java/org/apache/solr/api/AnnotatedApi.java |  4 +-
 .../org/apache/solr/api/ConfigurablePlugin.java    | 31 ++++++++++
 .../apache/solr/api/ContainerPluginsRegistry.java  | 68 +++++++++++++++++-----
 .../solr/handler/admin/ContainerPluginsApi.java    |  6 +-
 .../apache/solr/handler/TestContainerPlugin.java   | 57 ++++++++++++++++++
 .../apache/solr/common/util/JsonSchemaCreator.java |  1 +
 6 files changed, 150 insertions(+), 17 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 f9f97a4..df373ea 100644
--- a/solr/core/src/java/org/apache/solr/api/AnnotatedApi.java
+++ b/solr/core/src/java/org/apache/solr/api/AnnotatedApi.java
@@ -33,6 +33,7 @@ import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 
+import com.fasterxml.jackson.databind.DeserializationFeature;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.common.SolrException;
@@ -222,7 +223,8 @@ public class AnnotatedApi extends Api implements PermissionNameProvider , Closea
     final String command;
     final MethodHandle method;
     final Object obj;
-    ObjectMapper mapper = SolrJacksonAnnotationInspector.createObjectMapper();
+    ObjectMapper mapper = SolrJacksonAnnotationInspector.createObjectMapper()
+        .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
     int paramsCount;
     @SuppressWarnings({"rawtypes"})
     Class parameterClass;
diff --git a/solr/core/src/java/org/apache/solr/api/ConfigurablePlugin.java b/solr/core/src/java/org/apache/solr/api/ConfigurablePlugin.java
new file mode 100644
index 0000000..969cc8d
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/api/ConfigurablePlugin.java
@@ -0,0 +1,31 @@
+/*
+ * 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;
+
+/**Implement this interface if your plugin needs to accept some configuration
+ * 
+ * @param <T> the configuration Object type
+ */
+public interface ConfigurablePlugin<T> {
+
+  /**This is invoked soon after the Object is initialized
+   * 
+   * @param cfg value deserialized from JSON
+   */
+  void initConfig(T cfg);
+}
diff --git a/solr/core/src/java/org/apache/solr/api/ContainerPluginsRegistry.java b/solr/core/src/java/org/apache/solr/api/ContainerPluginsRegistry.java
index 8d0267c..690aee8 100644
--- a/solr/core/src/java/org/apache/solr/api/ContainerPluginsRegistry.java
+++ b/solr/core/src/java/org/apache/solr/api/ContainerPluginsRegistry.java
@@ -22,6 +22,8 @@ import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.Modifier;
+import java.lang.reflect.ParameterizedType;
+import java.lang.reflect.Type;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -31,6 +33,7 @@ import java.util.Optional;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.function.Supplier;
 
+import com.fasterxml.jackson.databind.DeserializationFeature;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.lucene.util.ResourceLoaderAware;
 import org.apache.solr.client.solrj.SolrRequest;
@@ -69,7 +72,8 @@ import static org.apache.solr.common.util.Utils.makeMap;
 public class ContainerPluginsRegistry implements ClusterPropertiesListener, MapWriter, Closeable {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  private final ObjectMapper mapper = SolrJacksonAnnotationInspector.createObjectMapper();
+  private static final ObjectMapper mapper = SolrJacksonAnnotationInspector.createObjectMapper()
+      .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
 
   private final List<PluginRegistryListener> listeners = new CopyOnWriteArrayList<>();
 
@@ -114,6 +118,16 @@ public class ContainerPluginsRegistry implements ClusterPropertiesListener, MapW
     return currentPlugins.get(name);
   }
 
+  static class PluginMetaHolder {
+    private final Map<String, Object> original;
+    private final PluginMeta meta;
+
+    PluginMetaHolder(Map<String, Object> original) throws IOException {
+      this.original = original;
+      meta = mapper.readValue(Utils.toJSON(original), PluginMeta.class);
+    }
+  }
+  @SuppressWarnings("unchecked")
   public synchronized void refresh() {
     Map<String, Object> pluginInfos = null;
     try {
@@ -122,19 +136,18 @@ public class ContainerPluginsRegistry implements ClusterPropertiesListener, MapW
       log.error("Could not read plugins data", e);
       return;
     }
-    Map<String,PluginMeta> newState = new HashMap<>(pluginInfos.size());
+    Map<String,PluginMetaHolder> 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));
+        newState.put(e.getKey(),new PluginMetaHolder((Map<String, Object>) e.getValue()));
       } catch (Exception exp) {
         log.error("Invalid apiInfo configuration :", exp);
       }
     }
 
-    Map<String, PluginMeta> currentState = new HashMap<>();
+    Map<String, PluginMetaHolder> currentState = new HashMap<>();
     for (Map.Entry<String, ApiInfo> e : currentPlugins.entrySet()) {
-      currentState.put(e.getKey(), e.getValue().info);
+      currentState.put(e.getKey(), e.getValue().holder);
     }
     Map<String, Diff> diff = compareMaps(currentState, newState);
     if (diff == null) return;//nothing has changed
@@ -153,10 +166,10 @@ public class ContainerPluginsRegistry implements ClusterPropertiesListener, MapW
         }
       } else {
         //ADDED or UPDATED
-        PluginMeta info = newState.get(e.getKey());
+        PluginMetaHolder info = newState.get(e.getKey());
         ApiInfo apiInfo = null;
         List<String> errs = new ArrayList<>();
-        apiInfo = new ApiInfo(info, errs);
+        apiInfo = new ApiInfo(info,errs);
         if (!errs.isEmpty()) {
           log.error(StrUtils.join(errs, ','));
           continue;
@@ -243,8 +256,10 @@ public class ContainerPluginsRegistry implements ClusterPropertiesListener, MapW
   public class ApiInfo implements ReflectMapWriter {
     List<ApiHolder> holders;
 
+    private final PluginMetaHolder holder;
+
     @JsonProperty
-    private final PluginMeta info;
+    private PluginMeta info;
 
     @JsonProperty(value = "package")
     public final String pkg;
@@ -272,8 +287,9 @@ public class ContainerPluginsRegistry implements ClusterPropertiesListener, MapW
       return info.copy();
     }
     @SuppressWarnings({"unchecked","rawtypes"})
-    public ApiInfo(PluginMeta info, List<String> errs) {
-      this.info = info;
+    public ApiInfo(PluginMetaHolder infoHolder, List<String> errs) {
+      this.holder = infoHolder;
+      this.info = infoHolder.meta;
       PluginInfo.ClassName klassInfo = new PluginInfo.ClassName(info.klass);
       pkg = klassInfo.pkg;
       if (pkg != null) {
@@ -349,7 +365,7 @@ public class ContainerPluginsRegistry implements ClusterPropertiesListener, MapW
       }
     }
 
-    @SuppressWarnings({"rawtypes"})
+    @SuppressWarnings({"rawtypes","unchecked"})
     public void init() throws Exception {
       if (this.holders != null) return;
       Constructor constructor = klas.getConstructors()[0];
@@ -360,6 +376,14 @@ public class ContainerPluginsRegistry implements ClusterPropertiesListener, MapW
       } else {
         throw new RuntimeException("Must have a no-arg constructor or CoreContainer constructor ");
       }
+      if (instance instanceof ConfigurablePlugin) {
+        Class c = getConfigObj(instance);
+        if(c != null) {
+          Object initVal =  mapper.readValue(Utils.toJSON(holder.original), c);
+          ((ConfigurablePlugin) instance).initConfig(initVal);
+        }
+
+      }
       if (instance instanceof ResourceLoaderAware) {
         try {
           ((ResourceLoaderAware) instance).inform(pkgVersion.getLoader());
@@ -372,9 +396,27 @@ public class ContainerPluginsRegistry implements ClusterPropertiesListener, MapW
         holders.add(new ApiHolder((AnnotatedApi) api));
       }
     }
+
+    @SuppressWarnings("rawtypes")
+    private Class getConfigObj(Object o) {
+      Type[] interfaces = o.getClass().getGenericInterfaces();
+      for (Type type : interfaces) {
+        if (type instanceof ParameterizedType) {
+          ParameterizedType parameterizedType = (ParameterizedType) type;
+          if(parameterizedType.getRawType() == ConfigurablePlugin.class) {
+            return (Class) parameterizedType.getActualTypeArguments()[0];
+          }
+        }
+      }
+      return null;
+    }
   }
 
-  public ApiInfo createInfo(PluginMeta info, List<String> errs) {
+  public ApiInfo createInfo(Map<String,Object> info, List<String> errs) throws IOException {
+    return new ApiInfo(new PluginMetaHolder(info), errs);
+
+  }
+  public ApiInfo createInfo(PluginMetaHolder info, List<String> errs) {
     return new ApiInfo(info, errs);
 
   }
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
index aff5484..4e3f7f5 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/ContainerPluginsApi.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/ContainerPluginsApi.java
@@ -94,7 +94,7 @@ public class ContainerPluginsApi {
           payload.addError(info.name + " already exists");
           return null;
         }
-        map.put(info.name, info);
+        map.put(info.name, payload.getDataMap());
         return map;
       });
     }
@@ -129,7 +129,7 @@ public class ContainerPluginsApi {
     }
   }
 
-  private void validateConfig(PayloadObj<PluginMeta> payload, PluginMeta info) {
+  private void validateConfig(PayloadObj<PluginMeta> payload, PluginMeta info) throws IOException {
     if (info.klass.indexOf(':') > 0) {
       if (info.version == null) {
         payload.addError("Using package. must provide a packageVersion");
@@ -137,7 +137,7 @@ public class ContainerPluginsApi {
       }
     }
     List<String> errs = new ArrayList<>();
-    ContainerPluginsRegistry.ApiInfo apiInfo = coreContainer.getContainerPluginsRegistry().createInfo(info, errs);
+    ContainerPluginsRegistry.ApiInfo apiInfo = coreContainer.getContainerPluginsRegistry().createInfo( payload.getDataMap(),  errs);
     if (!errs.isEmpty()) {
       for (String err : errs) payload.addError(err);
       return;
diff --git a/solr/core/src/test/org/apache/solr/handler/TestContainerPlugin.java b/solr/core/src/test/org/apache/solr/handler/TestContainerPlugin.java
index 3465a5b..79e3458 100644
--- a/solr/core/src/test/org/apache/solr/handler/TestContainerPlugin.java
+++ b/solr/core/src/test/org/apache/solr/handler/TestContainerPlugin.java
@@ -28,6 +28,7 @@ import org.apache.commons.io.IOUtils;
 import org.apache.lucene.util.ResourceLoader;
 import org.apache.lucene.util.ResourceLoaderAware;
 import org.apache.solr.api.Command;
+import org.apache.solr.api.ConfigurablePlugin;
 import org.apache.solr.api.EndPoint;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrServerException;
@@ -41,6 +42,7 @@ import org.apache.solr.cloud.ClusterSingleton;
 import org.apache.solr.cloud.MiniSolrCloudCluster;
 import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.common.NavigableObject;
+import org.apache.solr.common.annotation.JsonProperty;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.SolrResourceLoader;
@@ -185,6 +187,28 @@ public class TestContainerPlugin extends SolrCloudTestCase {
       assertTrue("ccProvided", C6.ccProvided);
       assertTrue("startCalled", C6.startCalled);
       assertFalse("stopCalled", C6.stopCalled);
+
+      CConfig p = new CConfig();
+      p.boolVal = Boolean.TRUE;
+      p.strVal = "Something";
+      p.longVal = 1234L;
+      p.name = "hello";
+      p.klass = CC.class.getName();
+
+      new V2Request.Builder("/cluster/plugin")
+          .forceV2(true)
+          .withMethod(POST)
+          .withPayload(singletonMap("add", p))
+          .build()
+          .process(cluster.getSolrClient());
+      TestDistribPackageStore.assertResponseValues(10,
+          () -> new V2Request.Builder("hello/plugin")
+              .forceV2(true)
+              .withMethod(GET)
+              .build().process(cluster.getSolrClient()),
+          ImmutableMap.of("/config/boolVal", "true", "/config/strVal", "Something","/config/longVal", "1234" ));
+
+
       // kill the Overseer leader
       for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
         if (!jetty.getCoreContainer().getZkController().getOverseer().isClosed()) {
@@ -312,6 +336,39 @@ public class TestContainerPlugin extends SolrCloudTestCase {
     }
   }
 
+  public static class CC implements ConfigurablePlugin<CConfig> {
+    private CConfig cfg;
+
+
+
+    @Override
+    public void initConfig(CConfig cfg) {
+      this.cfg = cfg;
+
+    }
+
+    @EndPoint(method = GET,
+        path = "/hello/plugin",
+        permission = PermissionNameProvider.Name.READ_PERM)
+    public void m2(SolrQueryRequest req, SolrQueryResponse rsp) {
+      rsp.add("config", cfg);
+    }
+
+  }
+
+  public static class CConfig extends PluginMeta {
+
+    @JsonProperty
+    public String strVal;
+
+    @JsonProperty
+    public Long longVal;
+
+    @JsonProperty
+    public Boolean boolVal;
+
+  }
+
   public static class C6 implements ClusterSingleton {
     static boolean startCalled = false;
     static boolean stopCalled = false;
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/JsonSchemaCreator.java b/solr/solrj/src/java/org/apache/solr/common/util/JsonSchemaCreator.java
index e6cef30..bc32d94 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/JsonSchemaCreator.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/JsonSchemaCreator.java
@@ -87,6 +87,7 @@ public class JsonSchemaCreator {
       if(p.required()) required.add(name);
     }
     if(!required.isEmpty()) map.put("required", new ArrayList<>(required));
+     map.put("additionalProperties", true);
 
   }
 }