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/07/20 22:12:34 UTC

[GitHub] [lucene-solr] murblanc opened a new pull request #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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


   SOLR-14613: strongly typed initial proposal for plugin interface to replace Autoscaling
   
   This is not meant to be merged, it's to share early for feedback.
   Still WIP, any feedback most welcome before I invest more time.


----------------------------------------------------------------
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] noblepaul edited a comment on pull request #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

Posted by GitBox <gi...@apache.org>.
noblepaul edited a comment on pull request #1684:
URL: https://github.com/apache/lucene-solr/pull/1684#issuecomment-683420028


   Here is an example of how to have strong typing for each atribute without creating  so many different interfaces
   
   `public interface PropertyFetcher {`
     `PropertyFetcher onNode(String nodeName);`
    ` PropertyFetcher withCoreCount();`
     `PropertyFetcher withSystemProperty(String name);`
     
     `PropertyValues fetchValues();`
     
    ` interface PropertyValues {`
       `Optional<Integer> coreCount();`
       `Optional<String> systemProperty(String propertyName);}}`
   `


----------------------------------------------------------------
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] chatman commented on pull request #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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


   > If the plugin requests data node by node, it's either sequential or forces the plugin to implement the concurrency mechanism itself, making it more complicated.
   
   Plugin making concurrent requests is "complicated"? Java is hard, but I can help you write a plugin that can make concurrent requests cleanly. Please don't pollute solr-core because of your perceived inability to do so cleanly from a plugin.


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementPluginFactoryFacade.java
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.cluster.placement.PlacementPlugin;
+import org.apache.solr.cluster.placement.PlacementPluginConfig;
+import org.apache.solr.cluster.placement.PlacementPluginFactory;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.core.PluginInfo;
+import org.apache.solr.core.SolrResourceLoader;
+
+/**
+ * <p>The internal class instantiating the configured {@link PlacementPluginFactory} and creating a {@link PlacementPlugin}
+ * instance by passing the the factory the appropriate configuration created from the {@code <placementPluginFactory>}
+ * element in {@code solr.xml}.
+ *
+ * <p>A single instance of {@link PlacementPlugin} is used for all placement computations and therefore must be reentrant.
+ * When configuration changes, a new instance of {@link PlacementPlugin} will be created by calling again
+ * {@link PlacementPluginFactory#createPluginInstance(PlacementPluginConfig)}.
+ */
+public class PlacementPluginFactoryFacade {

Review comment:
       Oh, indeed - sorry for the confusion.




----------------------------------------------------------------
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] chatman commented on pull request #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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


   > Gumi means “elastic” in Hebrew. I’ll change that package name to... what?
   I see that @noblepaul has used the name "o.a.s.cloud.sdk" in the other PR for SolrCloud specific interfaces. One day, I would hope all our internal code should use these interfaces, and having "sdk" then, IMHO, wouldn't be ideal. How about "o.a.s.cloud.interfaces" for that?
   
   Regarding other interfaces introduced here, how about "o.a.s.cloud.placements.interfaces" or "o.a.s.cloud.assignments.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 pull request #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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


   @sigram @noblepaul I have removed the `PropertyValue` concept. Now once `PropertyKeys` are fetched, the value is accessed directly on the key (no more returned map). This simplifies plugin code and removes `instanceof` and cast operations related to keys in the plugins.
   
   I started (just started) work on the more realistic plugin `SamplePluginAffinityReplicaPlacement` (did describe in the Javadoc how I intend to approach it), and as I see which methods I need there, I'm updating the API with anything that would likely make sense for anybody writing another plugin.


----------------------------------------------------------------
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] noblepaul commented on pull request #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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


   I think we should we commit the interfaces and one sample impl together


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/plugins/SamplePluginMinimizeCores.java
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.Comparator;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Set;
+import java.util.Map;
+
+import com.google.common.collect.Ordering;
+import com.google.common.collect.TreeMultimap;
+import org.apache.solr.cluster.placement.Cluster;
+import org.apache.solr.cluster.placement.CoresCountPropertyValue;
+import org.apache.solr.cluster.placement.CreateNewCollectionPlacementRequest;
+import org.apache.solr.cluster.placement.Node;
+import org.apache.solr.cluster.placement.PlacementException;
+import org.apache.solr.cluster.placement.PlacementPlugin;
+import org.apache.solr.cluster.placement.PropertyKey;
+import org.apache.solr.cluster.placement.PropertyKeyFactory;
+import org.apache.solr.cluster.placement.PropertyValue;
+import org.apache.solr.cluster.placement.PropertyValueFetcher;
+import org.apache.solr.cluster.placement.Replica;
+import org.apache.solr.cluster.placement.ReplicaPlacement;
+import org.apache.solr.cluster.placement.PlacementRequest;
+import org.apache.solr.cluster.placement.PlacementPlan;
+import org.apache.solr.cluster.placement.PlacementPlanFactory;
+import org.apache.solr.common.util.SuppressForbidden;
+
+/**
+ * Implements placing replicas to minimize number of cores per {@link Node}, while not placing two replicas of the same
+ * shard on the same node.
+ *
+ * TODO: code not tested and never run, there are no implementation yet for used interfaces
+ */
+public class SamplePluginMinimizeCores implements PlacementPlugin {
+
+  @SuppressForbidden(reason = "Ordering.arbitrary() has no equivalent in Comparator class. Rather reuse than copy.")
+  public PlacementPlan computePlacement(Cluster cluster, PlacementRequest placementRequest, PropertyKeyFactory propertyFactory,
+                                        PropertyValueFetcher propertyFetcher, PlacementPlanFactory placementPlanFactory) throws PlacementException {
+    // This plugin only supports Creating a collection.
+    if (!(placementRequest instanceof CreateNewCollectionPlacementRequest)) {
+      throw new PlacementException("This toy plugin only supports creating collections");
+    }
+
+    final CreateNewCollectionPlacementRequest reqCreateCollection = (CreateNewCollectionPlacementRequest) placementRequest;
+
+    final int totalReplicasPerShard = reqCreateCollection.getNrtReplicationFactor() +
+        reqCreateCollection.getTlogReplicationFactor() + reqCreateCollection.getPullReplicationFactor();
+
+    if (cluster.getLiveNodes().size() < totalReplicasPerShard) {
+      throw new PlacementException("Cluster size too small for number of replicas per shard");
+    }
+
+    // Get number of cores on each Node
+    TreeMultimap<Integer, Node> nodesByCores = TreeMultimap.create(Comparator.naturalOrder(), Ordering.arbitrary());
+
+    // Get the number of cores on each node and sort the nodes by increasing number of cores
+    for (Node node : cluster.getLiveNodes()) {
+      // TODO: redo this. It is potentially less efficient to call propertyFetcher.getProperties() multiple times rather than once
+      final PropertyKey coresCountPropertyKey = propertyFactory.createCoreCountKey(node);
+      Map<PropertyKey, PropertyValue> propMap = propertyFetcher.fetchProperties(Collections.singleton(coresCountPropertyKey));
+      PropertyValue returnedValue = propMap.get(coresCountPropertyKey);
+      if (returnedValue == null) {
+        throw new PlacementException("Can't get number of cores in " + node);

Review comment:
       Yes. This is a PoC implementation, not a real one. Used it to guide me in building the 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 #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/gumi/SolrCollection.java
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.cloud.gumi;
+
+import java.util.Set;
+
+/**
+ * Represents a Collection in SolrCloud. Although naming this class "Collection" is possible it would be confusing.
+ */
+public interface SolrCollection {

Review comment:
       Same as above for `Replica`, here the class is even more complex.
   For example we've been having discussions on getting rid of the term "Slice" and standardizing on "Shard". If we expose `DocCollection` in its current state, `Slice` can't be renamed.
   
   Also, if we want to build a simulation framework later for plugin developers, that layer will have a clean and easy interface to build on, since that's all the plugins are seeing. Building it will not require reaching into Solr code in various places and add hooks for the simulator. And removing it will be a lot easier 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] sigram commented on a change in pull request #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PlacementPlugin.java
##########
@@ -0,0 +1,41 @@
+/*
+ * 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;
+
+/**
+ * Implemented by external plugins to control replica placement and movement on the search cluster (as well as other things
+ * such as cluster elasticity?) when cluster changes are required (initiated elsewhere, most likely following a Collection
+ * API call).
+ */
+public interface PlacementPlugin {

Review comment:
       Configurable factory + throwaway stateless request processor make sense. This pattern is used in many other places in Solr and Lucene.




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/CreateNewCollectionRequest.java
##########
@@ -0,0 +1,62 @@
+/*
+ * 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;
+
+import java.util.Set;
+
+/**
+ * <p>Request for creating a new collection with a given set of shards and replication factor for various replica types.
+ * The expected {@link WorkOrder} corresponding to this {@link Request} is created using
+ * {@link WorkOrderFactory#createWorkOrderNewCollection}
+ *
+ * <p>Note there is no need at this stage to allow the plugin to know each shard hash range for example, this can be handled
+ * by the Solr side implementation of this interface without needing the plugin to worry about it (the implementation of this interface on
+ * the Solr side can maintain the ranges for each shard).
+ *
+ * <p>Same goes for the {@link org.apache.solr.core.ConfigSet} name or other collection parameters. They are needed for
+ * creating a Collection but likely do not have to be exposed to the plugin (this can easily be changed if needed by
+ * adding accessors here, the underlying Solr side implementation of this interface has the information).
+ */
+public interface CreateNewCollectionRequest extends Request {
+  /**
+   * <p>The name of the collection to be created and for which placement should be computed.
+   *
+   * <p>Compare this method with {@link AddReplicasRequest#getCollection()}, there the collection already exists so can be
+   * directly passed in the {@link Request}.
+   *
+   * <p>When processing this request, plugin code doesn't have to worry about existing {@link Replica}'s for the collection
+   * given that the collection is assumed not to exist.
+   */
+  String getCollectionName();
+
+  Set<String> getShardNames();
+
+  /**
+   * <p>Properties passed through the Collection API by the client creating the collection.
+   * See {@link SolrCollection#getCustomProperty(String)}.
+   *
+   * <p>Given this {@link Request} is for creating a new collection, it is not possible to pass the custom property values through
+   * the {@link SolrCollection} object. That instance does not exist yet, and is the reason {@link #getCollectionName()} exists
+   * rather than a method returning {@link SolrCollection}...
+   */
+  String getCustomProperty(String customPropertyName);

Review comment:
       Ok.

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/AddReplicasRequest.java
##########
@@ -0,0 +1,62 @@
+/*
+ * 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;
+
+import java.util.Set;
+
+/**
+ * <p>Request for creating one or more {@link Replica}'s for one or more {@link Shard}'s of an existing {@link SolrCollection}.
+ * The shard might or might not already exist, plugin code can easily find out by using {@link SolrCollection#getShards()}
+ * and verifying if the shard name(s) from {@link #getShardNames()} are there.
+ *
+ * <p>As opposed to {@link CreateNewCollectionRequest}, the set of {@link Node}s on which the replicas should be placed
+ * is specified (defaults to being equal to the set returned by {@link Cluster#getLiveNodes()}).
+ *
+ * <p>There is no extension between this interface and {@link CreateNewCollectionRequest} in either direction
+ * or from a common ancestor for readability. An ancestor could make sense and would be an "abstract interface" not intended
+ * to be implemented directly, but this does not exist in Java.
+ *
+ * <p>Plugin code would likely treat the two types of requests differently since here existing {@link Replica}'s must be taken
+ * into account for placement whereas in {@link CreateNewCollectionRequest} no {@link Replica}'s are assumed to exist.
+ */
+public interface AddReplicasRequest extends Request {
+  /**
+   * The {@link SolrCollection} to add {@link Replica}(s) to. The replicas are to be added to a shard that might or might
+   * not yet exist when the plugin's {@link PlacementPlugin#computePlacement} is called.
+   */
+  SolrCollection getCollection();
+
+  /**
+   * <p>Shard name(s) for which new replicas placement should be computed. The shard(s) might exist or not (that's why this
+   * method returns a {@link Set} of {@link String}'s and not directly a set of {@link Shard} instances).
+   *
+   * <p>Note the Collection API allows specifying the shard name or a {@code _route_} parameter. The Solr implementation will
+   * convert either specification into the relevant shard name so the plugin code doesn't have to worry about this.
+   */
+  Set<String> getShardNames();
+
+  /** Replicas should only be placed on nodes from the set returned by this method. */
+  Set<Node> getTargetNodes();

Review comment:
       Ok, but if we specify the semantics of this method more precisely then it will be a better guide to the implementors. As it is now it's not clear what to return when the target set is the whole cluster.

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/ReplicaPlacement.java
##########
@@ -0,0 +1,29 @@
+/*
+ * 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;
+
+/**
+ * <p>Placement decision for a single {@link Replica}. Note this placement decision is used as part of a {@link WorkOrder},
+ * it does not directly lead to the plugin code getting a corresponding {@link Replica} instance, nor does it require the
+ * plugin to provide a {@link Shard} instance (the plugin code gets such instances for existing replicas and shards in the
+ * cluster but does not create them directly for adding new replicas for new or existing shards).
+ *
+ * <p>Captures the {@link Shard} (via the shard name), {@link Node} and {@link Replica.ReplicaType} of a Replica to be created.
+ */
+public interface ReplicaPlacement {

Review comment:
       It's a common problem today (or at least in 8x) to actually understand why the policy made the placement decisions it did. Imagine having to investigate it without having a handy reference to the request that caused these decisions.
   
   So the purpose of this reference is not for the plugin, it's for the user to be able to track why a particular placement decision was made. We could leave this to the plugins to implement it but I think it's an important aspect of auditing and debugging and we should enforce it on plugins using the interfaces.
   
   Even if a plugin just randomly assigns placements there is always a cause for the assignments to be generated. Whether we include the whole Request or just an id, I think we should explcitly require this information to be tracked.

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/Cluster.java
##########
@@ -0,0 +1,46 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * <p>A representation of the (initial) cluster state, providing information on which nodes are part of the cluster and a way
+ * to get to more detailed info.
+ *
+ * <p>This instance can also be used as a {@link PropertyValueSource} if {@link PropertyKey}'s need to be specified with
+ * a global cluster target.
+ */
+public interface Cluster extends PropertyValueSource {
+  /**
+   * @return current set of live nodes. Never <code>null</code>, never empty (Solr wouldn't call the plugin if empty
+   * since no useful could then be done).
+   */
+  Set<Node> getLiveNodes();
+
+  /**
+   * <p>Returns info about the given collection if one exists. Because it is not expected for plugins to request info about
+   * a large number of collections, requests can only be made one by one.
+   *
+   * <p>This is also the reason we do not return a {@link java.util.Map} or {@link Set} of {@link SolrCollection}'s here: it would be
+   * wasteful to fetch all data and fill such a map when plugin code likely needs info about at most one or two collections.
+   */
+  Optional<SolrCollection> getCollection(String collectionName) throws IOException;

Review comment:
       Right, I thought of `withCollection`. But there are other cases where we may want to avoid placing replicas on nodes with replicas from other collections. Eg. at Lucidworks we heavily use several predefined system collections for logging and analytics jobs, and we want to avoid putting search collections on those nodes if possible. This is currently achieved by totally separating the nodes using exclusive policy properties, but a different softer mechanism could be better on resource-constrained clusters - but then you need to know the placements of other collections...

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PropertyKeyFactory.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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;
+
+/**
+ * Factory used by the plugin to create property keys to request property values from Solr.<p>
+ *
+ * Building of a {@link PropertyKey} requires specifying the target (context) from which the value of that key should be
+ * obtained. This is done by specifying the appropriate {@link PropertyValueSource}.<br>
+ * For clarity, when only a single type of target is acceptable, the corresponding subtype of {@link PropertyValueSource} is used instead
+ * (for example {@link Node}).
+ */
+public interface PropertyKeyFactory {
+  /**
+   * Returns a property key to request the number of cores on a {@link Node}.
+   */
+  PropertyKey createCoreCountKey(Node node);
+
+  /**
+   * Returns a property key to request disk related info on a {@link Node}.
+   */
+  PropertyKey createDiskInfoKey(Node node);
+
+  /**
+   * Returns a property key to request the value of a system property on a {@link Node}.
+   * @param systemPropertyName the name of the system property to retrieve.
+   */
+  PropertyKey createSystemPropertyKey(Node node, String systemPropertyName);
+
+  /**
+   * Returns a property key to request the value of a metric.<p>
+   *
+   * Not all metrics make sense everywhere, but metrics can be applied to different objects. For example
+   * <code>SEARCHER.searcher.indexCommitSize</code> would make sense for a given replica of a given shard of a given collection,
+   * and possibly in other contexts.<p>
+   *
+   * @param metricSource The registry of the metric. For example a specific {@link Replica}.
+   * @param metricName for example <code>SEARCHER.searcher.indexCommitSize</code>.
+   */
+  PropertyKey createMetricKey(PropertyValueSource metricSource, String metricName);

Review comment:
       .. and yes, the metrics in `solr.jvm` are a totally different set of metrics from those in `solr.node`.




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/gumi/CreateCollectionRequest.java
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.cloud.gumi;
+
+/**
+ * <p>Request for creating a new collection with a given number of shards and replication factor for various replica types.</p>
+ *
+ * Note there is no need at this stage to allow the request to convey each shard hash range for example, this can be handled
+ * by the Solr side implementation without needing the plugin to worry about it.
+ *
+ */
+public interface CreateCollectionRequest extends Request {
+  String getCollectionName();
+
+  int getShardCount();

Review comment:
       Shard index can likely go away altogether and we only keep shard names (+ matching routing info).
   
   I'm trying to understand if naming the shards should be the responsibility of the Autoscaling plugin or rather of Solr (would prefer the plugin to always deal with preexisting shard names, makes things simpler).
   If it's the responsibility of Solr, when the API collection create call does not specify names, Solr first creates the corresponding names and then calls the plugin.
   
   I'm interested in comments for or against this approach.




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PlacementPlugin.java
##########
@@ -0,0 +1,41 @@
+/*
+ * 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;
+
+/**
+ * Implemented by external plugins to control replica placement and movement on the search cluster (as well as other things
+ * such as cluster elasticity?) when cluster changes are required (initiated elsewhere, most likely following a Collection
+ * API call).
+ */
+public interface PlacementPlugin {

Review comment:
       I believe Solr side would keep the configuration (map) somewhere and just pass a reference to it on every call to the plugin. So cost would be minimal. If the plugin code needs to retrieve a config value, it will basically do a `get` on that map, very low cost here as well.
   I don't see how a plugin can keep state between invocations with a semantic that's different from a static field, unless we create a notion of higher level context shared between invocations (and different such higher level contexts will not be shared). I suggest we skip that aspect for now and reconsider later when we implement real plugins. For now I believe the model in which the plugin is instantiated (no arg constructor, or constructor getting the config map or something simple like that) before every call to the computePlacement method would be sufficient. 




----------------------------------------------------------------
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] noblepaul commented on pull request #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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


   Pleas refer to #1694 as a separate effort to make this simpler
   
   I would prefer not to give names like `gumi` anywhere. Let's use simple english terms like `Assign` , `AssignStrategy` and the package name can be `assign`


----------------------------------------------------------------
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] chatman commented on pull request #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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


   > Wow. I'm not even sure that would be polite to say to a junior engineer (that may be presumed to not know concurrency) than a very senior one here. I think you know full well that Ilan doesn't need your help writing concurrent code. Yet why would you offer help? I think this is a false offer (you know Ilan will not take you up on this); -- it's a pejorative offer that belittles who you offer it to as being beneath the abilities of your own. I wonder why you would say that? Maybe you don't believe it's worth it to make it easier for plugin writers at the expense of Solr having such concurrent code (i.e. where does the concurrency/complexity go) -- fare, but then just say that without belittling anyone's intelligence.
   
   Ilan is a reasonably new committer, so I am not fully aware of his background or his "seniority". I just offered help in writing concurrent code that I consider clean. But, now you have made couple of assertions that I will keep in mind:
   "Ilan doesn't need your help writing concurrent code"
   "Ilan will not take you up on this"
   
   
   > "pejorative offer that belittles who you offer it"
   > "as being beneath the abilities of your own"
   > "belittling [..] intelligence"
   
   Even though I won't stop you representing/misrepresenting Ilan as you have done above (by claiming he is not open to help in this area), at least please don't put misrepresent me with the motives behind my words. Just to clarify: I thought Ilan was inclined towards having tons of classes (and thereby "polluting solr-core") because he wanted to avoid having the plugins make concurrent requests, since I thought he perceived doing so as complicated. I am of course not doubting anyone's ability to write concurrent code (and of course not "belittling anyone's intelligence"), but I wanted to offer suggestions on doing so *cleanly* (without complications).


----------------------------------------------------------------
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] noblepaul commented on pull request #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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


   The consistent theme I see here is 
   
   A. Here's is my elegant solution to the problem
   
   B. I think the solution is 
   
   


----------------------------------------------------------------
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] dsmiley commented on pull request #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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


   I think use of solr.xml for this is pragmatic/realistic with the Solr we have today.  And it's flexible -- can go on the node or in ZK as you desire, and can hold structured information (not just a bag of name-value pairs).  This matter can be improved in the future.
   
   Some of the other discussion seems to be rooted in a trade-off: assuming there's an efficient plugin (minimally communicates with nodes to get data to make decisions), where does the complexity go relating to concurrency of node communication or knowledge of what nodes to even talk to?  If this is in Solr, it can be independently optimized of plugins and makes an autoscaling plugin simpler to write _well_ (because it doesn't need to concern itself with many efficiencies).  If we leave the complexity to a plugin writer, Solr is leaner.  I don't think this is related to the problems of the previous autoscaling code we got rid of -- I don't think that either path will lead to eventual removal of what's added.
   
   A third matter is more a matter of taste on wether it's better/worse to have typed properties (not just in the primitive nature but it's identity / semantic).  Trade-offs.  I don't like an explosion of classes but I think this is highly mitigated by them being defined as simple inner classes of one outer class, as Ilan has done.


----------------------------------------------------------------
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] chatman edited a comment on pull request #1684: SOLR-14613: strongly typed placement plugin interface and implementation

Posted by GitBox <gi...@apache.org>.
chatman edited a comment on pull request #1684:
URL: https://github.com/apache/lucene-solr/pull/1684#issuecomment-686808345


   > Wow. I'm not even sure that would be polite to say to a junior engineer (that may be presumed to not know concurrency) than a very senior one here. 
   
   I don't understand your ideas of politeness, but if someone offered me help/advice in writing code in any area, no matter how "senior" or "junior" I am or he/she who is offering help is, I would be grateful.
   
   > I think you know full well that Ilan doesn't need your help writing concurrent code. Yet why would you offer help? I think this is a false offer (you know Ilan will not take you up on this); -- it's a pejorative offer that belittles who you offer it to as being beneath the abilities of your own. I wonder why you would say that? Maybe you don't believe it's worth it to make it easier for plugin writers at the expense of Solr having such concurrent code (i.e. where does the concurrency/complexity go) -- fare, but then just say that without belittling anyone's intelligence.
   
   Ilan is a reasonably new committer, so I am not fully aware of his background or his "seniority". I just offered help in writing concurrent code that I consider clean. But, now you have made couple of assertions that I will keep in mind:
   "Ilan doesn't need your help writing concurrent code"
   "Ilan will not take you up on this"
   
   
   > "pejorative offer that belittles who you offer it"
   > "as being beneath the abilities of your own"
   > "belittling [..] intelligence"
   
   Even though I won't stop you representing/misrepresenting Ilan as you have done above (by claiming he is not open to help in this area), at least please don't put misrepresent me with the motives behind my words. Just to clarify: I thought Ilan was inclined towards having tons of classes (and thereby "polluting solr-core") because he wanted to avoid having the plugins make concurrent requests, since I thought he perceived doing so as complicated. I am of course not doubting anyone's ability to write concurrent code (and of course not "belittling anyone's intelligence"), but I wanted to offer suggestions on doing so *cleanly* (without complications).


----------------------------------------------------------------
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] noblepaul edited a comment on pull request #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

Posted by GitBox <gi...@apache.org>.
noblepaul edited a comment on pull request #1684:
URL: https://github.com/apache/lucene-solr/pull/1684#issuecomment-683488170


   >There's no other place to set default config (not hard code!) for which placement plugin to use 
   
   What is wrong with hard coding?
   
   >t's easy to have a single PropertyValue interface with all possible getter methods and have all but one return an empty optional. 
   
   Easy ? The whole code looks pretty bad with a million properties and factories. This looks like we are building a J2EE project. Let's make life simple


----------------------------------------------------------------
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] chatman removed a comment on pull request #1684: SOLR-14613: strongly typed placement plugin interface and implementation

Posted by GitBox <gi...@apache.org>.
chatman removed a comment on pull request #1684:
URL: https://github.com/apache/lucene-solr/pull/1684#issuecomment-686320915


   > Yeah, the solution to API surface area problem is to make them inner classes/interfaces. This is such a elegant & simple proposition. We should try the same in other places in Solr where there is a surface area problem.
   
   Totally disagree, @noblepaul. I hope you're not serious here. Having getter methods like getFreeDiskSpace() etc. are better than having a FreeDiskProperty class, no matter it is inner or outer. Imagine the 100s (possibly 1000s) of different metrics/properties that would end up as inner classes with such a design!


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/AddReplicasRequest.java
##########
@@ -0,0 +1,62 @@
+/*
+ * 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;
+
+import java.util.Set;
+
+/**
+ * <p>Request for creating one or more {@link Replica}'s for one or more {@link Shard}'s of an existing {@link SolrCollection}.
+ * The shard might or might not already exist, plugin code can easily find out by using {@link SolrCollection#getShards()}
+ * and verifying if the shard name(s) from {@link #getShardNames()} are there.
+ *
+ * <p>As opposed to {@link CreateNewCollectionRequest}, the set of {@link Node}s on which the replicas should be placed
+ * is specified (defaults to being equal to the set returned by {@link Cluster#getLiveNodes()}).
+ *
+ * <p>There is no extension between this interface and {@link CreateNewCollectionRequest} in either direction
+ * or from a common ancestor for readability. An ancestor could make sense and would be an "abstract interface" not intended
+ * to be implemented directly, but this does not exist in Java.
+ *
+ * <p>Plugin code would likely treat the two types of requests differently since here existing {@link Replica}'s must be taken
+ * into account for placement whereas in {@link CreateNewCollectionRequest} no {@link Replica}'s are assumed to exist.
+ */
+public interface AddReplicasRequest extends Request {
+  /**
+   * The {@link SolrCollection} to add {@link Replica}(s) to. The replicas are to be added to a shard that might or might
+   * not yet exist when the plugin's {@link PlacementPlugin#computePlacement} is called.
+   */
+  SolrCollection getCollection();
+
+  /**
+   * <p>Shard name(s) for which new replicas placement should be computed. The shard(s) might exist or not (that's why this
+   * method returns a {@link Set} of {@link String}'s and not directly a set of {@link Shard} instances).
+   *
+   * <p>Note the Collection API allows specifying the shard name or a {@code _route_} parameter. The Solr implementation will
+   * convert either specification into the relevant shard name so the plugin code doesn't have to worry about this.
+   */
+  Set<String> getShardNames();
+
+  /** Replicas should only be placed on nodes from the set returned by this method. */
+  Set<Node> getTargetNodes();

Review comment:
       And also `Request` -> `PlacementRequest`




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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


   @noblepaul & @chatman I find the tone of your latest comments offensive - that's no way to build a consensus. Please think twice before posting and calm down - if you have a different opinion about technical merits of this PR then I'm sure you can express it without personal attacks.
   
   Hiding the complexity of multi-node interactions from plugin implementations is a worthy goal IMO - and not because we don't know how to write multi-threaded code, but because most devs know how to do it poorly ;) If we can hide this complexity from the plugin writers then we increase the chance for the plugins to work well and efficiently.
   
   > I could as well create a full framework and give a PR. 
   
   By all means, if you disagree so strongly with the approach presented here then please do so - just be sure that you actually will do it instead of just complaining. I'll gladly review it and I will gladly collaborate with you if it moves us forward toward the same goal - which is a strongly-typed API for placement plugins :)
   
   Please also remember that we are on a deadline - we need to have some kind of replacement for the autoscaling in 9.0. The impl. can be barebones, but the API that we define is going to stay with us for the next couple years at least, so it pays off to think seriously how to do it in an elegant and efficient way.


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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


   With the last commit just now, the API code is functionally complete for a basic assign replicas use case except config and getting properties.
   Didn't test anything yet though (still on PTO technically, working off a 4MB RAM very very entry level Windows laptop)


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PropertyKeyFactory.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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;
+
+/**
+ * Factory used by the plugin to create property keys to request property values from Solr.<p>
+ *
+ * Building of a {@link PropertyKey} requires specifying the target (context) from which the value of that key should be
+ * obtained. This is done by specifying the appropriate {@link PropertyValueSource}.<br>
+ * For clarity, when only a single type of target is acceptable, the corresponding subtype of {@link PropertyValueSource} is used instead
+ * (for example {@link Node}).
+ */
+public interface PropertyKeyFactory {
+  /**
+   * Returns a property key to request the number of cores on a {@link Node}.
+   */
+  PropertyKey createCoreCountKey(Node node);
+
+  /**
+   * Returns a property key to request disk related info on a {@link Node}.
+   */
+  PropertyKey createDiskInfoKey(Node node);
+
+  /**
+   * Returns a property key to request the value of a system property on a {@link Node}.
+   * @param systemPropertyName the name of the system property to retrieve.
+   */
+  PropertyKey createSystemPropertyKey(Node node, String systemPropertyName);
+
+  /**
+   * Returns a property key to request the value of a metric.<p>
+   *
+   * Not all metrics make sense everywhere, but metrics can be applied to different objects. For example
+   * <code>SEARCHER.searcher.indexCommitSize</code> would make sense for a given replica of a given shard of a given collection,
+   * and possibly in other contexts.<p>
+   *
+   * @param metricSource The registry of the metric. For example a specific {@link Replica}.
+   * @param metricName for example <code>SEARCHER.searcher.indexCommitSize</code>.
+   */
+  PropertyKey createMetricKey(PropertyValueSource metricSource, String metricName);

Review comment:
       One node usually hosts many replicas. Each of these replicas has a unique registry name, in the form of `solr.core.<replicaName>`, so we could build PropertyKey from Replica because all components of the full metrics name are known.
   
   This is not the case with `node`, `jvm` and `jetty` - I think we need to explicitly specify the registry name in these cases.
   
   (Edit: or implement a PropertyValueSource that is a facade for registry name, to keep the API here consistent)




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/ReplicaPlacement.java
##########
@@ -0,0 +1,29 @@
+/*
+ * 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;
+
+/**
+ * <p>Placement decision for a single {@link Replica}. Note this placement decision is used as part of a {@link WorkOrder},
+ * it does not directly lead to the plugin code getting a corresponding {@link Replica} instance, nor does it require the
+ * plugin to provide a {@link Shard} instance (the plugin code gets such instances for existing replicas and shards in the
+ * cluster but does not create them directly for adding new replicas for new or existing shards).
+ *
+ * <p>Captures the {@link Shard} (via the shard name), {@link Node} and {@link Replica.ReplicaType} of a Replica to be created.
+ */
+public interface ReplicaPlacement {

Review comment:
       Ok, added. Note that the interfaces in `org.apache.solr.cluster.placement` only condition what the plugin code is seeing, not what's present in the objects. The `Request` is present in the `WorkOrder` instances and I added it to the interface, but even if it's not in the interface, Solr side code would have it.
   My thinking (by not putting it in the interface) is that plugin code is creating these instances so it should know which instance is which and doesn't need a method on that instance to find out.




----------------------------------------------------------------
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] chatman commented on pull request #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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


   > Please also remember that we are on a deadline - we need to have some kind of replacement for the autoscaling in 9.0.
   
   +1, this is the cause for maximum urgency at the moment.
   
   We need to:
   1. design the interfaces, 
   2. design the proper user interaction layer (DSL?), 
   3. build efficient implementations. 
   
   We cannot afford to be stuck at step 1. However, building a frankenstein framework as part of 1 is not conducive for overall goals of making/keeping Solr lean and clean.


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PlacementPlugin.java
##########
@@ -0,0 +1,41 @@
+/*
+ * 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;
+
+/**
+ * Implemented by external plugins to control replica placement and movement on the search cluster (as well as other things
+ * such as cluster elasticity?) when cluster changes are required (initiated elsewhere, most likely following a Collection
+ * API call).
+ */
+public interface PlacementPlugin {

Review comment:
       I believe we should let the plug-in manage this type of requirements rather than try to control it by the timing of when configs are passed. If there are licences to check the plug-in should cache the result and only confirm each time? 




----------------------------------------------------------------
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 edited a comment on pull request #1684: SOLR-14613: strongly typed initial proposal for plugin interface

Posted by GitBox <gi...@apache.org>.
murblanc edited a comment on pull request #1684:
URL: https://github.com/apache/lucene-solr/pull/1684#issuecomment-664046112


   There’s no direct mapping between work orders and collection API. I consider work orders the basic building blocs from which collection API actions can be built. Creating a collection (adding the data in ZK, checking for the config set etc) is different from adding a replica to an existing collection. Hence the two work orders. The “new collection” doesn’t add any replicas to the collection but prepares the system to accept adding replicas replicas for the collection. Obviously when adding a replica to an existing collection, that preparation work is not required.


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/gumi/Replica.java
##########
@@ -0,0 +1,34 @@
+/*
+ * 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.cloud.gumi;
+
+/**
+ * An instantiation (or one of the copies) of a given {@link Shard} of a given {@link SolrCollection}.
+ * Objects of this type are returned by the Solr framework to the plugin, they are not built by the plugin. When the
+ * plugin wants to add a replica it goes through {@link WorkOrderFactory#createWorkOrderCreateReplica}).
+ * TODO is there an elegant way to have this type also used by the plugin to add replicas? (insisting on elegant)
+ */
+public interface Replica {
+  Shard getShard();
+
+  ReplicaType getType();

Review comment:
       We probably need ReplicaState here too, otherwise it will be difficult to determine when movements are needed.

##########
File path: solr/core/src/java/org/apache/solr/cloud/gumi/PropertyKeyFactory.java
##########
@@ -0,0 +1,34 @@
+/*
+ * 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.cloud.gumi;
+
+/**
+ * Factory used by the plugin to create property keys to request property values from Solr
+ */
+public interface PropertyKeyFactory {
+  /**
+   * Returns a property key allowing to request the number of cores. There are no parameters for this key.
+   */
+  CoresCountPropertyKey createCoreCountKey();
+
+  /**
+   * Returns a property key allowing to request the value of a system property (sorry property used twice in two different
+   * contexts). The parameter is the name of the system property to retrieve.
+   */
+  SystemPropertyPropertyKey createSystemPropertyKey(String systemPropertyName);

Review comment:
       We also need several node-level metrics, so maybe add a MetricsPropertyKey?

##########
File path: solr/core/src/java/org/apache/solr/cloud/gumi/CreateCollectionRequest.java
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.cloud.gumi;
+
+/**
+ * <p>Request for creating a new collection with a given number of shards and replication factor for various replica types.</p>
+ *
+ * Note there is no need at this stage to allow the request to convey each shard hash range for example, this can be handled
+ * by the Solr side implementation without needing the plugin to worry about it.
+ *
+ */
+public interface CreateCollectionRequest extends Request {
+  String getCollectionName();
+
+  int getShardCount();

Review comment:
       Users must be able to request specific shard names, not just the count, eg. when implicit router is used. Other parameters are needed too, such as the router parameters, non-default shard hash ranges etc. See CollectionAdminRequest for details.

##########
File path: solr/core/src/java/org/apache/solr/cloud/gumi/Shard.java
##########
@@ -0,0 +1,32 @@
+/*
+ * 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.cloud.gumi;
+
+import java.util.Set;
+
+/**
+ * Shard in a {@link SolrCollection}, i.e. a subset of the data indexed in that collection.
+ */
+public interface Shard {
+  /**
+   * 0 numbered index of the {@link Shard} in the {@link SolrCollection}.
+   */
+  int getShardIndex();

Review comment:
       We really need a name here - shards can have arbitrary names.




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PlacementPlanFactory.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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;
+
+import java.util.Set;
+
+/**
+ * Allows plugins to create {@link PlacementPlan}s telling the Solr layer where to create replicas following the processing of
+ * a {@link PlacementRequest}. The Solr layer can (and will) check that the {@link PlacementPlan} conforms to the {@link PlacementRequest} (and
+ * if it does not, the requested operation will fail).
+ */
+public interface PlacementPlanFactory {
+  /**
+   * <p>Creates a {@link PlacementPlan} for adding a new collection and its replicas.
+   *
+   * <p>This is in support of {@link org.apache.solr.cloud.api.collections.CreateCollectionCmd}.
+   */
+  PlacementPlan createPlacementPlanNewCollection(CreateNewCollectionPlacementRequest request, String CollectionName, Set<ReplicaPlacement> replicaPlacements);
+
+  /**
+   * <p>Creates a {@link PlacementPlan} for adding replicas to a given shard of an existing collection.
+   *
+   * <p>This is in support (directly or indirectly) of {@link org.apache.solr.cloud.api.collections.AddReplicaCmd},
+   * {@link org.apache.solr.cloud.api.collections.CreateShardCmd}, {@link org.apache.solr.cloud.api.collections.ReplaceNodeCmd},
+   * {@link org.apache.solr.cloud.api.collections.MoveReplicaCmd}, {@link org.apache.solr.cloud.api.collections.SplitShardCmd},
+   * {@link org.apache.solr.cloud.api.collections.RestoreCmd} and {@link org.apache.solr.cloud.api.collections.MigrateCmd}.
+   * (as well as of {@link org.apache.solr.cloud.api.collections.CreateCollectionCmd} in the specific case of
+   * {@link org.apache.solr.common.params.CollectionAdminParams#WITH_COLLECTION} but this should be removed shortly and
+   * the section in parentheses of this comment should be removed when the {@code withCollection} javadoc link appears broken).
+   */
+  PlacementPlan createPlacementPlanAddReplicas(AddReplicasPlacementRequest request, String CollectionName, Set<ReplicaPlacement> replicaPlacements);

Review comment:
       That was my initial intention but then I switched to the factory approach. We can expose all accessors in PlacementPlan and let the plug-in build instances of it as it wishes. Maybe we should expose all accessors anyway, but I thought rather than letting each plug-in implement these interfaces again it's simpler to provide a factory. Also I saw no real added value from the plug-in perspective to have its own instance type. 
   A provided factory allows to introduce constraints on acceptable parameters in a programmer friendly way (what a method accepts, different method names), if Solr side accepts any instance it would have to do some checks at runtime.




----------------------------------------------------------------
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] HoustonPutman commented on pull request #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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


   I completely agree with David and Andrzej on all points. No one is going through the comments with a fine tooth comb, it is blatantly disrespectful. And doubling down instead of acknowledging and apologizing makes it even worse. 
   
   To your point Noble, Apache's motto is "Community over Code". There is no reason to put up with rudeness because someone graces us with a PR review. It is easier to review a PR and be dismissive and rude, but it is infinitely healthier and more constructive to be empathetic and kind. It also leads to a community that is more willing to contribute and collaborate. It's reasonable to expect mutual respect within the Lucene/Solr community. If we are in the place where we should accept any type of language when someone graces us with a review, then that is something we need to seriously address.


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PlacementPlanFactory.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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;
+
+import java.util.Set;
+
+/**
+ * Allows plugins to create {@link PlacementPlan}s telling the Solr layer where to create replicas following the processing of
+ * a {@link PlacementRequest}. The Solr layer can (and will) check that the {@link PlacementPlan} conforms to the {@link PlacementRequest} (and
+ * if it does not, the requested operation will fail).
+ */
+public interface PlacementPlanFactory {
+  /**
+   * <p>Creates a {@link PlacementPlan} for adding a new collection and its replicas.
+   *
+   * <p>This is in support of {@link org.apache.solr.cloud.api.collections.CreateCollectionCmd}.
+   */
+  PlacementPlan createPlacementPlanNewCollection(CreateNewCollectionPlacementRequest request, String CollectionName, Set<ReplicaPlacement> replicaPlacements);
+
+  /**
+   * <p>Creates a {@link PlacementPlan} for adding replicas to a given shard of an existing collection.
+   *
+   * <p>This is in support (directly or indirectly) of {@link org.apache.solr.cloud.api.collections.AddReplicaCmd},
+   * {@link org.apache.solr.cloud.api.collections.CreateShardCmd}, {@link org.apache.solr.cloud.api.collections.ReplaceNodeCmd},
+   * {@link org.apache.solr.cloud.api.collections.MoveReplicaCmd}, {@link org.apache.solr.cloud.api.collections.SplitShardCmd},
+   * {@link org.apache.solr.cloud.api.collections.RestoreCmd} and {@link org.apache.solr.cloud.api.collections.MigrateCmd}.
+   * (as well as of {@link org.apache.solr.cloud.api.collections.CreateCollectionCmd} in the specific case of
+   * {@link org.apache.solr.common.params.CollectionAdminParams#WITH_COLLECTION} but this should be removed shortly and
+   * the section in parentheses of this comment should be removed when the {@code withCollection} javadoc link appears broken).
+   */
+  PlacementPlan createPlacementPlanAddReplicas(AddReplicasPlacementRequest request, String CollectionName, Set<ReplicaPlacement> replicaPlacements);

Review comment:
       Is the move replica command picking up the destination or is the destination specified in the API call? If the latter, there will be no call to the placement plugin.
   And if the former, the fact that no files are to be moved is relatively transparent to the plugin. The plugin doesn’t do any work but just tells Solr where to put things. Solr code would then either create or move depending on what command it was executing.
   The only difference could be move placement computation (if there’s one) should take into account the lower load on the source node (since replicas will be moved off of 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] chatman commented on pull request #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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


   > @noblepaul & @chatman I find the tone of your latest comments offensive - that's no way to build a consensus. Please think twice before posting and calm down - if you have a different opinion about technical merits of this PR then I'm sure you can express it without personal attacks.
   
   I don't see how Noble's comments can construed as offensive. I may be biased in favour of my own comments, but I apologise if they were perceived as such. In any case, there is no personal attack anywhere.
   
   > By all means, if you disagree so strongly with the approach presented here then please do so - just be sure that you actually will do it instead of just complaining.
   
   I find choice of such words (" instead of just complaining") as unprofessional. This is a proposal, and comments are added to critique the design, not complain.
   
   On the other hand, Ilan wrote this on Slack:
   
   > If there’s consensus for Noble’s approach (or for that matter no consensus that goals 1-3 above are good guiding principles), I will stop work on SOLR-14613 and move on to other unrelated topics.
   
   Such threats of "stop work" unless one's design is agreed upon should cease, and constructive ways to collaborate should be explored.


----------------------------------------------------------------
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] chatman commented on pull request #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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


   > You need to make a realistic proposal here IMO @noblepaul.
   > And we need something that already exists. If placement plugins depend on some
   > future rehaul of configuration in SolrCloud, we're pushing this way out because
   > from past discussion there doesn't seem to be a consensus on how/if to 
   > remove solr.xml and how to support its features differently.
   
   Noble has made an instructive example of how to deal with configuration properly (using clusterprops.json, not solr.xml): https://github.com/apache/lucene-solr/pull/1813
   How much more hand holding do you need in order to do the right thing? If you want solr.xml to go away completely before using clusterprops, then let us wait until that is done. Using solr.xml (per node config) for a cluster wide replica assignment policy is not acceptable. The reasons have been stated many times, but I'm happy to do one more round and explain to you as simply or in as much detail as you would like.
   
   
   > I think use of solr.xml for this is pragmatic/realistic with the Solr we have today.
   
   This is absolutely untrue. Pragmatic/realistic is a joke. Using solr.xml means new configuration cannot take effect unless that file is hand-edited and node is restarted. Even bigger problem, in context of replica assignment/placement, is the possibility of different nodes having different configurations for the placement plugins, and hence cause cluster wide inconsistency.
   
   > And it's flexible -- can go on the node or in ZK as you desire, and can hold structured
   > information (not just a bag of name-value pairs). This matter can be improved in the future.
   
   Oh, my goodness! Such thinking is the reason why our project is in a total mess today. Putting solr.xml in ZK is a hack, which we don't need to carry forward any longer. Even though I have expressed my reluctant agreement to carry on with solr.xml for now and switch to clusterprops.json before release, such flawed reasoning in favour of using solr.xml pushes me towards -1'ing this issue on this basis.
   
   > the other one [requirement] was efficiency: being able to fetch ALL needed properties
   > from a remote Node at once 
   
   Efficiency is a matter of implementation, and not a characteristic of the interfaces.
   
   Lets look at two approaches where the plugin requests for 10 properties.
   
   Design 1 (your design): Interface contains 10 different classes for the 10 properties, and solr-core returns 10 different objects for those properties to the plugin.
   Design 2 (proper design): Interface contains 1 class containing (at least) 10 getter methods for those properties (all typed as Optional). Solr-core returns that 1 object containing those 10 optional values filled in.
   
   In terms of interface design, design 2 is clearly more efficient because it requires the creation of 10 less objects.
   Lets look at efficiency from implementation point of view.
   
   In the design 2, solr-core (on a remote node) can populate all properties in the same object (even if it needs 10 multi-threaded calls). If there are multiple nodes from where those 10 properties need to be fetched, the plugin can make n multi-threaded calls to n remote nodes and receive n objects (each containing those 10 properties that are needed).
   
   How is any of this inefficient?


----------------------------------------------------------------
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] chatman commented on pull request #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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


   Regarding the work orders, can we decouple them from the collection API commands that are used? I see CreateReplicaWorkOrder and NewCollectionWorkOrder. AFAICT, the former needs replica placement for one replica, and the latter needs placement for potentially multiple replicas. Why can't we have a generic interface for providing placements that are agnostic of the collection API used? If we don't, we'll end up with more such work orders like move replica, split shard, restore, rebalance, utilize etc. etc., which would be large surface area to deal with.


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/Cluster.java
##########
@@ -0,0 +1,46 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * <p>A representation of the (initial) cluster state, providing information on which nodes are part of the cluster and a way
+ * to get to more detailed info.
+ *
+ * <p>This instance can also be used as a {@link PropertyValueSource} if {@link PropertyKey}'s need to be specified with
+ * a global cluster target.
+ */
+public interface Cluster extends PropertyValueSource {
+  /**
+   * @return current set of live nodes. Never <code>null</code>, never empty (Solr wouldn't call the plugin if empty
+   * since no useful could then be done).
+   */
+  Set<Node> getLiveNodes();
+
+  /**
+   * <p>Returns info about the given collection if one exists. Because it is not expected for plugins to request info about
+   * a large number of collections, requests can only be made one by one.
+   *
+   * <p>This is also the reason we do not return a {@link java.util.Map} or {@link Set} of {@link SolrCollection}'s here: it would be
+   * wasteful to fetch all data and fill such a map when plugin code likely needs info about at most one or two collections.
+   */
+  Optional<SolrCollection> getCollection(String collectionName) throws IOException;

Review comment:
       The new framework does not mandate any specific way to do this. The scenario you describe (partitioning nodes by system properties) is totally possible to implement, then one would be using `[PropertyValue.Sysprop](https://github.com/apache/lucene-solr/pull/1684/files#diff-d4ace968d26586330e5b20a0b698e9a7R125)` to implement that.




----------------------------------------------------------------
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] chatman commented on pull request #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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


   What does "gumi" mean?


----------------------------------------------------------------
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 edited a comment on pull request #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

Posted by GitBox <gi...@apache.org>.
murblanc edited a comment on pull request #1684:
URL: https://github.com/apache/lucene-solr/pull/1684#issuecomment-683433584


   > Here is an example of how to have strong typing for each atribute without creating so many different interfaces
   
   It's easy to have a single `PropertyValue` interface with all possible getter methods and have all but one return an empty optional. I don't see how that's better.
   
   Also I don't want to assume the plugin code knows which node a request should be sent to (i.e. passing the node to `onNode()`). It does know the node for most requests but not all. When requesting data about a given shard or replica for example (say number of docs, or index size), it is not the role of the plugin to find out which node that replica is on to route the request there. I prefer the behind the scenes implementation to do that.
   
   [Edit: also, a single fetch call from plugin to fetch all data on all nodes rather than node by node allows request optimization on the implementation side. The most obvious example is using multiple concurrent messages to the different nodes (multithreading or in any other way). If the plugin requests data node by node, it's either sequential or forces the plugin to implement the concurrency mechanism itself, making it more complicated.
   There are even more ambitious optimizations that can be made on the implementation side if we get all the data to distribute right away, by implementing for example a hierarchical distribution, to reduce for example the number messages of messages that have to cross Availability Zones.]


----------------------------------------------------------------
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] chatman edited a comment on pull request #1684: SOLR-14613: strongly typed placement plugin interface and implementation

Posted by GitBox <gi...@apache.org>.
chatman edited a comment on pull request #1684:
URL: https://github.com/apache/lucene-solr/pull/1684#issuecomment-686320915


   > Yeah, the solution to API surface area problem is to make them inner classes/interfaces. This is such a elegant & simple proposition. We should try the same in other places in Solr where there is a surface area problem.
   
   Totally disagree, @noblepaul. I hope you're not serious here. Having getter methods like getFreeDiskSpace() etc. are better than having a FreeDiskProperty class, no matter it is inner or outer. Imagine the 100s (possibly 1000s) of different metrics/properties that would end up as inner classes with such a design!


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PlacementPlanFactory.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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;
+
+import java.util.Set;
+
+/**
+ * Allows plugins to create {@link PlacementPlan}s telling the Solr layer where to create replicas following the processing of
+ * a {@link PlacementRequest}. The Solr layer can (and will) check that the {@link PlacementPlan} conforms to the {@link PlacementRequest} (and
+ * if it does not, the requested operation will fail).
+ */
+public interface PlacementPlanFactory {
+  /**
+   * <p>Creates a {@link PlacementPlan} for adding a new collection and its replicas.
+   *
+   * <p>This is in support of {@link org.apache.solr.cloud.api.collections.CreateCollectionCmd}.
+   */
+  PlacementPlan createPlacementPlanNewCollection(CreateNewCollectionPlacementRequest request, String CollectionName, Set<ReplicaPlacement> replicaPlacements);
+
+  /**
+   * <p>Creates a {@link PlacementPlan} for adding replicas to a given shard of an existing collection.
+   *
+   * <p>This is in support (directly or indirectly) of {@link org.apache.solr.cloud.api.collections.AddReplicaCmd},
+   * {@link org.apache.solr.cloud.api.collections.CreateShardCmd}, {@link org.apache.solr.cloud.api.collections.ReplaceNodeCmd},
+   * {@link org.apache.solr.cloud.api.collections.MoveReplicaCmd}, {@link org.apache.solr.cloud.api.collections.SplitShardCmd},
+   * {@link org.apache.solr.cloud.api.collections.RestoreCmd} and {@link org.apache.solr.cloud.api.collections.MigrateCmd}.
+   * (as well as of {@link org.apache.solr.cloud.api.collections.CreateCollectionCmd} in the specific case of
+   * {@link org.apache.solr.common.params.CollectionAdminParams#WITH_COLLECTION} but this should be removed shortly and
+   * the section in parentheses of this comment should be removed when the {@code withCollection} javadoc link appears broken).
+   */
+  PlacementPlan createPlacementPlanAddReplicas(AddReplicasPlacementRequest request, String CollectionName, Set<ReplicaPlacement> replicaPlacements);
+
+  /**
+   * Creates a {@link ReplicaPlacement} needed to be passed to some/all {@link PlacementPlan} factory methods.
+   */
+  ReplicaPlacement createReplicaPlacement(String shardName, Node node, Replica.ReplicaType replicaType);

Review comment:
       That’s the way for the plugin to build the replica placements it has decided in order to pass the set to the appropriate PlacementPlanFactory method.




----------------------------------------------------------------
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] noblepaul commented on pull request #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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


   > don't like an explosion of classes but I think this is highly mitigated by them being defined as simple inner classes of one outer class, as Ilan has done.
   
   Yeah, the solution to API surface area problem is to make them inner classes/interfaces. This is such a elegant & simple proposition. We should try the same in other places in Solr where there is a surface area problem. 


----------------------------------------------------------------
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] noblepaul commented on pull request #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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


   I'm a bit busy too. We will compare notes later


----------------------------------------------------------------
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 edited a comment on pull request #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

Posted by GitBox <gi...@apache.org>.
murblanc edited a comment on pull request #1684:
URL: https://github.com/apache/lucene-solr/pull/1684#issuecomment-683433584


   > Here is an example of how to have strong typing for each atribute without creating so many different interfaces
   
   It's easy to have a single `PropertyValue` interface with all possible getter methods and have all but one return an empty optional. I don't see how that's better.
   
   Also I don't want to assume the plugin code knows which node a request should be sent to (i.e. passing the node to `onNode()`). It does know the node for most requests but not all. When requesting data about a given shard or replica for example (say number of docs, or index size), it is not the role of the plugin to find out which node that replica is on to route the request there. I prefer the behind the scenes implementation to do that.
   
   [Edit: also, a single fetch call from plugin to fetch all data on all nodes rather than node by node allows request optimization on the implementation side. The most obvious example is using multiple concurrent messages to the different nodes (multithreading or in any other way). If the plugin requests data node by node, it's either sequential or forces the plugin to implement the concurrency mechanism itself, making it more complicated.
   There are even more ambitious optimizations that can be made on the implementation side if we get all the data to distribute right away, by implementing for example a hierarchical distribution, to reduce the number of messages that have to cross Availability Zones.]


----------------------------------------------------------------
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] vthacker commented on pull request #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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


   > The fact is, all of has the same objective: to make the product better.
   
   > he purpose is to ensure that the feature/change is
   
   >    correct
   >    performant/efficient
   >    user-friendly
   
   I think everyone agrees on this. I really wish we can be better while giving feedback being nicer. We'd be able to collaborate better and keep the focus on the design decisions
   
   What are the current concerns with the current PR?
   1. The verbosity?
   2. Where the config lives?
   
   
   


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/AddReplicasRequest.java
##########
@@ -0,0 +1,62 @@
+/*
+ * 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;
+
+import java.util.Set;
+
+/**
+ * <p>Request for creating one or more {@link Replica}'s for one or more {@link Shard}'s of an existing {@link SolrCollection}.
+ * The shard might or might not already exist, plugin code can easily find out by using {@link SolrCollection#getShards()}
+ * and verifying if the shard name(s) from {@link #getShardNames()} are there.
+ *
+ * <p>As opposed to {@link CreateNewCollectionRequest}, the set of {@link Node}s on which the replicas should be placed
+ * is specified (defaults to being equal to the set returned by {@link Cluster#getLiveNodes()}).
+ *
+ * <p>There is no extension between this interface and {@link CreateNewCollectionRequest} in either direction
+ * or from a common ancestor for readability. An ancestor could make sense and would be an "abstract interface" not intended
+ * to be implemented directly, but this does not exist in Java.
+ *
+ * <p>Plugin code would likely treat the two types of requests differently since here existing {@link Replica}'s must be taken
+ * into account for placement whereas in {@link CreateNewCollectionRequest} no {@link Replica}'s are assumed to exist.
+ */
+public interface AddReplicasRequest extends Request {
+  /**
+   * The {@link SolrCollection} to add {@link Replica}(s) to. The replicas are to be added to a shard that might or might
+   * not yet exist when the plugin's {@link PlacementPlugin#computePlacement} is called.
+   */
+  SolrCollection getCollection();
+
+  /**
+   * <p>Shard name(s) for which new replicas placement should be computed. The shard(s) might exist or not (that's why this
+   * method returns a {@link Set} of {@link String}'s and not directly a set of {@link Shard} instances).
+   *
+   * <p>Note the Collection API allows specifying the shard name or a {@code _route_} parameter. The Solr implementation will
+   * convert either specification into the relevant shard name so the plugin code doesn't have to worry about this.
+   */
+  Set<String> getShardNames();
+
+  /** Replicas should only be placed on nodes from the set returned by this method. */
+  Set<Node> getTargetNodes();

Review comment:
       This will be implemented once (by me for example) and only consumed by the plugins. But I will clarify.
   The `Request`s are received by the plugins. The plugins send back `WorkOrder`s (and create them through a factory interface so don't have to worry about implementation anyway).
   If you can think of better names for these abstractions, I'm interested (esp work order doesn't feel totally right).




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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


   I have implemented the cluster state abstractions and added some (naive and temporary) wiring to select this assign strategy.
   A lot of missing parts, this can't be merged anymore at this stage. Work In Progress.


----------------------------------------------------------------
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] chatman commented on pull request #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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


   I withdraw all outstanding concerns. Verbosity, clunkiness of configuration
   etc are all my "perceptions" that I don't want to come in the way of the
   completion of this effort.
   
   >
   


----------------------------------------------------------------
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] noblepaul commented on pull request #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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


   I see a consistent theme here. We are always combing through comments with a lens to see if the tone is correct, if somebody uses the correct case in typing or if the feedback is warm and fuzzy and caring.
   
   While it's a good in general, we miss the whole point of a review and feedback. The purpose is to ensure that the feature/change is 
   
   - correct
   - performant/efficient
   - user-friendly
   
   The person who is doing a review is already doing a great favour by spending time to do a review. Let's be aware that the PR owner is probably paid to do that work and the reviewer is not. So, I have deep respect for anyone who reviews my PRs and gives suggestions (@dsmiley ) you are one of the devs who review a lot of my PRs and I'm grateful for what you do and you have the nicest demeanour while doing it. There are others like Uwe or Muir, who occasionally  gives me a feedback without being very kind. I'm grateful to them because they have spent time to go through my PR using their personal time. They also bring a wealth of expertise that I probably don't have.
   
   The fact is, all of has the same objective: to make the product better. We are all not paid by the same employer or some of us may not even be paid at all. Let's focus more on the content and less on the form. 
   
   cheers.


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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


   > On the other hand, Ilan wrote this on Slack:
   > 
   > > If there’s consensus for Noble’s approach (or for that matter no consensus that goals 1-3 above are good guiding principles), I will stop work on SOLR-14613 and move on to other unrelated topics.
   > 
   > Such threats of "stop work" unless one's design is agreed upon should cease, and constructive ways to collaborate should be explored.
   
   @chatman I likely didn't express myself clearly enough: all (all!) my work in this PR is guided by these 3 principles. If we don't agree on those, my work doesn't make sense and is indeed too verbose and complex for no reason!
   
   I did not ask anybody to agree with my design, but to agree with the goals.
   Implementing directly to the `Assign` interface is what we did at Salesforce to run 8x (too broken otherwise) and what we can and possibly will do to run SolrCloud in production. Goals 1-3 (copied below from Slack for reference) are to make this type of dev easier by contributors that might not have the expertise or time to code directly to existing Solr abstractions.
   
   > **1. placement plugin writing is easy**, **2. implementation for this placement API is efficient**, and **3. plugins do not break when Solr is refactored** (plugins might be client private so can’t be refactored when Solr code changes).


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/CreateNewCollectionRequest.java
##########
@@ -0,0 +1,62 @@
+/*
+ * 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;
+
+import java.util.Set;
+
+/**
+ * <p>Request for creating a new collection with a given set of shards and replication factor for various replica types.
+ * The expected {@link WorkOrder} corresponding to this {@link Request} is created using
+ * {@link WorkOrderFactory#createWorkOrderNewCollection}
+ *
+ * <p>Note there is no need at this stage to allow the plugin to know each shard hash range for example, this can be handled
+ * by the Solr side implementation of this interface without needing the plugin to worry about it (the implementation of this interface on
+ * the Solr side can maintain the ranges for each shard).
+ *
+ * <p>Same goes for the {@link org.apache.solr.core.ConfigSet} name or other collection parameters. They are needed for
+ * creating a Collection but likely do not have to be exposed to the plugin (this can easily be changed if needed by
+ * adding accessors here, the underlying Solr side implementation of this interface has the information).
+ */
+public interface CreateNewCollectionRequest extends Request {
+  /**
+   * <p>The name of the collection to be created and for which placement should be computed.
+   *
+   * <p>Compare this method with {@link AddReplicasRequest#getCollection()}, there the collection already exists so can be
+   * directly passed in the {@link Request}.
+   *
+   * <p>When processing this request, plugin code doesn't have to worry about existing {@link Replica}'s for the collection
+   * given that the collection is assumed not to exist.
+   */
+  String getCollectionName();
+
+  Set<String> getShardNames();
+
+  /**
+   * <p>Properties passed through the Collection API by the client creating the collection.
+   * See {@link SolrCollection#getCustomProperty(String)}.
+   *
+   * <p>Given this {@link Request} is for creating a new collection, it is not possible to pass the custom property values through
+   * the {@link SolrCollection} object. That instance does not exist yet, and is the reason {@link #getCollectionName()} exists
+   * rather than a method returning {@link SolrCollection}...
+   */
+  String getCustomProperty(String customPropertyName);

Review comment:
       I can add such an enumeration (but then would skip non `String` properties, or just `toString()` everything) but unclear to me how a plugin would be basing placement decisions on properties it doesn't know about.
   Indeed the general idea is that the plugin does not do the calls and does not need to access all the information from the Collection API call. It is called for placement computation, the code on the Solr side knows everything the Collection API call has provided and will handle the CREATE command (to Overseer) like it does 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


[GitHub] [lucene-solr] murblanc commented on pull request #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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


   And please DO NOT DELETE the murblanc:SOLR-14613-gumi branch, I need it around for a while.


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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


   > Let us take a step back and reassess at what we have today.
   ...
   > There is already an interface called `AssignStrategy` .Can you believe that it's as simple as
   ...
   > My suggestion is to not build anything new . Just use this interface
   
   You certainly noted in the PR the placement plugins are already doing what you suggest here: just using this interface.
   Like `LEGACY`, `RULES` (and in 8x `POLICY`) are. This interface is not touched by this PR.
   
   The placement plugins code should be compared (if we want comparisons) to the removed code of Autoscaling.


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PlacementException.java
##########
@@ -0,0 +1,48 @@
+/*
+ * 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;
+
+/**
+ * Exception thrown by a {@link PlacementPlugin} when it is unable to compute placement for whatever reason (except an

Review comment:
       Skipping on this one for now. I believe we need to better identify what problems plugins will be running into (that are not Solr side exceptions bubbling up) and how the Solr side implementation could use such a reason before being able to make useful design decisions.
   If you do have specific use cases in mind, please let me know.




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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


   FYI I stop work on this PR until we better define its scope.


----------------------------------------------------------------
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] noblepaul edited a comment on pull request #1684: SOLR-14613: strongly typed placement plugin interface and implementation

Posted by GitBox <gi...@apache.org>.
noblepaul edited a comment on pull request #1684:
URL: https://github.com/apache/lucene-solr/pull/1684#issuecomment-686359748


   > By all means, if you disagree so strongly with the approach presented here then please do so
   
   I clearly have objections to the verbosity of the proposals here. This does not mean that what I propose will be perfect. Nobody can design "the perfect system". A perfect system does not exist. I'm trying to give suggestions so that we can eliminate those obvious problems.
   
   At this point, two of you are invested in this feature and I totally understand your compulsions. 
   
   > The impl. can be barebones, but the API that we define is going to stay with us for the next couple years at least, so it pays off to think seriously how to do it in an elegant and efficient way.
   
   What I see is a largescale undertaking to rewrite everything that we have in Solr . How else will you justify the presence of a class such as this https://github.com/apache/lucene-solr/blob/e385e7083bd4e0263e563d35555cf0cb53b0e03e/solr/core/src/java/org/apache/solr/cluster/placement/Replica.java
   in  a framework that is designed to just find out nodes to place replicas in a cluster? This seems too grandiose
   
   We do not want to support such a massive framework . 
   I strongly recommend that we do not make duplicates for what we have today just for this particular framework. We already have an `AssignStrategy` interface. AFAIK, the only place where we have problem is in the `NodeStateProvider` . Let's add a method `NodeStateProvider#getPropertyFetcher()` and a new interfaces `PropertyFetcher`, `PropertyValues`  and start reusing the `AssignStrategy` interface itself. That way we do not need any new framework.  Most likely there won't be too many implementations of this framework. Now, if you wish me to make a PR with this suggestion, I'm happy to do that
   
   Just in case you wish to load this plugin from packages , My PR (#1684 ) has the necessary code to load this prom package. (That's optional at this point)
   


----------------------------------------------------------------
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] noblepaul edited a comment on pull request #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

Posted by GitBox <gi...@apache.org>.
noblepaul edited a comment on pull request #1684:
URL: https://github.com/apache/lucene-solr/pull/1684#issuecomment-683771821


   >I’d be happy to change the current proposal for something easier to use (not pretending it’s perfect) but I’m not willing to sacrifice on performance in order to do so.
   
   I totally disagree that my proposal will be not performant. It has zero impact on performance whatsoever. The current policy framework fetches multiple values in a single request. The only issue I see with that is "it's not strongly typed". It's a matter of aesthetics at this point. if you ask me should I add 20 new interfaces or have "strongly typed" I would say let's have weak typing. If you are gonna add 2-3 neat interfaces and are able to introduce strong typing , I would say it's worth 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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PlacementPlugin.java
##########
@@ -0,0 +1,41 @@
+/*
+ * 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;
+
+/**
+ * Implemented by external plugins to control replica placement and movement on the search cluster (as well as other things
+ * such as cluster elasticity?) when cluster changes are required (initiated elsewhere, most likely following a Collection
+ * API call).
+ */
+public interface PlacementPlugin {

Review comment:
       So what do you suggest here and how would the plug-in code deal with it? And how different it would be from the plug-in storing/caching whatever processing it has done in static variables?
   
   I thought about this and we can introduce a concept of "Config session" during which which the config doesn't change.
   The plug-in would implement such a newConfigSession method that would accept a config and return an instance of the compute plug-in. That instance would then be called by the framework to do computation. When the framework sees a change of config, it would call again newConfigSession. By using the appropriate classes without storing references in funny places, garbage collection would clean up old configs...
   
   Just having a Solr callback into the existing plug-in passing config is not elegant IMO: forces to synchronize access to config (JMM) and to deal with first compute request coming before first config + race conditions between the two. 
   




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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


   > So, please feel free to ignore #1714 . You can use #1694 in your framework, if you like it. If not, you go ahead .
   
   Thanks @noblepaul! It's still a bit hard for me to navigate who does what and how here (here being open source).
   
   I'll carry forward with this PR using for now the cluster abstractions introduced here. When [#1694](https://github.com/apache/lucene-solr/pull/1694) is done we can reconsider (we'll also have a better idea of the needs of this PR).
   
   I hope to commit it this week (only interfaces, no implementation) to then start work on the actual implementation (but I'll be mostly away for two weeks at the end of this week).


----------------------------------------------------------------
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] chatman commented on pull request #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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


   > Yeah, the solution to API surface area problem is to make them inner classes/interfaces. This is such a elegant & simple proposition. We should try the same in other places in Solr where there is a surface area problem.
   
   Totally disagree, @noblepaul. I hope you're not serious here. Having getter methods like getFreeDiskSpace() etc. are better than having a FreeDiskProperty class, no matter it is inner or outer.


----------------------------------------------------------------
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] noblepaul edited a comment on pull request #1684: SOLR-14613: strongly typed placement plugin interface and implementation

Posted by GitBox <gi...@apache.org>.
noblepaul edited a comment on pull request #1684:
URL: https://github.com/apache/lucene-solr/pull/1684#issuecomment-686359748


   > By all means, if you disagree so strongly with the approach presented here then please do so
   
   I clearly have objections to the verbosity of the proposals here. This does not mean that what I propose will be perfect. Nobody can design "the perfect system". A perfect system does not exist. I'm trying to give suggestions so that we can eliminate those obvious problems.
   
   At this point, two of you are invested in this feature and I totally understand your compulsions. 
   
   > The impl. can be barebones, but the API that we define is going to stay with us for the next couple years at least, so it pays off to think seriously how to do it in an elegant and efficient way.
   
   What I see is a largescale undertaking to rewrite everything that we have in Solr . How else will you justify the presence of a class such as this https://github.com/apache/lucene-solr/blob/e385e7083bd4e0263e563d35555cf0cb53b0e03e/solr/core/src/java/org/apache/solr/cluster/placement/Replica.java
   in  a framework that is designed to just find out nodes to place replicas in a cluster? This seems too grandiose
   
   We do not want to support such a massive framework . 
   I strongly recommend that we do not make duplicates for what we have today just for this particular framework. We already have an `AssignStrategy` interface. AFAIK, the only place where we have problem is in the `NodeStateProvider` . Let's add a method `NodeStateProvider#getPropertyFetcher()` and a new interface `NodePropertyFetcher`  and start reusing the `AssignStrategy` interface itself. That way we do not need any new framework.  Most likely there won't be too many implementations of this framework. Now, if you wish me to make a PR with this suggestion, I'm happy to do that
   
   Just in case you wish to load this plugin from packages , My PR (#1684 ) has the necessary code to load this prom package. (That's optional at this point)
   


----------------------------------------------------------------
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] chatman edited a comment on pull request #1684: SOLR-14613: strongly typed placement plugin interface and implementation

Posted by GitBox <gi...@apache.org>.
chatman edited a comment on pull request #1684:
URL: https://github.com/apache/lucene-solr/pull/1684#issuecomment-686355702


   > @noblepaul & @chatman I find the tone of your latest comments offensive - that's no way to build a consensus. Please think twice before posting and calm down - if you have a different opinion about technical merits of this PR then I'm sure you can express it without personal attacks.
   
   I don't see how Noble's comments can construed as offensive. I may be biased in favour of my own comments, but I *apologise* if they were perceived as such. In any case, there is no personal attack anywhere.
   
   > By all means, if you disagree so strongly with the approach presented here then please do so - just be sure that you actually will do it instead of just complaining.
   
   I find choice of such words (" instead of just complaining") as unprofessional. This is a proposal, and comments are added to critique the design, not complain.
   
   On the other hand, Ilan wrote this on Slack:
   
   > If there’s consensus for Noble’s approach (or for that matter no consensus that goals 1-3 above are good guiding principles), I will stop work on SOLR-14613 and move on to other unrelated topics.
   
   Such threats of "stop work" unless one's design is agreed upon should cease, and constructive ways to collaborate should be explored.


----------------------------------------------------------------
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] noblepaul commented on pull request #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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


   > Here's an example @noblepaul of how your proposal would be used (and if I misunderstood something, please correct). nodes are the cluster nodes of interest:
   
   
   I'm not writing full PRs here.The objective was to give you hints as to how to do things differently .
   
    With a little more imagination, the code would look as follows. 
   
   The following could
   
   - make requests in multiple thread
   - minimal code 
   
   ```
       Iterable<PropertyValues> values =  someFactory.createPropertyFetcher()
       .withSystemProperty("AvailabilityZone")
      .withCoresCount()
   .fetchFromAllNodes()
   ```
   or
   
   ```
       PropertyValues values =  someFactory.createPropertyFetcher()
       .withSystemProperty("AvailabilityZone")
      .withCoresCount()
   .fetchFromNode("nodeName")
   ```
   The common theme I see here is instead of trying to understand the spirit of suggestions, you are trying to pick holes in the suggestions. I could as well create a full framework and give a PR. Then you would say that "I do not want to work on somebody else's design". 


----------------------------------------------------------------
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] noblepaul commented on pull request #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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


   > the way forward should be to maybe use Solr.XML only
   for now to unblock you, but revisit it later to leverage clusterprops
   before the release. @noble, is that okay for you?
   
   NO


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PlacementPlugin.java
##########
@@ -0,0 +1,41 @@
+/*
+ * 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;
+
+/**
+ * Implemented by external plugins to control replica placement and movement on the search cluster (as well as other things
+ * such as cluster elasticity?) when cluster changes are required (initiated elsewhere, most likely following a Collection
+ * API call).
+ */
+public interface PlacementPlugin {

Review comment:
       I see a few options for passing config to the plugin:
   
   - Pass the config map in the `computePlacement` call along with the rest of the parameters
   - Provide in one of the passed factories the ability of the plugin code to call back into Solr and get the config
   - Add a `configure` call on the plugin as you suggest (then the Solr infra decides when to call this method)
   - Pass the config when the plugin class is instantiated. This might be equivalent to passing it in the `computePlacement` method if a new plugin class instance is created for each new computation.
   
   So the real choice is do we create a new plugin class instance per placement computation or reuse a given one? Creating a new one for each call is likely a simpler programming model for the plugin developer, it can use class member variables freely and if it wants to keep some state it has to make it static...
   
   With a new instance per computation, there would be no notion of "configuration update". A separate call into the plugin as you suggest or passing the config in `computePlacement` is equivalent (and the latter likely easier to handle in plugin code).
   
   I'm tempted to pass the config with each call to `computePlacement` (assuming the saving of not passing it when the plugin doesn't need it are non measurable). I'm also tempted to make it `Map<String, String>` given it will (most likely?) come from XML and the plugin code would have to deal with what the config means anyway and what types to cast it to...




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/AddReplicasPlacementRequest.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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;
+
+import java.util.Set;
+
+/**
+ * <p>Request passed by Solr to a {@link PlacementPlugin} to compute placement for one or more {@link Replica}'s for one
+ * or more {@link Shard}'s of an existing {@link SolrCollection}.
+ * The shard might or might not already exist, plugin code can easily find out by using {@link SolrCollection#getShards()}
+ * and verifying if the shard name(s) from {@link #getShardNames()} are there.
+ *
+ * <p>As opposed to {@link CreateNewCollectionPlacementRequest}, the set of {@link Node}s on which the replicas should be placed
+ * is specified (defaults to being equal to the set returned by {@link Cluster#getLiveNodes()}).
+ *
+ * <p>There is no extension between this interface and {@link CreateNewCollectionPlacementRequest} in either direction
+ * or from a common ancestor for readability. An ancestor could make sense and would be an "abstract interface" not intended
+ * to be implemented directly, but this does not exist in Java.
+ *
+ * <p>Plugin code would likely treat the two types of requests differently since here existing {@link Replica}'s must be taken
+ * into account for placement whereas in {@link CreateNewCollectionPlacementRequest} no {@link Replica}'s are assumed to exist.
+ */
+public interface AddReplicasPlacementRequest extends PlacementRequest {
+  /**
+   * The {@link SolrCollection} to add {@link Replica}(s) to. The replicas are to be added to a shard that might or might
+   * not yet exist when the plugin's {@link PlacementPlugin#computePlacement} is called.
+   */
+  SolrCollection getCollection();

Review comment:
       What's the value of returning the name rather than the object itself? If the plug-in requests info on a collection it's to access that info I assume, so returning an object saves an additional step of "get Collection From Name" or similar call in plugin code. 




----------------------------------------------------------------
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 edited a comment on pull request #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

Posted by GitBox <gi...@apache.org>.
murblanc edited a comment on pull request #1684:
URL: https://github.com/apache/lucene-solr/pull/1684#issuecomment-683433584


   > Here is an example of how to have strong typing for each atribute without creating so many different interfaces
   
   It's easy to have a single `PropertyValue` interface with all possible getter methods and have all but one return an empty optional. I don't see how that's better.
   
   Also I don't want to assume the plugin code knows which node a request should be sent to (i.e. passing the node to `onNode()`). It does know the node for most requests but not all. When requesting data about a given shard or replica for example (say number of docs, or index size), it is not the role of the plugin to find out which node that replica is on to route the request there. I prefer the behind the scenes implementation to do that.
   
   [Edit: also, a single fetch call from plugin to fetch all data on all nodes rather than node by node allows request optimization on the implementation side with the most obvious example is using multiple concurrent messages to the different nodes (multithreading or in any other way). If the plugin requests data node by node, it's either sequential or forces the plugin to implement the concurrency mechanism itself, making it more complicated.
   There are even more ambitious optimizations that can be made on the implementation side if we get all the data to distribute right away, bu implementing for example a hierarchical distribution, to reduce for example the number messages of messages that have to cross Availability Zones.]


----------------------------------------------------------------
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] noblepaul commented on pull request #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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


   >I’d be happy to change the current proposal for something easier to use (not pretending it’s perfect) but I’m not willing to sacrifice on performance in order to do so.
   
   I totally disagree that my proposal will be not performant. It has zero impact on performance whatsoever


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/Cluster.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * <p>A representation of the (initial) cluster state, providing information on which nodes are part of the cluster and a way
+ * to get to more detailed info.
+ *
+ * <p>This instance can also be used as a {@link PropertyValueSource} if {@link PropertyKey}'s need to be specified with
+ * a global cluster target.
+ */
+public interface Cluster extends PropertyValueSource {
+  /**
+   * @return current set of live nodes. Never <code>null</code>, never empty (Solr wouldn't call the plugin if empty
+   * since no useful work could then be done).
+   */
+  Set<Node> getLiveNodes();
+
+  /**
+   * <p>Returns info about the given collection if one exists. Because it is not expected for plugins to request info about
+   * a large number of collections, requests can only be made one by one.
+   *
+   * <p>This is also the reason we do not return a {@link java.util.Map} or {@link Set} of {@link SolrCollection}'s here: it would be
+   * wasteful to fetch all data and fill such a map when plugin code likely needs info about at most one or two collections.
+   */
+  Optional<SolrCollection> getCollection(String collectionName) throws IOException;
+
+  /**
+   * <p>Allows getting all {@link SolrCollection} present in the cluster.
+   *
+   * <p><b>WARNING:</b> this call might be extremely inefficient on large clusters. Usage is discouraged.
+   */
+  Set<SolrCollection> getAllCollections();

Review comment:
       Yes, that's the case today, but other parts of Solr suffer from this problem too, and there are already discussions to move away from one ClusterState to `Collection<String> getCollectionNames()` + `DocCollection getCollection(name)`.




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PlacementPlanFactory.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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;
+
+import java.util.Set;
+
+/**
+ * Allows plugins to create {@link PlacementPlan}s telling the Solr layer where to create replicas following the processing of
+ * a {@link PlacementRequest}. The Solr layer can (and will) check that the {@link PlacementPlan} conforms to the {@link PlacementRequest} (and
+ * if it does not, the requested operation will fail).
+ */
+public interface PlacementPlanFactory {
+  /**
+   * <p>Creates a {@link PlacementPlan} for adding a new collection and its replicas.
+   *
+   * <p>This is in support of {@link org.apache.solr.cloud.api.collections.CreateCollectionCmd}.
+   */
+  PlacementPlan createPlacementPlanNewCollection(CreateNewCollectionPlacementRequest request, String CollectionName, Set<ReplicaPlacement> replicaPlacements);
+
+  /**
+   * <p>Creates a {@link PlacementPlan} for adding replicas to a given shard of an existing collection.
+   *
+   * <p>This is in support (directly or indirectly) of {@link org.apache.solr.cloud.api.collections.AddReplicaCmd},
+   * {@link org.apache.solr.cloud.api.collections.CreateShardCmd}, {@link org.apache.solr.cloud.api.collections.ReplaceNodeCmd},
+   * {@link org.apache.solr.cloud.api.collections.MoveReplicaCmd}, {@link org.apache.solr.cloud.api.collections.SplitShardCmd},
+   * {@link org.apache.solr.cloud.api.collections.RestoreCmd} and {@link org.apache.solr.cloud.api.collections.MigrateCmd}.
+   * (as well as of {@link org.apache.solr.cloud.api.collections.CreateCollectionCmd} in the specific case of
+   * {@link org.apache.solr.common.params.CollectionAdminParams#WITH_COLLECTION} but this should be removed shortly and
+   * the section in parentheses of this comment should be removed when the {@code withCollection} javadoc link appears broken).
+   */
+  PlacementPlan createPlacementPlanAddReplicas(AddReplicasPlacementRequest request, String CollectionName, Set<ReplicaPlacement> replicaPlacements);

Review comment:
       @sigram I will indeed unify these two requests to a single one. When creating a new collection, the command currently first creates the collection ZK node then does the Assign request, and there's already a DocCollection instance available. Therefore no need to distinguish new collection creation from adding replicas to an existing one. Given the plugin will have to deal with the more complex task of adding replicas (compared to creating a new collection, since positions of existing replicas have to be taken into account), there will be no added work there (minor loss of efficiency for computing placement for new collections, likely will not even be measurable).




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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


   > Here is an example of how to have strong typing for each atribute without creating so many different interfaces
   
   It's easy to have a single `PropertyValue` interface with all possible getter methods and have all but one return an empty optional. I don't see how that's better.
   
   Also I don't want to assume the plugin code knows which node a request should be sent to (i.e. passing the node to `onNode()`). It does know the node for most requests but not all. When requesting data about a given shard or replica for example (say number of docs, or index size), it is not the role of the plugin to find out which node that replica is on to route the request there. I prefer the behind the scenes implementation to do that.


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/gumi/PropertyKeyFactory.java
##########
@@ -0,0 +1,34 @@
+/*
+ * 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.cloud.gumi;
+
+/**
+ * Factory used by the plugin to create property keys to request property values from Solr
+ */
+public interface PropertyKeyFactory {
+  /**
+   * Returns a property key allowing to request the number of cores. There are no parameters for this key.
+   */
+  CoresCountPropertyKey createCoreCountKey();
+
+  /**
+   * Returns a property key allowing to request the value of a system property (sorry property used twice in two different
+   * contexts). The parameter is the name of the system property to retrieve.
+   */
+  SystemPropertyPropertyKey createSystemPropertyKey(String systemPropertyName);

Review comment:
       @sigram @noblepaul I need some help here...
   
   I assume "metrics" are the things that look like `"metrics:solr.core.Collection.shardXX.replicaYY:actual.metric.name"`). They don't seem to be at the "node level" but relate to a replica (maybe to other things).
   Do you know how getting the corresponding metric value is implemented? Is it sent from Overseer running Autoscaling to the node hosting the relevant replica for evaluation?
   
   I wonder how the `PropertyKey` instances (that will include a `MetricsPropertyKey` soon) should be grouped by the plugin when it requests to have values fetched so that the fetching is efficient (I assume plugins will be single threaded and do synchronous calls, to keep things simple). In other words, I want to allow limiting to a single exchange with each node for fetching all data (all properties, including metrics), so I need to let the plugin group its requests accordingly.
   
   In the first commit, the `Node` class had a `getProperties()` method. I was considering that the plugin knows what it wants to ask from each `Node` and can group these requests and send only once per node.
   
   Now if the plugin also needs for example `metrics:solr.core.testEstimatedIndexSize_collection.shard1.replica_n2:SEARCHER.searcher.indexCommitSize` (random value copied from a test), and if that metric requires a round trip with the node hosting that specific replica, there's no easy way for the plugin to group requests per `Node` (I don't want the plugin to have to do complicated groupings on data just to request metrics).
   
   Therefore I'm considering moving the `getProperties()` away from `Node` and into a central place, and having each property key specify what context it is for: the whole cluster, a specific node, a specific collection, a specific shard or a specific replica. I assume that covers current needs.
   This way a single call to get all the property values can be dispatched efficiently by the implementation by grouping all keys that need to go to the same node into a single message.
   (in order not to prematurely optimize, I'm resisting and not introducing wildcard constructs such as `#EACH`... under the assumption that key building is of negligible cost compared to the shipping to the nodes and response collection, and we will need to expand wildcard requests anyway).
   
   Assuming the metric target is specified independently (by having a reference to the corresponding API object), a metric is then fully defined by passing in the metric name? (the `SEARCHER.searcher.indexCommitSize`). That's all I need to pass then in the corresponding `PropertyKeyFactory` metric building call?




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/Cluster.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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;

Review comment:
       Didn't want to hijack what might be a useful package name for other uses, but of course this can be moved (I can move it if "eventually" is now).




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PlacementPlugin.java
##########
@@ -0,0 +1,41 @@
+/*
+ * 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;
+
+/**
+ * Implemented by external plugins to control replica placement and movement on the search cluster (as well as other things
+ * such as cluster elasticity?) when cluster changes are required (initiated elsewhere, most likely following a Collection
+ * API call).
+ */
+public interface PlacementPlugin {

Review comment:
       I think that we need to add an explicit mechanism for configuration of plugins, otherwise plugin implementors will have to use other Solr facilities anyway.
   
   Maybe add a `configure(Map<String, Object> config)` method?




----------------------------------------------------------------
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] noblepaul commented on pull request #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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


   Here is an example of how to have strong typing for each atribute without creating  so many different interfaces
   
   `public interface PropertyFetcher {`
     `PropertyFetcher onNode();`
    ` PropertyFetcher withCoreCount();`
     `PropertyFetcher withSystemProperty(String name);`
     
     `PropertyValues fetchValues();`
     
    ` interface PropertyValues {`
       `Optional<Integer> coreCount();`
       `Optional<String> systemProperty(String propertyName);}}`
   `


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PlacementPlugin.java
##########
@@ -0,0 +1,41 @@
+/*
+ * 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;
+
+/**
+ * Implemented by external plugins to control replica placement and movement on the search cluster (as well as other things
+ * such as cluster elasticity?) when cluster changes are required (initiated elsewhere, most likely following a Collection
+ * API call).
+ */
+public interface PlacementPlugin {

Review comment:
       Either constructor convention or - I prefer but not sure how popular it is in the Solr codebase - when configuring a plugin what is added to the Solr config is a plugin factory and that factory is called to get plugin instances. One more interface but cleaner code.




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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


   I did answer above your comment and alternate proposal about fetching property values. I believe our users will not be tormented if we provide them an efficient implementation.
   
   I’d be happy to change the current proposal for something easier to use (not pretending it’s perfect) but I’m not willing to sacrifice on performance in order to do so.


----------------------------------------------------------------
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] noblepaul edited a comment on pull request #1684: SOLR-14613: strongly typed initial proposal for plugin interface

Posted by GitBox <gi...@apache.org>.
noblepaul edited a comment on pull request #1684:
URL: https://github.com/apache/lucene-solr/pull/1684#issuecomment-669008011


   I'm a bit busy too. We will compare notes later once you are done


----------------------------------------------------------------
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 edited a comment on pull request #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

Posted by GitBox <gi...@apache.org>.
murblanc edited a comment on pull request #1684:
URL: https://github.com/apache/lucene-solr/pull/1684#issuecomment-683434613


   > So, I have two main problems with this PR
   
   I consider this progress 😃 
   
   > 1. Using `solr.xml`
   
   There's no other place to set default config (not hard code!) for which placement plugin to use (as well as configuration parameters for that plugin). `solr.xml` is such a place, is used in that way for other values so makes perfectly sense here. Previous Autoscaling framework used `solr.xml` for config (for autoscaling trigger). It did not use it for the preferences and policy because these were collection specific, but you suggested placement plugins be a cluster level config so I followed that route instead.
   
   > 2. having 2 interfaces for each property
   
   Untrue. There's only one interface, for the value. Keys all share the same interface.
   
   


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/gumi/ReplicaType.java
##########
@@ -0,0 +1,25 @@
+/*
+ * 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.cloud.gumi;
+
+/**
+ * TODO: move into {@link Replica}?
+ */
+public enum ReplicaType {

Review comment:
       I didn't want to expose `Replica` as stated above and I therefore can't use `Replica.Type`.
   If `Type` were in its own Java file and a pure enum, reuse would have been possible. But even then, not sure it would have been a good idea: an internal replica type might eventually include things we don't want a plugin to be able to manipulate.




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/Cluster.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * <p>A representation of the (initial) cluster state, providing information on which nodes are part of the cluster and a way
+ * to get to more detailed info.
+ *
+ * <p>This instance can also be used as a {@link PropertyValueSource} if {@link PropertyKey}'s need to be specified with
+ * a global cluster target.
+ */
+public interface Cluster extends PropertyValueSource {
+  /**
+   * @return current set of live nodes. Never <code>null</code>, never empty (Solr wouldn't call the plugin if empty
+   * since no useful work could then be done).
+   */
+  Set<Node> getLiveNodes();
+
+  /**
+   * <p>Returns info about the given collection if one exists. Because it is not expected for plugins to request info about
+   * a large number of collections, requests can only be made one by one.
+   *
+   * <p>This is also the reason we do not return a {@link java.util.Map} or {@link Set} of {@link SolrCollection}'s here: it would be
+   * wasteful to fetch all data and fill such a map when plugin code likely needs info about at most one or two collections.
+   */
+  Optional<SolrCollection> getCollection(String collectionName) throws IOException;
+
+  /**
+   * <p>Allows getting all {@link SolrCollection} present in the cluster.
+   *
+   * <p><b>WARNING:</b> this call might be extremely inefficient on large clusters. Usage is discouraged.
+   */
+  Set<SolrCollection> getAllCollections();

Review comment:
       Side note: I need (and plan to eventually make) SolrCloud to scale to hundred of thousand collections. Anything that’s in O(n) or worse in number of collections will not fly for this scale. Implementations can be inefficient (and can change) but let’s try to keep interfaces efficient.




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PropertyKeyFactory.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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;
+
+/**
+ * Factory used by the plugin to create property keys to request property values from Solr.<p>
+ *
+ * Building of a {@link PropertyKey} requires specifying the target (context) from which the value of that key should be
+ * obtained. This is done by specifying the appropriate {@link PropertyValueSource}.<br>
+ * For clarity, when only a single type of target is acceptable, the corresponding subtype of {@link PropertyValueSource} is used instead
+ * (for example {@link Node}).
+ */
+public interface PropertyKeyFactory {
+  /**
+   * Returns a property key to request the number of cores on a {@link Node}.
+   */
+  PropertyKey createCoreCountKey(Node node);

Review comment:
       Keep in mind it's not subclassing just for the pleasure of subclassing. In this example of `createCoreCountKey()`, we end up building an `AbstractNodePropertyKey.CoreCountImpl` and that's where the definition is that this key will correspond to `ImplicitSnitch.CORES`. If we make creation code generic, we'll have to move the actual differences in implementation to somewhere else.
   
   Another key create by `createMetricKey` does even more in `AbstractNodePropertyKey.NodeMetricImpl` since it translates the abstracted registry (from `NodeMetricRegistry`) into the actual registry and assembles the metric name.
   
   In other words, there's specific code to write regardless for each new key type assuming we don't want to expose the internal implementation of Solr (and parameters such as `"metrics:"` and `"totaldisk"` etc used in messages).




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementPluginFactoryFacade.java
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.cluster.placement.PlacementPlugin;
+import org.apache.solr.cluster.placement.PlacementPluginConfig;
+import org.apache.solr.cluster.placement.PlacementPluginFactory;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.core.PluginInfo;
+import org.apache.solr.core.SolrResourceLoader;
+
+/**
+ * <p>The internal class instantiating the configured {@link PlacementPluginFactory} and creating a {@link PlacementPlugin}
+ * instance by passing the the factory the appropriate configuration created from the {@code <placementPluginFactory>}
+ * element in {@code solr.xml}.
+ *
+ * <p>A single instance of {@link PlacementPlugin} is used for all placement computations and therefore must be reentrant.
+ * When configuration changes, a new instance of {@link PlacementPlugin} will be created by calling again
+ * {@link PlacementPluginFactory#createPluginInstance(PlacementPluginConfig)}.
+ */
+public class PlacementPluginFactoryFacade {

Review comment:
       It is not part of the API. It's in the `impl` package.




----------------------------------------------------------------
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] noblepaul commented on pull request #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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


   >There's no other place to set default config (not hard code!) for which placement plugin to use 
   
   What is wrong with hard coding?


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/Cluster.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * <p>A representation of the (initial) cluster state, providing information on which nodes are part of the cluster and a way
+ * to get to more detailed info.
+ *
+ * <p>This instance can also be used as a {@link PropertyValueSource} if {@link PropertyKey}'s need to be specified with
+ * a global cluster target.
+ */
+public interface Cluster extends PropertyValueSource {
+  /**
+   * @return current set of live nodes. Never <code>null</code>, never empty (Solr wouldn't call the plugin if empty
+   * since no useful work could then be done).
+   */
+  Set<Node> getLiveNodes();
+
+  /**
+   * <p>Returns info about the given collection if one exists. Because it is not expected for plugins to request info about
+   * a large number of collections, requests can only be made one by one.
+   *
+   * <p>This is also the reason we do not return a {@link java.util.Map} or {@link Set} of {@link SolrCollection}'s here: it would be
+   * wasteful to fetch all data and fill such a map when plugin code likely needs info about at most one or two collections.
+   */
+  Optional<SolrCollection> getCollection(String collectionName) throws IOException;
+
+  /**
+   * <p>Allows getting all {@link SolrCollection} present in the cluster.
+   *
+   * <p><b>WARNING:</b> this call might be extremely inefficient on large clusters. Usage is discouraged.
+   */
+  Set<SolrCollection> getAllCollections();

Review comment:
       I think there was no call to get only names (away from computer for a week or more). The only call on cluster state returned DocCollection set or map...




----------------------------------------------------------------
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] noblepaul edited a comment on pull request #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

Posted by GitBox <gi...@apache.org>.
noblepaul edited a comment on pull request #1684:
URL: https://github.com/apache/lucene-solr/pull/1684#issuecomment-683778043


   Let us take a step back and reassess at what we have today.
   
   https://github.com/apache/lucene-solr/blob/4a2a2da462ef017154856edd92b9f4f8c549c084/solr/core/src/java/org/apache/solr/cloud/api/collections/Assign.java#L385
   
   There is already an interface called `AssignStrategy` .Can you believe that it's as simple as
   
   `  public interface AssignStrategy {`
      ` List<ReplicaPosition> assign(SolrCloudManager solrCloudManager, AssignRequest assignRequest)`
           `throws Assign.AssignmentException, IOException, InterruptedException;`
    ` }`
   
   This is simple interface is doing the trick today (there are 3 different impls using this interface). We are proposing to introduce a few dozen classes and interfaces to improve upon this. In reality, I do not see anybody other than a couple of Solr devs actually using this new framework. This is going to be a huge ball and chain. We will be struggling to deprecate this an year down the line. Let's not do this
   
   My suggestion is to not build anything new . Just use this interface


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/ReplicaPlacement.java
##########
@@ -0,0 +1,29 @@
+/*
+ * 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;
+
+/**
+ * <p>Placement decision for a single {@link Replica}. Note this placement decision is used as part of a {@link WorkOrder},
+ * it does not directly lead to the plugin code getting a corresponding {@link Replica} instance, nor does it require the
+ * plugin to provide a {@link Shard} instance (the plugin code gets such instances for existing replicas and shards in the
+ * cluster but does not create them directly for adding new replicas for new or existing shards).
+ *
+ * <p>Captures the {@link Shard} (via the shard name), {@link Node} and {@link Replica.ReplicaType} of a Replica to be created.
+ */
+public interface ReplicaPlacement {

Review comment:
       Right, the plugin doesn't need this - but other parts of Solr may need it to simplify error handling if the error is too far away in the code from the original Request.

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PlacementPlanFactory.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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;
+
+import java.util.Set;
+
+/**
+ * Allows plugins to create {@link PlacementPlan}s telling the Solr layer where to create replicas following the processing of
+ * a {@link PlacementRequest}. The Solr layer can (and will) check that the {@link PlacementPlan} conforms to the {@link PlacementRequest} (and
+ * if it does not, the requested operation will fail).
+ */
+public interface PlacementPlanFactory {
+  /**
+   * <p>Creates a {@link PlacementPlan} for adding a new collection and its replicas.
+   *
+   * <p>This is in support of {@link org.apache.solr.cloud.api.collections.CreateCollectionCmd}.
+   */
+  PlacementPlan createPlacementPlanNewCollection(CreateNewCollectionPlacementRequest request, String CollectionName, Set<ReplicaPlacement> replicaPlacements);
+
+  /**
+   * <p>Creates a {@link PlacementPlan} for adding replicas to a given shard of an existing collection.
+   *
+   * <p>This is in support (directly or indirectly) of {@link org.apache.solr.cloud.api.collections.AddReplicaCmd},
+   * {@link org.apache.solr.cloud.api.collections.CreateShardCmd}, {@link org.apache.solr.cloud.api.collections.ReplaceNodeCmd},
+   * {@link org.apache.solr.cloud.api.collections.MoveReplicaCmd}, {@link org.apache.solr.cloud.api.collections.SplitShardCmd},
+   * {@link org.apache.solr.cloud.api.collections.RestoreCmd} and {@link org.apache.solr.cloud.api.collections.MigrateCmd}.
+   * (as well as of {@link org.apache.solr.cloud.api.collections.CreateCollectionCmd} in the specific case of
+   * {@link org.apache.solr.common.params.CollectionAdminParams#WITH_COLLECTION} but this should be removed shortly and
+   * the section in parentheses of this comment should be removed when the {@code withCollection} javadoc link appears broken).
+   */
+  PlacementPlan createPlacementPlanAddReplicas(AddReplicasPlacementRequest request, String CollectionName, Set<ReplicaPlacement> replicaPlacements);

Review comment:
       Eh, it looks to me like we need just one method:
   `PlacementPlan createPlacementPlan(PlacementRequest request, String collectionName, Set<ReplicaPlacement> replicaPlacements)`
   
   Also ... I'm not sure how useful this interface is. Do we really need to dictate such granular internal API for the plugins? It's the responsibility of `PlacementPlugin` to create a `PlacementPlan`, and it should be free to do it whichever way it wants, not necessarily going through this factory - it seems like an additional hurdle and not a convenience.

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PlacementPlugin.java
##########
@@ -0,0 +1,41 @@
+/*
+ * 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;
+
+/**
+ * Implemented by external plugins to control replica placement and movement on the search cluster (as well as other things
+ * such as cluster elasticity?) when cluster changes are required (initiated elsewhere, most likely following a Collection
+ * API call).
+ */
+public interface PlacementPlugin {

Review comment:
       Exactly - but where should that data be cached? we don't offer the plugin any way to do it, so implementors will come up with all sorts of nasty ways to circumvent the API sandbox to do it (accessing ZK directly, using external shared FS, or even something worse ;) ).




----------------------------------------------------------------
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] noblepaul commented on a change in pull request #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/gumi/PropertyKeyFactory.java
##########
@@ -0,0 +1,34 @@
+/*
+ * 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.cloud.gumi;
+
+/**
+ * Factory used by the plugin to create property keys to request property values from Solr
+ */
+public interface PropertyKeyFactory {

Review comment:
       PrpertyKey can be an enum?




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/ReplicaPlacement.java
##########
@@ -0,0 +1,29 @@
+/*
+ * 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;
+
+/**
+ * <p>Placement decision for a single {@link Replica}. Note this placement decision is used as part of a {@link WorkOrder},
+ * it does not directly lead to the plugin code getting a corresponding {@link Replica} instance, nor does it require the
+ * plugin to provide a {@link Shard} instance (the plugin code gets such instances for existing replicas and shards in the
+ * cluster but does not create them directly for adding new replicas for new or existing shards).
+ *
+ * <p>Captures the {@link Shard} (via the shard name), {@link Node} and {@link Replica.ReplicaType} of a Replica to be created.
+ */
+public interface ReplicaPlacement {

Review comment:
       It does include the `Node`. See `WorkOrderFactory.createReplicaPlacement()`. It does not directly refer to a `Request`, the reference to `Request` is captured in the `WorkOrder` created using the same factory and in which the `ReplicaPlacement` are used.
   Everything passed to the creation factories can be made accessible on the returned instances if needed (given it's captured in the underlying implementations), but I'm not convinced it's useful, so kept it simple. Assumption is that plugin code creates these instances so plugin code knows why and keeps track of what each created instance refers to... But again, easy to add here and in other instances returned by factories (might need to define subinterfaces then to make the appropriate values accessible - BTW that's how I started coding locally, but then simplified to limit the number of interfaces and for the reasons exposed above).




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/Replica.java
##########
@@ -0,0 +1,46 @@
+/*
+ * 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;
+
+/**
+ * An instantiation (or one of the copies) of a given {@link Shard} of a given {@link SolrCollection}.
+ * Objects of this type are returned by the Solr framework to the plugin, they are not directly built by the plugin. When the
+ * plugin wants to add a replica it goes through appropriate method in {@link PlacementPlanFactory}).
+ */
+public interface Replica extends PropertyValueSource {
+  Shard getShard();
+
+  ReplicaType getType();
+  ReplicaState getState();

Review comment:
       Leader is fetched from `Shard` in this API. Of course can move it here. Didn't find examples where it would be more convenient to have it on the `Replica`.




----------------------------------------------------------------
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] chatman commented on pull request #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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


   I used to run Windows 3.11 on my 4MB RAM 486DX computer back in early 90s.
   Your comment brought back those memories. Developers used to be so careful
   about memory usage!
   
   On Mon, Aug 17, 2020 at 8:57 PM Ilan Ginzburg <no...@github.com>
   wrote:
   
   > With the last commit just now, the API code is functionally complete for a
   > basic assign replicas use case except config and getting properties.
   > Didn't test anything yet though (still on PTO technically, working off a
   > 4MB RAM very very entry level Windows laptop)
   >
   > —
   > You are receiving this because you were mentioned.
   > Reply to this email directly, view it on GitHub
   > <https://github.com/apache/lucene-solr/pull/1684#issuecomment-674948465>,
   > or unsubscribe
   > <https://github.com/notifications/unsubscribe-auth/ABDCR5GQKEMMZEU7YVJL2KLSBFD6FANCNFSM4PC3HDKQ>
   > .
   >
   


----------------------------------------------------------------
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 edited a comment on pull request #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

Posted by GitBox <gi...@apache.org>.
murblanc edited a comment on pull request #1684:
URL: https://github.com/apache/lucene-solr/pull/1684#issuecomment-683433584


   > Here is an example of how to have strong typing for each atribute without creating so many different interfaces
   
   It's easy to have a single `PropertyValue` interface with all possible getter methods and have all but one return an empty optional. I don't see how that's better. [Edit: this is not exactly what you suggested though]
   
   Also I don't want to assume the plugin code knows which node a request should be sent to (i.e. passing the node to `onNode()`). It does know the node for most requests but not all. When requesting data about a given shard or replica for example (say number of docs, or index size), it is not the role of the plugin to find out which node that replica is on to route the request there. I prefer the behind the scenes implementation to do that.
   
   [Edit: also, a single fetch call from plugin to fetch all data on all nodes rather than node by node allows request optimization on the implementation side. The most obvious example is using multiple concurrent messages to the different nodes (multithreading or in any other way). If the plugin requests data node by node, it's either sequential or forces the plugin to implement the concurrency mechanism itself, making it more complicated.
   There are even more ambitious optimizations that can be made on the implementation side if we get all the data to distribute right away, by implementing for example a hierarchical distribution, to reduce the number of messages that have to cross Availability Zones.]


----------------------------------------------------------------
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] thelabdude commented on a change in pull request #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/Cluster.java
##########
@@ -0,0 +1,46 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * <p>A representation of the (initial) cluster state, providing information on which nodes are part of the cluster and a way
+ * to get to more detailed info.
+ *
+ * <p>This instance can also be used as a {@link PropertyValueSource} if {@link PropertyKey}'s need to be specified with
+ * a global cluster target.
+ */
+public interface Cluster extends PropertyValueSource {
+  /**
+   * @return current set of live nodes. Never <code>null</code>, never empty (Solr wouldn't call the plugin if empty
+   * since no useful could then be done).
+   */
+  Set<Node> getLiveNodes();
+
+  /**
+   * <p>Returns info about the given collection if one exists. Because it is not expected for plugins to request info about
+   * a large number of collections, requests can only be made one by one.
+   *
+   * <p>This is also the reason we do not return a {@link java.util.Map} or {@link Set} of {@link SolrCollection}'s here: it would be
+   * wasteful to fetch all data and fill such a map when plugin code likely needs info about at most one or two collections.
+   */
+  Optional<SolrCollection> getCollection(String collectionName) throws IOException;

Review comment:
       You can use system property rules for ensuring replicas for one collection don't get placed on nodes hosting another collection vs. the new framework trying to compute this by looking at all the nodes hosting a specific collection (which can change anyway). Here's an example using the current framework: 
   https://github.com/lucidworks/fusion-cloud-native/blob/master/policy.json#L19
   
   This approach is also simple to reason about and is in-use in a number of production environments. Any ops person will be comfortable with setting JVM props or ENV vars to indicate unique Solr node properties that they want to use for placement decisions.




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/SolrCollection.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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;
+
+import java.util.Map;
+
+/**
+ * Represents a Collection in SolrCloud (unrelated to {@link java.util.Collection} that uses the nicer name).
+ */
+public interface SolrCollection {
+  /**
+   * The collection name (value passed to {@link Cluster#getCollection(String)}).
+   */
+  String getName();
+
+  /**
+   * <p>The {@link Shard}'s over which the data of this {@link SolrCollection} is distributed.
+   *
+   * <p>The map is from {@link Shard#getShardName()} to {@link Shard} instance.
+   */
+  Map<String, Shard> getShards();

Review comment:
       What use case is served by an iterator that only iterates on shard names and that doesn't "materialize" them into `Shard` instances?
   If it's for finding a `Shard` whose name matches some pattern, we should devise more efficient ways.
   I'm not against having such an iterator but need help understanding when it might be useful (assuming we already have an iterator that returns `Shard` instances and a method to fetch a given `Shard` by name).




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PlacementPlugin.java
##########
@@ -0,0 +1,41 @@
+/*
+ * 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;
+
+/**
+ * Implemented by external plugins to control replica placement and movement on the search cluster (as well as other things
+ * such as cluster elasticity?) when cluster changes are required (initiated elsewhere, most likely following a Collection
+ * API call).
+ */
+public interface PlacementPlugin {

Review comment:
       Perfectly fine with me. The contract will be for the plugin to provide a constructor accepting config to create a new Plugin instance, then that instance will be called for each placement computation (and if the plugin doesn’t care about config, no arg constructor would be fine). Multiple plugin instances might be in use at the same time if so called by Solr (config changes or other reasons).
   This means the plugin instance must be reentrant and its member variables can’t directly be used as if specific to a single computation. Not a big deal, plugin implementor can delegate internally to an instance of another class if they want that option.




----------------------------------------------------------------
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] dsmiley commented on pull request #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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


   I am a conflict-avoider (not a good thing) but need to say I fully agree with @sigram 's impression of the tone.  I can tell, personally, as a subjective measure in how my own body reacts.  When I read conflict, my heart starts racing (fight/flight response?), and it started racing on some of Noble's comments and Ishan alike.  It can be hard to objectively measure tone; I think it's fundamentally subjective, though some particulars we point to can be illustrative of behavioral problems.  I will be specific:
   
   @chatman wrote
   > Plugin making concurrent requests is "complicated"? Java is hard, but I can help you write a plugin that can make concurrent requests cleanly. Please don't pollute solr-core because of your perceived inability to do so cleanly from a plugin.
   
   Wow.  I'm not even sure that would be polite to say to a junior engineer (that may be presumed to not know concurrency) than a very senior one here.  I think you know full well that Ilan doesn't need your help writing concurrent code.   Yet why would you offer help?  I think this is a false offer (you know Ilan will not take you up on this); -- it's a pejorative offer that belittles who you offer it to as being beneath the abilities of your own.  I wonder why you would say that?  Maybe you don't believe it's worth it to make it easier for plugin writers at the expense of Solr having such concurrent code (i.e. where does the concurrency/complexity go) -- fare, but then just say that without belittling anyone's intelligence.
   
   At least at the moment it seems we're at a point of better understanding.
   
   p.s. This PR is so big/long that my browser (Safari) struggles to keep up as I type here. I copy-paste from a text-editor.t


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/Cluster.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * <p>A representation of the (initial) cluster state, providing information on which nodes are part of the cluster and a way
+ * to get to more detailed info.
+ *
+ * <p>This instance can also be used as a {@link PropertyValueSource} if {@link PropertyKey}'s need to be specified with
+ * a global cluster target.
+ */
+public interface Cluster extends PropertyValueSource {
+  /**
+   * @return current set of live nodes. Never <code>null</code>, never empty (Solr wouldn't call the plugin if empty
+   * since no useful work could then be done).
+   */
+  Set<Node> getLiveNodes();
+
+  /**
+   * <p>Returns info about the given collection if one exists. Because it is not expected for plugins to request info about
+   * a large number of collections, requests can only be made one by one.
+   *
+   * <p>This is also the reason we do not return a {@link java.util.Map} or {@link Set} of {@link SolrCollection}'s here: it would be
+   * wasteful to fetch all data and fill such a map when plugin code likely needs info about at most one or two collections.
+   */
+  Optional<SolrCollection> getCollection(String collectionName) throws IOException;
+
+  /**
+   * <p>Allows getting all {@link SolrCollection} present in the cluster.
+   *
+   * <p><b>WARNING:</b> this call might be extremely inefficient on large clusters. Usage is discouraged.
+   */
+  Set<SolrCollection> getAllCollections();

Review comment:
       Ok. Can return names here.
   Still didn’t get what’s the use case for this method though. I’d assume if we need to fetch collection names we don’t know, we might want to fetch names that verify a given pattern. Maybe make this method accept some form of filtering? (Something that can be implemented efficiently if we ever want to, not an “accept” function that forces iterating over all collection names anyway).




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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


   I have implemented the configuration part of the placement plugins. This happens in solr.xml that has been updated with an example.
   I was able to run collection creation and see the plugin code get called, compute placement and replicas get created.
   Property fetching (number of cores per node, metrics, system properties) is not yet implemented. That's the next 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 #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PropertyValue.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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;
+
+/**
+ *  <p>The value corresponding to a specific {@link PropertyKey}, in a specific context (e.g. property of a specific
+ *  {@link Node} instance). The context is tracked in the {@link PropertyKey} using a {@link PropertyValueSource}.
+ *
+ *  <p>Instances are obtained by first getting a key using {@link PropertyKeyFactory} then getting the corresponding
+ *  {@link PropertyValue} using {@link PropertyValueFetcher}.
+ */
+public interface PropertyValue {
+  /**
+   * The property key used for retrieving this property value.
+   */
+  PropertyKey getKey();
+
+  /**
+   *  Instances are obtained by first getting a key using {@link PropertyKeyFactory#createCoreCountKey} then calling
+   *  {@link PropertyValueFetcher#fetchProperties}, retrieving the appropriate {@link PropertyValue} from the returned map
+   *  using the {@link PropertyKey} as key and finally casting it to {@link PropertyValue.CoresCount}.
+   */
+  interface CoresCount extends PropertyValue {

Review comment:
       I was thinking of getting rid of the `PropertyValue` interface and classes and adding an accessor to get the value from the `PropertyKey` directly. This accessor would necessarily be an `Optional`, and would return empty if the property keys were not fetched yet or if the corresponding value was not returned by the fetch (sysprop not defined for example). This would force defining subinterfaces of `PropertyKey` (currently these are opaque objects), it would free the caller from manipulation the fetch result map (the fetch function would be `void` and only work by side effect on the keys passed to 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 pull request #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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


   The work orders are the actions decided by the plugin to satisfy a request or requests sent by a Collection API command implementation.
   The chain is: Collection API -> command execution builds requests -> plugin code running, returns work orders -> collection API command execution executes the work orders resulting in cluster state changes and node changes (creating/deleting replicas).
   Maybe create collection doesn’t have to be decided by plugin code since it has to happen in any case when a new collection is created. It could be added by the collection api command itself (but then tricky interaction with the cluster state classes given it’s a partially built collection).
   
   Gumi means “elastic” in Hebrew. I’ll change that package name to... what?
   
   I’ll remove the cluster state classes (SolrCollection, Shard, Replica) and assume plugins get their view of cluster state elsewhere.


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PropertyKeyFactory.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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;
+
+/**
+ * Factory used by the plugin to create property keys to request property values from Solr.<p>
+ *
+ * Building of a {@link PropertyKey} requires specifying the target (context) from which the value of that key should be
+ * obtained. This is done by specifying the appropriate {@link PropertyValueSource}.<br>
+ * For clarity, when only a single type of target is acceptable, the corresponding subtype of {@link PropertyValueSource} is used instead
+ * (for example {@link Node}).
+ */
+public interface PropertyKeyFactory {
+  /**
+   * Returns a property key to request the number of cores on a {@link Node}.
+   */
+  PropertyKey createCoreCountKey(Node node);
+
+  /**
+   * Returns a property key to request disk related info on a {@link Node}.
+   */
+  PropertyKey createDiskInfoKey(Node node);
+
+  /**
+   * Returns a property key to request the value of a system property on a {@link Node}.
+   * @param systemPropertyName the name of the system property to retrieve.
+   */
+  PropertyKey createSystemPropertyKey(Node node, String systemPropertyName);
+
+  /**
+   * Returns a property key to request the value of a metric.<p>
+   *
+   * Not all metrics make sense everywhere, but metrics can be applied to different objects. For example
+   * <code>SEARCHER.searcher.indexCommitSize</code> would make sense for a given replica of a given shard of a given collection,
+   * and possibly in other contexts.<p>
+   *
+   * @param metricSource The registry of the metric. For example a specific {@link Replica}.
+   * @param metricName for example <code>SEARCHER.searcher.indexCommitSize</code>.
+   */
+  PropertyKey createMetricKey(PropertyValueSource metricSource, String metricName);

Review comment:
       Maybe for metrics there's a need to add a registry name in addition to the `PropertyValueSource`? That would likely solve this issue.
   My thinking was that we might have metrics on other things than a `Node` or the JVM where a `Node` runs. Like metrics on a collection...
   If this is not the case, then I guess metrics should have a value source of `Node`, and then a registry name (that might better be expressed as enum if there are only two).




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/MetricPropertyValue.java
##########
@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+/**
+ * A {@link PropertyValue} representing a metric on the target {@link PropertyValueSource}.
+ * Note there might be overlap with {@link SystemLoadPropertyValue} (only applicable to {@link Node}'s), may need to clarify.
+ */
+public interface MetricPropertyValue extends PropertyValue {
+  /**
+   * Returns the metric value from the {@link PropertyValueSource} on which it was retrieved.
+   * TODO: what type should the metric be? Maybe offer multiple getters for different java types and have each metric implement the right one and throw from the wrong ones? This avoids casting...

Review comment:
       I've created the number and string property value types but I'm unclear on the Map one so we need to discuss this further.




----------------------------------------------------------------
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] thelabdude commented on a change in pull request #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PropertyValue.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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;
+
+/**
+ *  <p>The value corresponding to a specific {@link PropertyKey}, in a specific context (e.g. property of a specific
+ *  {@link Node} instance). The context is tracked in the {@link PropertyKey} using a {@link PropertyValueSource}.
+ *
+ *  <p>Instances are obtained by first getting a key using {@link PropertyKeyFactory} then getting the corresponding
+ *  {@link PropertyValue} using {@link PropertyValueFetcher}.
+ */
+public interface PropertyValue {
+  /**
+   * The property key used for retrieving this property value.
+   */
+  PropertyKey getKey();
+
+  /**
+   *  Instances are obtained by first getting a key using {@link PropertyKeyFactory#createCoreCountKey} then calling
+   *  {@link PropertyValueFetcher#fetchProperties}, retrieving the appropriate {@link PropertyValue} from the returned map
+   *  using the {@link PropertyKey} as key and finally casting it to {@link PropertyValue.CoresCount}.
+   */
+  interface CoresCount extends PropertyValue {
+    /**
+     * Returns the number of cores on the {@link Node}) this instance was obtained from (i.e. instance
+     * passed to {@link PropertyKeyFactory#createCoreCountKey(Node)}).
+     */
+    int getCoresCount();
+  }
+
+  /**
+   *  Instances are obtained by first getting a key using {@link PropertyKeyFactory#createDiskTypeKey} then getting the
+   *  {@link org.apache.solr.cluster.placement.PropertyValue.DiskType} using {@link PropertyValueFetcher#fetchProperties} and retrieving (then casting) the
+   *  appropriate {@link PropertyValue} from the returned map using the {@link PropertyKey} as key.
+   */
+  interface DiskType extends PropertyValue {
+    /**
+     * Type of storage hardware used for the partition on which cores are stored on the {@link Node}) from which this instance
+     * was obtained (i.e. instance passed to {@link PropertyKeyFactory#createDiskTypeKey(Node)}).
+     */
+    HardwareType getHardwareType();
+
+    enum HardwareType {
+      SSD, ROTATIONAL, UNKNOWN
+    }
+  }
+
+  /**
+   *  Instances are obtained by first getting a key using {@link PropertyKeyFactory#createTotalDiskKey(Node)} (Node)} then getting the
+   *  {@link org.apache.solr.cluster.placement.PropertyValue.TotalDisk} using {@link PropertyValueFetcher#fetchProperties} and retrieving (then casting) the
+   *  appropriate {@link PropertyValue} fetched from the returned map using the {@link PropertyKey} as key.
+   */
+  interface TotalDisk extends PropertyValue {
+    /**
+     * Total disk size of the partition on which cores are stored on the {@link Node}) from which this instance was obtained
+     * (i.e. instance passed to {@link PropertyKeyFactory#createTotalDiskKey(Node)}).
+     */
+    long getTotalSizeGB();
+  }
+
+  /**
+   *  Instances are obtained by first getting a key using {@link PropertyKeyFactory#createFreeDiskKey(Node)} then getting the
+   *  {@link org.apache.solr.cluster.placement.PropertyValue.FreeDisk} using {@link PropertyValueFetcher#fetchProperties} and retrieving (then casting) the
+   *  appropriate {@link PropertyValue} fetched from the returned map using the {@link PropertyKey} as key.
+   */
+  interface FreeDisk extends PropertyValue {
+    /**
+     * Free disk size of the partition on which cores are stored on the {@link Node}) from which this instance was obtained
+     *  (i.e. instance passed to {@link PropertyKeyFactory#createDiskTypeKey(Node)}).
+     */
+    long getFreeSizeGB();
+  }
+
+
+  /**
+   *  Instances are obtained by first getting a key using {@link PropertyKeyFactory#createHeapUsageKey(Node)} then calling
+   *  {@link PropertyValueFetcher#fetchProperties}, retrieving the appropriate {@link PropertyValue} from the returned map
+   *  using the {@link PropertyKey} as key and finally casting it to {@link org.apache.solr.cluster.placement.PropertyValue.HeapUsage}.
+   */
+  interface HeapUsage extends PropertyValue {
+
+    /**
+     * Percentage between 0 and 100 of used heap over max heap.
+     */
+    double getUsedHeapMemoryUsage();
+  }
+
+  /**
+   * A {@link PropertyValue} representing a metric on the target {@link PropertyValueSource}.
+   *
+   * <p>Instances are obtained by first getting a key using {@link PropertyKeyFactory#createMetricKey(PropertyValueSource, String)}
+   * or {@link PropertyKeyFactory#createMetricKey(Node, String, PropertyKeyFactory.NodeMetricRegistry)} then calling
+   * {@link PropertyValueFetcher#fetchProperties}, retrieving the appropriate {@link PropertyValue} from the returned map
+   * using the {@link PropertyKey} as key and finally casting it to {@link org.apache.solr.cluster.placement.PropertyValue.Metric}.
+   */
+  interface Metric extends PropertyValue {
+    /**
+     * Returns the metric value from the {@link PropertyValueSource} from which it was retrieved.
+     */
+    Double getNumberValue();
+  }
+
+  /**
+   * A {@link PropertyValue} representing a sysprop (or System property) on the target {@link Node}.
+   *
+   *  Instances are obtained by first getting a key using {@link PropertyKeyFactory#createSyspropKey} then calling
+   *  {@link PropertyValueFetcher#fetchProperties}, retrieving the appropriate {@link PropertyValue} from the returned map
+   *  using the {@link PropertyKey} as key and finally casting it to {@link org.apache.solr.cluster.placement.PropertyValue.Sysprop}.
+   */
+  interface Sysprop extends PropertyValue {
+    /**

Review comment:
       Be great to support an ENV var as well (which is typically easier to work with in containerized deployments like Kubernetes). Works similar to Sysprop just the value comes from an Environment variable.




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/SolrCollection.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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;
+
+import java.util.Map;
+
+/**
+ * Represents a Collection in SolrCloud (unrelated to {@link java.util.Collection} that uses the nicer name).
+ */
+public interface SolrCollection {
+  /**
+   * The collection name (value passed to {@link Cluster#getCollection(String)}).
+   */
+  String getName();
+
+  /**
+   * <p>The {@link Shard}'s over which the data of this {@link SolrCollection} is distributed.
+   *
+   * <p>The map is from {@link Shard#getShardName()} to {@link Shard} instance.
+   */
+  Map<String, Shard> getShards();

Review comment:
       What about replacing this by `Shard getShard(String name);` and `Iterator<Shard> getShardIterator();` ?
   These can be implemented efficiently.
   
   Same thing should then be done for `Cluster.getAllCollectionNames()` that I've added reluctantly, maybe returning an `Iterator` is better?




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PropertyKeyFactory.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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;
+
+/**
+ * Factory used by the plugin to create property keys to request property values from Solr.<p>
+ *
+ * Building of a {@link PropertyKey} requires specifying the target (context) from which the value of that key should be
+ * obtained. This is done by specifying the appropriate {@link PropertyValueSource}.<br>
+ * For clarity, when only a single type of target is acceptable, the corresponding subtype of {@link PropertyValueSource} is used instead
+ * (for example {@link Node}).
+ */
+public interface PropertyKeyFactory {
+  /**
+   * Returns a property key to request the number of cores on a {@link Node}.
+   */
+  PropertyKey createCoreCountKey(Node node);
+
+  /**
+   * Returns a property key to request disk related info on a {@link Node}.
+   */
+  PropertyKey createDiskInfoKey(Node node);
+
+  /**
+   * Returns a property key to request the value of a system property on a {@link Node}.
+   * @param systemPropertyName the name of the system property to retrieve.
+   */
+  PropertyKey createSystemPropertyKey(Node node, String systemPropertyName);
+
+  /**
+   * Returns a property key to request the value of a metric.<p>
+   *
+   * Not all metrics make sense everywhere, but metrics can be applied to different objects. For example
+   * <code>SEARCHER.searcher.indexCommitSize</code> would make sense for a given replica of a given shard of a given collection,
+   * and possibly in other contexts.<p>
+   *
+   * @param metricSource The registry of the metric. For example a specific {@link Replica}.
+   * @param metricName for example <code>SEARCHER.searcher.indexCommitSize</code>.
+   */
+  PropertyKey createMetricKey(PropertyValueSource metricSource, String metricName);

Review comment:
       Is `solr.jetty` of any use or `solr.node` and `solr.jvm` are sufficient?
   
   I see `SolrInfoBean.Group` entries other than `node`, `jvm`, `jetty` are:
   - `collection`, `shard`, `cluster`: these can be inferred from `PropertyValueSource` type,
   - `core`: likely not needed for placement computation,
   - `overseer`: doesn't seem to be used




----------------------------------------------------------------
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] chatman commented on pull request #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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


   I see. In that case, the way forward should be to maybe use Solr.XML only
   for now to unblock you, but revisit it later to leverage clusterprops
   before the release. @noble, is that okay for you?
   
   On Mon, 31 Aug, 2020, 11:54 am Ilan Ginzburg, <no...@github.com>
   wrote:
   
   > Ishan, Noble is suggesting not to use solr.xml and hard code the plugin to
   > use if I understand his point correctly.
   >
   > What you suggest is how it’s in the PR: legacy will be used unless plugins
   > are defined in solr.xml (or if the Collection has rules).
   >
   > —
   > You are receiving this because you were mentioned.
   > Reply to this email directly, view it on GitHub
   > <https://github.com/apache/lucene-solr/pull/1684#issuecomment-683584869>,
   > or unsubscribe
   > <https://github.com/notifications/unsubscribe-auth/ABDCR5AOFZTTWFPHU43D6VLSDM6ZXANCNFSM4PC3HDKQ>
   > .
   >
   


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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


   Thanks @murblanc for taking the lead on this issue, and for your patience!


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/AddReplicasPlacementRequest.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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;
+
+import java.util.Set;
+
+/**
+ * <p>Request passed by Solr to a {@link PlacementPlugin} to compute placement for one or more {@link Replica}'s for one
+ * or more {@link Shard}'s of an existing {@link SolrCollection}.
+ * The shard might or might not already exist, plugin code can easily find out by using {@link SolrCollection#getShards()}
+ * and verifying if the shard name(s) from {@link #getShardNames()} are there.
+ *
+ * <p>As opposed to {@link CreateNewCollectionPlacementRequest}, the set of {@link Node}s on which the replicas should be placed
+ * is specified (defaults to being equal to the set returned by {@link Cluster#getLiveNodes()}).
+ *
+ * <p>There is no extension between this interface and {@link CreateNewCollectionPlacementRequest} in either direction
+ * or from a common ancestor for readability. An ancestor could make sense and would be an "abstract interface" not intended
+ * to be implemented directly, but this does not exist in Java.
+ *
+ * <p>Plugin code would likely treat the two types of requests differently since here existing {@link Replica}'s must be taken
+ * into account for placement whereas in {@link CreateNewCollectionPlacementRequest} no {@link Replica}'s are assumed to exist.
+ */
+public interface AddReplicasPlacementRequest extends PlacementRequest {
+  /**
+   * The {@link SolrCollection} to add {@link Replica}(s) to. The replicas are to be added to a shard that might or might
+   * not yet exist when the plugin's {@link PlacementPlugin#computePlacement} is called.
+   */
+  SolrCollection getCollection();

Review comment:
       @noblepaul you requested a change here. Do you mind elaborating based on my reply to your request?




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PlacementPlugin.java
##########
@@ -0,0 +1,41 @@
+/*
+ * 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;
+
+/**
+ * Implemented by external plugins to control replica placement and movement on the search cluster (as well as other things
+ * such as cluster elasticity?) when cluster changes are required (initiated elsewhere, most likely following a Collection
+ * API call).
+ */
+public interface PlacementPlugin {

Review comment:
       I'm strongly against using static variables. 
   
   We can either assume that plugins are basically stateless and configured on the fly, or stateful - and in the latter case they could be re-configurable (which has its own problems) or not - that is, you have to create a new instance of the plugin if configuration has changed. In this case there's no problem with synchronization - the old instance of the plugin completes its work using the old config.




----------------------------------------------------------------
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] chatman edited a comment on pull request #1684: SOLR-14613: strongly typed initial proposal for plugin interface

Posted by GitBox <gi...@apache.org>.
chatman edited a comment on pull request #1684:
URL: https://github.com/apache/lucene-solr/pull/1684#issuecomment-664051353


   > Gumi means “elastic” in Hebrew. I’ll change that package name to... what?
   
   I see that @noblepaul has used the name "o.a.s.common.cloud.sdk" in the other PR for SolrCloud specific interfaces. One day, I would hope all our internal code should use these interfaces, and having "sdk" then, IMHO, wouldn't be ideal. How about "o.a.s.cloud.interfaces" for that?
   
   Regarding other interfaces introduced here, how about "o.a.s.cloud.placements.interfaces" or "o.a.s.cloud.assignments.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 pull request #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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


   Thanks @sigram for the comments. They're useful, will update the PR tomorrow.


----------------------------------------------------------------
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] noblepaul closed pull request #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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


   


----------------------------------------------------------------
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 edited a comment on pull request #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

Posted by GitBox <gi...@apache.org>.
murblanc edited a comment on pull request #1684:
URL: https://github.com/apache/lucene-solr/pull/1684#issuecomment-674948465


   With the last commit just now, the API code is functionally complete for a basic assign replicas use case except config and getting properties.
   Didn't test anything yet though (still on PTO technically, working off a 4MB [edit: 4GB of course] RAM very very entry level Windows laptop)


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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


   There’s no direct mapping between work orders and collection API. I consider work orders the basic building blocs from which collection API actions can be built. Creating a replica (adding the data in ZK, checking for the config set etc) is different from adding a replica to an existing collection. Hence the two work orders. The new collection one doesn’t add any replicas to the collection but prepares the system to accept replicas for the collection (work that is not required when adding a replica to an existing collection).


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/gumi/Replica.java
##########
@@ -0,0 +1,34 @@
+/*
+ * 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.cloud.gumi;
+
+/**
+ * An instantiation (or one of the copies) of a given {@link Shard} of a given {@link SolrCollection}.
+ * Objects of this type are returned by the Solr framework to the plugin, they are not built by the plugin. When the
+ * plugin wants to add a replica it goes through {@link WorkOrderFactory#createWorkOrderCreateReplica}).
+ * TODO is there an elegant way to have this type also used by the plugin to add replicas? (insisting on elegant)
+ */
+public interface Replica {

Review comment:
       Couple reasons not to use `org.apache.solr.common.cloud.Replica`:
   
   The abstraction level there is "weak". For example from a replica getting a shard returns a String with Shard name. The plugin can't do much with a String. We'd have to provide the plugin with a way to convert that String into data, adding complexity. It's easier to have a real object (interface) that can be used right away (in our case to get the index of the shard in the collection, possibly other data will be added as other use cases are inventoried).
   
   There are or might be things in this class (it's not even an interface) that the plugin doesn't need and shouldn't know about. No reason to expose those.
   
   Also, exposing an internal implementation class to external plugin code (code the community will not necessarily have access to) forces not to change that internal class to not risk breaking the external plugins. This might be a problem, it limits refactoring. By having an insulation layer (Adapter/Wrapper pattern) we can refactor `org.apache.solr.common.cloud.Replica` as much as we want, and will adapt the (internal) implementation provided for `org.apache.solr.cloud.gumi.Replica` so client plugin code is not impacted.




----------------------------------------------------------------
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] noblepaul commented on a change in pull request #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/plugins/SamplePluginMinimizeCores.java
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.Comparator;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Set;
+import java.util.Map;
+
+import com.google.common.collect.Ordering;
+import com.google.common.collect.TreeMultimap;
+import org.apache.solr.cluster.placement.Cluster;
+import org.apache.solr.cluster.placement.CoresCountPropertyValue;
+import org.apache.solr.cluster.placement.CreateNewCollectionPlacementRequest;
+import org.apache.solr.cluster.placement.Node;
+import org.apache.solr.cluster.placement.PlacementException;
+import org.apache.solr.cluster.placement.PlacementPlugin;
+import org.apache.solr.cluster.placement.PropertyKey;
+import org.apache.solr.cluster.placement.PropertyKeyFactory;
+import org.apache.solr.cluster.placement.PropertyValue;
+import org.apache.solr.cluster.placement.PropertyValueFetcher;
+import org.apache.solr.cluster.placement.Replica;
+import org.apache.solr.cluster.placement.ReplicaPlacement;
+import org.apache.solr.cluster.placement.PlacementRequest;
+import org.apache.solr.cluster.placement.PlacementPlan;
+import org.apache.solr.cluster.placement.PlacementPlanFactory;
+import org.apache.solr.common.util.SuppressForbidden;
+
+/**
+ * Implements placing replicas to minimize number of cores per {@link Node}, while not placing two replicas of the same
+ * shard on the same node.
+ *
+ * TODO: code not tested and never run, there are no implementation yet for used interfaces
+ */
+public class SamplePluginMinimizeCores implements PlacementPlugin {
+
+  @SuppressForbidden(reason = "Ordering.arbitrary() has no equivalent in Comparator class. Rather reuse than copy.")
+  public PlacementPlan computePlacement(Cluster cluster, PlacementRequest placementRequest, PropertyKeyFactory propertyFactory,
+                                        PropertyValueFetcher propertyFetcher, PlacementPlanFactory placementPlanFactory) throws PlacementException {
+    // This plugin only supports Creating a collection.
+    if (!(placementRequest instanceof CreateNewCollectionPlacementRequest)) {
+      throw new PlacementException("This toy plugin only supports creating collections");
+    }
+
+    final CreateNewCollectionPlacementRequest reqCreateCollection = (CreateNewCollectionPlacementRequest) placementRequest;
+
+    final int totalReplicasPerShard = reqCreateCollection.getNrtReplicationFactor() +
+        reqCreateCollection.getTlogReplicationFactor() + reqCreateCollection.getPullReplicationFactor();
+
+    if (cluster.getLiveNodes().size() < totalReplicasPerShard) {
+      throw new PlacementException("Cluster size too small for number of replicas per shard");
+    }
+
+    // Get number of cores on each Node
+    TreeMultimap<Integer, Node> nodesByCores = TreeMultimap.create(Comparator.naturalOrder(), Ordering.arbitrary());

Review comment:
       I believe the property fetching is overly complicated . We should probably make it a lot simpler. 
   
   Basically, the only requirement is strong typing. 
   
   `TreeMultimap<Integer, Node> nodesByCores = TreeMultimap.create(Comparator.naturalOrder(), Ordering.arbitrary());`
   
   This definitely is not the easiest code we could write. A user just wants to get an integer value for the # of cores in a node. 
   
   

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/plugins/SamplePluginMinimizeCores.java
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.Comparator;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Set;
+import java.util.Map;
+
+import com.google.common.collect.Ordering;
+import com.google.common.collect.TreeMultimap;
+import org.apache.solr.cluster.placement.Cluster;
+import org.apache.solr.cluster.placement.CoresCountPropertyValue;
+import org.apache.solr.cluster.placement.CreateNewCollectionPlacementRequest;
+import org.apache.solr.cluster.placement.Node;
+import org.apache.solr.cluster.placement.PlacementException;
+import org.apache.solr.cluster.placement.PlacementPlugin;
+import org.apache.solr.cluster.placement.PropertyKey;
+import org.apache.solr.cluster.placement.PropertyKeyFactory;
+import org.apache.solr.cluster.placement.PropertyValue;
+import org.apache.solr.cluster.placement.PropertyValueFetcher;
+import org.apache.solr.cluster.placement.Replica;
+import org.apache.solr.cluster.placement.ReplicaPlacement;
+import org.apache.solr.cluster.placement.PlacementRequest;
+import org.apache.solr.cluster.placement.PlacementPlan;
+import org.apache.solr.cluster.placement.PlacementPlanFactory;
+import org.apache.solr.common.util.SuppressForbidden;
+
+/**
+ * Implements placing replicas to minimize number of cores per {@link Node}, while not placing two replicas of the same
+ * shard on the same node.
+ *
+ * TODO: code not tested and never run, there are no implementation yet for used interfaces
+ */
+public class SamplePluginMinimizeCores implements PlacementPlugin {
+
+  @SuppressForbidden(reason = "Ordering.arbitrary() has no equivalent in Comparator class. Rather reuse than copy.")
+  public PlacementPlan computePlacement(Cluster cluster, PlacementRequest placementRequest, PropertyKeyFactory propertyFactory,
+                                        PropertyValueFetcher propertyFetcher, PlacementPlanFactory placementPlanFactory) throws PlacementException {
+    // This plugin only supports Creating a collection.
+    if (!(placementRequest instanceof CreateNewCollectionPlacementRequest)) {
+      throw new PlacementException("This toy plugin only supports creating collections");
+    }
+
+    final CreateNewCollectionPlacementRequest reqCreateCollection = (CreateNewCollectionPlacementRequest) placementRequest;
+
+    final int totalReplicasPerShard = reqCreateCollection.getNrtReplicationFactor() +
+        reqCreateCollection.getTlogReplicationFactor() + reqCreateCollection.getPullReplicationFactor();
+
+    if (cluster.getLiveNodes().size() < totalReplicasPerShard) {
+      throw new PlacementException("Cluster size too small for number of replicas per shard");
+    }
+
+    // Get number of cores on each Node
+    TreeMultimap<Integer, Node> nodesByCores = TreeMultimap.create(Comparator.naturalOrder(), Ordering.arbitrary());

Review comment:
       Having 2 classes for each property will just multiply the no:of classes in the API. Basically
   
   The only difference between 2 properties is property name and the type. Property name can be supplied as a method, type can be a generic
   
   

##########
File path: solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
##########
@@ -85,6 +86,7 @@
   public static final java.util.List<String> MODIFIABLE_COLLECTION_PROPERTIES = Arrays.asList(
       RULE,
       SNITCH,
+      PLACEMENT,

Review comment:
       let's make this a cluster-wide property and not a per collection property

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/AddReplicasPlacementRequest.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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;
+
+import java.util.Set;
+
+/**
+ * <p>Request passed by Solr to a {@link PlacementPlugin} to compute placement for one or more {@link Replica}'s for one
+ * or more {@link Shard}'s of an existing {@link SolrCollection}.
+ * The shard might or might not already exist, plugin code can easily find out by using {@link SolrCollection#getShards()}
+ * and verifying if the shard name(s) from {@link #getShardNames()} are there.
+ *
+ * <p>As opposed to {@link CreateNewCollectionPlacementRequest}, the set of {@link Node}s on which the replicas should be placed
+ * is specified (defaults to being equal to the set returned by {@link Cluster#getLiveNodes()}).
+ *
+ * <p>There is no extension between this interface and {@link CreateNewCollectionPlacementRequest} in either direction
+ * or from a common ancestor for readability. An ancestor could make sense and would be an "abstract interface" not intended
+ * to be implemented directly, but this does not exist in Java.
+ *
+ * <p>Plugin code would likely treat the two types of requests differently since here existing {@link Replica}'s must be taken
+ * into account for placement whereas in {@link CreateNewCollectionPlacementRequest} no {@link Replica}'s are assumed to exist.
+ */
+public interface AddReplicasPlacementRequest extends PlacementRequest {
+  /**
+   * The {@link SolrCollection} to add {@link Replica}(s) to. The replicas are to be added to a shard that might or might
+   * not yet exist when the plugin's {@link PlacementPlugin#computePlacement} is called.
+   */
+  SolrCollection getCollection();

Review comment:
       preferably use a name of the collection




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PropertyValue.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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;
+
+/**
+ *  <p>The value corresponding to a specific {@link PropertyKey}, in a specific context (e.g. property of a specific
+ *  {@link Node} instance). The context is tracked in the {@link PropertyKey} using a {@link PropertyValueSource}.
+ *
+ *  <p>Instances are obtained by first getting a key using {@link PropertyKeyFactory} then getting the corresponding
+ *  {@link PropertyValue} using {@link PropertyValueFetcher}.
+ */
+public interface PropertyValue {
+  /**
+   * The property key used for retrieving this property value.
+   */
+  PropertyKey getKey();
+
+  /**
+   *  Instances are obtained by first getting a key using {@link PropertyKeyFactory#createCoreCountKey} then calling
+   *  {@link PropertyValueFetcher#fetchProperties}, retrieving the appropriate {@link PropertyValue} from the returned map
+   *  using the {@link PropertyKey} as key and finally casting it to {@link PropertyValue.CoresCount}.
+   */
+  interface CoresCount extends PropertyValue {
+    /**
+     * Returns the number of cores on the {@link Node}) this instance was obtained from (i.e. instance
+     * passed to {@link PropertyKeyFactory#createCoreCountKey(Node)}).
+     */
+    int getCoresCount();
+  }
+
+  /**
+   *  Instances are obtained by first getting a key using {@link PropertyKeyFactory#createDiskTypeKey} then getting the
+   *  {@link org.apache.solr.cluster.placement.PropertyValue.DiskType} using {@link PropertyValueFetcher#fetchProperties} and retrieving (then casting) the
+   *  appropriate {@link PropertyValue} from the returned map using the {@link PropertyKey} as key.
+   */
+  interface DiskType extends PropertyValue {
+    /**
+     * Type of storage hardware used for the partition on which cores are stored on the {@link Node}) from which this instance
+     * was obtained (i.e. instance passed to {@link PropertyKeyFactory#createDiskTypeKey(Node)}).
+     */
+    HardwareType getHardwareType();
+
+    enum HardwareType {
+      SSD, ROTATIONAL, UNKNOWN
+    }
+  }
+
+  /**
+   *  Instances are obtained by first getting a key using {@link PropertyKeyFactory#createTotalDiskKey(Node)} (Node)} then getting the
+   *  {@link org.apache.solr.cluster.placement.PropertyValue.TotalDisk} using {@link PropertyValueFetcher#fetchProperties} and retrieving (then casting) the
+   *  appropriate {@link PropertyValue} fetched from the returned map using the {@link PropertyKey} as key.
+   */
+  interface TotalDisk extends PropertyValue {
+    /**
+     * Total disk size of the partition on which cores are stored on the {@link Node}) from which this instance was obtained
+     * (i.e. instance passed to {@link PropertyKeyFactory#createTotalDiskKey(Node)}).
+     */
+    long getTotalSizeGB();
+  }
+
+  /**
+   *  Instances are obtained by first getting a key using {@link PropertyKeyFactory#createFreeDiskKey(Node)} then getting the
+   *  {@link org.apache.solr.cluster.placement.PropertyValue.FreeDisk} using {@link PropertyValueFetcher#fetchProperties} and retrieving (then casting) the
+   *  appropriate {@link PropertyValue} fetched from the returned map using the {@link PropertyKey} as key.
+   */
+  interface FreeDisk extends PropertyValue {
+    /**
+     * Free disk size of the partition on which cores are stored on the {@link Node}) from which this instance was obtained
+     *  (i.e. instance passed to {@link PropertyKeyFactory#createDiskTypeKey(Node)}).
+     */
+    long getFreeSizeGB();
+  }
+
+
+  /**
+   *  Instances are obtained by first getting a key using {@link PropertyKeyFactory#createHeapUsageKey(Node)} then calling
+   *  {@link PropertyValueFetcher#fetchProperties}, retrieving the appropriate {@link PropertyValue} from the returned map
+   *  using the {@link PropertyKey} as key and finally casting it to {@link org.apache.solr.cluster.placement.PropertyValue.HeapUsage}.
+   */
+  interface HeapUsage extends PropertyValue {
+
+    /**
+     * Percentage between 0 and 100 of used heap over max heap.
+     */
+    double getUsedHeapMemoryUsage();
+  }
+
+  /**
+   * A {@link PropertyValue} representing a metric on the target {@link PropertyValueSource}.
+   *
+   * <p>Instances are obtained by first getting a key using {@link PropertyKeyFactory#createMetricKey(PropertyValueSource, String)}
+   * or {@link PropertyKeyFactory#createMetricKey(Node, String, PropertyKeyFactory.NodeMetricRegistry)} then calling
+   * {@link PropertyValueFetcher#fetchProperties}, retrieving the appropriate {@link PropertyValue} from the returned map
+   * using the {@link PropertyKey} as key and finally casting it to {@link org.apache.solr.cluster.placement.PropertyValue.Metric}.
+   */
+  interface Metric extends PropertyValue {
+    /**
+     * Returns the metric value from the {@link PropertyValueSource} from which it was retrieved.
+     */
+    Double getNumberValue();
+  }
+
+  /**
+   * A {@link PropertyValue} representing a sysprop (or System property) on the target {@link Node}.
+   *
+   *  Instances are obtained by first getting a key using {@link PropertyKeyFactory#createSyspropKey} then calling
+   *  {@link PropertyValueFetcher#fetchProperties}, retrieving the appropriate {@link PropertyValue} from the returned map
+   *  using the {@link PropertyKey} as key and finally casting it to {@link org.apache.solr.cluster.placement.PropertyValue.Sysprop}.
+   */
+  interface Sysprop extends PropertyValue {
+    /**

Review comment:
       Had them initially and removed ;)
   Can add back if that PR ever makes 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] chatman edited a comment on pull request #1684: SOLR-14613: strongly typed placement plugin interface and implementation

Posted by GitBox <gi...@apache.org>.
chatman edited a comment on pull request #1684:
URL: https://github.com/apache/lucene-solr/pull/1684#issuecomment-686380223


   I broadly agree with those principals, and I'm sure everyone does.
   
   >    1. placement plugin writing is easy, 
   
   Agree, but not at the cost of complicated server side code that resides in solr-core.
   
   > 2. implementation for this placement API is efficient, and 
   
   +1
   
   > 3. plugins do not break when Solr is refactored (plugins might be client private so can’t be refactored when Solr code changes).
   
   Mostly agree, but not at the cost of thousands of classes. There are simpler ways, as I think we now are clear on, to achieve the same goal.
   
   I would also like you to consider another goal (which, IMHO, is *most important*), which is to keeping the solr-core as lean, clean and the footprint of doing so as minimal as possible. Everything that can be done outside solr-core should be done. Autoscaling is not a first class citizen in a search engine, but replica assignment/placement can arguably be so.
   
   > Can we make this assumption? Plugin requests a set of properties from a set of nodes, and this is done efficiently.
   If so I'll gladly simplify my proposal
   
   +1. I'm glad we're all on the same page (at least on the verbosity aspect). Thanks for your work, @murblanc.


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/impl/PluginInteractionsImpl.java
##########
@@ -0,0 +1,190 @@
+/*

Review comment:
       Heh, ok ... I guess there's no way to make everyone happy ;)




----------------------------------------------------------------
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] noblepaul removed a comment on pull request #1684: SOLR-14613: strongly typed placement plugin interface and implementation

Posted by GitBox <gi...@apache.org>.
noblepaul removed a comment on pull request #1684:
URL: https://github.com/apache/lucene-solr/pull/1684#issuecomment-686318299


   > don't like an explosion of classes but I think this is highly mitigated by them being defined as simple inner classes of one outer class, as Ilan has done.
   
   Yeah, the solution to API surface area problem is to make them inner classes/interfaces. This is such a elegant & simple proposition. We should try the same in other places in Solr where there is a surface area problem. 


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PlacementPlanFactory.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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;
+
+import java.util.Set;
+
+/**
+ * Allows plugins to create {@link PlacementPlan}s telling the Solr layer where to create replicas following the processing of
+ * a {@link PlacementRequest}. The Solr layer can (and will) check that the {@link PlacementPlan} conforms to the {@link PlacementRequest} (and
+ * if it does not, the requested operation will fail).
+ */
+public interface PlacementPlanFactory {
+  /**
+   * <p>Creates a {@link PlacementPlan} for adding a new collection and its replicas.
+   *
+   * <p>This is in support of {@link org.apache.solr.cloud.api.collections.CreateCollectionCmd}.
+   */
+  PlacementPlan createPlacementPlanNewCollection(CreateNewCollectionPlacementRequest request, String CollectionName, Set<ReplicaPlacement> replicaPlacements);
+
+  /**
+   * <p>Creates a {@link PlacementPlan} for adding replicas to a given shard of an existing collection.
+   *
+   * <p>This is in support (directly or indirectly) of {@link org.apache.solr.cloud.api.collections.AddReplicaCmd},
+   * {@link org.apache.solr.cloud.api.collections.CreateShardCmd}, {@link org.apache.solr.cloud.api.collections.ReplaceNodeCmd},
+   * {@link org.apache.solr.cloud.api.collections.MoveReplicaCmd}, {@link org.apache.solr.cloud.api.collections.SplitShardCmd},
+   * {@link org.apache.solr.cloud.api.collections.RestoreCmd} and {@link org.apache.solr.cloud.api.collections.MigrateCmd}.
+   * (as well as of {@link org.apache.solr.cloud.api.collections.CreateCollectionCmd} in the specific case of
+   * {@link org.apache.solr.common.params.CollectionAdminParams#WITH_COLLECTION} but this should be removed shortly and
+   * the section in parentheses of this comment should be removed when the {@code withCollection} javadoc link appears broken).
+   */
+  PlacementPlan createPlacementPlanAddReplicas(AddReplicasPlacementRequest request, String CollectionName, Set<ReplicaPlacement> replicaPlacements);

Review comment:
       I meant above I'm unifying the creation requests as well as the placement plans.




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/plugins/SamplePluginMinimizeCores.java
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.Comparator;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Set;
+import java.util.Map;
+
+import com.google.common.collect.Ordering;
+import com.google.common.collect.TreeMultimap;
+import org.apache.solr.cluster.placement.Cluster;
+import org.apache.solr.cluster.placement.CoresCountPropertyValue;
+import org.apache.solr.cluster.placement.CreateNewCollectionPlacementRequest;
+import org.apache.solr.cluster.placement.Node;
+import org.apache.solr.cluster.placement.PlacementException;
+import org.apache.solr.cluster.placement.PlacementPlugin;
+import org.apache.solr.cluster.placement.PropertyKey;
+import org.apache.solr.cluster.placement.PropertyKeyFactory;
+import org.apache.solr.cluster.placement.PropertyValue;
+import org.apache.solr.cluster.placement.PropertyValueFetcher;
+import org.apache.solr.cluster.placement.Replica;
+import org.apache.solr.cluster.placement.ReplicaPlacement;
+import org.apache.solr.cluster.placement.PlacementRequest;
+import org.apache.solr.cluster.placement.PlacementPlan;
+import org.apache.solr.cluster.placement.PlacementPlanFactory;
+import org.apache.solr.common.util.SuppressForbidden;
+
+/**
+ * Implements placing replicas to minimize number of cores per {@link Node}, while not placing two replicas of the same
+ * shard on the same node.
+ *
+ * TODO: code not tested and never run, there are no implementation yet for used interfaces
+ */
+public class SamplePluginMinimizeCores implements PlacementPlugin {
+
+  @SuppressForbidden(reason = "Ordering.arbitrary() has no equivalent in Comparator class. Rather reuse than copy.")
+  public PlacementPlan computePlacement(Cluster cluster, PlacementRequest placementRequest, PropertyKeyFactory propertyFactory,
+                                        PropertyValueFetcher propertyFetcher, PlacementPlanFactory placementPlanFactory) throws PlacementException {
+    // This plugin only supports Creating a collection.
+    if (!(placementRequest instanceof CreateNewCollectionPlacementRequest)) {
+      throw new PlacementException("This toy plugin only supports creating collections");
+    }
+
+    final CreateNewCollectionPlacementRequest reqCreateCollection = (CreateNewCollectionPlacementRequest) placementRequest;
+
+    final int totalReplicasPerShard = reqCreateCollection.getNrtReplicationFactor() +
+        reqCreateCollection.getTlogReplicationFactor() + reqCreateCollection.getPullReplicationFactor();
+
+    if (cluster.getLiveNodes().size() < totalReplicasPerShard) {
+      throw new PlacementException("Cluster size too small for number of replicas per shard");
+    }
+
+    // Get number of cores on each Node
+    TreeMultimap<Integer, Node> nodesByCores = TreeMultimap.create(Comparator.naturalOrder(), Ordering.arbitrary());

Review comment:
       Strong typing was one requirement (and strong typing is not only return type, it's also nice names for methods to access these values), the other one was efficiency: being able to fetch ALL needed properties from a remote Node at once (i.e. not requesting metrics first and system properties second which would force the Solr side framework to do two round trips - caching and optimizations aside).
   That's why we have the notion of property key, to be able to "accumulate" the properties to fetch, then we have asingle interface per property, allowing clean access to the value (for cores it's the call coresCountPropertyValue.getCoresCount() line 85 below).
   
   The "complexity" of the structure here is the plug-in implementation, not the API, using sorted data structures to compute a placement quickly. This should be compared (in complexity) to the previous approach of generating all possible placements and comparing them. 




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PlacementPlugin.java
##########
@@ -0,0 +1,41 @@
+/*
+ * 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;
+
+/**
+ * Implemented by external plugins to control replica placement and movement on the search cluster (as well as other things
+ * such as cluster elasticity?) when cluster changes are required (initiated elsewhere, most likely following a Collection
+ * API call).
+ */
+public interface PlacementPlugin {

Review comment:
       **Please ignore this comment**. I thought you were referring to how various plugins get selected for doing work (currently hard coded in `AssignStrategyFactory.create`) 
   
   > The configuration part I didn't really do yet. Unclear to me how the `configure` method here would be used, since in order to get to it the plugin has to be loaded already...
   > I was thinking defining the plugin class or classes in some solr configuration file (with the rest of the config). At least a single default plugin implementation that would be used for all placement needs or a default + other ones ones (with names that can then be selected by callers of the Collection API passing a `placement` parameter as suggested in the changes to `CollectionAdminRequest.java`).
   > Are there similar examples in Solr code (loading plugins possibly on a per collection basis) that I can get inspiration from or reuse?




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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


   I agree about the naming, will change, will try to make these interfaces seem more like a generic thing for SolrCloud rather than something that’s Autoscaling specific.
   
   Regarding “Pleas refer to #1694 as a separate effort to make this simpler.”, I will not at this stage.
   
   I can participate in that other PR (already did) but it duplicates seriously the effort I’ve started here.
   
   Maybe we carry the two approaches in parallel and then we’ll be able to compare what’s best in one or the other and be able to build a third one that’s even better?


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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


   > So, I have two main problems with this PR
   
   I consider this progress 😃 
   
   > 1. Using `solr.xml`
   
   There's no other place to set default config (not hard code!) for which placement plugin to use. `solr.xml` is such a place, is used in that way for other values so makes perfectly sense here. Previous Autoscaling framework used `solr.xml` for config (for autoscaling trigger). It did not use it for the preferences and policy because these were collection specific, but you suggested placement plugins be a cluster level config so I followed that route instead.
   
   > 2. having 2 interfaces for each property
   
   Untrue. There's only one interface, for the value. Keys all share the same interface.
   
   


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/Replica.java
##########
@@ -0,0 +1,46 @@
+/*
+ * 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;
+
+/**
+ * An instantiation (or one of the copies) of a given {@link Shard} of a given {@link SolrCollection}.
+ * Objects of this type are returned by the Solr framework to the plugin, they are not directly built by the plugin. When the
+ * plugin wants to add a replica it goes through appropriate method in {@link PlacementPlanFactory}).
+ */
+public interface Replica extends PropertyValueSource {
+  Shard getShard();
+
+  ReplicaType getType();
+  ReplicaState getState();
+
+  String getReplicaName();
+  String getCoreName();
+
+  /**
+   * {@link Node} on which this {@link Replica} is located.
+   */
+  Node getNode();

Review comment:
       We're going to manipulate `Node` instances in the plugin code. Therefore it's easier that all concepts of `Node` are the nodes themselves and not just their names. Current implementation only defines the name on the node, but this can change. Nodes are also used as targets for fetching values (`PropertyValueSource`), so it's convenient to have them implement the same interfaces as other targets. On the other hand, most key building ended up accepting a `Node` directly, and I don't know if key that are not implemented yet will require flexibility to be applicable to either `Node`, `Shard`, `SolrCollection` or `Replica` transparently.
   I found it more consistent and easier to read to manipulate `Nodes` rather than `Strings` basically if you suggestion is to get rid of `Node` completely.




----------------------------------------------------------------
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] noblepaul commented on pull request #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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


   @murblanc 
   
   The objective of my PRs is not to duplicate your work. It was more or less an idea as a PR.  
   
   - #1694 . This is something I eventually plan to do . Not necessarily for assign/placement framework, But, for across Solr codebase. If you use these interfaces, it will be a good example of how clean interfaces can be used across our codebase
   - #1714 . I created it because you asked me to build a sample. I have no intention to duplicate your work here. 
   
   So, please feel free to ignore #1714 . You can use #1694 in your framework, if you like it. If not, you go ahead .
   
   


----------------------------------------------------------------
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] noblepaul commented on a change in pull request #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/gumi/Replica.java
##########
@@ -0,0 +1,34 @@
+/*
+ * 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.cloud.gumi;
+
+/**
+ * An instantiation (or one of the copies) of a given {@link Shard} of a given {@link SolrCollection}.
+ * Objects of this type are returned by the Solr framework to the plugin, they are not built by the plugin. When the
+ * plugin wants to add a replica it goes through {@link WorkOrderFactory#createWorkOrderCreateReplica}).
+ * TODO is there an elegant way to have this type also used by the plugin to add replicas? (insisting on elegant)
+ */
+public interface Replica {

Review comment:
       Do we need to add a class for this? Isn't it good enough to use the existing class?

##########
File path: solr/core/src/java/org/apache/solr/cloud/gumi/PropertyKeyFactory.java
##########
@@ -0,0 +1,34 @@
+/*
+ * 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.cloud.gumi;
+
+/**
+ * Factory used by the plugin to create property keys to request property values from Solr
+ */
+public interface PropertyKeyFactory {

Review comment:
       PrpertyKey can be an enum?

##########
File path: solr/core/src/java/org/apache/solr/cloud/gumi/CreateCollectionRequest.java
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.cloud.gumi;
+
+/**
+ * <p>Request for creating a new collection with a given number of shards and replication factor for various replica types.</p>
+ *
+ * Note there is no need at this stage to allow the request to convey each shard hash range for example, this can be handled
+ * by the Solr side implementation without needing the plugin to worry about it.
+ *
+ */
+public interface CreateCollectionRequest extends Request {
+  String getCollectionName();
+
+  int getShardCount();
+
+  int getNRTReplicationFactor();

Review comment:
       `getNrtReplicaCount()` ? and so on ?

##########
File path: solr/core/src/java/org/apache/solr/cloud/gumi/ReplicaType.java
##########
@@ -0,0 +1,25 @@
+/*
+ * 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.cloud.gumi;
+
+/**
+ * TODO: move into {@link Replica}?
+ */
+public enum ReplicaType {

Review comment:
       Why not reuse the existing Replica.Type enum ?

##########
File path: solr/core/src/java/org/apache/solr/cloud/gumi/SolrCollection.java
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.cloud.gumi;
+
+import java.util.Set;
+
+/**
+ * Represents a Collection in SolrCloud. Although naming this class "Collection" is possible it would be confusing.
+ */
+public interface SolrCollection {

Review comment:
       Why don't we use the standard DocCollection class ?




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PropertyKeyFactory.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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;
+
+/**
+ * Factory used by the plugin to create property keys to request property values from Solr.<p>
+ *
+ * Building of a {@link PropertyKey} requires specifying the target (context) from which the value of that key should be
+ * obtained. This is done by specifying the appropriate {@link PropertyValueSource}.<br>
+ * For clarity, when only a single type of target is acceptable, the corresponding subtype of {@link PropertyValueSource} is used instead
+ * (for example {@link Node}).
+ */
+public interface PropertyKeyFactory {
+  /**
+   * Returns a property key to request the number of cores on a {@link Node}.
+   */
+  PropertyKey createCoreCountKey(Node node);
+
+  /**
+   * Returns a property key to request disk related info on a {@link Node}.
+   */
+  PropertyKey createDiskInfoKey(Node node);
+
+  /**
+   * Returns a property key to request the value of a system property on a {@link Node}.
+   * @param systemPropertyName the name of the system property to retrieve.
+   */
+  PropertyKey createSystemPropertyKey(Node node, String systemPropertyName);
+
+  /**
+   * Returns a property key to request the value of a metric.<p>
+   *
+   * Not all metrics make sense everywhere, but metrics can be applied to different objects. For example
+   * <code>SEARCHER.searcher.indexCommitSize</code> would make sense for a given replica of a given shard of a given collection,
+   * and possibly in other contexts.<p>
+   *
+   * @param metricSource The registry of the metric. For example a specific {@link Replica}.
+   * @param metricName for example <code>SEARCHER.searcher.indexCommitSize</code>.
+   */
+  PropertyKey createMetricKey(PropertyValueSource metricSource, String metricName);

Review comment:
       So these would be metrics that live on a node but that are accessed differently or with a different name? If we were able to distinguish by some other mean, would Node be an appropriate PropertyValueSource? Another type of PropertyValueSource can be introduced, but then would have to point to a specific JVM...
   Can you point me to examples of these two metrics in 8x or trunk?




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/AddReplicasPlacementRequest.java
##########
@@ -0,0 +1,64 @@
+/*
+ * 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;
+
+import java.util.Set;
+
+/**
+ * <p>Request passed by Solr to a {@link PlacementPlugin} to compute placement for one or more {@link Replica}'s for one
+ * or more {@link Shard}'s of an existing {@link SolrCollection}.
+ * The shard might or might not already exist, plugin code can easily find out by using {@link SolrCollection#getShards()}
+ * and verifying if the shard name(s) from {@link #getShardNames()} are there.
+ *
+ * <p>The set of {@link Node}s on which the replicas should be placed
+ * is specified (defaults to being equal to the set returned by {@link Cluster#getLiveNodes()}).
+ */
+public interface AddReplicasPlacementRequest extends PlacementRequest {
+  /**
+   * The {@link SolrCollection} to add {@link Replica}(s) to.
+   */
+  SolrCollection getCollection();
+
+  /**
+   * <p>Shard name(s) for which new replicas placement should be computed. The shard(s) might exist or not (that's why this
+   * method returns a {@link Set} of {@link String}'s and not directly a set of {@link Shard} instances).
+   *
+   * <p>Note the Collection API allows specifying the shard name or a {@code _route_} parameter. The Solr implementation will
+   * convert either specification into the relevant shard name so the plugin code doesn't have to worry about this.
+   */
+  Set<String> getShardNames();
+
+  /**
+   * <p>Replicas should only be placed on nodes in the set returned by this method.
+   *
+   * <p>When Collection API calls do not specify a specific set of target nodes, replicas can be placed on any live node of
+   * the cluster. In such cases, this set will be equal to the set of all live nodes. The plugin placement code does not
+   * need to worry (or care) if a set of nodes was explicitly specified or not.
+   *
+   * @return never {@code null} and never empty set (if that set was to be empty for any reason, no placement would be
+   * possible and the Solr infrastructure driving the plugin code would detect the error itself rather than calling the plugin).
+   */
+  Set<Node> getTargetNodes();

Review comment:
       In the implementation in `AddReplicasPlacementRequestImpl.toPlacementRequest()`, when no value is passed, the set of live nodes from `Cluster.getLiveNodes()` is passed here. So no memory allocation.




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PlacementException.java
##########
@@ -0,0 +1,48 @@
+/*
+ * 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;
+
+/**
+ * Exception thrown by a {@link PlacementPlugin} when it is unable to compute placement for whatever reason (except an

Review comment:
       Some kind of policy violation comes to mind, but you're right, it's too vague. OTOH with the Policy engine these kind of exceptions where invaluable for tracking the reasons for failures (policy rule violation vs. other types of low-level errors), and in that case we could usually get detailed and well-structured (JSON maps) errors. Something to keep in mind.

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PropertyKeyFactory.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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;
+
+/**
+ * Factory used by the plugin to create property keys to request property values from Solr.<p>
+ *
+ * Building of a {@link PropertyKey} requires specifying the target (context) from which the value of that key should be
+ * obtained. This is done by specifying the appropriate {@link PropertyValueSource}.<br>
+ * For clarity, when only a single type of target is acceptable, the corresponding subtype of {@link PropertyValueSource} is used instead
+ * (for example {@link Node}).
+ */
+public interface PropertyKeyFactory {
+  /**
+   * Returns a property key to request the number of cores on a {@link Node}.
+   */
+  PropertyKey createCoreCountKey(Node node);

Review comment:
       Ok.

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/AddReplicasRequest.java
##########
@@ -0,0 +1,62 @@
+/*
+ * 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;
+
+import java.util.Set;
+
+/**
+ * <p>Request for creating one or more {@link Replica}'s for one or more {@link Shard}'s of an existing {@link SolrCollection}.
+ * The shard might or might not already exist, plugin code can easily find out by using {@link SolrCollection#getShards()}
+ * and verifying if the shard name(s) from {@link #getShardNames()} are there.
+ *
+ * <p>As opposed to {@link CreateNewCollectionRequest}, the set of {@link Node}s on which the replicas should be placed
+ * is specified (defaults to being equal to the set returned by {@link Cluster#getLiveNodes()}).
+ *
+ * <p>There is no extension between this interface and {@link CreateNewCollectionRequest} in either direction
+ * or from a common ancestor for readability. An ancestor could make sense and would be an "abstract interface" not intended
+ * to be implemented directly, but this does not exist in Java.
+ *
+ * <p>Plugin code would likely treat the two types of requests differently since here existing {@link Replica}'s must be taken
+ * into account for placement whereas in {@link CreateNewCollectionRequest} no {@link Replica}'s are assumed to exist.
+ */
+public interface AddReplicasRequest extends Request {
+  /**
+   * The {@link SolrCollection} to add {@link Replica}(s) to. The replicas are to be added to a shard that might or might
+   * not yet exist when the plugin's {@link PlacementPlugin#computePlacement} is called.
+   */
+  SolrCollection getCollection();
+
+  /**
+   * <p>Shard name(s) for which new replicas placement should be computed. The shard(s) might exist or not (that's why this
+   * method returns a {@link Set} of {@link String}'s and not directly a set of {@link Shard} instances).
+   *
+   * <p>Note the Collection API allows specifying the shard name or a {@code _route_} parameter. The Solr implementation will
+   * convert either specification into the relevant shard name so the plugin code doesn't have to worry about this.
+   */
+  Set<String> getShardNames();
+
+  /** Replicas should only be placed on nodes from the set returned by this method. */
+  Set<Node> getTargetNodes();

Review comment:
       Hmmm ... `WorkOrder` -> `Change`, `ChangeTask`, `ClusterChange`, `ClusterModification` (mouthful) ... 

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/ReplicaPlacement.java
##########
@@ -0,0 +1,29 @@
+/*
+ * 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;
+
+/**
+ * <p>Placement decision for a single {@link Replica}. Note this placement decision is used as part of a {@link WorkOrder},
+ * it does not directly lead to the plugin code getting a corresponding {@link Replica} instance, nor does it require the
+ * plugin to provide a {@link Shard} instance (the plugin code gets such instances for existing replicas and shards in the
+ * cluster but does not create them directly for adding new replicas for new or existing shards).
+ *
+ * <p>Captures the {@link Shard} (via the shard name), {@link Node} and {@link Replica.ReplicaType} of a Replica to be created.
+ */
+public interface ReplicaPlacement {

Review comment:
       > WorkOrder to tell Solr where to do what, it passes a reference to the Request
   Hmm, WorkOrder is an empty interface. I think we should add it there explicitly, either the Request or at least the id.

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PlacementPlugin.java
##########
@@ -0,0 +1,41 @@
+/*
+ * 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;
+
+/**
+ * Implemented by external plugins to control replica placement and movement on the search cluster (as well as other things
+ * such as cluster elasticity?) when cluster changes are required (initiated elsewhere, most likely following a Collection
+ * API call).
+ */
+public interface PlacementPlugin {

Review comment:
       Passing configuration in every request means that the plugin is forced to be stateless. There are of course pros and cons... If we force the stateless model, and plugins has to keep some state, implementors will need to figure out how to do this - static fields are NOT the solution. Maybe pass around some kind of long-lived shared context?
   
   Stateless model makes it easier to reconfigure on the fly for every request ... but how realistic is this scenario? I think reconfiguration is only an occasional event. Furthermore, initialization may be costly so it would make sense to allow plugins to keep some kind of context between invokations.

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PlacementPlugin.java
##########
@@ -0,0 +1,41 @@
+/*
+ * 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;
+
+/**
+ * Implemented by external plugins to control replica placement and movement on the search cluster (as well as other things
+ * such as cluster elasticity?) when cluster changes are required (initiated elsewhere, most likely following a Collection
+ * API call).
+ */
+public interface PlacementPlugin {

Review comment:
       And yeas, in Solr it's usually the responsibility of the parent component to configure its children, at least initially.




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PlacementPluginConfig.java
##########
@@ -0,0 +1,60 @@
+/*
+ * 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;
+
+/**
+ * <p>Configuration passed by Solr to {@link PlacementPluginFactory#createPluginInstance(PlacementPluginConfig)} so that plugin instances
+ * ({@link PlacementPlugin}) created by the factory can easily retrieve their configuration.
+ */
+public interface PlacementPluginConfig {

Review comment:
       It could. But I still would like then that the plugin visible one and the internal ones to be different, so we can change internal ones without impacting plugins.
   Unless we reach such a high level of maturity on such interfaces that we are confident to use the same ones inside and outside...




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/gumi/SolrCollection.java
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.cloud.gumi;
+
+import java.util.Set;
+
+/**
+ * Represents a Collection in SolrCloud. Although naming this class "Collection" is possible it would be confusing.
+ */
+public interface SolrCollection {

Review comment:
       BTW the two sample plugins ([SamplePluginMinimizeCores](https://github.com/apache/lucene-solr/pull/1684/files#diff-ddbe185b5e7922b91b90dfabfc50df4c) and [SamplePluginRandomPlacement](https://github.com/apache/lucene-solr/pull/1684/files#diff-a707d47e9a1c89551d455e2ad167f5a5)) show how the interfaces are used.
   
   If we wanted to use existing classes instead of some of the interfaces in the proposal (even though I think it's a bad idea because it will make refactoring challenging), we can easily play with the various interfaces and see what it means and how the plugin code evolves.




----------------------------------------------------------------
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] chatman edited a comment on pull request #1684: SOLR-14613: strongly typed initial proposal for plugin interface

Posted by GitBox <gi...@apache.org>.
chatman edited a comment on pull request #1684:
URL: https://github.com/apache/lucene-solr/pull/1684#issuecomment-664051353


   > Gumi means “elastic” in Hebrew. I’ll change that package name to... what?
   
   I see that @noblepaul has used the name "o.a.s.cloud.sdk" in the other PR for SolrCloud specific interfaces. One day, I would hope all our internal code should use these interfaces, and having "sdk" then, IMHO, wouldn't be ideal. How about "o.a.s.cloud.interfaces" for that?
   
   Regarding other interfaces introduced here, how about "o.a.s.cloud.placements.interfaces" or "o.a.s.cloud.assignments.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] noblepaul commented on a change in pull request #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/gumi/Shard.java
##########
@@ -0,0 +1,32 @@
+/*
+ * 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.cloud.gumi;
+
+import java.util.Set;
+
+/**
+ * Shard in a {@link SolrCollection}, i.e. a subset of the data indexed in that collection.
+ */
+public interface Shard {
+  /**
+   * 0 numbered index of the {@link Shard} in the {@link SolrCollection}.
+   */
+  int getShardIndex();

Review comment:
       shards do not have index. This must not exist




----------------------------------------------------------------
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] noblepaul commented on pull request #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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


   Let us take a step back and reassess at what we have today.
   
   https://github.com/apache/lucene-solr/blob/4a2a2da462ef017154856edd92b9f4f8c549c084/solr/core/src/java/org/apache/solr/cloud/api/collections/Assign.java#L385
   
   There is already an interface called `AssignStrategy` .Can you believe that it's as simple as
   
   `  public interface AssignStrategy {`
      ` List<ReplicaPosition> assign(SolrCloudManager solrCloudManager, AssignRequest assignRequest)`
           `throws Assign.AssignmentException, IOException, InterruptedException;`
    ` }`
   
   This is simple interface is doing the trick today. We are proposing to introduce a few dozen classes and interfaces to improve upon this. In reality, I do not see anybody other than a couple of Solr devs actually using this new framework. This is going to be a huge ball and chain. We will be struggling to deprecate this an year down the line. Let's not do this
   
   My suggestion is to not build anything new . Just use this interface


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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


   The PR has reached what I consider a satisfying and ready to merge state with the last commit.
   Please see some description in the [Jira](https://issues.apache.org/jira/browse/SOLR-14613?focusedCommentId=17171841&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-17171841).
   
   I do hope to merge it by the end of the week, leaving time for discussion until then.


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/SolrCollection.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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;
+
+import java.util.Map;
+
+/**
+ * Represents a Collection in SolrCloud (unrelated to {@link java.util.Collection} that uses the nicer name).
+ */
+public interface SolrCollection {
+  /**
+   * The collection name (value passed to {@link Cluster#getCollection(String)}).
+   */
+  String getName();
+
+  /**
+   * <p>The {@link Shard}'s over which the data of this {@link SolrCollection} is distributed.
+   *
+   * <p>The map is from {@link Shard#getShardName()} to {@link Shard} instance.
+   */
+  Map<String, Shard> getShards();

Review comment:
       What use case is served by an iterator that only iterates on shard names and that doesn't "materialize" them into `Shard` instances?
   If it's for finding a `Shard` whose name matches some pattern, we should devise more efficient ways.
   I'm not against having such an iterator but need help understanding when it might be useful (given an iterator that returns `Shard` instances and a method to fetch a given `Shard` by name).




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PropertyKeyFactory.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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;
+
+/**
+ * Factory used by the plugin to create property keys to request property values from Solr.<p>
+ *
+ * Building of a {@link PropertyKey} requires specifying the target (context) from which the value of that key should be
+ * obtained. This is done by specifying the appropriate {@link PropertyValueSource}.<br>
+ * For clarity, when only a single type of target is acceptable, the corresponding subtype of {@link PropertyValueSource} is used instead
+ * (for example {@link Node}).
+ */
+public interface PropertyKeyFactory {
+  /**
+   * Returns a property key to request the number of cores on a {@link Node}.
+   */
+  PropertyKey createCoreCountKey(Node node);

Review comment:
       If we add new types of `PropertyKeys` we will have to add implementations for these new keys. Wouldn't we need to touch the Solr codebase anyway? Clients (plugins) using the interface would have to know about the new implementation classes and update their code to use them. Technically they could pass a class name through config or other means to use new implementations without code change, but is it a realistic scenario? What would they do with these keys? What are the values these keys will fetch and how will they be used?
   I'm not against making generic and highly flexible code but only if it's really needed. So if you have a real use case in mind that we should support, I'm open. Otherwise I'd rather keep things strongly typed for now (and as long as we only add stuff to these interfaces we're not breaking anything so we can add later).




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PropertyKeyFactory.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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;
+
+/**
+ * Factory used by the plugin to create property keys to request property values from Solr.<p>
+ *
+ * Building of a {@link PropertyKey} requires specifying the target (context) from which the value of that key should be
+ * obtained. This is done by specifying the appropriate {@link PropertyValueSource}.<br>
+ * For clarity, when only a single type of target is acceptable, the corresponding subtype of {@link PropertyValueSource} is used instead
+ * (for example {@link Node}).
+ */
+public interface PropertyKeyFactory {
+  /**
+   * Returns a property key to request the number of cores on a {@link Node}.
+   */
+  PropertyKey createCoreCountKey(Node node);
+
+  /**
+   * Returns a property key to request disk related info on a {@link Node}.
+   */
+  PropertyKey createDiskInfoKey(Node node);
+
+  /**
+   * Returns a property key to request the value of a system property on a {@link Node}.
+   * @param systemPropertyName the name of the system property to retrieve.
+   */
+  PropertyKey createSystemPropertyKey(Node node, String systemPropertyName);
+
+  /**
+   * Returns a property key to request the value of a metric.<p>
+   *
+   * Not all metrics make sense everywhere, but metrics can be applied to different objects. For example
+   * <code>SEARCHER.searcher.indexCommitSize</code> would make sense for a given replica of a given shard of a given collection,
+   * and possibly in other contexts.<p>
+   *
+   * @param metricSource The registry of the metric. For example a specific {@link Replica}.
+   * @param metricName for example <code>SEARCHER.searcher.indexCommitSize</code>.
+   */
+  PropertyKey createMetricKey(PropertyValueSource metricSource, String metricName);

Review comment:
       `SolrDispatchFilter.setupJvmMetrics` initializes per-JVM metrics. They appear in a separate `solr.jvm` registry, which is different from `solr.node`.
   
   In 99% cases (or practically always in production) Solr node maps 1:1 to a JVM instance. In some cases (most notably tests) there can be multiple Solr nodes running in a single JVM, so it's a N:1 - but never the other way around because it wouldn't make sense. So in some rare cases we will have multiple `solr.node` registries in one JVM (reachable via different API endpoints), but always a single `solr.jvm` registry (also reachable via different endpoints).




----------------------------------------------------------------
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] chatman commented on pull request #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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


   > I can participate in that other PR (already did) but it duplicates seriously the effort I’ve started here. Maybe we carry the two approaches in parallel and then we’ll be able to compare what’s best in one or the other and be able to build a third one that’s even better?
   
   I think we should work on SolrCloud interfaces that packages should use in that other PR, and not define them here. I'm -1 on carrying the two approaches in parallel, but I just want for us to separate the two concerns in two PRs (this one concerning with the interfaces specific to assignment/replica placement, and the other one to define SolrCloud interfaces for the benefit of this effort as well as other plugins/packages that may want to use them).


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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


   Ishan, Noble is suggesting not to use solr.xml and hard code the plugin to use if I understand his point correctly.
   
   What you suggest is how it’s in the PR: legacy will be used unless plugins are defined in solr.xml (or if the Collection has rules).


----------------------------------------------------------------
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] vthacker edited a comment on pull request #1684: SOLR-14613: strongly typed placement plugin interface and implementation

Posted by GitBox <gi...@apache.org>.
vthacker edited a comment on pull request #1684:
URL: https://github.com/apache/lucene-solr/pull/1684#issuecomment-687318445


   > The fact is, all of has the same objective: to make the product better.
   
   > he purpose is to ensure that the feature/change is
   
   >    correct
   >    performant/efficient
   >    user-friendly
   
   I think everyone agrees on this. I really wish we can be nicer while giving feedback. We'd be able to collaborate better and keep the focus on the design decisions
   
   What are the current concerns with the current PR?
   1. The verbosity?
   2. Where the config lives?
   
   
   


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/AddReplicasRequest.java
##########
@@ -0,0 +1,62 @@
+/*
+ * 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;
+
+import java.util.Set;
+
+/**
+ * <p>Request for creating one or more {@link Replica}'s for one or more {@link Shard}'s of an existing {@link SolrCollection}.
+ * The shard might or might not already exist, plugin code can easily find out by using {@link SolrCollection#getShards()}
+ * and verifying if the shard name(s) from {@link #getShardNames()} are there.
+ *
+ * <p>As opposed to {@link CreateNewCollectionRequest}, the set of {@link Node}s on which the replicas should be placed
+ * is specified (defaults to being equal to the set returned by {@link Cluster#getLiveNodes()}).
+ *
+ * <p>There is no extension between this interface and {@link CreateNewCollectionRequest} in either direction
+ * or from a common ancestor for readability. An ancestor could make sense and would be an "abstract interface" not intended
+ * to be implemented directly, but this does not exist in Java.
+ *
+ * <p>Plugin code would likely treat the two types of requests differently since here existing {@link Replica}'s must be taken
+ * into account for placement whereas in {@link CreateNewCollectionRequest} no {@link Replica}'s are assumed to exist.
+ */
+public interface AddReplicasRequest extends Request {
+  /**
+   * The {@link SolrCollection} to add {@link Replica}(s) to. The replicas are to be added to a shard that might or might
+   * not yet exist when the plugin's {@link PlacementPlugin#computePlacement} is called.
+   */
+  SolrCollection getCollection();
+
+  /**
+   * <p>Shard name(s) for which new replicas placement should be computed. The shard(s) might exist or not (that's why this
+   * method returns a {@link Set} of {@link String}'s and not directly a set of {@link Shard} instances).
+   *
+   * <p>Note the Collection API allows specifying the shard name or a {@code _route_} parameter. The Solr implementation will
+   * convert either specification into the relevant shard name so the plugin code doesn't have to worry about this.
+   */
+  Set<String> getShardNames();
+
+  /** Replicas should only be placed on nodes from the set returned by this method. */
+  Set<Node> getTargetNodes();

Review comment:
       My motivation here was to not have the plugin worry about it and if no specific set of subnodes was passed, just make this set equivalent to all live nodes. That way no specific logic on the plugin side for dealing with this.




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/gumi/CreateCollectionRequest.java
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.cloud.gumi;
+
+/**
+ * <p>Request for creating a new collection with a given number of shards and replication factor for various replica types.</p>
+ *
+ * Note there is no need at this stage to allow the request to convey each shard hash range for example, this can be handled
+ * by the Solr side implementation without needing the plugin to worry about it.
+ *
+ */
+public interface CreateCollectionRequest extends Request {
+  String getCollectionName();
+
+  int getShardCount();
+
+  int getNRTReplicationFactor();

Review comment:
       Not sure we need the count here. This is a creation request for a new collection, not a representation of the cluster state.
   If you look at the [Shard](https://github.com/apache/lucene-solr/pull/1684/files#diff-9633f5e169fa3095062451599daac213) class, that's where cluster state is passed to the plugin and there we do have access to the replicas.




----------------------------------------------------------------
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] noblepaul edited a comment on pull request #1684: SOLR-14613: strongly typed placement plugin interface and implementation

Posted by GitBox <gi...@apache.org>.
noblepaul edited a comment on pull request #1684:
URL: https://github.com/apache/lucene-solr/pull/1684#issuecomment-686359748


   > By all means, if you disagree so strongly with the approach presented here then please do so
   
   I clearly have objections to the verbosity of the proposals here. This does not mean that what I propose will be perfect. Nobody can design "the perfect system". A perfect system does not exist. I'm trying to give suggestions so that we can eliminate those obvious problems.
   
   At this point, two of you are invested in this feature and I totally understand your compulsions. 
   
   > The impl. can be barebones, but the API that we define is going to stay with us for the next couple years at least, so it pays off to think seriously how to do it in an elegant and efficient way.
   
   What I see is a largescale undertaking to rewrite everything that we have in Solr . How else will you justify the presence of a class such as this https://github.com/apache/lucene-solr/blob/e385e7083bd4e0263e563d35555cf0cb53b0e03e/solr/core/src/java/org/apache/solr/cluster/placement/Replica.java
   in  a framework that is designed to just find out nodes to place replicas in a cluster?
   
   We do not want to support such a massive framework . 
   I strongly recommend that we do not make duplicates for what we have today just for this particular framework. We already have an `AssignStrategy` interface. AFAIK, the only place where we have problem is in the `NodeStateProvider` . Let's add a method `NodeStateProvider#getPropertyFetcher()` and a new interface `NodePropertyFetcher`  and start reusing the `AssignStrategy` interface itself. That way we do not need any new framework.  Most likely there won't be too many implementations of this framework. Now, if you wish me to make a PR with this suggestion, I'm happy to do that
   
   Just in case you wish to load this plugin from packages , My PR (#1684 ) has the necessary code to load this prom package. (That's optional at this point)
   


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/SolrCollection.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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;
+
+import java.util.Map;
+
+/**
+ * Represents a Collection in SolrCloud (unrelated to {@link java.util.Collection} that uses the nicer name).
+ */
+public interface SolrCollection {
+  /**
+   * The collection name (value passed to {@link Cluster#getCollection(String)}).
+   */
+  String getName();
+
+  /**
+   * <p>The {@link Shard}'s over which the data of this {@link SolrCollection} is distributed.
+   *
+   * <p>The map is from {@link Shard#getShardName()} to {@link Shard} instance.
+   */
+  Map<String, Shard> getShards();

Review comment:
       If it's an Iterator then I think it should iterate on names only, or maybe have two iterators, one for names and one for Shard-s? I'm beginning to like the `SimpleMap` model from @noblepaul because it gives you a getter + iterators for names and/or values in one abstraction (the name of that class is confusing though ...).




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/impl/PluginInteractionsImpl.java
##########
@@ -0,0 +1,190 @@
+/*

Review comment:
       I clearly wanted to reduce the number of **files** because somehow it became an issue...
   
   All the implementation classes are already in an `*.impl` package.




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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


   I am closing this PR now. Will create a new one with the up to date state of the code as resulting from the discussions here, but things became way too slow to still be usable in this PR.


----------------------------------------------------------------
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 closed pull request #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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


   


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/Cluster.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * <p>A representation of the (initial) cluster state, providing information on which nodes are part of the cluster and a way
+ * to get to more detailed info.
+ *
+ * <p>This instance can also be used as a {@link PropertyValueSource} if {@link PropertyKey}'s need to be specified with
+ * a global cluster target.
+ */
+public interface Cluster extends PropertyValueSource {
+  /**
+   * @return current set of live nodes. Never <code>null</code>, never empty (Solr wouldn't call the plugin if empty
+   * since no useful work could then be done).
+   */
+  Set<Node> getLiveNodes();
+
+  /**
+   * <p>Returns info about the given collection if one exists. Because it is not expected for plugins to request info about
+   * a large number of collections, requests can only be made one by one.
+   *
+   * <p>This is also the reason we do not return a {@link java.util.Map} or {@link Set} of {@link SolrCollection}'s here: it would be
+   * wasteful to fetch all data and fill such a map when plugin code likely needs info about at most one or two collections.
+   */
+  Optional<SolrCollection> getCollection(String collectionName) throws IOException;
+
+  /**
+   * <p>Allows getting all {@link SolrCollection} present in the cluster.
+   *
+   * <p><b>WARNING:</b> this call might be extremely inefficient on large clusters. Usage is discouraged.
+   */
+  Set<SolrCollection> getAllCollections();

Review comment:
       Side note: I need (and plan to eventually make) SolrCloud to scale to hundreds of thousand collections. Anything that’s in O(n) or worse in number of collections will not fly for this scale. Implementations can be inefficient (and can change) but let’s try to keep interfaces efficient.




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/Cluster.java
##########
@@ -0,0 +1,46 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * <p>A representation of the (initial) cluster state, providing information on which nodes are part of the cluster and a way
+ * to get to more detailed info.
+ *
+ * <p>This instance can also be used as a {@link PropertyValueSource} if {@link PropertyKey}'s need to be specified with
+ * a global cluster target.
+ */
+public interface Cluster extends PropertyValueSource {
+  /**
+   * @return current set of live nodes. Never <code>null</code>, never empty (Solr wouldn't call the plugin if empty
+   * since no useful could then be done).
+   */
+  Set<Node> getLiveNodes();
+
+  /**
+   * <p>Returns info about the given collection if one exists. Because it is not expected for plugins to request info about
+   * a large number of collections, requests can only be made one by one.
+   *
+   * <p>This is also the reason we do not return a {@link java.util.Map} or {@link Set} of {@link SolrCollection}'s here: it would be
+   * wasteful to fetch all data and fill such a map when plugin code likely needs info about at most one or two collections.
+   */
+  Optional<SolrCollection> getCollection(String collectionName) throws IOException;

Review comment:
       Ok. I don't really see a use case for this (interested if you have something specific in mind) but will add. My thinking was that plugins will be interested in the Collection they need to compute placement for and other specific collections if "their" collection properties reference another collection (for example something along the lines of `withCollection`, even though we remove it, a plugin could reimplement).




----------------------------------------------------------------
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] chatman commented on pull request #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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


   Legacy can be hard-coded. It will be used Unless overridden by solr.xml (to
   use something else).
   
   On Mon, 31 Aug, 2020, 11:41 am Ilan Ginzburg, <no...@github.com>
   wrote:
   
   > If we hard code a default plugin, how do users get to chose if they’d
   > rather keep legacy or rules based assign instead? How do they pass
   > parameters to the plugin?
   >
   > —
   > You are receiving this because you were mentioned.
   > Reply to this email directly, view it on GitHub
   > <https://github.com/apache/lucene-solr/pull/1684#issuecomment-683580121>,
   > or unsubscribe
   > <https://github.com/notifications/unsubscribe-auth/ABDCR5GWCP3P7GUEN745ZTDSDM5I3ANCNFSM4PC3HDKQ>
   > .
   >
   


----------------------------------------------------------------
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] noblepaul edited a comment on pull request #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

Posted by GitBox <gi...@apache.org>.
noblepaul edited a comment on pull request #1684:
URL: https://github.com/apache/lucene-solr/pull/1684#issuecomment-683488170


   >There's no other place to set default config (not hard code!) for which placement plugin to use 
   
   What is wrong with hard coding?
   
   >t's easy to have a single PropertyValue interface with all possible getter methods and have all but one return an empty optional. 
   
   Easy ? The whole code looks pretty bad with a million properties and factories. This looks like we are building a J2EE project. Let's make life simple
   
   IMHO, you have made this like EJBs . I do not want this PR to be merged in the current form and torment our users


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/ReplicaPlacement.java
##########
@@ -0,0 +1,29 @@
+/*
+ * 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;
+
+/**
+ * <p>Placement decision for a single {@link Replica}. Note this placement decision is used as part of a {@link WorkOrder},
+ * it does not directly lead to the plugin code getting a corresponding {@link Replica} instance, nor does it require the
+ * plugin to provide a {@link Shard} instance (the plugin code gets such instances for existing replicas and shards in the
+ * cluster but does not create them directly for adding new replicas for new or existing shards).
+ *
+ * <p>Captures the {@link Shard} (via the shard name), {@link Node} and {@link Replica.ReplicaType} of a Replica to be created.
+ */
+public interface ReplicaPlacement {

Review comment:
       It is already tracked. As the plugin creates and returns a `WorkOrder` to tell Solr where to do what, it passes a reference to the `Request`. Solr code would be totally able to log the request and the corresponding placement decisions. `ReplicaPlacement` instances do not exist outside a `WorkOrder` in Solr (they do in plugin code but if not added to a `WorkOrder` it's as if they were not created).




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/Cluster.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * <p>A representation of the (initial) cluster state, providing information on which nodes are part of the cluster and a way
+ * to get to more detailed info.
+ *
+ * <p>This instance can also be used as a {@link PropertyValueSource} if {@link PropertyKey}'s need to be specified with
+ * a global cluster target.
+ */
+public interface Cluster extends PropertyValueSource {
+  /**
+   * @return current set of live nodes. Never <code>null</code>, never empty (Solr wouldn't call the plugin if empty
+   * since no useful work could then be done).
+   */
+  Set<Node> getLiveNodes();
+
+  /**
+   * <p>Returns info about the given collection if one exists. Because it is not expected for plugins to request info about
+   * a large number of collections, requests can only be made one by one.
+   *
+   * <p>This is also the reason we do not return a {@link java.util.Map} or {@link Set} of {@link SolrCollection}'s here: it would be
+   * wasteful to fetch all data and fill such a map when plugin code likely needs info about at most one or two collections.
+   */
+  Optional<SolrCollection> getCollection(String collectionName) throws IOException;
+
+  /**
+   * <p>Allows getting all {@link SolrCollection} present in the cluster.
+   *
+   * <p><b>WARNING:</b> this call might be extremely inefficient on large clusters. Usage is discouraged.
+   */
+  Set<SolrCollection> getAllCollections();

Review comment:
       I meant just the list of names ... `Collection<String>`, otherwise I agree it can be very inefficient.

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PlacementPlanFactory.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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;
+
+import java.util.Set;
+
+/**
+ * Allows plugins to create {@link PlacementPlan}s telling the Solr layer where to create replicas following the processing of
+ * a {@link PlacementRequest}. The Solr layer can (and will) check that the {@link PlacementPlan} conforms to the {@link PlacementRequest} (and
+ * if it does not, the requested operation will fail).
+ */
+public interface PlacementPlanFactory {
+  /**
+   * <p>Creates a {@link PlacementPlan} for adding a new collection and its replicas.
+   *
+   * <p>This is in support of {@link org.apache.solr.cloud.api.collections.CreateCollectionCmd}.
+   */
+  PlacementPlan createPlacementPlanNewCollection(CreateNewCollectionPlacementRequest request, String CollectionName, Set<ReplicaPlacement> replicaPlacements);
+
+  /**
+   * <p>Creates a {@link PlacementPlan} for adding replicas to a given shard of an existing collection.
+   *
+   * <p>This is in support (directly or indirectly) of {@link org.apache.solr.cloud.api.collections.AddReplicaCmd},
+   * {@link org.apache.solr.cloud.api.collections.CreateShardCmd}, {@link org.apache.solr.cloud.api.collections.ReplaceNodeCmd},
+   * {@link org.apache.solr.cloud.api.collections.MoveReplicaCmd}, {@link org.apache.solr.cloud.api.collections.SplitShardCmd},
+   * {@link org.apache.solr.cloud.api.collections.RestoreCmd} and {@link org.apache.solr.cloud.api.collections.MigrateCmd}.
+   * (as well as of {@link org.apache.solr.cloud.api.collections.CreateCollectionCmd} in the specific case of
+   * {@link org.apache.solr.common.params.CollectionAdminParams#WITH_COLLECTION} but this should be removed shortly and
+   * the section in parentheses of this comment should be removed when the {@code withCollection} javadoc link appears broken).
+   */
+  PlacementPlan createPlacementPlanAddReplicas(AddReplicasPlacementRequest request, String CollectionName, Set<ReplicaPlacement> replicaPlacements);

Review comment:
       Should we include MOVEREPLICA here, too? Some placement plans need to know in advance whether it's an unrelated ADD or a part of MOVE operation - eg. if the replica files are on a shared FS then MOVE is not equivalent to ADD + DELETE because the actual files are not being moved.
   
   So I think we need both `createAddReplicas` and `createMoveReplicas` and `createDeleteReplicas`.

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PlacementRequest.java
##########
@@ -0,0 +1,28 @@
+/*
+ * 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;
+
+/**
+ * A cluster related placement request that Solr asks a {@link PlacementPlugin} plugin to resolve and compute a {@link PlacementPlan} for.
+ */
+public interface PlacementRequest {
+  /**
+   * "Unique" request ID that can be used for logging in the plugin code and that will also be used in logs on the Solr side.
+   */
+  String getUniqueRequestId();

Review comment:
       `getRequestId`. Javadoc can clarify that it's unique.

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PlacementPlanFactory.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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;
+
+import java.util.Set;
+
+/**
+ * Allows plugins to create {@link PlacementPlan}s telling the Solr layer where to create replicas following the processing of
+ * a {@link PlacementRequest}. The Solr layer can (and will) check that the {@link PlacementPlan} conforms to the {@link PlacementRequest} (and
+ * if it does not, the requested operation will fail).
+ */
+public interface PlacementPlanFactory {
+  /**
+   * <p>Creates a {@link PlacementPlan} for adding a new collection and its replicas.
+   *
+   * <p>This is in support of {@link org.apache.solr.cloud.api.collections.CreateCollectionCmd}.
+   */
+  PlacementPlan createPlacementPlanNewCollection(CreateNewCollectionPlacementRequest request, String CollectionName, Set<ReplicaPlacement> replicaPlacements);
+
+  /**
+   * <p>Creates a {@link PlacementPlan} for adding replicas to a given shard of an existing collection.
+   *
+   * <p>This is in support (directly or indirectly) of {@link org.apache.solr.cloud.api.collections.AddReplicaCmd},
+   * {@link org.apache.solr.cloud.api.collections.CreateShardCmd}, {@link org.apache.solr.cloud.api.collections.ReplaceNodeCmd},
+   * {@link org.apache.solr.cloud.api.collections.MoveReplicaCmd}, {@link org.apache.solr.cloud.api.collections.SplitShardCmd},
+   * {@link org.apache.solr.cloud.api.collections.RestoreCmd} and {@link org.apache.solr.cloud.api.collections.MigrateCmd}.
+   * (as well as of {@link org.apache.solr.cloud.api.collections.CreateCollectionCmd} in the specific case of
+   * {@link org.apache.solr.common.params.CollectionAdminParams#WITH_COLLECTION} but this should be removed shortly and
+   * the section in parentheses of this comment should be removed when the {@code withCollection} javadoc link appears broken).
+   */
+  PlacementPlan createPlacementPlanAddReplicas(AddReplicasPlacementRequest request, String CollectionName, Set<ReplicaPlacement> replicaPlacements);
+
+  /**
+   * Creates a {@link ReplicaPlacement} needed to be passed to some/all {@link PlacementPlan} factory methods.
+   */
+  ReplicaPlacement createReplicaPlacement(String shardName, Node node, Replica.ReplicaType replicaType);

Review comment:
       Do we need to expose this as a separate public method? I would expect this to be an internal detail of the implementation.

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PlacementPlugin.java
##########
@@ -0,0 +1,41 @@
+/*
+ * 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;
+
+/**
+ * Implemented by external plugins to control replica placement and movement on the search cluster (as well as other things
+ * such as cluster elasticity?) when cluster changes are required (initiated elsewhere, most likely following a Collection
+ * API call).
+ */
+public interface PlacementPlugin {

Review comment:
       >  get on that map, very low cost here as well
   Oh, I wasn't worried about that .. My point was that the plugin may need to initialize costly internal structures based on the config (like fetching some additional params from an external system to eg. verify license constraints or cost schedule for allocating additional resources).

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/plugins/SamplePluginMinimizeCores.java
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.Comparator;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Set;
+import java.util.Map;
+
+import com.google.common.collect.Ordering;
+import com.google.common.collect.TreeMultimap;
+import org.apache.solr.cluster.placement.Cluster;
+import org.apache.solr.cluster.placement.CoresCountPropertyValue;
+import org.apache.solr.cluster.placement.CreateNewCollectionPlacementRequest;
+import org.apache.solr.cluster.placement.Node;
+import org.apache.solr.cluster.placement.PlacementException;
+import org.apache.solr.cluster.placement.PlacementPlugin;
+import org.apache.solr.cluster.placement.PropertyKey;
+import org.apache.solr.cluster.placement.PropertyKeyFactory;
+import org.apache.solr.cluster.placement.PropertyValue;
+import org.apache.solr.cluster.placement.PropertyValueFetcher;
+import org.apache.solr.cluster.placement.Replica;
+import org.apache.solr.cluster.placement.ReplicaPlacement;
+import org.apache.solr.cluster.placement.PlacementRequest;
+import org.apache.solr.cluster.placement.PlacementPlan;
+import org.apache.solr.cluster.placement.PlacementPlanFactory;
+import org.apache.solr.common.util.SuppressForbidden;
+
+/**
+ * Implements placing replicas to minimize number of cores per {@link Node}, while not placing two replicas of the same
+ * shard on the same node.
+ *
+ * TODO: code not tested and never run, there are no implementation yet for used interfaces
+ */
+public class SamplePluginMinimizeCores implements PlacementPlugin {
+
+  @SuppressForbidden(reason = "Ordering.arbitrary() has no equivalent in Comparator class. Rather reuse than copy.")
+  public PlacementPlan computePlacement(Cluster cluster, PlacementRequest placementRequest, PropertyKeyFactory propertyFactory,
+                                        PropertyValueFetcher propertyFetcher, PlacementPlanFactory placementPlanFactory) throws PlacementException {
+    // This plugin only supports Creating a collection.
+    if (!(placementRequest instanceof CreateNewCollectionPlacementRequest)) {
+      throw new PlacementException("This toy plugin only supports creating collections");
+    }
+
+    final CreateNewCollectionPlacementRequest reqCreateCollection = (CreateNewCollectionPlacementRequest) placementRequest;
+
+    final int totalReplicasPerShard = reqCreateCollection.getNrtReplicationFactor() +
+        reqCreateCollection.getTlogReplicationFactor() + reqCreateCollection.getPullReplicationFactor();
+
+    if (cluster.getLiveNodes().size() < totalReplicasPerShard) {
+      throw new PlacementException("Cluster size too small for number of replicas per shard");
+    }
+
+    // Get number of cores on each Node
+    TreeMultimap<Integer, Node> nodesByCores = TreeMultimap.create(Comparator.naturalOrder(), Ordering.arbitrary());
+
+    // Get the number of cores on each node and sort the nodes by increasing number of cores
+    for (Node node : cluster.getLiveNodes()) {
+      // TODO: redo this. It is potentially less efficient to call propertyFetcher.getProperties() multiple times rather than once
+      final PropertyKey coresCountPropertyKey = propertyFactory.createCoreCountKey(node);
+      Map<PropertyKey, PropertyValue> propMap = propertyFetcher.fetchProperties(Collections.singleton(coresCountPropertyKey));
+      PropertyValue returnedValue = propMap.get(coresCountPropertyKey);
+      if (returnedValue == null) {
+        throw new PlacementException("Can't get number of cores in " + node);

Review comment:
       This doesn't have to be a fatal error - in some edge-cases the node could have gone down between the calls to `getLiveNodes` and `fetchProperties`, it could just mean that it's not eligible for placement.

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PropertyKeyFactory.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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;
+
+/**
+ * Factory used by the plugin to create property keys to request property values from Solr.<p>
+ *
+ * Building of a {@link PropertyKey} requires specifying the target (context) from which the value of that key should be
+ * obtained. This is done by specifying the appropriate {@link PropertyValueSource}.<br>
+ * For clarity, when only a single type of target is acceptable, the corresponding subtype of {@link PropertyValueSource} is used instead
+ * (for example {@link Node}).
+ */
+public interface PropertyKeyFactory {
+  /**
+   * Returns a property key to request the number of cores on a {@link Node}.
+   */
+  PropertyKey createCoreCountKey(Node node);
+
+  /**
+   * Returns a property key to request disk related info on a {@link Node}.
+   */
+  PropertyKey createDiskInfoKey(Node node);
+
+  /**
+   * Returns a property key to request the value of a system property on a {@link Node}.
+   * @param systemPropertyName the name of the system property to retrieve.
+   */
+  PropertyKey createSystemPropertyKey(Node node, String systemPropertyName);
+
+  /**
+   * Returns a property key to request the value of a metric.<p>
+   *
+   * Not all metrics make sense everywhere, but metrics can be applied to different objects. For example
+   * <code>SEARCHER.searcher.indexCommitSize</code> would make sense for a given replica of a given shard of a given collection,
+   * and possibly in other contexts.<p>
+   *
+   * @param metricSource The registry of the metric. For example a specific {@link Replica}.
+   * @param metricName for example <code>SEARCHER.searcher.indexCommitSize</code>.
+   */
+  PropertyKey createMetricKey(PropertyValueSource metricSource, String metricName);

Review comment:
       One node usually hosts many replicas. Each of these replicas has a unique registry name, in the form of `solr.core.<replicaName>`, so we could build PropertyKey from Replica because all components of the full metrics name are known.
   
   This is not the case with `node`, `jvm` and `jetty` - I think we need to explicitly specify the registry name in these cases.




----------------------------------------------------------------
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] chatman commented on pull request #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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


   > There’s no direct mapping between work orders and collection API. I consider work orders the basic building blocs from which collection API actions can be built. Creating a collection (adding the data in ZK, checking for the config set etc) is different from adding a replica to an existing collection. Hence the two work orders. The “new collection” doesn’t add any replicas to the collection but prepares the system to accept adding replicas replicas for the collection. Obviously when adding a replica to an existing collection, that preparation work is not required.
   
   Why can't we have a single type of work order that requests for the preparation of n number of replicas (where n is 1 in case of ADDREPLICA and n is numShards*replicationFactor in case of CREATE)?


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/Cluster.java
##########
@@ -0,0 +1,46 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * <p>A representation of the (initial) cluster state, providing information on which nodes are part of the cluster and a way
+ * to get to more detailed info.
+ *
+ * <p>This instance can also be used as a {@link PropertyValueSource} if {@link PropertyKey}'s need to be specified with
+ * a global cluster target.
+ */
+public interface Cluster extends PropertyValueSource {
+  /**
+   * @return current set of live nodes. Never <code>null</code>, never empty (Solr wouldn't call the plugin if empty
+   * since no useful could then be done).
+   */
+  Set<Node> getLiveNodes();
+
+  /**
+   * <p>Returns info about the given collection if one exists. Because it is not expected for plugins to request info about
+   * a large number of collections, requests can only be made one by one.
+   *
+   * <p>This is also the reason we do not return a {@link java.util.Map} or {@link Set} of {@link SolrCollection}'s here: it would be
+   * wasteful to fetch all data and fill such a map when plugin code likely needs info about at most one or two collections.
+   */
+  Optional<SolrCollection> getCollection(String collectionName) throws IOException;

Review comment:
       The new framework does not mandate any specific way to do this. The scenario you describe (partitioning nodes by system properties) is totally possible to implement, then one would be using [PropertyValue.Sysprop](https://github.com/apache/lucene-solr/pull/1684/files#diff-d4ace968d26586330e5b20a0b698e9a7R125) to implement that.




----------------------------------------------------------------
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] noblepaul commented on pull request #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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


   I don't believe we should have a set of interfaces that duplicate existing classes just for this functionality. This is a common mistake that we all do. When we design a feature we think that is the most important thing. We endup over designing and over engineering things. This feature will remain a tiny part of Solr. Anyone who wishes to implement this should not require to learn a lot before even getting started.  Let's try to have a minimal set of interfaces so that people who try to implement them do not have a huge learning cure. 
   
   Let's try to understand the requirement
   
   * Solr wants a set of positions to place a few replicas
   * The implementation wants to know what is the current state of the cluster so that it can make those decisions
   
   24 interfaces to do this is definitely over engineering
   
   


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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


   > I totally disagree that my proposal will be not performant. It has zero impact on performance whatsoever.
   
   Here's an example @noblepaul of how your proposal would be used (and if I misunderstood something, please correct). nodes are the cluster nodes of interest:
   
   ```
   for (Node node : nodes) {
       PropertyFetcher pf = someFactory.createPropertyFetcher();
       pf.onNode(node).withSystemProperty("AvailabilityZone").withCoresCount();
       PropertyValues pv = pf.fetchValues();
       // Use the values pv.coreCount() and pv.systemProperty("AvailabilityZone") if they got returned, store them somewhere...
   }
   ```
   The calls to `fetchValues()` are sequential in the plugin code. The underlying implementation **cannot** make them concurrent because it gets them one by one and needs to answer first.
   If a plugin wants to fetch in parallel (say a large cluster), there's complexity involved in implementing parallel fetches: create threads, start them, join then and collect results, dispose of the threads... and we likely prefer not to have plugin start to create threads in uncontrolled ways.
   
   That's why I say it's inefficient. It's not the fetching itself that would identical to the proposal in the PR.
   
   The proposal in the PR is the only way I've found to push the complexity of multithreading or other to the implementation (that **we write**, not plugin implementors) and make plugins dev easier and immediately efficient (or not efficient, but then we can improve our implementation).
   It also covers the case when plugin code doesn't know which node to send the request to (for example once we add  `withDocCount(collection, shard)` there's nothing telling which node this should go to, yet a specific node has to be picked to serve that data and I want this request to be "batchable" with other requests for data to the node as above).
   
   > The only issue I see with that is "it's not strongly typed". It's a matter of aesthetics at this point.
   
   Your proposal above **is** strongly typed so from that perspective I'm fine with it. If you find a way with this proposal to **group the fetching of everything from each node in one go** (the `withDocCount` issue that we don't know where it should go), and **if there's a way for the framework - **not the plugin writer** - to invest and make this efficient** I'm okay with it.
   
   
   If really what you don't like is the number of PropertyValue interfaces (for the record, they have just one method each) i'm willing to use only a single per return type, or even a single one having getters returning Optional<type> for all supported types (`HardwareType` for disk is one, unless we want to return a string for it, then of course `Integer`, `Long`, `Double`, `String`). This will save a few interfaces (3 exactly as of now, but then when we add new types of keys the interface will likely already be available), and it will simplify (to a lesser degree) the implementations because even though two property values might return the same type, the value is not extracted in the same way. If you look at `AbstractNodePropertyKey` in the PR, the constructor is where the snitch type is matched, so if we have fewer implementation we'll have to move that code to some other place.


----------------------------------------------------------------
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] noblepaul commented on pull request #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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


   So, I have two main  problems with this PR
   
   1. Using `solr.xml` 
   2. having 2 interfaces for each property


----------------------------------------------------------------
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] chatman edited a comment on pull request #1684: SOLR-14613: strongly typed placement plugin interface and implementation

Posted by GitBox <gi...@apache.org>.
chatman edited a comment on pull request #1684:
URL: https://github.com/apache/lucene-solr/pull/1684#issuecomment-687329334


   I withdraw all outstanding concerns. Verbosity, clunkiness/ineffectiveness/misplacement of configuration
   etc are all my "perceptions" that I don't want to come in the way of the completion of this effort.


----------------------------------------------------------------
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] noblepaul commented on pull request #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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


   > By all means, if you disagree so strongly with the approach presented here then please do so
   
   I clearly have objections to the verbosity of the proposals here. This does not mean that what I propose will be perfect. Nobody can design "the perfect system". A perfect system does not exist. I'm trying to give suggestions so that we can eliminate those obvious problems.
   
   At this point, two of you are invested in this feature and I totally understand your compulsions. 
   
   > The impl. can be barebones, but the API that we define is going to stay with us for the next couple years at least, so it pays off to think seriously how to do it in an elegant and efficient way.
   
   What I see is a largescale undertaking to rewrite everything that we have in Solr . How else will you justify the presence of a class such as this https://github.com/apache/lucene-solr/blob/e385e7083bd4e0263e563d35555cf0cb53b0e03e/solr/core/src/java/org/apache/solr/cluster/placement/Replica.java
   in  a framework that is designed to just find out nodes to place replicas in a cluster?
   
   We do not want to support such a massive framework . 
   I strongly recommend that we do not make duplicates for what we have today just for this particular framework. We already have an `AssignStrategy` interface. AFAIK, the only place where we have problem is in the `NodeStateProvider` . Let's add a method `SolrCloudManager#getNodepropertyFetcher()` and a `NodePropertyFetcher` interface and start reusing the `AssignStrategy` interface itself. That way we do not need any new framework.  Most likely there won't be too many implementations of this framework. 
   
   Just in case you wish to load this plugin from packages , My PR (#1684 ) has the necessary code to load this prom package. (That's optional at this point)
   


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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


   If we hard code a default plugin, how do users get to chose if they’d rather keep legacy or rules based assign instead? How do they pass parameters to the plugin?
   


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/gumi/PropertyKeyFactory.java
##########
@@ -0,0 +1,34 @@
+/*
+ * 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.cloud.gumi;
+
+/**
+ * Factory used by the plugin to create property keys to request property values from Solr
+ */
+public interface PropertyKeyFactory {
+  /**
+   * Returns a property key allowing to request the number of cores. There are no parameters for this key.
+   */
+  CoresCountPropertyKey createCoreCountKey();
+
+  /**
+   * Returns a property key allowing to request the value of a system property (sorry property used twice in two different
+   * contexts). The parameter is the name of the system property to retrieve.
+   */
+  SystemPropertyPropertyKey createSystemPropertyKey(String systemPropertyName);

Review comment:
       Sure, this is just the very initial drop to show the overall approach. By no means complete! Actually I'll go through all the Collection API commands as well as the Autoscaling tests to see what was used and needs to be added to these 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 #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/gumi/CreateCollectionRequest.java
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.cloud.gumi;
+
+/**
+ * <p>Request for creating a new collection with a given number of shards and replication factor for various replica types.</p>
+ *
+ * Note there is no need at this stage to allow the request to convey each shard hash range for example, this can be handled
+ * by the Solr side implementation without needing the plugin to worry about it.
+ *
+ */
+public interface CreateCollectionRequest extends Request {
+  String getCollectionName();
+
+  int getShardCount();
+
+  int getNRTReplicationFactor();

Review comment:
       Not sure we need the count here. This is a creation request for a new collection, not a representation of the cluster state.
   If you look at the `[Shard](https://github.com/apache/lucene-solr/pull/1684/files#diff-9633f5e169fa3095062451599daac213)` class, that's where cluster state is passed to the plugin and there we do have access to the replicas.




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/ReplicaPlacement.java
##########
@@ -0,0 +1,29 @@
+/*
+ * 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;
+
+/**
+ * <p>Placement decision for a single {@link Replica}. Note this placement decision is used as part of a {@link WorkOrder},
+ * it does not directly lead to the plugin code getting a corresponding {@link Replica} instance, nor does it require the
+ * plugin to provide a {@link Shard} instance (the plugin code gets such instances for existing replicas and shards in the
+ * cluster but does not create them directly for adding new replicas for new or existing shards).
+ *
+ * <p>Captures the {@link Shard} (via the shard name), {@link Node} and {@link Replica.ReplicaType} of a Replica to be created.
+ */
+public interface ReplicaPlacement {

Review comment:
       Solr code will not be using these interfaces. This will be clearer once I'm back from vacation and implement the solr side of things. Solr will only be using the concrete implementations of the interfaces so will have access to everything it needs. 




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/Node.java
##########
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+/**
+ * Representation of a SolrCloud node or server in the SolrCloud cluster.
+ */
+public interface Node extends PropertyValueSource {
+  String getNodeName();

Review comment:
       `getName`? we already know it's a node.

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/Cluster.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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;

Review comment:
       Eventually we can put it in `org.apache.solr.cluster` (maybe replace `SolrCluster`).

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/AddReplicasPlacementRequest.java
##########
@@ -0,0 +1,64 @@
+/*
+ * 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;
+
+import java.util.Set;
+
+/**
+ * <p>Request passed by Solr to a {@link PlacementPlugin} to compute placement for one or more {@link Replica}'s for one
+ * or more {@link Shard}'s of an existing {@link SolrCollection}.
+ * The shard might or might not already exist, plugin code can easily find out by using {@link SolrCollection#getShards()}
+ * and verifying if the shard name(s) from {@link #getShardNames()} are there.
+ *
+ * <p>The set of {@link Node}s on which the replicas should be placed
+ * is specified (defaults to being equal to the set returned by {@link Cluster#getLiveNodes()}).
+ */
+public interface AddReplicasPlacementRequest extends PlacementRequest {
+  /**
+   * The {@link SolrCollection} to add {@link Replica}(s) to.
+   */
+  SolrCollection getCollection();
+
+  /**
+   * <p>Shard name(s) for which new replicas placement should be computed. The shard(s) might exist or not (that's why this
+   * method returns a {@link Set} of {@link String}'s and not directly a set of {@link Shard} instances).
+   *
+   * <p>Note the Collection API allows specifying the shard name or a {@code _route_} parameter. The Solr implementation will
+   * convert either specification into the relevant shard name so the plugin code doesn't have to worry about this.
+   */
+  Set<String> getShardNames();
+
+  /**
+   * <p>Replicas should only be placed on nodes in the set returned by this method.
+   *
+   * <p>When Collection API calls do not specify a specific set of target nodes, replicas can be placed on any live node of
+   * the cluster. In such cases, this set will be equal to the set of all live nodes. The plugin placement code does not
+   * need to worry (or care) if a set of nodes was explicitly specified or not.
+   *
+   * @return never {@code null} and never empty set (if that set was to be empty for any reason, no placement would be
+   * possible and the Solr infrastructure driving the plugin code would detect the error itself rather than calling the plugin).
+   */
+  Set<Node> getTargetNodes();

Review comment:
       I'm concerned about the memory impact here ... consider a cluster of 10,000 nodes :) Maybe it's enough to return the set of node names (strings)? Also, a common scenario is that the requestor doesn't care about specific nodes, so defining that "empty set == all live nodes" makes more sense to me...

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/impl/AbstractPropertyValue.java
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.cluster.placement.PropertyKey;
+import org.apache.solr.cluster.placement.PropertyValue;
+
+/**
+ * This class and static subclasses implement all subtypes of {@link PropertyValue}
+ */
+public abstract class AbstractPropertyValue implements PropertyValue {

Review comment:
       See my other comments about simplifying this hierarchy at the cost of relaxing the strong-typing here.

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PropertyValue.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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;
+
+/**
+ *  <p>The value corresponding to a specific {@link PropertyKey}, in a specific context (e.g. property of a specific
+ *  {@link Node} instance). The context is tracked in the {@link PropertyKey} using a {@link PropertyValueSource}.
+ *
+ *  <p>Instances are obtained by first getting a key using {@link PropertyKeyFactory} then getting the corresponding
+ *  {@link PropertyValue} using {@link PropertyValueFetcher}.
+ */
+public interface PropertyValue {
+  /**
+   * The property key used for retrieving this property value.
+   */
+  PropertyKey getKey();
+
+  /**
+   *  Instances are obtained by first getting a key using {@link PropertyKeyFactory#createCoreCountKey} then calling
+   *  {@link PropertyValueFetcher#fetchProperties}, retrieving the appropriate {@link PropertyValue} from the returned map
+   *  using the {@link PropertyKey} as key and finally casting it to {@link PropertyValue.CoresCount}.
+   */
+  interface CoresCount extends PropertyValue {
+    /**
+     * Returns the number of cores on the {@link Node}) this instance was obtained from (i.e. instance
+     * passed to {@link PropertyKeyFactory#createCoreCountKey(Node)}).
+     */
+    int getCoresCount();
+  }
+
+  /**
+   *  Instances are obtained by first getting a key using {@link PropertyKeyFactory#createDiskTypeKey} then getting the
+   *  {@link org.apache.solr.cluster.placement.PropertyValue.DiskType} using {@link PropertyValueFetcher#fetchProperties} and retrieving (then casting) the
+   *  appropriate {@link PropertyValue} from the returned map using the {@link PropertyKey} as key.
+   */
+  interface DiskType extends PropertyValue {
+    /**
+     * Type of storage hardware used for the partition on which cores are stored on the {@link Node}) from which this instance
+     * was obtained (i.e. instance passed to {@link PropertyKeyFactory#createDiskTypeKey(Node)}).
+     */
+    HardwareType getHardwareType();
+
+    enum HardwareType {
+      SSD, ROTATIONAL, UNKNOWN
+    }
+  }
+
+  /**
+   *  Instances are obtained by first getting a key using {@link PropertyKeyFactory#createTotalDiskKey(Node)} (Node)} then getting the
+   *  {@link org.apache.solr.cluster.placement.PropertyValue.TotalDisk} using {@link PropertyValueFetcher#fetchProperties} and retrieving (then casting) the
+   *  appropriate {@link PropertyValue} fetched from the returned map using the {@link PropertyKey} as key.
+   */
+  interface TotalDisk extends PropertyValue {
+    /**
+     * Total disk size of the partition on which cores are stored on the {@link Node}) from which this instance was obtained
+     * (i.e. instance passed to {@link PropertyKeyFactory#createTotalDiskKey(Node)}).
+     */
+    long getTotalSizeGB();
+  }
+
+  /**
+   *  Instances are obtained by first getting a key using {@link PropertyKeyFactory#createFreeDiskKey(Node)} then getting the
+   *  {@link org.apache.solr.cluster.placement.PropertyValue.FreeDisk} using {@link PropertyValueFetcher#fetchProperties} and retrieving (then casting) the
+   *  appropriate {@link PropertyValue} fetched from the returned map using the {@link PropertyKey} as key.
+   */
+  interface FreeDisk extends PropertyValue {
+    /**
+     * Free disk size of the partition on which cores are stored on the {@link Node}) from which this instance was obtained
+     *  (i.e. instance passed to {@link PropertyKeyFactory#createDiskTypeKey(Node)}).
+     */
+    long getFreeSizeGB();
+  }
+
+
+  /**
+   *  Instances are obtained by first getting a key using {@link PropertyKeyFactory#createHeapUsageKey(Node)} then calling
+   *  {@link PropertyValueFetcher#fetchProperties}, retrieving the appropriate {@link PropertyValue} from the returned map
+   *  using the {@link PropertyKey} as key and finally casting it to {@link org.apache.solr.cluster.placement.PropertyValue.HeapUsage}.
+   */
+  interface HeapUsage extends PropertyValue {
+
+    /**
+     * Percentage between 0 and 100 of used heap over max heap.
+     */
+    double getUsedHeapMemoryUsage();
+  }
+
+  /**
+   * A {@link PropertyValue} representing a metric on the target {@link PropertyValueSource}.
+   *
+   * <p>Instances are obtained by first getting a key using {@link PropertyKeyFactory#createMetricKey(PropertyValueSource, String)}
+   * or {@link PropertyKeyFactory#createMetricKey(Node, String, PropertyKeyFactory.NodeMetricRegistry)} then calling
+   * {@link PropertyValueFetcher#fetchProperties}, retrieving the appropriate {@link PropertyValue} from the returned map
+   * using the {@link PropertyKey} as key and finally casting it to {@link org.apache.solr.cluster.placement.PropertyValue.Metric}.
+   */
+  interface Metric extends PropertyValue {
+    /**
+     * Returns the metric value from the {@link PropertyValueSource} from which it was retrieved.
+     */
+    Double getNumberValue();
+  }
+
+  /**
+   * A {@link PropertyValue} representing a sysprop (or System property) on the target {@link Node}.
+   *
+   *  Instances are obtained by first getting a key using {@link PropertyKeyFactory#createSyspropKey} then calling
+   *  {@link PropertyValueFetcher#fetchProperties}, retrieving the appropriate {@link PropertyValue} from the returned map
+   *  using the {@link PropertyKey} as key and finally casting it to {@link org.apache.solr.cluster.placement.PropertyValue.Sysprop}.
+   */
+  interface Sysprop extends PropertyValue {
+    /**

Review comment:
       +1.

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/Replica.java
##########
@@ -0,0 +1,46 @@
+/*
+ * 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;
+
+/**
+ * An instantiation (or one of the copies) of a given {@link Shard} of a given {@link SolrCollection}.
+ * Objects of this type are returned by the Solr framework to the plugin, they are not directly built by the plugin. When the
+ * plugin wants to add a replica it goes through appropriate method in {@link PlacementPlanFactory}).
+ */
+public interface Replica extends PropertyValueSource {
+  Shard getShard();
+
+  ReplicaType getType();
+  ReplicaState getState();

Review comment:
       SOLR-14680 doesn't include these, but includes `isLeader` (and some other props).

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/Replica.java
##########
@@ -0,0 +1,46 @@
+/*
+ * 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;
+
+/**
+ * An instantiation (or one of the copies) of a given {@link Shard} of a given {@link SolrCollection}.
+ * Objects of this type are returned by the Solr framework to the plugin, they are not directly built by the plugin. When the
+ * plugin wants to add a replica it goes through appropriate method in {@link PlacementPlanFactory}).
+ */
+public interface Replica extends PropertyValueSource {
+  Shard getShard();
+
+  ReplicaType getType();
+  ReplicaState getState();
+
+  String getReplicaName();
+  String getCoreName();
+
+  /**
+   * {@link Node} on which this {@link Replica} is located.
+   */
+  Node getNode();

Review comment:
       Do we need a Node instance or just a node name here?

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PropertyValue.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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;
+
+/**
+ *  <p>The value corresponding to a specific {@link PropertyKey}, in a specific context (e.g. property of a specific
+ *  {@link Node} instance). The context is tracked in the {@link PropertyKey} using a {@link PropertyValueSource}.
+ *
+ *  <p>Instances are obtained by first getting a key using {@link PropertyKeyFactory} then getting the corresponding
+ *  {@link PropertyValue} using {@link PropertyValueFetcher}.
+ */
+public interface PropertyValue {
+  /**
+   * The property key used for retrieving this property value.
+   */
+  PropertyKey getKey();
+
+  /**
+   *  Instances are obtained by first getting a key using {@link PropertyKeyFactory#createCoreCountKey} then calling
+   *  {@link PropertyValueFetcher#fetchProperties}, retrieving the appropriate {@link PropertyValue} from the returned map
+   *  using the {@link PropertyKey} as key and finally casting it to {@link PropertyValue.CoresCount}.
+   */
+  interface CoresCount extends PropertyValue {

Review comment:
       I see the same problem here as with `PropertyKey`. The API surface here becomes very large.
   
   Also, in practice users of this API will always have to do an `instanceof` check and then cast `PropertyValue` to one of its subclasses. Is this significantly better than retrieving a predefined key from a Map and casting an Object to Integer? I'm not so sure, but I'm sure it would be much easier to use.

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/impl/PluginInteractionsImpl.java
##########
@@ -0,0 +1,190 @@
+/*

Review comment:
       That's frankly a very confusing name and there's little reason for grouping these classes here. Why not put all implementation classes separately into an *.impl package?
   
   (I know, this increases the number of files :) but that's directly the function of the number of defined interfaces.)

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PlacementPluginConfig.java
##########
@@ -0,0 +1,60 @@
+/*
+ * 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;
+
+/**
+ * <p>Configuration passed by Solr to {@link PlacementPluginFactory#createPluginInstance(PlacementPluginConfig)} so that plugin instances
+ * ({@link PlacementPlugin}) created by the factory can easily retrieve their configuration.
+ */
+public interface PlacementPluginConfig {

Review comment:
       This in fact could be a general-purpose interface to many other configs that are based on key/value properties. See eg. `ZkNodeProps`.

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/plugins/SamplePluginMinimizeCores.java
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.Comparator;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Set;
+import java.util.Map;
+
+import com.google.common.collect.Ordering;
+import com.google.common.collect.TreeMultimap;
+import org.apache.solr.cluster.placement.Cluster;
+import org.apache.solr.cluster.placement.CoresCountPropertyValue;
+import org.apache.solr.cluster.placement.CreateNewCollectionPlacementRequest;
+import org.apache.solr.cluster.placement.Node;
+import org.apache.solr.cluster.placement.PlacementException;
+import org.apache.solr.cluster.placement.PlacementPlugin;
+import org.apache.solr.cluster.placement.PropertyKey;
+import org.apache.solr.cluster.placement.PropertyKeyFactory;
+import org.apache.solr.cluster.placement.PropertyValue;
+import org.apache.solr.cluster.placement.PropertyValueFetcher;
+import org.apache.solr.cluster.placement.Replica;
+import org.apache.solr.cluster.placement.ReplicaPlacement;
+import org.apache.solr.cluster.placement.PlacementRequest;
+import org.apache.solr.cluster.placement.PlacementPlan;
+import org.apache.solr.cluster.placement.PlacementPlanFactory;
+import org.apache.solr.common.util.SuppressForbidden;
+
+/**
+ * Implements placing replicas to minimize number of cores per {@link Node}, while not placing two replicas of the same
+ * shard on the same node.
+ *
+ * TODO: code not tested and never run, there are no implementation yet for used interfaces
+ */
+public class SamplePluginMinimizeCores implements PlacementPlugin {
+
+  @SuppressForbidden(reason = "Ordering.arbitrary() has no equivalent in Comparator class. Rather reuse than copy.")
+  public PlacementPlan computePlacement(Cluster cluster, PlacementRequest placementRequest, PropertyKeyFactory propertyFactory,
+                                        PropertyValueFetcher propertyFetcher, PlacementPlanFactory placementPlanFactory) throws PlacementException {
+    // This plugin only supports Creating a collection.
+    if (!(placementRequest instanceof CreateNewCollectionPlacementRequest)) {
+      throw new PlacementException("This toy plugin only supports creating collections");
+    }
+
+    final CreateNewCollectionPlacementRequest reqCreateCollection = (CreateNewCollectionPlacementRequest) placementRequest;
+
+    final int totalReplicasPerShard = reqCreateCollection.getNrtReplicationFactor() +
+        reqCreateCollection.getTlogReplicationFactor() + reqCreateCollection.getPullReplicationFactor();
+
+    if (cluster.getLiveNodes().size() < totalReplicasPerShard) {
+      throw new PlacementException("Cluster size too small for number of replicas per shard");
+    }
+
+    // Get number of cores on each Node
+    TreeMultimap<Integer, Node> nodesByCores = TreeMultimap.create(Comparator.naturalOrder(), Ordering.arbitrary());

Review comment:
       I agree with @murblanc about the need for efficient multi-fetching. But I also agree with @noblepaul :) that we could partially relax strong-typing requirements here in order to drastically reduce the API surface.

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/SolrCollection.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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;
+
+import java.util.Map;
+
+/**
+ * Represents a Collection in SolrCloud (unrelated to {@link java.util.Collection} that uses the nicer name).
+ */
+public interface SolrCollection {
+  /**
+   * The collection name (value passed to {@link Cluster#getCollection(String)}).
+   */
+  String getName();
+
+  /**
+   * <p>The {@link Shard}'s over which the data of this {@link SolrCollection} is distributed.
+   *
+   * <p>The map is from {@link Shard#getShardName()} to {@link Shard} instance.
+   */
+  Map<String, Shard> getShards();

Review comment:
       Please note that in some extreme (but real!) deployments the number of shards may be in the order of thousands and the number of replicas in the order of hundreds of thousands... Not so sure about a simple Map here - maybe a list of shard names + getShard accessor?
   
   (Or use a read-through pseudo-map like the one added with SOLR-14680 ?)

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementPluginFactoryFacade.java
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.cluster.placement.PlacementPlugin;
+import org.apache.solr.cluster.placement.PlacementPluginConfig;
+import org.apache.solr.cluster.placement.PlacementPluginFactory;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.core.PluginInfo;
+import org.apache.solr.core.SolrResourceLoader;
+
+/**
+ * <p>The internal class instantiating the configured {@link PlacementPluginFactory} and creating a {@link PlacementPlugin}
+ * instance by passing the the factory the appropriate configuration created from the {@code <placementPluginFactory>}
+ * element in {@code solr.xml}.
+ *
+ * <p>A single instance of {@link PlacementPlugin} is used for all placement computations and therefore must be reentrant.
+ * When configuration changes, a new instance of {@link PlacementPlugin} will be created by calling again
+ * {@link PlacementPluginFactory#createPluginInstance(PlacementPluginConfig)}.
+ */
+public class PlacementPluginFactoryFacade {

Review comment:
       I'm not convinced we need this as a part of the API - instead we could leave the details of `PlacementPlugin` life-cycle management to the `PlacementPluginFactory` implementation.

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PropertyKeyFactory.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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;
+
+/**
+ * Factory used by the plugin to create property keys to request property values from Solr.<p>
+ *
+ * Building of a {@link PropertyKey} requires specifying the target (context) from which the value of that key should be
+ * obtained. This is done by specifying the appropriate {@link PropertyValueSource}.<br>
+ * For clarity, when only a single type of target is acceptable, the corresponding subtype of {@link PropertyValueSource} is used instead
+ * (for example {@link Node}).
+ */
+public interface PropertyKeyFactory {
+  /**
+   * Returns a property key to request the number of cores on a {@link Node}.
+   */
+  PropertyKey createCoreCountKey(Node node);

Review comment:
       I think this part of the API is the most controversial, as it indeed leads to a large number of interfaces and impl. classes.
   
   On one hand this indeed provides the super-strong typing, the burden is on Solr-core to provide the implementation of these interfaces so it doesn't complicate the lives of plugin devs. But on the other hand this may be taking things too far ...
   
   I don't see yet how to simplify it without compromising the strong-typing too much - maybe just use a single `PropertyKey` interface + enum for key types and use one method like `PropertyKey createPropertyKey(PropertyKey.Type type, Node node, Object... params)`? This of course compromises the strong-typing, as it uses weakly-typed params and requires callers to know what parameters to supply ... so I'm not sure. But it would reduce the API surface significantly so maybe it's a compromise worth taking.




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PropertyKeyFactory.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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;
+
+/**
+ * Factory used by the plugin to create property keys to request property values from Solr.<p>
+ *
+ * Building of a {@link PropertyKey} requires specifying the target (context) from which the value of that key should be
+ * obtained. This is done by specifying the appropriate {@link PropertyValueSource}.<br>
+ * For clarity, when only a single type of target is acceptable, the corresponding subtype of {@link PropertyValueSource} is used instead
+ * (for example {@link Node}).
+ */
+public interface PropertyKeyFactory {
+  /**
+   * Returns a property key to request the number of cores on a {@link Node}.
+   */
+  PropertyKey createCoreCountKey(Node node);

Review comment:
       @sigram Keep in mind it's not subclassing just for the pleasure of subclassing. In this example of `createCoreCountKey()`, we end up building an `AbstractNodePropertyKey.CoreCountImpl` and that's where the definition is that this key will correspond to `ImplicitSnitch.CORES`. If we make creation code generic, we'll have to move the actual differences in implementation to somewhere else.
   
   Another key create by `createMetricKey` does even more in `AbstractNodePropertyKey.NodeMetricImpl` since it translates the abstracted registry (from `NodeMetricRegistry`) into the actual registry and assembles the metric name.
   
   In other words, there's specific code to write regardless for each new key type assuming we don't want to expose the internal implementation of Solr (and parameters such as `"metrics:"` and `"totaldisk"` etc used in messages).




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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


   > ```
   >     Iterable<PropertyValues> values =  someFactory.createPropertyFetcher()
   >     .withSystemProperty("AvailabilityZone")
   >    .withCoresCount()
   > .fetchFromAllNodes()
   > ```
   
   @noblepaul this can indeed lead to a simplification if we always fetch the same set of data from all nodes (from which we fetch data, doesn't have to be all nodes in the cluster). I didn't think of it this way and I was trying to design something more flexible. I like the approach above and indeed it can be as efficient (and less verbose) than my proposal.
   
   Can we make this assumption? Plugin requests a set of properties from a set of nodes, and this is done efficiently.
   If so I'll gladly simplify my proposal.


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/Assign.java
##########
@@ -569,14 +574,20 @@ public AssignStrategy create(ClusterState clusterState, DocCollection collection
         case RULES:
           List<Rule> rules = new ArrayList<>();
           for (Object map : ruleMaps) rules.add(new Rule((Map) map));
+          @SuppressWarnings({"rawtypes"})
+          List snitches = (List) collection.get(SNITCH);
           return new RulesBasedAssignStrategy(rules, snitches, clusterState);
+        case PLUGIN_PLACEMENT:
+          // TODO need to decide which plugin class to use. Global config (single plugin for all PLUGIN_PLACEMENT collections?) or per collection config?
+          // TODO hardconding a sample plugin for now. DO NOT MERGE this as is.
+          return new PlacementPluginAssignStrategy(new SamplePluginMinimizeCores());
         default:
           throw new Assign.AssignmentException("Unknown strategy type: " + strategy);
       }
     }
 
     private enum Strategy {
-      LEGACY, RULES;
+      LEGACY, RULES, PLUGIN_PLACEMENT;

Review comment:
       `Strategy` already describes how to perform placement. Maybe we should rename `Strategy` -> `Placement` and simply use `PLUGIN` here?

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/Cluster.java
##########
@@ -0,0 +1,46 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * <p>A representation of the (initial) cluster state, providing information on which nodes are part of the cluster and a way
+ * to get to more detailed info.
+ *
+ * <p>This instance can also be used as a {@link PropertyValueSource} if {@link PropertyKey}'s need to be specified with
+ * a global cluster target.
+ */
+public interface Cluster extends PropertyValueSource {
+  /**
+   * @return current set of live nodes. Never <code>null</code>, never empty (Solr wouldn't call the plugin if empty
+   * since no useful could then be done).

Review comment:
       `no useful` -> `no useful work`

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/SystemPropertyPropertyValue.java
##########
@@ -0,0 +1,28 @@
+/*
+ * 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;
+
+/**
+ * A {@link PropertyValue} representing a System property on the target {@link Node}.
+ */
+public interface SystemPropertyPropertyValue extends PropertyValue {

Review comment:
       Maybe rename it to `SyspropPropertyValue` to avoid this weird repetition?

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/MetricPropertyValue.java
##########
@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+/**
+ * A {@link PropertyValue} representing a metric on the target {@link PropertyValueSource}.
+ * Note there might be overlap with {@link SystemLoadPropertyValue} (only applicable to {@link Node}'s), may need to clarify.
+ */
+public interface MetricPropertyValue extends PropertyValue {
+  /**
+   * Returns the metric value from the {@link PropertyValueSource} on which it was retrieved.
+   * TODO: what type should the metric be? Maybe offer multiple getters for different java types and have each metric implement the right one and throw from the wrong ones? This avoids casting...

Review comment:
       Practically we can encounter just three general types here:
   *  `NumberMetricPropertyValue` for numeric values
   * `StringMetricPropertyValue` for plain strings
   * `MapMetricPropertyValue` for some types of pseudo-metrics (gauges such as eg. system properties), or fully detailed representation of many other metrics (eg. histograms, meters, timers, etc).

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PropertyKeyFactory.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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;
+
+/**
+ * Factory used by the plugin to create property keys to request property values from Solr.<p>
+ *
+ * Building of a {@link PropertyKey} requires specifying the target (context) from which the value of that key should be
+ * obtained. This is done by specifying the appropriate {@link PropertyValueSource}.<br>
+ * For clarity, when only a single type of target is acceptable, the corresponding subtype of {@link PropertyValueSource} is used instead
+ * (for example {@link Node}).
+ */
+public interface PropertyKeyFactory {
+  /**
+   * Returns a property key to request the number of cores on a {@link Node}.
+   */
+  PropertyKey createCoreCountKey(Node node);

Review comment:
       IMHO this model is very limiting .. because every time we add another type of `PropertyKey` we have to change this interface.
   
   How about something like this?
   
   `
   <T> T createPropertyKey(Node node, Class<T extends PropertyKey> propertyClass, String... props)
   `
   
   (or something like that ... not sure about the generics syntax here)

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/AddReplicasRequest.java
##########
@@ -0,0 +1,62 @@
+/*
+ * 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;
+
+import java.util.Set;
+
+/**
+ * <p>Request for creating one or more {@link Replica}'s for one or more {@link Shard}'s of an existing {@link SolrCollection}.
+ * The shard might or might not already exist, plugin code can easily find out by using {@link SolrCollection#getShards()}
+ * and verifying if the shard name(s) from {@link #getShardNames()} are there.
+ *
+ * <p>As opposed to {@link CreateNewCollectionRequest}, the set of {@link Node}s on which the replicas should be placed
+ * is specified (defaults to being equal to the set returned by {@link Cluster#getLiveNodes()}).
+ *
+ * <p>There is no extension between this interface and {@link CreateNewCollectionRequest} in either direction
+ * or from a common ancestor for readability. An ancestor could make sense and would be an "abstract interface" not intended
+ * to be implemented directly, but this does not exist in Java.
+ *
+ * <p>Plugin code would likely treat the two types of requests differently since here existing {@link Replica}'s must be taken
+ * into account for placement whereas in {@link CreateNewCollectionRequest} no {@link Replica}'s are assumed to exist.
+ */
+public interface AddReplicasRequest extends Request {
+  /**
+   * The {@link SolrCollection} to add {@link Replica}(s) to. The replicas are to be added to a shard that might or might
+   * not yet exist when the plugin's {@link PlacementPlugin#computePlacement} is called.
+   */
+  SolrCollection getCollection();
+
+  /**
+   * <p>Shard name(s) for which new replicas placement should be computed. The shard(s) might exist or not (that's why this
+   * method returns a {@link Set} of {@link String}'s and not directly a set of {@link Shard} instances).
+   *
+   * <p>Note the Collection API allows specifying the shard name or a {@code _route_} parameter. The Solr implementation will
+   * convert either specification into the relevant shard name so the plugin code doesn't have to worry about this.
+   */
+  Set<String> getShardNames();
+
+  /** Replicas should only be placed on nodes from the set returned by this method. */
+  Set<Node> getTargetNodes();

Review comment:
       Or null / empty to mean all live nodes?

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/CoresCountPropertyValue.java
##########
@@ -0,0 +1,26 @@
+/*
+ * 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;
+
+public interface CoresCountPropertyValue extends PropertyValue {
+  /**
+   * Returns the number of cores on the {@link Node}) this instance was obtained from (i.e. instance
+   * passed to {@link PropertyKeyFactory#createCoreCountKey(Node)}).
+   */
+  int getCoresCount();

Review comment:
       Later we may consider extending this to report loaded / transient / lazy cores.

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/ReplicaPlacement.java
##########
@@ -0,0 +1,29 @@
+/*
+ * 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;
+
+/**
+ * <p>Placement decision for a single {@link Replica}. Note this placement decision is used as part of a {@link WorkOrder},
+ * it does not directly lead to the plugin code getting a corresponding {@link Replica} instance, nor does it require the
+ * plugin to provide a {@link Shard} instance (the plugin code gets such instances for existing replicas and shards in the
+ * cluster but does not create them directly for adding new replicas for new or existing shards).
+ *
+ * <p>Captures the {@link Shard} (via the shard name), {@link Node} and {@link Replica.ReplicaType} of a Replica to be created.
+ */
+public interface ReplicaPlacement {

Review comment:
       I think we must be more explicit here - the placement refers to Request (very important for debugging and audit of the plugin decisions!), and it should at the very least include the target node.

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PropertyKeyFactory.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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;
+
+/**
+ * Factory used by the plugin to create property keys to request property values from Solr.<p>
+ *
+ * Building of a {@link PropertyKey} requires specifying the target (context) from which the value of that key should be
+ * obtained. This is done by specifying the appropriate {@link PropertyValueSource}.<br>
+ * For clarity, when only a single type of target is acceptable, the corresponding subtype of {@link PropertyValueSource} is used instead
+ * (for example {@link Node}).
+ */
+public interface PropertyKeyFactory {
+  /**
+   * Returns a property key to request the number of cores on a {@link Node}.
+   */
+  PropertyKey createCoreCountKey(Node node);
+
+  /**
+   * Returns a property key to request disk related info on a {@link Node}.
+   */
+  PropertyKey createDiskInfoKey(Node node);
+
+  /**
+   * Returns a property key to request the value of a system property on a {@link Node}.
+   * @param systemPropertyName the name of the system property to retrieve.
+   */
+  PropertyKey createSystemPropertyKey(Node node, String systemPropertyName);
+
+  /**
+   * Returns a property key to request the value of a metric.<p>
+   *
+   * Not all metrics make sense everywhere, but metrics can be applied to different objects. For example
+   * <code>SEARCHER.searcher.indexCommitSize</code> would make sense for a given replica of a given shard of a given collection,
+   * and possibly in other contexts.<p>
+   *
+   * @param metricSource The registry of the metric. For example a specific {@link Replica}.
+   * @param metricName for example <code>SEARCHER.searcher.indexCommitSize</code>.
+   */
+  PropertyKey createMetricKey(PropertyValueSource metricSource, String metricName);

Review comment:
       We have two sets of "global" metrics for a Solr instance: JVM and node-level. I understand we could use Node to select node-level metrics (from `solr.node` registry) but how to select `solr.jvm` metrics? Perhaps we need a `JVMPropertyValueSource`

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/Cluster.java
##########
@@ -0,0 +1,46 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * <p>A representation of the (initial) cluster state, providing information on which nodes are part of the cluster and a way
+ * to get to more detailed info.
+ *
+ * <p>This instance can also be used as a {@link PropertyValueSource} if {@link PropertyKey}'s need to be specified with
+ * a global cluster target.
+ */
+public interface Cluster extends PropertyValueSource {
+  /**
+   * @return current set of live nodes. Never <code>null</code>, never empty (Solr wouldn't call the plugin if empty
+   * since no useful could then be done).
+   */
+  Set<Node> getLiveNodes();
+
+  /**
+   * <p>Returns info about the given collection if one exists. Because it is not expected for plugins to request info about
+   * a large number of collections, requests can only be made one by one.
+   *
+   * <p>This is also the reason we do not return a {@link java.util.Map} or {@link Set} of {@link SolrCollection}'s here: it would be
+   * wasteful to fetch all data and fill such a map when plugin code likely needs info about at most one or two collections.
+   */
+  Optional<SolrCollection> getCollection(String collectionName) throws IOException;

Review comment:
       I think we're missing a method to actually list all collection names. Otherwise the plugins won't be able to make decisions based on more than 1 collection.

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/CreateNewCollectionRequest.java
##########
@@ -0,0 +1,62 @@
+/*
+ * 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;
+
+import java.util.Set;
+
+/**
+ * <p>Request for creating a new collection with a given set of shards and replication factor for various replica types.
+ * The expected {@link WorkOrder} corresponding to this {@link Request} is created using
+ * {@link WorkOrderFactory#createWorkOrderNewCollection}
+ *
+ * <p>Note there is no need at this stage to allow the plugin to know each shard hash range for example, this can be handled
+ * by the Solr side implementation of this interface without needing the plugin to worry about it (the implementation of this interface on
+ * the Solr side can maintain the ranges for each shard).
+ *
+ * <p>Same goes for the {@link org.apache.solr.core.ConfigSet} name or other collection parameters. They are needed for
+ * creating a Collection but likely do not have to be exposed to the plugin (this can easily be changed if needed by
+ * adding accessors here, the underlying Solr side implementation of this interface has the information).
+ */
+public interface CreateNewCollectionRequest extends Request {
+  /**
+   * <p>The name of the collection to be created and for which placement should be computed.
+   *
+   * <p>Compare this method with {@link AddReplicasRequest#getCollection()}, there the collection already exists so can be
+   * directly passed in the {@link Request}.
+   *
+   * <p>When processing this request, plugin code doesn't have to worry about existing {@link Replica}'s for the collection
+   * given that the collection is assumed not to exist.
+   */
+  String getCollectionName();
+
+  Set<String> getShardNames();
+
+  /**
+   * <p>Properties passed through the Collection API by the client creating the collection.
+   * See {@link SolrCollection#getCustomProperty(String)}.
+   *
+   * <p>Given this {@link Request} is for creating a new collection, it is not possible to pass the custom property values through
+   * the {@link SolrCollection} object. That instance does not exist yet, and is the reason {@link #getCollectionName()} exists
+   * rather than a method returning {@link SolrCollection}...
+   */
+  String getCustomProperty(String customPropertyName);

Review comment:
       I'm missing a method to enumerate the available properties, to be able to pass them on to the CREATE command. Solr side can be responsible for acting on these props but it needs to know that they exist - and checking for existence of every possible property seems too tedious.

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PlacementException.java
##########
@@ -0,0 +1,48 @@
+/*
+ * 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;
+
+/**
+ * Exception thrown by a {@link PlacementPlugin} when it is unable to compute placement for whatever reason (except an

Review comment:
       We could make it immediately more useful by providing a required `reason` arg, either an enum (easier to check but hard to extend) or one of the predefined String constants.

##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PropertyKey.java
##########
@@ -0,0 +1,29 @@
+/*
+ * 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;
+
+/**
+ * A property key used by plugins to request values from Solr
+ */
+public interface PropertyKey {
+  /**
+   * @return the target of this {@link PropertyKey}, i.e. from where the corresponding {@link PropertyValue}'s should be
+   * (or were) obtained.
+   */
+  PropertyValueSource getPropertyKeyTarget();

Review comment:
       Uh ... why not use `getPropertyValueSource` or `getTarget` ? mixing the "key", "value", "source" and "target" in one method is very confusing. Javadoc can explain what the method actually does.




----------------------------------------------------------------
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] noblepaul edited a comment on pull request #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

Posted by GitBox <gi...@apache.org>.
noblepaul edited a comment on pull request #1684:
URL: https://github.com/apache/lucene-solr/pull/1684#issuecomment-683488170


   >There's no other place to set default config (not hard code!) for which placement plugin to use 
   
   What is wrong with hard coding?
   
   >t's easy to have a single PropertyValue interface with all possible getter methods and have all but one return an empty optional. 
   
   Easy ? The whole code looks pretty bad with a million properties and factories. This looks like we are building a J2EE project. Let's make life simple
   
   IMHO, this is made this like EJBs . I do not want this PR to be merged in the current form and torment our users


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/Cluster.java
##########
@@ -0,0 +1,46 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * <p>A representation of the (initial) cluster state, providing information on which nodes are part of the cluster and a way
+ * to get to more detailed info.
+ *
+ * <p>This instance can also be used as a {@link PropertyValueSource} if {@link PropertyKey}'s need to be specified with
+ * a global cluster target.
+ */
+public interface Cluster extends PropertyValueSource {
+  /**
+   * @return current set of live nodes. Never <code>null</code>, never empty (Solr wouldn't call the plugin if empty
+   * since no useful could then be done).
+   */
+  Set<Node> getLiveNodes();
+
+  /**
+   * <p>Returns info about the given collection if one exists. Because it is not expected for plugins to request info about
+   * a large number of collections, requests can only be made one by one.
+   *
+   * <p>This is also the reason we do not return a {@link java.util.Map} or {@link Set} of {@link SolrCollection}'s here: it would be
+   * wasteful to fetch all data and fill such a map when plugin code likely needs info about at most one or two collections.
+   */
+  Optional<SolrCollection> getCollection(String collectionName) throws IOException;

Review comment:
       I will add this.
   To your use case: if the plugin knows the names of the predefined system collections not to colocate new replicas with, it can request them one by one by name. Where an iteration would be useful is if the plugin doesn't want to place replicas on any `Node` with replicas from any collection named `system-*-metric` for example. It would then have to iterate...
   I'm always thinking eventually SolrCloud will be scaling to hundreds of thousands collections per cluster so anything that's linear in execution with that number is not acceptable in such a case for placement computation. But I do understand it can be useful in low scale uses.




----------------------------------------------------------------
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] chatman commented on pull request #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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


   I broadly agree with those principals, and I'm sure everyone does.
   
   >    1. placement plugin writing is easy, 
   Agree, but not at the cost of complicated server side code that resides in solr-core.
   
   > 2. implementation for this placement API is efficient, and 
   +1
   
   > 3. plugins do not break when Solr is refactored (plugins might be client private so can’t be refactored when Solr code changes).
   Mostly agree, but not at the cost of thousands of classes. There are simpler ways, as I think we now are clear on, to achieve the same goal.
   
   I would also like you to consider another goal (which, IMHO, is *most important*), which is to keeping the solr-core as lean, clean and the footprint of doing so as minimal as possible. Everything that can be done outside solr-core should be done. Autoscaling is not a first class citizen in a search engine, but replica assignment/placement can arguably be so.
   
   > Can we make this assumption? Plugin requests a set of properties from a set of nodes, and this is done efficiently.
   If so I'll gladly simplify my proposal
   
   +1. I'm glad we're all on the same page (at least on the verbosity aspect). Thanks for your work, @murblanc.


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/plugins/SamplePluginMinimizeCores.java
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.Comparator;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Set;
+import java.util.Map;
+
+import com.google.common.collect.Ordering;
+import com.google.common.collect.TreeMultimap;
+import org.apache.solr.cluster.placement.Cluster;
+import org.apache.solr.cluster.placement.CoresCountPropertyValue;
+import org.apache.solr.cluster.placement.CreateNewCollectionPlacementRequest;
+import org.apache.solr.cluster.placement.Node;
+import org.apache.solr.cluster.placement.PlacementException;
+import org.apache.solr.cluster.placement.PlacementPlugin;
+import org.apache.solr.cluster.placement.PropertyKey;
+import org.apache.solr.cluster.placement.PropertyKeyFactory;
+import org.apache.solr.cluster.placement.PropertyValue;
+import org.apache.solr.cluster.placement.PropertyValueFetcher;
+import org.apache.solr.cluster.placement.Replica;
+import org.apache.solr.cluster.placement.ReplicaPlacement;
+import org.apache.solr.cluster.placement.PlacementRequest;
+import org.apache.solr.cluster.placement.PlacementPlan;
+import org.apache.solr.cluster.placement.PlacementPlanFactory;
+import org.apache.solr.common.util.SuppressForbidden;
+
+/**
+ * Implements placing replicas to minimize number of cores per {@link Node}, while not placing two replicas of the same
+ * shard on the same node.
+ *
+ * TODO: code not tested and never run, there are no implementation yet for used interfaces
+ */
+public class SamplePluginMinimizeCores implements PlacementPlugin {
+
+  @SuppressForbidden(reason = "Ordering.arbitrary() has no equivalent in Comparator class. Rather reuse than copy.")
+  public PlacementPlan computePlacement(Cluster cluster, PlacementRequest placementRequest, PropertyKeyFactory propertyFactory,
+                                        PropertyValueFetcher propertyFetcher, PlacementPlanFactory placementPlanFactory) throws PlacementException {
+    // This plugin only supports Creating a collection.
+    if (!(placementRequest instanceof CreateNewCollectionPlacementRequest)) {
+      throw new PlacementException("This toy plugin only supports creating collections");
+    }
+
+    final CreateNewCollectionPlacementRequest reqCreateCollection = (CreateNewCollectionPlacementRequest) placementRequest;
+
+    final int totalReplicasPerShard = reqCreateCollection.getNrtReplicationFactor() +
+        reqCreateCollection.getTlogReplicationFactor() + reqCreateCollection.getPullReplicationFactor();
+
+    if (cluster.getLiveNodes().size() < totalReplicasPerShard) {
+      throw new PlacementException("Cluster size too small for number of replicas per shard");
+    }
+
+    // Get number of cores on each Node
+    TreeMultimap<Integer, Node> nodesByCores = TreeMultimap.create(Comparator.naturalOrder(), Ordering.arbitrary());

Review comment:
       I challenge the "drastically reduce" assertion here @sigram. I think we can save a couple or so lines of code per property type, but maybe not much more. We can likely simplify the `PropertyValue` hierarchy to a few typed abstractions, see section at the end of [this comment](https://github.com/apache/lucene-solr/pull/1684#issuecomment-684063923).
   If we do keep the efficiency of multi fetching (all properties from a node at once) and want to hide from the plugin implementer the ability to fetch from multiple nodes concurrently, we do need to keep this whole logic of keys totally describing what they refer to.




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PlacementPlugin.java
##########
@@ -0,0 +1,41 @@
+/*
+ * 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;
+
+/**
+ * Implemented by external plugins to control replica placement and movement on the search cluster (as well as other things
+ * such as cluster elasticity?) when cluster changes are required (initiated elsewhere, most likely following a Collection
+ * API call).
+ */
+public interface PlacementPlugin {

Review comment:
       The configuration part I didn't really do yet. Unclear to me how the `configure` method here would be used, since in order to get to it the plugin has to be loaded already...
   I was thinking defining the plugin class or classes in some solr configuration file (with the rest of the config). At least a single default plugin implementation that would be used for all placement needs or a default + other ones ones (with names that can then be selected by callers of the Collection API passing a `placement` parameter as suggested in the changes to `CollectionAdminRequest.java`).
   Are there similar examples in Solr code (loading plugins possibly on a per collection basis) that I can get inspiration from or reuse?




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/AddReplicasRequest.java
##########
@@ -0,0 +1,62 @@
+/*
+ * 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;
+
+import java.util.Set;
+
+/**
+ * <p>Request for creating one or more {@link Replica}'s for one or more {@link Shard}'s of an existing {@link SolrCollection}.
+ * The shard might or might not already exist, plugin code can easily find out by using {@link SolrCollection#getShards()}
+ * and verifying if the shard name(s) from {@link #getShardNames()} are there.
+ *
+ * <p>As opposed to {@link CreateNewCollectionRequest}, the set of {@link Node}s on which the replicas should be placed
+ * is specified (defaults to being equal to the set returned by {@link Cluster#getLiveNodes()}).
+ *
+ * <p>There is no extension between this interface and {@link CreateNewCollectionRequest} in either direction
+ * or from a common ancestor for readability. An ancestor could make sense and would be an "abstract interface" not intended
+ * to be implemented directly, but this does not exist in Java.
+ *
+ * <p>Plugin code would likely treat the two types of requests differently since here existing {@link Replica}'s must be taken
+ * into account for placement whereas in {@link CreateNewCollectionRequest} no {@link Replica}'s are assumed to exist.
+ */
+public interface AddReplicasRequest extends Request {
+  /**
+   * The {@link SolrCollection} to add {@link Replica}(s) to. The replicas are to be added to a shard that might or might
+   * not yet exist when the plugin's {@link PlacementPlugin#computePlacement} is called.
+   */
+  SolrCollection getCollection();
+
+  /**
+   * <p>Shard name(s) for which new replicas placement should be computed. The shard(s) might exist or not (that's why this
+   * method returns a {@link Set} of {@link String}'s and not directly a set of {@link Shard} instances).
+   *
+   * <p>Note the Collection API allows specifying the shard name or a {@code _route_} parameter. The Solr implementation will
+   * convert either specification into the relevant shard name so the plugin code doesn't have to worry about this.
+   */
+  Set<String> getShardNames();
+
+  /** Replicas should only be placed on nodes from the set returned by this method. */
+  Set<Node> getTargetNodes();

Review comment:
       Used work order -> placement plan




----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed initial proposal for placement plugin interface

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


   > NO
   
   You need to make a realistic proposal here IMO @noblepaul.
   And we need something that **already exists**. If placement plugins depend on some future rehaul of configuration in SolrCloud, we're pushing this way out because from past discussion there doesn't seem to be a consensus on how/if to remove `solr.xml` and how to support its features differently.
   
   The **absolute minimal** needs this PR has as far as configuration:
   - Define a classname to be loaded as the placement plugin factory
   - Define configuration passed to that plugin (int, long, string, boolean etc). Assign default values to these configs (for example maxReplicasOnNodeForGivenShard=1, maxCoresPerNode=9999) and allow the user to easily change them on its system
   - Allow picking the placement plugin AssignStrategy or the LEGACY or RULES one. And allow the end user to decide.
   
   Another need that I think makes life easier (esp. in testing) is being able to set this configuration to persist across Zookeeper wipes during tests.
   
   The solution I see is passing all these configs (everything I've added in `solr.xml` in the PR) as command line arguments (system properties) to Solr. The startup script would have to be distributed in the same way `solr.xml` would have to, and `solr.xml` can be put in Zookeeper, something impossible with the startup script. From the perspective of this PR I can switch to that (and rely on funky naming of system properties to know they need to be passed to the placementPlugin - remember these properties are defined by the plugin writer, not by us).
   
   Hard coding (in code) the type of `AssignStrategy` to use (then has to be `LEGACY` so `RULES` continue to work for collections that have them) and forcing the user to prime `clusterprops.json` to use anything else or to change the default config parameters of placement plugins (such as maxCoresPerNode) is going to make testing a nightmare, and we'll likely introduce automated priming mechanism that read their value from somewhere and push it to `clusterprops.json` on start up. I know I'll do that because I want to run SolrCloud with a fresh ZK very often so I delete the dir where ZK puts its stuff.


----------------------------------------------------------------
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 #1684: SOLR-14613: strongly typed placement plugin interface and implementation

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



##########
File path: solr/core/src/java/org/apache/solr/cluster/placement/PropertyValue.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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;
+
+/**
+ *  <p>The value corresponding to a specific {@link PropertyKey}, in a specific context (e.g. property of a specific
+ *  {@link Node} instance). The context is tracked in the {@link PropertyKey} using a {@link PropertyValueSource}.
+ *
+ *  <p>Instances are obtained by first getting a key using {@link PropertyKeyFactory} then getting the corresponding
+ *  {@link PropertyValue} using {@link PropertyValueFetcher}.
+ */
+public interface PropertyValue {
+  /**
+   * The property key used for retrieving this property value.
+   */
+  PropertyKey getKey();
+
+  /**
+   *  Instances are obtained by first getting a key using {@link PropertyKeyFactory#createCoreCountKey} then calling
+   *  {@link PropertyValueFetcher#fetchProperties}, retrieving the appropriate {@link PropertyValue} from the returned map
+   *  using the {@link PropertyKey} as key and finally casting it to {@link PropertyValue.CoresCount}.
+   */
+  interface CoresCount extends PropertyValue {
+    /**
+     * Returns the number of cores on the {@link Node}) this instance was obtained from (i.e. instance
+     * passed to {@link PropertyKeyFactory#createCoreCountKey(Node)}).
+     */
+    int getCoresCount();
+  }
+
+  /**
+   *  Instances are obtained by first getting a key using {@link PropertyKeyFactory#createDiskTypeKey} then getting the
+   *  {@link org.apache.solr.cluster.placement.PropertyValue.DiskType} using {@link PropertyValueFetcher#fetchProperties} and retrieving (then casting) the
+   *  appropriate {@link PropertyValue} from the returned map using the {@link PropertyKey} as key.
+   */
+  interface DiskType extends PropertyValue {
+    /**
+     * Type of storage hardware used for the partition on which cores are stored on the {@link Node}) from which this instance
+     * was obtained (i.e. instance passed to {@link PropertyKeyFactory#createDiskTypeKey(Node)}).
+     */
+    HardwareType getHardwareType();
+
+    enum HardwareType {
+      SSD, ROTATIONAL, UNKNOWN
+    }
+  }
+
+  /**
+   *  Instances are obtained by first getting a key using {@link PropertyKeyFactory#createTotalDiskKey(Node)} (Node)} then getting the
+   *  {@link org.apache.solr.cluster.placement.PropertyValue.TotalDisk} using {@link PropertyValueFetcher#fetchProperties} and retrieving (then casting) the
+   *  appropriate {@link PropertyValue} fetched from the returned map using the {@link PropertyKey} as key.
+   */
+  interface TotalDisk extends PropertyValue {
+    /**
+     * Total disk size of the partition on which cores are stored on the {@link Node}) from which this instance was obtained
+     * (i.e. instance passed to {@link PropertyKeyFactory#createTotalDiskKey(Node)}).
+     */
+    long getTotalSizeGB();
+  }
+
+  /**
+   *  Instances are obtained by first getting a key using {@link PropertyKeyFactory#createFreeDiskKey(Node)} then getting the
+   *  {@link org.apache.solr.cluster.placement.PropertyValue.FreeDisk} using {@link PropertyValueFetcher#fetchProperties} and retrieving (then casting) the
+   *  appropriate {@link PropertyValue} fetched from the returned map using the {@link PropertyKey} as key.
+   */
+  interface FreeDisk extends PropertyValue {
+    /**
+     * Free disk size of the partition on which cores are stored on the {@link Node}) from which this instance was obtained
+     *  (i.e. instance passed to {@link PropertyKeyFactory#createDiskTypeKey(Node)}).
+     */
+    long getFreeSizeGB();
+  }
+
+
+  /**
+   *  Instances are obtained by first getting a key using {@link PropertyKeyFactory#createHeapUsageKey(Node)} then calling
+   *  {@link PropertyValueFetcher#fetchProperties}, retrieving the appropriate {@link PropertyValue} from the returned map
+   *  using the {@link PropertyKey} as key and finally casting it to {@link org.apache.solr.cluster.placement.PropertyValue.HeapUsage}.
+   */
+  interface HeapUsage extends PropertyValue {
+
+    /**
+     * Percentage between 0 and 100 of used heap over max heap.
+     */
+    double getUsedHeapMemoryUsage();
+  }
+
+  /**
+   * A {@link PropertyValue} representing a metric on the target {@link PropertyValueSource}.
+   *
+   * <p>Instances are obtained by first getting a key using {@link PropertyKeyFactory#createMetricKey(PropertyValueSource, String)}
+   * or {@link PropertyKeyFactory#createMetricKey(Node, String, PropertyKeyFactory.NodeMetricRegistry)} then calling
+   * {@link PropertyValueFetcher#fetchProperties}, retrieving the appropriate {@link PropertyValue} from the returned map
+   * using the {@link PropertyKey} as key and finally casting it to {@link org.apache.solr.cluster.placement.PropertyValue.Metric}.
+   */
+  interface Metric extends PropertyValue {
+    /**
+     * Returns the metric value from the {@link PropertyValueSource} from which it was retrieved.
+     */
+    Double getNumberValue();
+  }
+
+  /**
+   * A {@link PropertyValue} representing a sysprop (or System property) on the target {@link Node}.
+   *
+   *  Instances are obtained by first getting a key using {@link PropertyKeyFactory#createSyspropKey} then calling
+   *  {@link PropertyValueFetcher#fetchProperties}, retrieving the appropriate {@link PropertyValue} from the returned map
+   *  using the {@link PropertyKey} as key and finally casting it to {@link org.apache.solr.cluster.placement.PropertyValue.Sysprop}.
+   */
+  interface Sysprop extends PropertyValue {
+    /**

Review comment:
       Will add back. Making a few changes to the interfaces anyway as I write the more ambitious plugin @thelabdude suggested.




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