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/11/27 09:38:38 UTC

[GitHub] [lucene-solr] sigram opened a new pull request #2101: SOLR-15016 Replica placement plugins should use container plugins API / configs

sigram opened a new pull request #2101:
URL: https://github.com/apache/lucene-solr/pull/2101


   See Jira for more details.


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


[GitHub] [lucene-solr] murblanc commented on a change in pull request #2101: SOLR-15016 Replica placement plugins should use container plugins API / configs

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #2101:
URL: https://github.com/apache/lucene-solr/pull/2101#discussion_r535497621



##########
File path: solr/core/src/java/org/apache/solr/cluster/events/ClusterEventProducer.java
##########
@@ -26,7 +26,7 @@
 public interface ClusterEventProducer extends ClusterSingleton, Closeable {
 
   /** Unique name for the registration of a plugin-based implementation. */
-  String PLUGIN_NAME = "cluster-event-producer";
+  String PLUGIN_NAME = ".cluster-event-producer";

Review comment:
       I don't understand why "." was added, therefore I suggest adding a comment explaining it.




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


[GitHub] [lucene-solr] murblanc commented on a change in pull request #2101: SOLR-15016 Replica placement plugins should use container plugins API / configs

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #2101:
URL: https://github.com/apache/lucene-solr/pull/2101#discussion_r534205062



##########
File path: solr/core/src/java/org/apache/solr/core/CoreContainer.java
##########
@@ -896,6 +900,9 @@ public void load() {
       containerHandlers.getApiBag().registerObject(containerPluginsApi.readAPI);
       containerHandlers.getApiBag().registerObject(containerPluginsApi.editAPI);
 
+      // get the placement plugin

Review comment:
       I'd rather comment "get the placement plugin **factory**"
   And possibly specify what's the plugin factory lifecycle wrt configuration? Now that everything is a bit more implicit than it previously was, I don't get (yet...) when exactly the configuration is passed. I believe comments related to this would be useful.




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


[GitHub] [lucene-solr] TomMD commented on a change in pull request #2101: SOLR-15016 Replica placement plugins should use container plugins API / configs

Posted by GitBox <gi...@apache.org>.
TomMD commented on a change in pull request #2101:
URL: https://github.com/apache/lucene-solr/pull/2101#discussion_r539505535



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/plugins/RandomPlacementFactory.java
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.cluster.placement.plugins;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Random;
+import java.util.Set;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.solr.cluster.Cluster;
+import org.apache.solr.cluster.Node;
+import org.apache.solr.cluster.Replica;
+import org.apache.solr.cluster.SolrCollection;
+import org.apache.solr.cluster.placement.*;
+
+/**
+ * <p>Factory for creating {@link RandomPlacementPlugin}, a placement plugin implementing random placement for new
+ * collection creation while preventing two replicas of same shard from being placed on same node..</p>
+ *
+ * <p>See {@link AffinityPlacementFactory} for a more realistic example and documentation.</p>
+ */
+public class RandomPlacementFactory implements PlacementPluginFactory {
+
+  @Override
+  public PlacementPlugin createPluginInstance() {
+    return new RandomPlacementPlugin();
+  }
+
+  public static class RandomPlacementPlugin implements PlacementPlugin {
+    private Random random = new Random();

Review comment:
       @murblanc Sorry for the delay, this was missed.
   
   The above issue is produced by FSB which suggests `@suppressFBWarnings`.  See [here](http://findbugs.sourceforge.net/api/edu/umd/cs/findbugs/annotations/SuppressFBWarnings.html) for the method to suppress in a case by case basis.
   
   If you'd like to remove the warning globally then you can add `ignore = [ "PREDICTABLE_RANDOM" ]` to the `.muse/config.toml` file.




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


[GitHub] [lucene-solr] murblanc commented on a change in pull request #2101: SOLR-15016 Replica placement plugins should use container plugins API / configs

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #2101:
URL: https://github.com/apache/lucene-solr/pull/2101#discussion_r535333392



##########
File path: solr/core/src/java/org/apache/solr/api/ContainerPluginsRegistry.java
##########
@@ -239,8 +238,7 @@ private static String getActualPath(ApiInfo apiInfo, String path) {
 
   @SuppressWarnings({"rawtypes", "unchecked"})
   private static  Map<String, String> getTemplateVars(PluginMeta pluginMeta) {
-    Map result = makeMap("plugin-name", pluginMeta.name, "path-prefix", pluginMeta.pathPrefix);
-    return result;
+    return (Map) makeMap("plugin-name", pluginMeta.name, "path-prefix", pluginMeta.pathPrefix);

Review comment:
       Should replace the call to `makeMap` by `Map.of(...)`. This allows removing the cast and removing the `@SuppressWarnings`.




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


[GitHub] [lucene-solr] sigram closed pull request #2101: SOLR-15016 Replica placement plugins should use container plugins API / configs

Posted by GitBox <gi...@apache.org>.
sigram closed pull request #2101:
URL: https://github.com/apache/lucene-solr/pull/2101


   


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


[GitHub] [lucene-solr] murblanc commented on a change in pull request #2101: SOLR-15016 Replica placement plugins should use container plugins API / configs

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #2101:
URL: https://github.com/apache/lucene-solr/pull/2101#discussion_r534259966



##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/AddReplicaCmd.java
##########
@@ -144,7 +140,8 @@ public void call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"ra
       }
     }
 
-    List<CreateReplica> createReplicas = buildReplicaPositions(ocmh.cloudManager, clusterState, collectionName, message, replicaTypesVsCount)
+    List<CreateReplica> createReplicas = buildReplicaPositions(ocmh.cloudManager, clusterState, collectionName, message, replicaTypesVsCount,
+        ocmh.overseer.getCoreContainer().getPlacementPluginFactory().createPluginInstance())

Review comment:
       Ok, I see that's how we tell which one is configured by this value being `null`... Can't say I really like it, but at least it should be commented (here and in `PlacementPluginFactory` as well?).




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


[GitHub] [lucene-solr] murblanc commented on a change in pull request #2101: SOLR-15016 Replica placement plugins should use container plugins API / configs

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #2101:
URL: https://github.com/apache/lucene-solr/pull/2101#discussion_r536287423



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/plugins/RandomPlacementFactory.java
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.cluster.placement.plugins;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Random;
+import java.util.Set;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.solr.cluster.Cluster;
+import org.apache.solr.cluster.Node;
+import org.apache.solr.cluster.Replica;
+import org.apache.solr.cluster.SolrCollection;
+import org.apache.solr.cluster.placement.*;
+
+/**
+ * <p>Factory for creating {@link RandomPlacementPlugin}, a placement plugin implementing random placement for new
+ * collection creation while preventing two replicas of same shard from being placed on same node..</p>
+ *
+ * <p>See {@link AffinityPlacementFactory} for a more realistic example and documentation.</p>
+ */
+public class RandomPlacementFactory implements PlacementPluginFactory {
+
+  @Override
+  public PlacementPlugin createPluginInstance() {
+    return new RandomPlacementPlugin();
+  }
+
+  public static class RandomPlacementPlugin implements PlacementPlugin {
+    private Random random = new Random();

Review comment:
       @TomMD is there a way to annotate the code when we know/decide this is not an issue to silence muse-dev? (@madrob suggested to ask you).




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


[GitHub] [lucene-solr] murblanc commented on a change in pull request #2101: SOLR-15016 Replica placement plugins should use container plugins API / configs

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #2101:
URL: https://github.com/apache/lucene-solr/pull/2101#discussion_r536261667



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementPluginFactoryLoader.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.cluster.placement.impl;
+
+import org.apache.solr.api.ContainerPluginsRegistry;
+import org.apache.solr.client.solrj.request.beans.PluginMeta;
+import org.apache.solr.cluster.placement.PlacementPluginConfig;
+import org.apache.solr.cluster.placement.PlacementPluginFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.invoke.MethodHandles;
+
+/**
+ * Utility class to load the configured {@link PlacementPluginFactory} plugin and
+ * then keep it up to date as the plugin configuration changes.
+ */
+public class PlacementPluginFactoryLoader {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  public static void load(DelegatingPlacementPluginFactory pluginFactory, ContainerPluginsRegistry plugins) {
+    ContainerPluginsRegistry.ApiInfo pluginFactoryInfo = plugins.getPlugin(PlacementPluginFactory.PLUGIN_NAME);
+    if (pluginFactoryInfo != null && (pluginFactoryInfo.getInstance() instanceof PlacementPluginFactory)) {
+      pluginFactory.setDelegate((PlacementPluginFactory<? extends PlacementPluginConfig>) pluginFactoryInfo.getInstance());
+    }
+    ContainerPluginsRegistry.PluginRegistryListener pluginListener = new ContainerPluginsRegistry.PluginRegistryListener() {
+      @Override
+      public void added(ContainerPluginsRegistry.ApiInfo plugin) {
+        if (plugin == null || plugin.getInstance() == null) {
+          return;
+        }
+        Object instance = plugin.getInstance();
+        if (instance instanceof PlacementPluginFactory) {
+          setDelegate(plugin.getInfo(), instance);

Review comment:
       1/3: maybe we should cast the second parameter `instance` to `PlacementPluginFactory` here.




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


[GitHub] [lucene-solr] sigram commented on pull request #2101: SOLR-15016 Replica placement plugins should use container plugins API / configs

Posted by GitBox <gi...@apache.org>.
sigram commented on pull request #2101:
URL: https://github.com/apache/lucene-solr/pull/2101#issuecomment-741836807


   Merged.


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


[GitHub] [lucene-solr] muse-dev[bot] commented on a change in pull request #2101: SOLR-15016 Replica placement plugins should use container plugins API / configs

Posted by GitBox <gi...@apache.org>.
muse-dev[bot] commented on a change in pull request #2101:
URL: https://github.com/apache/lucene-solr/pull/2101#discussion_r531588535



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/plugins/AffinityPlacementFactory.java
##########
@@ -0,0 +1,537 @@
+/*
+ * 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.cluster.placement.plugins;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Ordering;
+import com.google.common.collect.TreeMultimap;
+import org.apache.solr.api.ConfigurablePlugin;
+import org.apache.solr.cluster.*;
+import org.apache.solr.cluster.placement.*;
+import org.apache.solr.common.util.Pair;
+import org.apache.solr.common.util.SuppressForbidden;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.invoke.MethodHandles;
+import java.util.*;
+import java.util.stream.Collectors;
+
+/**
+ * <p>This factory is instantiated by config from its class name. Using it is the only way to create instances of
+ * {@link AffinityPlacementPlugin}.</p>
+ *
+ * <p>In order to configure this plugin to be used for placement decisions, the following {@code curl} command (or something
+ * equivalent) has to be executed once the cluster is already running in order to set
+ * the appropriate Zookeeper stored configuration. Replace {@code localhost:8983} by one of your servers' IP address and port.</p>
+ *
+ * <pre>
+ *
+ * curl -X POST -H 'Content-type:application/json' -d '{
+ * "set-placement-plugin": {
+ * "class": "org.apache.solr.cluster.placement.plugins.AffinityPlacementFactory",
+ * "minimalFreeDiskGB": 10,
+ * "prioritizedFreeDiskGB": 50
+ * }
+ * }' http://localhost:8983/api/cluster
+ * </pre>
+ *
+ * <p>The consequence will be the creation of an element in the Zookeeper file {@code /clusterprops.json} as follows:</p>
+ *
+ * <pre>
+ *
+ * "placement-plugin":{
+ *     "class":"org.apache.solr.cluster.placement.plugins.AffinityPlacementFactory",
+ *     "minimalFreeDiskGB":10,
+ *     "prioritizedFreeDiskGB":50}
+ * </pre>
+ *
+ * <p>In order to delete the placement-plugin section from {@code /clusterprops.json} (and to fallback to either Legacy
+ * or rule based placement if configured for a collection), execute:</p>
+ *
+ * <pre>
+ *
+ * curl -X POST -H 'Content-type:application/json' -d '{
+ * "set-placement-plugin" : null
+ * }' http://localhost:8983/api/cluster
+ * </pre>
+ *
+ *
+ * <p>{@link AffinityPlacementPlugin} implements placing replicas in a way that replicate past Autoscaling config defined
+ * <a href="https://github.com/lucidworks/fusion-cloud-native/blob/master/policy.json#L16">here</a>.</p>
+ *
+ * <p>This specification is doing the following:
+ * <p><i>Spread replicas per shard as evenly as possible across multiple availability zones (given by a sys prop),
+ * assign replicas based on replica type to specific kinds of nodes (another sys prop), and avoid having more than
+ * one replica per shard on the same node.<br>
+ * Only after these constraints are satisfied do minimize cores per node or disk usage.</i></p>
+ *
+ * <p>Overall strategy of this plugin:</p>
+ * <ul><li>
+ *     The set of nodes in the cluster is obtained and transformed into 3 independent sets (that can overlap) of nodes
+ *     accepting each of the three replica types.
+ * </li><li>
+ *     For each shard on which placing replicas is required and then for each replica type to place (starting with NRT,
+ *     then TLOG then PULL): <ul>
+ *         <li>The set of candidates nodes corresponding to the replica type is used and from that set are removed nodes
+ *         that already have a replica (of any type) for that shard</li>
+ *         <li>If there are not enough nodes, an error is thrown (this is checked further down during processing).</li>
+ *         <li>The number of (already existing) replicas of the current type on each Availability Zone is collected.</li>
+ *         <li>Separate the set of available nodes to as many subsets (possibly some are empty) as there are Availability Zones
+ *         defined for the candidate nodes</li>
+ *         <li>In each AZ nodes subset, sort the nodes by increasing total number of cores count, with possibly a condition
+ *         that pushes nodes with low disk space to the end of the list? Or a weighted combination of the relative
+ *         importance of these two factors? Some randomization? Marking as non available nodes with not enough disk space?
+ *         These and other are likely aspects to be played with once the plugin is tested or observed to be running in prod,
+ *         don't expect the initial code drop(s) to do all of that.</li>
+ *         <li>Iterate over the number of replicas to place (for the current replica type for the current shard):
+ *         <ul>
+ *             <li>Based on the number of replicas per AZ collected previously, pick the non empty set of nodes having the
+ *             lowest number of replicas. Then pick the first node in that set. That's the node the replica is placed one.
+ *             Remove the node from the set of available nodes for the given AZ and increase the number of replicas placed
+ *             on that AZ.</li>
+ *         </ul></li>
+ *         <li>During this process, the number of cores on the nodes in general is tracked to take into account placement
+ *         decisions so that not all shards decide to put their replicas on the same nodes (they might though if these are
+ *         the less loaded nodes).</li>
+ *     </ul>
+ * </li>
+ * </ul>
+ *
+ * <p>This code is a realistic placement computation, based on a few assumptions. The code is written in such a way to
+ * make it relatively easy to adapt it to (somewhat) different assumptions. Configuration options could be introduced
+ * to allow configuration base option selection as well...</p>
+ */
+public class AffinityPlacementFactory implements PlacementPluginFactory, ConfigurablePlugin<AffinityPlacementConfig> {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  /**
+   * <p>Name of the system property on a node indicating which (public cloud) Availability Zone that node is in. The value
+   * is any string, different strings denote different availability zones.
+   *
+   * <p>Nodes on which this system property is not defined are considered being in the same Availability Zone
+   * {@link #UNDEFINED_AVAILABILITY_ZONE} (hopefully the value of this constant is not the name of a real Availability Zone :).
+   */
+  public static final String AVAILABILITY_ZONE_SYSPROP = "availability_zone";
+
+  /**
+   * <p>Name of the system property on a node indicating the type of replicas allowed on that node.
+   * The value of that system property is a comma separated list or a single string of value names of
+   * {@link org.apache.solr.cluster.Replica.ReplicaType} (case insensitive). If that property is not defined, that node is
+   * considered accepting all replica types (i.e. undefined is equivalent to {@code "NRT,Pull,tlog"}).
+   */
+  public static final String REPLICA_TYPE_SYSPROP = "replica_type";
+
+  /**
+   * This is the "AZ" name for nodes that do not define an AZ. Should not match a real AZ name (I think we're safe)
+   */
+  public static final String UNDEFINED_AVAILABILITY_ZONE = "uNd3f1NeD";
+
+  /**
+   * If a node has strictly less GB of free disk than this value, the node is excluded from assignment decisions.
+   * Set to 0 or less to disable.
+   */
+  public static final String MINIMAL_FREE_DISK_GB = "minimalFreeDiskGB";
+
+  /**
+   * Replica allocation will assign replicas to nodes with at least this number of GB of free disk space regardless
+   * of the number of cores on these nodes rather than assigning replicas to nodes with less than this amount of free
+   * disk space if that's an option (if that's not an option, replicas can still be assigned to nodes with less than this
+   * amount of free space).
+   */
+  public static final String PRIORITIZED_FREE_DISK_GB = "prioritizedFreeDiskGB";
+
+  private AffinityPlacementConfig config = AffinityPlacementConfig.DEFAULT;
+
+  /**
+   * Empty public constructor is used to instantiate this factory. Using a factory pattern to allow the factory to do one
+   * time costly operations if needed, and to only have to instantiate a default constructor class by name, rather than
+   * having to call a constructor with more parameters (if we were to instantiate the plugin class directly without going
+   * through a factory).
+   */
+  public AffinityPlacementFactory() {
+  }
+
+  @Override
+  public PlacementPlugin createPluginInstance() {
+    return new AffinityPlacementPlugin(config.minimalFreeDiskGB, config.prioritizedFreeDiskGB);
+  }
+
+  @Override
+  public void configure(AffinityPlacementConfig cfg) {
+    Objects.requireNonNull(cfg, "configuration must never be null");
+    this.config = cfg;
+  }
+
+  /**
+   * See {@link AffinityPlacementFactory} for instructions on how to configure a cluster to use this plugin and details
+   * on what the plugin does.
+   */
+  static class AffinityPlacementPlugin implements PlacementPlugin {
+
+    private final long minimalFreeDiskGB;
+
+    private final long prioritizedFreeDiskGB;
+
+    private Random random = new Random();

Review comment:
       *PREDICTABLE_RANDOM:*  This random generator (java.util.Random) is predictable [(details)](https://find-sec-bugs.github.io/bugs.htm#PREDICTABLE_RANDOM)

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/plugins/RandomPlacementFactory.java
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.cluster.placement.plugins;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Random;
+import java.util.Set;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.solr.cluster.Cluster;
+import org.apache.solr.cluster.Node;
+import org.apache.solr.cluster.Replica;
+import org.apache.solr.cluster.SolrCollection;
+import org.apache.solr.cluster.placement.*;
+
+/**
+ * <p>Factory for creating {@link RandomPlacementPlugin}, a placement plugin implementing random placement for new
+ * collection creation while preventing two replicas of same shard from being placed on same node..</p>
+ *
+ * <p>See {@link AffinityPlacementFactory} for a more realistic example and documentation.</p>
+ */
+public class RandomPlacementFactory implements PlacementPluginFactory {
+
+  @Override
+  public PlacementPlugin createPluginInstance() {
+    return new RandomPlacementPlugin();
+  }
+
+  public static class RandomPlacementPlugin implements PlacementPlugin {
+    private Random random = new Random();

Review comment:
       *PREDICTABLE_RANDOM:*  This random generator (java.util.Random) is predictable [(details)](https://find-sec-bugs.github.io/bugs.htm#PREDICTABLE_RANDOM)




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


[GitHub] [lucene-solr] sigram commented on a change in pull request #2101: SOLR-15016 Replica placement plugins should use container plugins API / configs

Posted by GitBox <gi...@apache.org>.
sigram commented on a change in pull request #2101:
URL: https://github.com/apache/lucene-solr/pull/2101#discussion_r534293404



##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/AddReplicaCmd.java
##########
@@ -144,7 +140,8 @@ public void call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"ra
       }
     }
 
-    List<CreateReplica> createReplicas = buildReplicaPositions(ocmh.cloudManager, clusterState, collectionName, message, replicaTypesVsCount)
+    List<CreateReplica> createReplicas = buildReplicaPositions(ocmh.cloudManager, clusterState, collectionName, message, replicaTypesVsCount,
+        ocmh.overseer.getCoreContainer().getPlacementPluginFactory().createPluginInstance())

Review comment:
       The plugin is null and `Assign.createAssignStrategy` provides a `LegacyAssignStrategy`.




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


[GitHub] [lucene-solr] sigram commented on pull request #2101: SOLR-15016 Replica placement plugins should use container plugins API / configs

Posted by GitBox <gi...@apache.org>.
sigram commented on pull request #2101:
URL: https://github.com/apache/lucene-solr/pull/2101#issuecomment-734742963


   This branch includes changes in branch `jira/solr-15004`.


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


[GitHub] [lucene-solr] murblanc commented on a change in pull request #2101: SOLR-15016 Replica placement plugins should use container plugins API / configs

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #2101:
URL: https://github.com/apache/lucene-solr/pull/2101#discussion_r535506478



##########
File path: solr/core/src/java/org/apache/solr/cluster/events/impl/ClusterEventProducerFactory.java
##########
@@ -128,13 +128,17 @@ public void added(ContainerPluginsRegistry.ApiInfo plugin) {
           ClusterEventListener listener = (ClusterEventListener) instance;
           clusterEventProducer.registerListener(listener);
         } else if (instance instanceof ClusterEventProducer) {
-          // replace the existing impl
-          if (cc.getClusterEventProducer() instanceof DelegatingClusterEventProducer) {
-            ((DelegatingClusterEventProducer) cc.getClusterEventProducer())
-                .setDelegate((ClusterEventProducer) instance);
+          if (ClusterEventProducer.PLUGIN_NAME.equals(plugin.getInfo().name)) {
+            // replace the existing impl
+            if (cc.getClusterEventProducer() instanceof DelegatingClusterEventProducer) {
+              ((DelegatingClusterEventProducer) cc.getClusterEventProducer())
+                  .setDelegate((ClusterEventProducer) instance);

Review comment:
       It seems we still have synchronization issues on access to `DelegatingClusterEventProducer.delegate`, right? Might not be on this PR though (and I might have it wrong because I don't fully understand this plugin and events architecture) but in `DelegatingClusterEventProducer` I see non sync access to `delegate` and I wonder...




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


[GitHub] [lucene-solr] murblanc commented on a change in pull request #2101: SOLR-15016 Replica placement plugins should use container plugins API / configs

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #2101:
URL: https://github.com/apache/lucene-solr/pull/2101#discussion_r534210911



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PlacementPluginFactory.java
##########
@@ -18,14 +18,22 @@
 package org.apache.solr.cluster.placement;
 
 /**
- * Factory implemented by client code and configured in {@code solr.xml} allowing the creation of instances of
+ * Factory implemented by client code and configured in container plugins allowing the creation of instances of
  * {@link PlacementPlugin} to be used for replica placement computation.
+ * <p>Note: configurable factory implementations should also implement
+ * {@link org.apache.solr.api.ConfigurablePlugin} with the appropriate configuration
+ * bean type.</p>
  */
 public interface PlacementPluginFactory {

Review comment:
       That would force every placement plugin to be configurable though I guess...




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


[GitHub] [lucene-solr] murblanc commented on a change in pull request #2101: SOLR-15016 Replica placement plugins should use container plugins API / configs

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #2101:
URL: https://github.com/apache/lucene-solr/pull/2101#discussion_r536262348



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementPluginFactoryLoader.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.cluster.placement.impl;
+
+import org.apache.solr.api.ContainerPluginsRegistry;
+import org.apache.solr.client.solrj.request.beans.PluginMeta;
+import org.apache.solr.cluster.placement.PlacementPluginConfig;
+import org.apache.solr.cluster.placement.PlacementPluginFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.invoke.MethodHandles;
+
+/**
+ * Utility class to load the configured {@link PlacementPluginFactory} plugin and
+ * then keep it up to date as the plugin configuration changes.
+ */
+public class PlacementPluginFactoryLoader {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  public static void load(DelegatingPlacementPluginFactory pluginFactory, ContainerPluginsRegistry plugins) {
+    ContainerPluginsRegistry.ApiInfo pluginFactoryInfo = plugins.getPlugin(PlacementPluginFactory.PLUGIN_NAME);
+    if (pluginFactoryInfo != null && (pluginFactoryInfo.getInstance() instanceof PlacementPluginFactory)) {
+      pluginFactory.setDelegate((PlacementPluginFactory<? extends PlacementPluginConfig>) pluginFactoryInfo.getInstance());
+    }
+    ContainerPluginsRegistry.PluginRegistryListener pluginListener = new ContainerPluginsRegistry.PluginRegistryListener() {
+      @Override
+      public void added(ContainerPluginsRegistry.ApiInfo plugin) {
+        if (plugin == null || plugin.getInstance() == null) {
+          return;
+        }
+        Object instance = plugin.getInstance();
+        if (instance instanceof PlacementPluginFactory) {
+          setDelegate(plugin.getInfo(), instance);
+        }
+      }
+
+      @Override
+      public void deleted(ContainerPluginsRegistry.ApiInfo plugin) {
+        if (plugin == null || plugin.getInstance() == null) {
+          return;
+        }
+        Object instance = plugin.getInstance();
+        if (instance instanceof PlacementPluginFactory) {
+          setDelegate(plugin.getInfo(), null);
+        }
+      }
+
+      @Override
+      public void modified(ContainerPluginsRegistry.ApiInfo old, ContainerPluginsRegistry.ApiInfo replacement) {
+        added(replacement);
+      }
+
+      private void setDelegate(PluginMeta pluginMeta, Object instance) {
+        if (PlacementPluginFactory.PLUGIN_NAME.equals(pluginMeta.name)) {
+          pluginFactory.setDelegate((PlacementPluginFactory<? extends PlacementPluginConfig>) instance);

Review comment:
       3/3: remove the cast here, `instance` would be of the expected type.




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


[GitHub] [lucene-solr] murblanc commented on a change in pull request #2101: SOLR-15016 Replica placement plugins should use container plugins API / configs

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #2101:
URL: https://github.com/apache/lucene-solr/pull/2101#discussion_r535520411



##########
File path: solr/core/src/java/org/apache/solr/cluster/events/impl/ClusterEventProducerFactory.java
##########
@@ -149,21 +153,25 @@ public void deleted(ContainerPluginsRegistry.ApiInfo plugin) {
           ClusterEventListener listener = (ClusterEventListener) instance;
           clusterEventProducer.unregisterListener(listener);
         } else if (instance instanceof ClusterEventProducer) {
-          // replace the existing impl with NoOp
-          if (cc.getClusterEventProducer() instanceof DelegatingClusterEventProducer) {
-            ((DelegatingClusterEventProducer) cc.getClusterEventProducer())
-                .setDelegate(new NoOpProducer(cc));
+          if (ClusterEventProducer.PLUGIN_NAME.equals(plugin.getInfo().name)) {

Review comment:
       There's quite some duplication between this listener method (`deleted`) and method `added` above. Likely could use some factoring out (and maybe slight rework of the logs to differentiate adding and deleting?).




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


[GitHub] [lucene-solr] murblanc commented on pull request #2101: SOLR-15016 Replica placement plugins should use container plugins API / configs

Posted by GitBox <gi...@apache.org>.
murblanc commented on pull request #2101:
URL: https://github.com/apache/lucene-solr/pull/2101#issuecomment-738942564


   Looks good to me!


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


[GitHub] [lucene-solr] murblanc commented on a change in pull request #2101: SOLR-15016 Replica placement plugins should use container plugins API / configs

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #2101:
URL: https://github.com/apache/lucene-solr/pull/2101#discussion_r535361197



##########
File path: solr/core/src/test/org/apache/solr/cluster/placement/impl/PlacementPluginIntegrationTest.java
##########
@@ -109,4 +132,108 @@ public void testMinimizeCores() throws Exception {
     assertEquals("min cores too low", 1, minCores);
   }
 
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testDynamicReconfiguration() throws Exception {
+    PlacementPluginFactory<? extends MapWriter> pluginFactory = cc.getPlacementPluginFactory();

Review comment:
       `<? extends PlacementPluginConfig>`?




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


[GitHub] [lucene-solr] murblanc commented on a change in pull request #2101: SOLR-15016 Replica placement plugins should use container plugins API / configs

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #2101:
URL: https://github.com/apache/lucene-solr/pull/2101#discussion_r535359225



##########
File path: solr/core/src/java/org/apache/solr/cluster/events/impl/CollectionsRepairEventListener.java
##########
@@ -76,10 +78,12 @@
   private int waitForSecond = DEFAULT_WAIT_FOR_SEC;
 
   private ScheduledThreadPoolExecutor waitForExecutor;
+  private PlacementPluginFactory<? extends MapWriter> placementPluginFactory;

Review comment:
       Should this rather be `<? extends PlacementPluginConfig>`?




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


[GitHub] [lucene-solr] sigram commented on a change in pull request #2101: SOLR-15016 Replica placement plugins should use container plugins API / configs

Posted by GitBox <gi...@apache.org>.
sigram commented on a change in pull request #2101:
URL: https://github.com/apache/lucene-solr/pull/2101#discussion_r536361329



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/plugins/MinimizeCoresPlacementFactory.java
##########
@@ -40,13 +40,23 @@
  *
  * <p>See {@link AffinityPlacementFactory} for a more realistic example and documentation.</p>
  */
-public class MinimizeCoresPlacementFactory implements PlacementPluginFactory {
+public class MinimizeCoresPlacementFactory implements PlacementPluginFactory<PlacementPluginFactory.NoConfig> {
 
   @Override
-  public PlacementPlugin createPluginInstance(PlacementPluginConfig config) {
+  public PlacementPlugin createPluginInstance() {
     return new MinimizeCoresPlacementPlugin();
   }
 
+  @Override
+  public void configure(NoConfig cfg) {
+    // no-op
+  }
+
+  @Override
+  public NoConfig getConfig() {

Review comment:
       I think it needs to return null here, I don't see any other way to return an instance that is assignable without ClassCastException to any `<T extends PlacementPluginConfig>`.




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


[GitHub] [lucene-solr] murblanc commented on a change in pull request #2101: SOLR-15016 Replica placement plugins should use container plugins API / configs

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #2101:
URL: https://github.com/apache/lucene-solr/pull/2101#discussion_r536262072



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementPluginFactoryLoader.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.cluster.placement.impl;
+
+import org.apache.solr.api.ContainerPluginsRegistry;
+import org.apache.solr.client.solrj.request.beans.PluginMeta;
+import org.apache.solr.cluster.placement.PlacementPluginConfig;
+import org.apache.solr.cluster.placement.PlacementPluginFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.invoke.MethodHandles;
+
+/**
+ * Utility class to load the configured {@link PlacementPluginFactory} plugin and
+ * then keep it up to date as the plugin configuration changes.
+ */
+public class PlacementPluginFactoryLoader {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  public static void load(DelegatingPlacementPluginFactory pluginFactory, ContainerPluginsRegistry plugins) {
+    ContainerPluginsRegistry.ApiInfo pluginFactoryInfo = plugins.getPlugin(PlacementPluginFactory.PLUGIN_NAME);
+    if (pluginFactoryInfo != null && (pluginFactoryInfo.getInstance() instanceof PlacementPluginFactory)) {
+      pluginFactory.setDelegate((PlacementPluginFactory<? extends PlacementPluginConfig>) pluginFactoryInfo.getInstance());
+    }
+    ContainerPluginsRegistry.PluginRegistryListener pluginListener = new ContainerPluginsRegistry.PluginRegistryListener() {
+      @Override
+      public void added(ContainerPluginsRegistry.ApiInfo plugin) {
+        if (plugin == null || plugin.getInstance() == null) {
+          return;
+        }
+        Object instance = plugin.getInstance();
+        if (instance instanceof PlacementPluginFactory) {
+          setDelegate(plugin.getInfo(), instance);
+        }
+      }
+
+      @Override
+      public void deleted(ContainerPluginsRegistry.ApiInfo plugin) {
+        if (plugin == null || plugin.getInstance() == null) {
+          return;
+        }
+        Object instance = plugin.getInstance();
+        if (instance instanceof PlacementPluginFactory) {
+          setDelegate(plugin.getInfo(), null);
+        }
+      }
+
+      @Override
+      public void modified(ContainerPluginsRegistry.ApiInfo old, ContainerPluginsRegistry.ApiInfo replacement) {
+        added(replacement);
+      }
+
+      private void setDelegate(PluginMeta pluginMeta, Object instance) {

Review comment:
       2/3: change the signature of this method so that `instance` is a `PlacementPluginFactory`.




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


[GitHub] [lucene-solr] murblanc commented on a change in pull request #2101: SOLR-15016 Replica placement plugins should use container plugins API / configs

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #2101:
URL: https://github.com/apache/lucene-solr/pull/2101#discussion_r534255115



##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/AddReplicaCmd.java
##########
@@ -144,7 +140,8 @@ public void call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"ra
       }
     }
 
-    List<CreateReplica> createReplicas = buildReplicaPositions(ocmh.cloudManager, clusterState, collectionName, message, replicaTypesVsCount)
+    List<CreateReplica> createReplicas = buildReplicaPositions(ocmh.cloudManager, clusterState, collectionName, message, replicaTypesVsCount,
+        ocmh.overseer.getCoreContainer().getPlacementPluginFactory().createPluginInstance())

Review comment:
       Unclear to me: what happens here when placement plugins are not configured and we use for example the legacy assign strategy?




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


[GitHub] [lucene-solr] sigram commented on a change in pull request #2101: SOLR-15016 Replica placement plugins should use container plugins API / configs

Posted by GitBox <gi...@apache.org>.
sigram commented on a change in pull request #2101:
URL: https://github.com/apache/lucene-solr/pull/2101#discussion_r535359830



##########
File path: solr/core/src/java/org/apache/solr/cluster/events/impl/CollectionsRepairEventListener.java
##########
@@ -76,10 +78,12 @@
   private int waitForSecond = DEFAULT_WAIT_FOR_SEC;
 
   private ScheduledThreadPoolExecutor waitForExecutor;
+  private PlacementPluginFactory<? extends MapWriter> placementPluginFactory;

Review comment:
       Yes.




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


[GitHub] [lucene-solr] sigram commented on a change in pull request #2101: SOLR-15016 Replica placement plugins should use container plugins API / configs

Posted by GitBox <gi...@apache.org>.
sigram commented on a change in pull request #2101:
URL: https://github.com/apache/lucene-solr/pull/2101#discussion_r534234770



##########
File path: solr/core/src/java/org/apache/solr/core/CoreContainer.java
##########
@@ -257,6 +260,7 @@ public CoreLoadFailure(CoreDescriptor cd, Exception loadFailure) {
 
   // initially these are the same to collect the plugin-based listeners during init
   private ClusterEventProducer clusterEventProducer;
+  private PlacementPluginFactory placementPluginFactory;

Review comment:
       Since we always use `DelegatingPlacementPluginFactory` I'm going to create a final instance of this wrapper - and then it will be initialized once the plugins registry is ready.




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


[GitHub] [lucene-solr] murblanc commented on a change in pull request #2101: SOLR-15016 Replica placement plugins should use container plugins API / configs

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #2101:
URL: https://github.com/apache/lucene-solr/pull/2101#discussion_r536270908



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/plugins/MinimizeCoresPlacementFactory.java
##########
@@ -40,13 +40,23 @@
  *
  * <p>See {@link AffinityPlacementFactory} for a more realistic example and documentation.</p>
  */
-public class MinimizeCoresPlacementFactory implements PlacementPluginFactory {
+public class MinimizeCoresPlacementFactory implements PlacementPluginFactory<PlacementPluginFactory.NoConfig> {
 
   @Override
-  public PlacementPlugin createPluginInstance(PlacementPluginConfig config) {
+  public PlacementPlugin createPluginInstance() {
     return new MinimizeCoresPlacementPlugin();
   }
 
+  @Override
+  public void configure(NoConfig cfg) {

Review comment:
       Maybe `configure()` in `ConfigurablePlugin<T extends MapWriter>` should have a default empty implementation so we don't have to repeat it (here, in `DelegatingPlacementPluginFactory` and `RandomPlacementFactory`).




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


[GitHub] [lucene-solr] murblanc commented on a change in pull request #2101: SOLR-15016 Replica placement plugins should use container plugins API / configs

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #2101:
URL: https://github.com/apache/lucene-solr/pull/2101#discussion_r535360258



##########
File path: solr/core/src/java/org/apache/solr/core/CoreContainer.java
##########
@@ -2180,6 +2188,10 @@ public ClusterEventProducer getClusterEventProducer() {
     return clusterEventProducer;
   }
 
+  public PlacementPluginFactory<? extends MapWriter> getPlacementPluginFactory() {

Review comment:
       Shouldn't this be `<? extends PlacementPluginConfig>`?




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


[GitHub] [lucene-solr] sigram commented on a change in pull request #2101: SOLR-15016 Replica placement plugins should use container plugins API / configs

Posted by GitBox <gi...@apache.org>.
sigram commented on a change in pull request #2101:
URL: https://github.com/apache/lucene-solr/pull/2101#discussion_r535527148



##########
File path: solr/core/src/java/org/apache/solr/cluster/events/impl/ClusterEventProducerFactory.java
##########
@@ -149,21 +153,25 @@ public void deleted(ContainerPluginsRegistry.ApiInfo plugin) {
           ClusterEventListener listener = (ClusterEventListener) instance;
           clusterEventProducer.unregisterListener(listener);
         } else if (instance instanceof ClusterEventProducer) {
-          // replace the existing impl with NoOp
-          if (cc.getClusterEventProducer() instanceof DelegatingClusterEventProducer) {
-            ((DelegatingClusterEventProducer) cc.getClusterEventProducer())
-                .setDelegate(new NoOpProducer(cc));
+          if (ClusterEventProducer.PLUGIN_NAME.equals(plugin.getInfo().name)) {

Review comment:
       I debated whether to do it in this PR or separately... strictly speaking it's not a part of this PR, but I refactored a similar pattern in a listener that belongs to this PR ... so maybe I'll do it here :)




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


[GitHub] [lucene-solr] murblanc commented on a change in pull request #2101: SOLR-15016 Replica placement plugins should use container plugins API / configs

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #2101:
URL: https://github.com/apache/lucene-solr/pull/2101#discussion_r535354318



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PlacementPluginFactory.java
##########
@@ -17,15 +17,37 @@
 
 package org.apache.solr.cluster.placement;
 
+import org.apache.solr.api.ConfigurablePlugin;
+
 /**
- * Factory implemented by client code and configured in {@code solr.xml} allowing the creation of instances of
+ * Factory implemented by client code and configured in container plugins allowing the creation of instances of
  * {@link PlacementPlugin} to be used for replica placement computation.
+ * <p>Note: configurable factory implementations should also implement
+ * {@link org.apache.solr.api.ConfigurablePlugin} with the appropriate configuration
+ * bean type.</p>
  */
-public interface PlacementPluginFactory {
+public interface PlacementPluginFactory<T extends PlacementPluginConfig> extends ConfigurablePlugin<T> {
   /**
-   * Returns an instance of the plugin that will be repeatedly (and concurrently) be called to compute placement. Multiple
+   * The key in the plugins registry under which this plugin and its configuration are defined.
+   */
+  String PLUGIN_NAME = ".placement-plugin";
+
+  /**
+   * Returns an instance of the plugin that will be repeatedly (and concurrently) called to compute placement. Multiple
    * instances of a plugin can be used in parallel (for example if configuration has to change, but plugin instances with
    * the previous configuration are still being used).
    */
-  PlacementPlugin createPluginInstance(PlacementPluginConfig config);
+  PlacementPlugin createPluginInstance();

Review comment:
       See previous comment in [AddReplicaCmd](https://github.com/apache/lucene-solr/pull/2101/files#diff-9190d76945a6b15eee98e2a3725de99d342c635b87cec2992cc2a443c247e3e7R144).
   A comment should be added here saying that when this returns null it means legacy assign strategy. 




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


[GitHub] [lucene-solr] sigram commented on a change in pull request #2101: SOLR-15016 Replica placement plugins should use container plugins API / configs

Posted by GitBox <gi...@apache.org>.
sigram commented on a change in pull request #2101:
URL: https://github.com/apache/lucene-solr/pull/2101#discussion_r535525525



##########
File path: solr/core/src/java/org/apache/solr/cluster/events/ClusterEventProducer.java
##########
@@ -26,7 +26,7 @@
 public interface ClusterEventProducer extends ClusterSingleton, Closeable {
 
   /** Unique name for the registration of a plugin-based implementation. */
-  String PLUGIN_NAME = "cluster-event-producer";
+  String PLUGIN_NAME = ".cluster-event-producer";

Review comment:
       There was a Slack discussion on the naming scheme for pre-defined plugins (on #solr-plugins) and the overall sentiment was to use a dot prefix for such names. It's a convention not a requirement - and it will be documented in SOLR-15022.




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


[GitHub] [lucene-solr] sigram commented on a change in pull request #2101: SOLR-15016 Replica placement plugins should use container plugins API / configs

Posted by GitBox <gi...@apache.org>.
sigram commented on a change in pull request #2101:
URL: https://github.com/apache/lucene-solr/pull/2101#discussion_r535356233



##########
File path: solr/core/src/java/org/apache/solr/api/ContainerPluginsRegistry.java
##########
@@ -422,7 +421,10 @@ public static Class getConfigClass(ConfigurablePlugin<?> o) {
       for (Type type : interfaces) {
         if (type instanceof ParameterizedType) {
           ParameterizedType parameterizedType = (ParameterizedType) type;
-          if (parameterizedType.getRawType() == ConfigurablePlugin.class) {
+          Type rawType = parameterizedType.getRawType();
+          if (rawType == ConfigurablePlugin.class ||

Review comment:
       Right, I wrestled with this expressions to make it work and forgot to simplify.




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


[GitHub] [lucene-solr] murblanc commented on a change in pull request #2101: SOLR-15016 Replica placement plugins should use container plugins API / configs

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #2101:
URL: https://github.com/apache/lucene-solr/pull/2101#discussion_r534209824



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PlacementPluginFactory.java
##########
@@ -18,14 +18,22 @@
 package org.apache.solr.cluster.placement;
 
 /**
- * Factory implemented by client code and configured in {@code solr.xml} allowing the creation of instances of
+ * Factory implemented by client code and configured in container plugins allowing the creation of instances of
  * {@link PlacementPlugin} to be used for replica placement computation.
+ * <p>Note: configurable factory implementations should also implement
+ * {@link org.apache.solr.api.ConfigurablePlugin} with the appropriate configuration
+ * bean type.</p>
  */
 public interface PlacementPluginFactory {

Review comment:
       Shouldn't this interface extend `ConfigurablePlugin<T extends MapWriter>` so that concrete config classes such as `AffinityPlacementFactory` do not have to implement multiple interfaces?




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


[GitHub] [lucene-solr] murblanc commented on a change in pull request #2101: SOLR-15016 Replica placement plugins should use container plugins API / configs

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #2101:
URL: https://github.com/apache/lucene-solr/pull/2101#discussion_r534219720



##########
File path: solr/core/src/java/org/apache/solr/core/CoreContainer.java
##########
@@ -257,6 +260,7 @@ public CoreLoadFailure(CoreDescriptor cd, Exception loadFailure) {
 
   // initially these are the same to collect the plugin-based listeners during init
   private ClusterEventProducer clusterEventProducer;
+  private PlacementPluginFactory placementPluginFactory;

Review comment:
       I believe we have a synchronization issue here on access to that variable. It is not `final` nor `volatile`, access is not synchronized but it is accessed from multiple threads (the command execution Overseer threads calling `getPlacementPluginFactory()`).




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


[GitHub] [lucene-solr] murblanc commented on a change in pull request #2101: SOLR-15016 Replica placement plugins should use container plugins API / configs

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #2101:
URL: https://github.com/apache/lucene-solr/pull/2101#discussion_r536256783



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PlacementPluginFactory.java
##########
@@ -17,15 +17,39 @@
 
 package org.apache.solr.cluster.placement;
 
+import org.apache.solr.api.ConfigurablePlugin;
+
 /**
- * Factory implemented by client code and configured in {@code solr.xml} allowing the creation of instances of
+ * Factory implemented by client code and configured in container plugins allowing the creation of instances of

Review comment:
       Can we add a reference to how "container plugins" are configured? Not obvious for me and I guess I'm likely not the only one...




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


[GitHub] [lucene-solr] murblanc commented on a change in pull request #2101: SOLR-15016 Replica placement plugins should use container plugins API / configs

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #2101:
URL: https://github.com/apache/lucene-solr/pull/2101#discussion_r535361621



##########
File path: solr/core/src/test/org/apache/solr/cluster/placement/impl/PlacementPluginIntegrationTest.java
##########
@@ -109,4 +132,108 @@ public void testMinimizeCores() throws Exception {
     assertEquals("min cores too low", 1, minCores);
   }
 
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testDynamicReconfiguration() throws Exception {
+    PlacementPluginFactory<? extends MapWriter> pluginFactory = cc.getPlacementPluginFactory();
+    assertTrue("wrong type " + pluginFactory.getClass().getName(), pluginFactory instanceof DelegatingPlacementPluginFactory);
+    DelegatingPlacementPluginFactory wrapper = (DelegatingPlacementPluginFactory) pluginFactory;
+
+    int version = wrapper.getVersion();
+    log.debug("--initial version={}", version);
+
+    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());
+
+    version = waitForVersionChange(version, wrapper, 10);
+
+    assertTrue("wrong version " + version, version > 0);
+    PlacementPluginFactory<? extends MapWriter> factory = wrapper.getDelegate();

Review comment:
       <? extends PlacementPluginConfig>?




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


[GitHub] [lucene-solr] murblanc commented on a change in pull request #2101: SOLR-15016 Replica placement plugins should use container plugins API / configs

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #2101:
URL: https://github.com/apache/lucene-solr/pull/2101#discussion_r536257122



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PlacementPluginFactory.java
##########
@@ -17,15 +17,39 @@
 
 package org.apache.solr.cluster.placement;
 
+import org.apache.solr.api.ConfigurablePlugin;
+
 /**
- * Factory implemented by client code and configured in {@code solr.xml} allowing the creation of instances of
+ * Factory implemented by client code and configured in container plugins allowing the creation of instances of

Review comment:
       (referencing the Affinity class and its configuration would likely be good enough here)




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


[GitHub] [lucene-solr] sigram commented on a change in pull request #2101: SOLR-15016 Replica placement plugins should use container plugins API / configs

Posted by GitBox <gi...@apache.org>.
sigram commented on a change in pull request #2101:
URL: https://github.com/apache/lucene-solr/pull/2101#discussion_r536353469



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PlacementPluginFactory.java
##########
@@ -17,15 +17,39 @@
 
 package org.apache.solr.cluster.placement;
 
+import org.apache.solr.api.ConfigurablePlugin;
+
 /**
- * Factory implemented by client code and configured in {@code solr.xml} allowing the creation of instances of
+ * Factory implemented by client code and configured in container plugins allowing the creation of instances of

Review comment:
       I'll add a reference to `ContainerPluginsApi#Edit` - JSON formats and the location of the config in `/clusterprops.json` are an implementation detail. Also, container plugin configuration is/will be described in the RefGuide (SOLR-15022).




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


[GitHub] [lucene-solr] murblanc commented on a change in pull request #2101: SOLR-15016 Replica placement plugins should use container plugins API / configs

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #2101:
URL: https://github.com/apache/lucene-solr/pull/2101#discussion_r536274851



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/plugins/MinimizeCoresPlacementFactory.java
##########
@@ -40,13 +40,23 @@
  *
  * <p>See {@link AffinityPlacementFactory} for a more realistic example and documentation.</p>
  */
-public class MinimizeCoresPlacementFactory implements PlacementPluginFactory {
+public class MinimizeCoresPlacementFactory implements PlacementPluginFactory<PlacementPluginFactory.NoConfig> {
 
   @Override
-  public PlacementPlugin createPluginInstance(PlacementPluginConfig config) {
+  public PlacementPlugin createPluginInstance() {
     return new MinimizeCoresPlacementPlugin();
   }
 
+  @Override
+  public void configure(NoConfig cfg) {
+    // no-op
+  }
+
+  @Override
+  public NoConfig getConfig() {

Review comment:
       Maybe similarly here, `PlacementPluginFactory` should provide a default implementation returning `NoConfig.INSTANCE` given that out of 4 implementations of the interface, only one returns something else (`AffinityPlacementFactory`).




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


[GitHub] [lucene-solr] murblanc commented on a change in pull request #2101: SOLR-15016 Replica placement plugins should use container plugins API / configs

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #2101:
URL: https://github.com/apache/lucene-solr/pull/2101#discussion_r535346975



##########
File path: solr/core/src/java/org/apache/solr/api/ContainerPluginsRegistry.java
##########
@@ -422,7 +421,10 @@ public static Class getConfigClass(ConfigurablePlugin<?> o) {
       for (Type type : interfaces) {
         if (type instanceof ParameterizedType) {
           ParameterizedType parameterizedType = (ParameterizedType) type;
-          if (parameterizedType.getRawType() == ConfigurablePlugin.class) {
+          Type rawType = parameterizedType.getRawType();
+          if (rawType == ConfigurablePlugin.class ||

Review comment:
       Is it worth keeping `rawType == ConfigurablePlugin.class` in the if condition? It's covered by the second operand of the OR.




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


[GitHub] [lucene-solr] murblanc commented on a change in pull request #2101: SOLR-15016 Replica placement plugins should use container plugins API / configs

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #2101:
URL: https://github.com/apache/lucene-solr/pull/2101#discussion_r535361621



##########
File path: solr/core/src/test/org/apache/solr/cluster/placement/impl/PlacementPluginIntegrationTest.java
##########
@@ -109,4 +132,108 @@ public void testMinimizeCores() throws Exception {
     assertEquals("min cores too low", 1, minCores);
   }
 
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testDynamicReconfiguration() throws Exception {
+    PlacementPluginFactory<? extends MapWriter> pluginFactory = cc.getPlacementPluginFactory();
+    assertTrue("wrong type " + pluginFactory.getClass().getName(), pluginFactory instanceof DelegatingPlacementPluginFactory);
+    DelegatingPlacementPluginFactory wrapper = (DelegatingPlacementPluginFactory) pluginFactory;
+
+    int version = wrapper.getVersion();
+    log.debug("--initial version={}", version);
+
+    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());
+
+    version = waitForVersionChange(version, wrapper, 10);
+
+    assertTrue("wrong version " + version, version > 0);
+    PlacementPluginFactory<? extends MapWriter> factory = wrapper.getDelegate();

Review comment:
       `<? extends PlacementPluginConfig>`?




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


[GitHub] [lucene-solr] sigram commented on a change in pull request #2101: SOLR-15016 Replica placement plugins should use container plugins API / configs

Posted by GitBox <gi...@apache.org>.
sigram commented on a change in pull request #2101:
URL: https://github.com/apache/lucene-solr/pull/2101#discussion_r535404901



##########
File path: solr/core/src/test/org/apache/solr/cluster/placement/impl/PlacementPluginIntegrationTest.java
##########
@@ -109,4 +132,108 @@ public void testMinimizeCores() throws Exception {
     assertEquals("min cores too low", 1, minCores);
   }
 
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testDynamicReconfiguration() throws Exception {
+    PlacementPluginFactory<? extends MapWriter> pluginFactory = cc.getPlacementPluginFactory();
+    assertTrue("wrong type " + pluginFactory.getClass().getName(), pluginFactory instanceof DelegatingPlacementPluginFactory);
+    DelegatingPlacementPluginFactory wrapper = (DelegatingPlacementPluginFactory) pluginFactory;
+
+    int version = wrapper.getVersion();
+    log.debug("--initial version={}", version);
+
+    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());
+
+    version = waitForVersionChange(version, wrapper, 10);
+
+    assertTrue("wrong version " + version, version > 0);
+    PlacementPluginFactory<? extends MapWriter> factory = wrapper.getDelegate();

Review comment:
       Gah .. well-spotted all these. I'll fix them later today.




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