You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by dw...@apache.org on 2021/03/10 10:09:55 UTC

[lucene] 23/33: SOLR-15016: Add unit test for dynamic reconfiguration. Remove old API endpoint.

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

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

commit 68536407cee0e2f4049704243071a916251c9453
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Fri Nov 27 13:19:56 2020 +0100

    SOLR-15016: Add unit test for dynamic reconfiguration. Remove old API endpoint.
---
 .../solr/cluster/placement/PlacementPlugin.java    |   5 -
 .../impl/PlacementPluginFactoryLoader.java         |  13 ++-
 .../plugins/AffinityPlacementFactory.java          |   5 +
 .../java/org/apache/solr/handler/ClusterAPI.java   |  23 ----
 .../impl/PlacementPluginIntegrationTest.java       | 122 +++++++++++++++++++--
 5 files changed, 129 insertions(+), 39 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/PlacementPlugin.java b/solr/core/src/java/org/apache/solr/cluster/placement/PlacementPlugin.java
index 11b08dd..bbb52cb 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/PlacementPlugin.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/PlacementPlugin.java
@@ -31,11 +31,6 @@ import org.apache.solr.cluster.Cluster;
  */
 public interface PlacementPlugin {
   /**
-   * Name of the property containing the factory class
-   */
-  String FACTORY_CLASS = "class";
-
-  /**
    * <p>Request from plugin code to compute placement. Note this method must be reentrant as a plugin instance may (read
    * will) get multiple such calls in parallel.
    *
diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementPluginFactoryLoader.java b/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementPluginFactoryLoader.java
index f444834..8207279 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementPluginFactoryLoader.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementPluginFactoryLoader.java
@@ -52,9 +52,11 @@ public class PlacementPluginFactoryLoader {
   /**
    * Helper class to support dynamic reloading of plugin implementations.
    */
-  private static final class DelegatingPlacementPluginFactory implements PlacementPluginFactory {
+  public static final class DelegatingPlacementPluginFactory implements PlacementPluginFactory {
 
     private PlacementPluginFactory delegate;
+    // support for tests to make sure the update is completed
+    private int version;
 
     @Override
     public PlacementPlugin createPluginInstance() {
@@ -67,6 +69,15 @@ public class PlacementPluginFactoryLoader {
 
     public void setDelegate(PlacementPluginFactory delegate) {
       this.delegate = delegate;
+      this.version++;
+    }
+
+    public PlacementPluginFactory getDelegate() {
+      return delegate;
+    }
+
+    public int getVersion() {
+      return version;
     }
   }
 }
diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/plugins/AffinityPlacementFactory.java b/solr/core/src/java/org/apache/solr/cluster/placement/plugins/AffinityPlacementFactory.java
index 4867ab1..a4faa7e 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/plugins/AffinityPlacementFactory.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/plugins/AffinityPlacementFactory.java
@@ -178,6 +178,11 @@ public class AffinityPlacementFactory implements PlacementPluginFactory, Configu
     this.config = cfg;
   }
 
+  @VisibleForTesting
+  public AffinityPlacementConfig getConfig() {
+    return config;
+  }
+
   /**
    * See {@link AffinityPlacementFactory} for instructions on how to configure a cluster to use this plugin and details
    * on what the plugin does.
diff --git a/solr/core/src/java/org/apache/solr/handler/ClusterAPI.java b/solr/core/src/java/org/apache/solr/handler/ClusterAPI.java
index 157c0e5..ee77e3d 100644
--- a/solr/core/src/java/org/apache/solr/handler/ClusterAPI.java
+++ b/solr/core/src/java/org/apache/solr/handler/ClusterAPI.java
@@ -27,9 +27,6 @@ import org.apache.solr.client.solrj.request.beans.ClusterPropInfo;
 import org.apache.solr.client.solrj.request.beans.CreateConfigInfo;
 import org.apache.solr.client.solrj.request.beans.RateLimiterMeta;
 import org.apache.solr.cloud.OverseerConfigSetMessageHandler;
-import org.apache.solr.cluster.placement.PlacementPlugin;
-import org.apache.solr.cluster.placement.PlacementPluginFactory;
-import org.apache.solr.common.MapWriterMap;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.annotation.JsonProperty;
 import org.apache.solr.common.cloud.ClusterProperties;
@@ -244,26 +241,6 @@ public class ClusterAPI {
       collectionsHandler.handleRequestBody(wrapParams(obj.getRequest(), m), obj.getResponse());
     }
 
-    @Command(name = "set-placement-plugin")
-    public void setPlacementPlugin(PayloadObj<Map<String, Object>> obj) {
-      Map<String, Object> placementPluginConfig = obj.getDataMap();
-      if(placementPluginConfig.isEmpty()) placementPluginConfig = null;
-      ClusterProperties clusterProperties = new ClusterProperties(getCoreContainer().getZkController().getZkClient());
-      // When the json contains { "set-placement-plugin" : null }, the map is empty, not null.
-      // Very basic sanity check. Real validation will be done when the config is used...
-      if (!(placementPluginConfig == null) && !placementPluginConfig.containsKey(PlacementPlugin.FACTORY_CLASS)) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Must contain " + PlacementPlugin.FACTORY_CLASS + " attribute (or be null)");
-      }
-      try {
-        clusterProperties.update(placementPluginConfig == null?
-            null:
-            new MapWriterMap(placementPluginConfig),
-            PlacementPluginFactory.PLUGIN_NAME);
-      } catch (Exception e) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error in API", e);
-      }
-    }
-
     @Command(name = "set-ratelimiter")
     public void setRateLimiters(PayloadObj<RateLimiterMeta> payLoad) {
       RateLimiterMeta rateLimiterConfig = payLoad.get();
diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/PlacementPluginIntegrationTest.java b/solr/core/src/test/org/apache/solr/cluster/placement/impl/PlacementPluginIntegrationTest.java
index 90e3965..8c218e7 100644
--- a/solr/core/src/test/org/apache/solr/cluster/placement/impl/PlacementPluginIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/PlacementPluginIntegrationTest.java
@@ -3,19 +3,29 @@ package org.apache.solr.cluster.placement.impl;
 import org.apache.solr.client.solrj.cloud.SolrCloudManager;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.request.V2Request;
+import org.apache.solr.client.solrj.request.beans.PluginMeta;
 import org.apache.solr.client.solrj.response.CollectionAdminResponse;
+import org.apache.solr.client.solrj.response.V2Response;
 import org.apache.solr.cloud.SolrCloudTestCase;
-import org.apache.solr.cluster.placement.PlacementPlugin;
+import org.apache.solr.cluster.placement.PlacementPluginFactory;
+import org.apache.solr.cluster.placement.plugins.AffinityPlacementConfig;
+import org.apache.solr.cluster.placement.plugins.AffinityPlacementFactory;
 import org.apache.solr.cluster.placement.plugins.MinimizeCoresPlacementFactory;
 import org.apache.solr.common.cloud.ClusterProperties;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.util.TimeSource;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.util.TimeOut;
 import org.junit.After;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import static java.util.Collections.singletonMap;
@@ -29,6 +39,7 @@ public class PlacementPluginIntegrationTest extends SolrCloudTestCase {
 
   private static ClusterProperties clusterProperties;
   private static SolrCloudManager cloudManager;
+  private static CoreContainer cc;
 
   @BeforeClass
   public static void setupCluster() throws Exception {
@@ -37,29 +48,38 @@ public class PlacementPluginIntegrationTest extends SolrCloudTestCase {
     configureCluster(3)
         .addConfig("conf", configset("cloud-minimal"))
         .configure();
-    cloudManager = cluster.getJettySolrRunner(0).getCoreContainer().getZkController().getSolrCloudManager();
+    cc = cluster.getJettySolrRunner(0).getCoreContainer();
+    cloudManager = cc.getZkController().getSolrCloudManager();
     clusterProperties = new ClusterProperties(cluster.getZkClient());
   }
 
   @After
   public void cleanup() throws Exception {
     cluster.deleteAllCollections();
-    V2Request req = new V2Request.Builder("/cluster")
+    V2Request req = new V2Request.Builder("/cluster/plugin")
         .forceV2(true)
-        .POST()
-        .withPayload(singletonMap("set-placement-plugin", Map.of()))
+        .GET()
         .build();
-    req.process(cluster.getSolrClient());
-
+    V2Response rsp = req.process(cluster.getSolrClient());
+    if (rsp._get(Arrays.asList("plugin", PlacementPluginFactory.PLUGIN_NAME), null) != null) {
+      req = new V2Request.Builder("/cluster/plugin")
+          .forceV2(true)
+          .POST()
+          .withPayload("{remove: " + PlacementPluginFactory.PLUGIN_NAME + "}")
+          .build();
+      req.process(cluster.getSolrClient());
+    }
   }
 
   @Test
   public void testMinimizeCores() throws Exception {
-    Map<String, Object> config = Map.of(PlacementPlugin.FACTORY_CLASS, MinimizeCoresPlacementFactory.class.getName());
-    V2Request req = new V2Request.Builder("/cluster")
+    PluginMeta plugin = new PluginMeta();
+    plugin.name = PlacementPluginFactory.PLUGIN_NAME;
+    plugin.klass = MinimizeCoresPlacementFactory.class.getName();
+    V2Request req = new V2Request.Builder("/cluster/plugin")
         .forceV2(true)
         .POST()
-        .withPayload(singletonMap("set-placement-plugin", config))
+        .withPayload(singletonMap("add", plugin))
         .build();
     req.process(cluster.getSolrClient());
 
@@ -91,4 +111,86 @@ public class PlacementPluginIntegrationTest extends SolrCloudTestCase {
     assertEquals("min cores too low", 1, minCores);
   }
 
+  @Test
+  public void testDynamicReconfiguration() throws Exception {
+    PluginMeta plugin = new PluginMeta();
+    plugin.name = PlacementPluginFactory.PLUGIN_NAME;
+    plugin.klass = MinimizeCoresPlacementFactory.class.getName();
+    V2Request req = new V2Request.Builder("/cluster/plugin")
+        .forceV2(true)
+        .POST()
+        .withPayload(singletonMap("add", plugin))
+        .build();
+    req.process(cluster.getSolrClient());
+
+    PlacementPluginFactory pluginFactory = cc.getPlacementPluginFactory();
+    assertTrue("wrong type " + pluginFactory.getClass().getName(), pluginFactory instanceof PlacementPluginFactoryLoader.DelegatingPlacementPluginFactory);
+    PlacementPluginFactoryLoader.DelegatingPlacementPluginFactory wrapper = (PlacementPluginFactoryLoader.DelegatingPlacementPluginFactory) pluginFactory;
+    // should already have some updates
+    int version = wrapper.getVersion();
+    assertTrue("wrong version " + version, version > 0);
+    PlacementPluginFactory factory = wrapper.getDelegate();
+    assertTrue("wrong type " + factory.getClass().getName(), factory instanceof MinimizeCoresPlacementFactory);
+
+    // reconfigure
+    plugin.klass = AffinityPlacementFactory.class.getName();
+    plugin.config = new AffinityPlacementConfig(1, 2);
+    req = new V2Request.Builder("/cluster/plugin")
+        .forceV2(true)
+        .POST()
+        .withPayload(singletonMap("update", plugin))
+        .build();
+    req.process(cluster.getSolrClient());
+
+    version = waitForVersionChange(version, wrapper);
+
+    factory = wrapper.getDelegate();
+    assertTrue("wrong type " + factory.getClass().getName(), factory instanceof AffinityPlacementFactory);
+    AffinityPlacementConfig config = ((AffinityPlacementFactory) factory).getConfig();
+    assertEquals("minimalFreeDiskGB", 1, config.minimalFreeDiskGB);
+    assertEquals("prioritizedFreeDiskGB", 2, config.prioritizedFreeDiskGB);
+
+    // change plugin config
+    plugin.config = new AffinityPlacementConfig(3, 4);
+    req = new V2Request.Builder("/cluster/plugin")
+        .forceV2(true)
+        .POST()
+        .withPayload(singletonMap("update", plugin))
+        .build();
+    req.process(cluster.getSolrClient());
+
+    version = waitForVersionChange(version, wrapper);
+    factory = wrapper.getDelegate();
+    assertTrue("wrong type " + factory.getClass().getName(), factory instanceof AffinityPlacementFactory);
+    config = ((AffinityPlacementFactory) factory).getConfig();
+    assertEquals("minimalFreeDiskGB", 3, config.minimalFreeDiskGB);
+    assertEquals("prioritizedFreeDiskGB", 4, config.prioritizedFreeDiskGB);
+
+    // remove plugin
+    req = new V2Request.Builder("/cluster/plugin")
+        .forceV2(true)
+        .POST()
+        .withPayload("{remove: " + PlacementPluginFactory.PLUGIN_NAME + "}")
+        .build();
+    req.process(cluster.getSolrClient());
+    version = waitForVersionChange(version, wrapper);
+    factory = wrapper.getDelegate();
+    assertNull("no factory should be present", factory);
+  }
+
+  private int waitForVersionChange(int currentVersion, PlacementPluginFactoryLoader.DelegatingPlacementPluginFactory wrapper) throws Exception {
+    TimeOut timeout = new TimeOut(60, TimeUnit.SECONDS, TimeSource.NANO_TIME);
+
+    while (!timeout.hasTimedOut()) {
+      int newVersion = wrapper.getVersion();
+      if (newVersion < currentVersion) {
+        throw new Exception("Invalid version - went back! currentVersion=" + currentVersion +
+            " newVersion=" + newVersion);
+      } else if (currentVersion < newVersion) {
+        return newVersion;
+      }
+      timeout.sleep(200);
+    }
+    throw new TimeoutException("version didn't change in time, currentVersion=" + currentVersion);
+  }
 }