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/16 15:50:23 UTC

[GitHub] [lucene-solr] sigram opened a new pull request #1678: SOLR-14613: Initial POC based on Ilan's proposal.

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


   See Jira for background.
   
   This PR is based on Ilan's proposal for a minimal API that is totally independent of Solr internals. I renamed some of the interfaces for convenience, and implemented a few concrete classes (POJOs) to pass data around.
   
   So far I implemented a round-trip shim between SolrCloudManager and AssignerClusterState and its components - see TestPolicy8x for details. This is sufficient to support the Policy engine calculations using the new API + shim. 
   
   I didn't start converting the Solr side of things to actually provide the new APIs to the Assigner implementations, and I didn't hook up the creation of Assigner yet.


----------------------------------------------------------------
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 #1678: SOLR-14613: Initial POC based on Ilan's proposal.

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/assign/AddReplicaRequest.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.cloud.api.collections.assign;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+import org.apache.solr.common.cloud.Replica;
+
+/**
+ *
+ */
+public class AddReplicaRequest extends BaseAssignRequest {
+  private final Replica.Type type;
+  private final Map<String, Object> params = new HashMap<>();
+  private final String targetNode;
+  private final String coreName;

Review comment:
       Right, I'll fix 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 #1678: SOLR-14613: Initial POC based on Ilan's proposal.

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/assign/AssignDecision.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.cloud.api.collections.assign;
+
+/**
+ *
+ */
+public interface AssignDecision {

Review comment:
       Added value of this interface unclear to me given we have a base class and concrete implementations.
   Would make more sense IMO if we had interfaces for all possible decision types, and provide actual concrete implementations as an optional convenience to the plugin writer.




----------------------------------------------------------------
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 #1678: SOLR-14613: Initial POC based on Ilan's proposal.

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/assign/MoveReplicaRequest.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.cloud.api.collections.assign;
+
+/**
+ *
+ */
+public interface MoveReplicaRequest extends AssignRequest {

Review comment:
       What is the plugin asked to decide/compute here?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] murblanc commented on a change in pull request #1678: SOLR-14613: Initial POC based on Ilan's proposal.

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/assign/AssignerNodeState.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.api.collections.assign;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ *
+ */
+public interface AssignerNodeState {
+  Collection<AssignerReplica> getReplicas();
+
+  long getTotalDiskGB();
+
+  long getFreeDiskGB();

Review comment:
       Maybe make number of cores an explicit metric as well?
   Note in current Autoscaling framework it is always requested, as is free disk space.
   
   Unclear to me how total disk space would be used by 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] sigram commented on a change in pull request #1678: SOLR-14613: Initial POC based on Ilan's proposal.

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/assign/AssignerReplica.java
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.api.collections.assign;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.solr.common.cloud.Replica;
+
+/**
+ * Represents basic information about a replica.
+ */
+public class AssignerReplica {
+  private final String name;
+  private final String node;
+  private final String collection;
+  private final String shard;
+  private final String core;
+  private final Replica.Type type;
+  private final Replica.State state;

Review comment:
       I'll wrap them. Moving Type and State would cause massive diffs across the whole code base.




----------------------------------------------------------------
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 #1678: SOLR-14613: Initial POC based on Ilan's proposal.

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/assign/policy8x/AssignerCloudManager.java
##########
@@ -0,0 +1,81 @@
+package org.apache.solr.cloud.api.collections.assign.policy8x;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.SolrResponse;
+import org.apache.solr.client.solrj.cloud.DistribStateManager;
+import org.apache.solr.client.solrj.cloud.DistributedQueueFactory;
+import org.apache.solr.client.solrj.cloud.NodeStateProvider;
+import org.apache.solr.client.solrj.cloud.SolrCloudManager;
+import org.apache.solr.client.solrj.impl.ClusterStateProvider;
+import org.apache.solr.cloud.api.collections.assign.AssignerClusterState;
+import org.apache.solr.common.util.ObjectCache;
+import org.apache.solr.common.util.TimeSource;
+
+/**
+ *
+ */
+public class AssignerCloudManager implements SolrCloudManager {
+  private final ObjectCache objectCache = new ObjectCache();
+  private final TimeSource timeSource;
+  private final AssignerClusterState assignerClusterState;
+  private final AssignerClusterStateProvider clusterStateProvider;
+  private final AssignerNodeStateProvider nodeStateProvider;
+  private final AssignerDistribStateManager distribStateManager;
+
+  public AssignerCloudManager(AssignerClusterState assignerClusterState,

Review comment:
       This package is a proof of concept that the Policy engine can fully function as a plugin when it only has the information provided by the new API (`AssignerClusterState`).




----------------------------------------------------------------
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 #1678: SOLR-14613: Initial POC based on Ilan's proposal.

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


   You lost me when I got to the policy8x package classes.
   My understanding is that the assigner driving the plugins would build the state as seen by the plugins from SolrCloud internal state, would build the requests as seen by the plugins from the internal requests (Hint thing most likely?), then the plugin will be called, then the decisions will be unpacked into SolrCloud actions.
   
   Internal code should not have to unpack the state in the format used by the plugins... Am I missing something?


----------------------------------------------------------------
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 #1678: SOLR-14613: Initial POC based on Ilan's proposal.

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/assign/MoveReplicaRequest.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.cloud.api.collections.assign;
+
+/**
+ *
+ */
+public interface MoveReplicaRequest extends AssignRequest {

Review comment:
       In some cases you know exactly where you want to move a replica. In other cases you're asking for the best place other than the current place. This supports both scenarios - if `targetNode` is specified then this translates directly to a MOVEREPLICA cmd without any computation. If `targetNode` is null then the plugin needs to compute the best new placement.




----------------------------------------------------------------
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 #1678: SOLR-14613: Initial POC based on Ilan's proposal.

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/assign/AssignerReplica.java
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.api.collections.assign;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.solr.common.cloud.Replica;
+
+/**
+ * Represents basic information about a replica.
+ */
+public class AssignerReplica {
+  private final String name;
+  private final String node;
+  private final String collection;
+  private final String shard;
+  private final String core;
+  private final Replica.Type type;
+  private final Replica.State state;

Review comment:
       I'd prefer Type and State to be moved outside of an implementation class so won't bring any dependencies or leak any internal class 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 #1678: SOLR-14613: Initial POC based on Ilan's proposal.

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/assign/DeleteReplicaRequest.java
##########
@@ -0,0 +1,33 @@
+/*
+ * 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.api.collections.assign;
+
+/**
+ *
+ */
+public class DeleteReplicaRequest extends BaseAssignRequest {

Review comment:
       Again, if it's just a delete and we already know what to delete, the plugin doesn't add much value.
   Solr calls plugins to ask a question and the plugins provide answers. What's the question and what are the possible answers?




----------------------------------------------------------------
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 #1678: SOLR-14613: Initial POC based on Ilan's proposal.

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/assign/policy8x/AssignerCloudManager.java
##########
@@ -0,0 +1,81 @@
+package org.apache.solr.cloud.api.collections.assign.policy8x;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.SolrResponse;
+import org.apache.solr.client.solrj.cloud.DistribStateManager;
+import org.apache.solr.client.solrj.cloud.DistributedQueueFactory;
+import org.apache.solr.client.solrj.cloud.NodeStateProvider;
+import org.apache.solr.client.solrj.cloud.SolrCloudManager;
+import org.apache.solr.client.solrj.impl.ClusterStateProvider;
+import org.apache.solr.cloud.api.collections.assign.AssignerClusterState;
+import org.apache.solr.common.util.ObjectCache;
+import org.apache.solr.common.util.TimeSource;
+
+/**
+ *
+ */
+public class AssignerCloudManager implements SolrCloudManager {

Review comment:
       The current Policy engine relies on `SolrCloudManager` API. This class is a shim - it provides a synthetic `SolrCloudManager` that uses data from `AssignerClusterState`, and nothing else.




----------------------------------------------------------------
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 #1678: SOLR-14613: Initial POC based on Ilan's proposal.

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/assign/ReplicaState.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.cloud.api.collections.assign;
+
+import java.util.Locale;
+
+/**
+ *
+ */
+public enum ReplicaState {

Review comment:
       How is it any different from `Replia.State` ?




----------------------------------------------------------------
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 #1678: SOLR-14613: Initial POC based on Ilan's proposal.

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/assign/policy8x/AssignerCloudManager.java
##########
@@ -0,0 +1,81 @@
+package org.apache.solr.cloud.api.collections.assign.policy8x;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.SolrResponse;
+import org.apache.solr.client.solrj.cloud.DistribStateManager;
+import org.apache.solr.client.solrj.cloud.DistributedQueueFactory;
+import org.apache.solr.client.solrj.cloud.NodeStateProvider;
+import org.apache.solr.client.solrj.cloud.SolrCloudManager;
+import org.apache.solr.client.solrj.impl.ClusterStateProvider;
+import org.apache.solr.cloud.api.collections.assign.AssignerClusterState;
+import org.apache.solr.common.util.ObjectCache;
+import org.apache.solr.common.util.TimeSource;
+
+/**
+ *
+ */
+public class AssignerCloudManager implements SolrCloudManager {

Review comment:
       This is an internal class, right? The plugins do not see 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 #1678: SOLR-14613: Initial POC based on Ilan's proposal.

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/assign/DeleteReplicaDecision.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.cloud.api.collections.assign;
+
+/**
+ *
+ */
+public class DeleteReplicaDecision extends BaseAssignDecision {

Review comment:
       Missing details on what to delete here, no? Otherwise what's the plugin role in the delete?




----------------------------------------------------------------
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 #1678: SOLR-14613: Initial POC based on Ilan's proposal.

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/assign/AssignerReplica.java
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.api.collections.assign;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.solr.common.cloud.Replica;
+
+/**
+ * Represents basic information about a replica.
+ */
+public class AssignerReplica {
+  private final String name;
+  private final String node;
+  private final String collection;
+  private final String shard;
+  private final String core;
+  private final Replica.Type type;
+  private final Replica.State state;

Review comment:
       (or wrap them if we don't want to touch Replica at this stage)




----------------------------------------------------------------
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 #1678: SOLR-14613: Initial POC based on Ilan's proposal.

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/assign/AssignDecisions.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.cloud.api.collections.assign;
+
+import java.util.Collection;
+
+/**
+ *
+ */
+public class AssignDecisions {
+
+  private final AssignerClusterState outputClusterState;
+  private final Collection<AssignDecision> decisions;
+
+  public AssignDecisions(AssignerClusterState outputClusterState, Collection<AssignDecision> decisions) {

Review comment:
       Why should the plugin pass an `outputClusterState`?




----------------------------------------------------------------
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 #1678: SOLR-14613: Initial POC based on Ilan's proposal.

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/assign/AssignerNodeState.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.api.collections.assign;
+
+import java.util.Collection;
+import java.util.Map;
+
+/**
+ *
+ */
+public interface AssignerNodeState {
+  Collection<AssignerReplica> getReplicas();
+
+  long getTotalDiskGB();
+
+  long getFreeDiskGB();
+
+  int getNumCores();
+
+  Map<String, Object> getProperties();

Review comment:
       What are these properties? javadocs missing?




----------------------------------------------------------------
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 #1678: SOLR-14613: Initial POC based on Ilan's proposal.

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/assign/policy8x/AssignerCloudManager.java
##########
@@ -0,0 +1,81 @@
+package org.apache.solr.cloud.api.collections.assign.policy8x;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.SolrResponse;
+import org.apache.solr.client.solrj.cloud.DistribStateManager;
+import org.apache.solr.client.solrj.cloud.DistributedQueueFactory;
+import org.apache.solr.client.solrj.cloud.NodeStateProvider;
+import org.apache.solr.client.solrj.cloud.SolrCloudManager;
+import org.apache.solr.client.solrj.impl.ClusterStateProvider;
+import org.apache.solr.cloud.api.collections.assign.AssignerClusterState;
+import org.apache.solr.common.util.ObjectCache;
+import org.apache.solr.common.util.TimeSource;
+
+/**
+ *
+ */
+public class AssignerCloudManager implements SolrCloudManager {
+  private final ObjectCache objectCache = new ObjectCache();
+  private final TimeSource timeSource;
+  private final AssignerClusterState assignerClusterState;
+  private final AssignerClusterStateProvider clusterStateProvider;
+  private final AssignerNodeStateProvider nodeStateProvider;
+  private final AssignerDistribStateManager distribStateManager;
+
+  public AssignerCloudManager(AssignerClusterState assignerClusterState,

Review comment:
       I finally got that following our Slack exchange... I'm not sure that's a proof this will really work in a non trivial case, because we're missing the logic that goes around properties/variables and their types.




----------------------------------------------------------------
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 #1678: SOLR-14613: Initial POC based on Ilan's proposal.

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/assign/DeleteReplicaRequest.java
##########
@@ -0,0 +1,33 @@
+/*
+ * 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.api.collections.assign;
+
+/**
+ *
+ */
+public class DeleteReplicaRequest extends BaseAssignRequest {

Review comment:
       That's not always the case - when you want to decommission a node you know exactly what replicas to delete.




----------------------------------------------------------------
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 #1678: SOLR-14613: Initial POC based on Ilan's proposal.

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/assign/DeleteReplicaDecision.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.cloud.api.collections.assign;
+
+/**
+ *
+ */
+public class DeleteReplicaDecision extends BaseAssignDecision {

Review comment:
       All necessary details are already in the enclosed DeleteReplicaRequest.




----------------------------------------------------------------
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 #1678: SOLR-14613: Initial POC based on Ilan's proposal.

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/assign/AssignerShardState.java
##########
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.cloud.api.collections.assign;
+
+import java.util.Collection;
+import java.util.Map;
+
+/**
+ *
+ */
+public interface AssignerShardState {
+  String getName();
+
+  Collection<AssignerReplica> getReplicas();
+
+  Map<String, Object> getProperties();

Review comment:
       what are shard properties? Examples?




----------------------------------------------------------------
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 #1678: SOLR-14613: Initial POC based on Ilan's proposal.

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/assign/Assigner.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.cloud.api.collections.assign;
+
+import java.util.List;
+
+/**
+ * Interface implemented by assign strategies.
+ */
+public interface Assigner {
+
+  AssignDecisions computeAssignments(AssignerClusterState initialState,
+                                     List<AssignRequest> requests) throws InterruptedException, AssignerException;

Review comment:
       Nice touch here to spare the plugin from having to deal with interruptions!




----------------------------------------------------------------
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 #1678: SOLR-14613: Initial POC based on Ilan's proposal.

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/assign/AssignDecisions.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.cloud.api.collections.assign;
+
+import java.util.Collection;
+
+/**
+ *
+ */
+public class AssignDecisions {
+
+  private final AssignerClusterState outputClusterState;
+  private final Collection<AssignDecision> decisions;
+
+  public AssignDecisions(AssignerClusterState outputClusterState, Collection<AssignDecision> decisions) {

Review comment:
       The Solr side support framework (didn't read all classes yet) might want to apply the decisions to another cluster state maintained centrally, used as a "master copy" including all changes from all computations and passed (in an immutable form so the plugin doesn't become crazy) to the next computation. Internally we don't have to make copies of everything, we can share data but give consistent point in time reads.




----------------------------------------------------------------
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 #1678: SOLR-14613: Initial POC based on Ilan's proposal.

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/assign/DeleteReplicaDecision.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.cloud.api.collections.assign;
+
+/**
+ *
+ */
+public class DeleteReplicaDecision extends BaseAssignDecision {

Review comment:
       So why are we talking to the plugin for 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 #1678: SOLR-14613: Initial POC based on Ilan's proposal.

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/assign/DeleteReplicaRequest.java
##########
@@ -0,0 +1,33 @@
+/*
+ * 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.api.collections.assign;
+
+/**
+ *
+ */
+public class DeleteReplicaRequest extends BaseAssignRequest {

Review comment:
       Likely a delete request should only specify the shard and let the plugin pick which replica to kill...
   Otherwise no need to ask the plugin, just delete the replica directly.




----------------------------------------------------------------
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 #1678: SOLR-14613: Initial POC based on Ilan's proposal.

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/assign/MoveReplicaDecision.java
##########
@@ -0,0 +1,36 @@
+/*
+ * 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.api.collections.assign;
+
+import org.apache.solr.common.cloud.Replica;
+
+/**
+ *
+ */
+public interface MoveReplicaDecision extends AssignDecision {
+  String getSourceNodeName();
+
+  String getTargetNodeName();
+
+  String getCollectionName();
+
+  String getSliceName();

Review comment:
       shard 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 #1678: SOLR-14613: Initial POC based on Ilan's proposal.

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/assign/policy8x/AssignerCloudManager.java
##########
@@ -0,0 +1,81 @@
+package org.apache.solr.cloud.api.collections.assign.policy8x;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.SolrResponse;
+import org.apache.solr.client.solrj.cloud.DistribStateManager;
+import org.apache.solr.client.solrj.cloud.DistributedQueueFactory;
+import org.apache.solr.client.solrj.cloud.NodeStateProvider;
+import org.apache.solr.client.solrj.cloud.SolrCloudManager;
+import org.apache.solr.client.solrj.impl.ClusterStateProvider;
+import org.apache.solr.cloud.api.collections.assign.AssignerClusterState;
+import org.apache.solr.common.util.ObjectCache;
+import org.apache.solr.common.util.TimeSource;
+
+/**
+ *
+ */
+public class AssignerCloudManager implements SolrCloudManager {
+  private final ObjectCache objectCache = new ObjectCache();
+  private final TimeSource timeSource;
+  private final AssignerClusterState assignerClusterState;
+  private final AssignerClusterStateProvider clusterStateProvider;
+  private final AssignerNodeStateProvider nodeStateProvider;
+  private final AssignerDistribStateManager distribStateManager;
+
+  public AssignerCloudManager(AssignerClusterState assignerClusterState,

Review comment:
       I don't get the logic. I'm assuming package `*.policy8x` is internal. It should not depend on state as seen by the plugins but should **build** that state based on internal SolrCloud state.
   Here and in other classes in this package I have the impression we do things the wrong 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] sigram commented on a change in pull request #1678: SOLR-14613: Initial POC based on Ilan's proposal.

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/assign/ReplicaState.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.cloud.api.collections.assign;
+
+import java.util.Locale;
+
+/**
+ *
+ */
+public enum ReplicaState {

Review comment:
       No difference. This is just to separate this api from Replica class. I considered moving the inner Replica.State and Replica.Type to top level but this causes massive diffs all over the place. Eventually we may want to do this but now it would just obscure the important 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] sigram commented on a change in pull request #1678: SOLR-14613: Initial POC based on Ilan's proposal.

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/assign/AssignerNodeState.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.api.collections.assign;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ *
+ */
+public interface AssignerNodeState {
+  Collection<AssignerReplica> getReplicas();
+
+  long getTotalDiskGB();
+
+  long getFreeDiskGB();

Review comment:
       Yes, we should add cores here. Freedisk and totaldisk are very much needed to decide where to move replicas - you want to move them to a node that has enough free disk, and one that is not too "full".




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] sigram closed pull request #1678: SOLR-14613: Initial POC based on Ilan's proposal.

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


   


----------------------------------------------------------------
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 #1678: SOLR-14613: Initial POC based on Ilan's proposal.

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


   Another variant of this has already been merged.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] noblepaul commented on a change in pull request #1678: SOLR-14613: Initial POC based on Ilan's proposal.

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/assign/ReplicaType.java
##########
@@ -0,0 +1,47 @@
+/*
+ * 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.api.collections.assign;
+
+import java.util.Locale;
+
+/**
+ *
+ */
+public enum ReplicaType {

Review comment:
       How is this different from `Replica.Type` ?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] murblanc commented on a change in pull request #1678: SOLR-14613: Initial POC based on Ilan's proposal.

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/assign/AddReplicaRequest.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.cloud.api.collections.assign;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+import org.apache.solr.common.cloud.Replica;
+
+/**
+ *
+ */
+public class AddReplicaRequest extends BaseAssignRequest {

Review comment:
       Requests for work such as this one (passed from Solr to the plug-in implementing the assignment decisions) should likely be interfaces, not classes.
   Also I'd prefer `Replica.Type` to be made a toplevel class (enum) so we don't have to expose `Replica` to external code (as `Replica` brings many other classes with 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] sigram commented on a change in pull request #1678: SOLR-14613: Initial POC based on Ilan's proposal.

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/assign/MoveReplicaDecision.java
##########
@@ -0,0 +1,36 @@
+/*
+ * 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.api.collections.assign;
+
+import org.apache.solr.common.cloud.Replica;
+
+/**
+ *
+ */
+public interface MoveReplicaDecision extends AssignDecision {
+  String getSourceNodeName();
+
+  String getTargetNodeName();
+
+  String getCollectionName();
+
+  String getSliceName();

Review comment:
       Yes :)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] noblepaul commented on a change in pull request #1678: SOLR-14613: Initial POC based on Ilan's proposal.

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



##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/assign/AddReplicaRequest.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.cloud.api.collections.assign;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+import org.apache.solr.common.cloud.Replica;
+
+/**
+ *
+ */
+public class AddReplicaRequest extends BaseAssignRequest {
+  private final Replica.Type type;
+  private final Map<String, Object> params = new HashMap<>();
+  private final String targetNode;
+  private final String coreName;

Review comment:
       do you mean the replica Name? The core name and the replica name are not required. The system always assign a unique name

##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/assign/AssignDecision.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.cloud.api.collections.assign;
+
+/**
+ *
+ */
+public interface AssignDecision {

Review comment:
       I would prefer 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