[GitHub] [helix] narendly commented on a change in pull request #466: Integrate the WAGED rebalancer with all the related components.

2019-09-18 Thread GitBox
narendly commented on a change in pull request #466: Integrate the WAGED 
rebalancer with all the related components.
URL: https://github.com/apache/helix/pull/466#discussion_r325871225
 
 

 ##
 File path: 
helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintBasedAlgorithm.java
 ##
 @@ -66,29 +66,26 @@
   @Override
   public OptimalAssignment calculate(ClusterModel clusterModel) throws 
HelixRebalanceException {
 OptimalAssignment optimalAssignment = new OptimalAssignment();
-Map> replicasByResource = 
clusterModel.getAssignableReplicaMap();
 List nodes = new 
ArrayList<>(clusterModel.getAssignableNodes().values());
-
-// TODO: different orders of resource/replica could lead to different 
greedy assignments, will
-// revisit and improve the performance
-for (String resource : replicasByResource.keySet()) {
-  for (AssignableReplica replica : replicasByResource.get(resource)) {
-Optional maybeBestNode =
-getNodeWithHighestPoints(replica, nodes, 
clusterModel.getContext(), optimalAssignment);
-// stop immediately if any replica cannot find best assignable node
-if (optimalAssignment.hasAnyFailure()) {
-  String errorMessage = String.format(
-  "Unable to find any available candidate node for partition %s; 
Fail reasons: %s",
-  replica.getPartitionName(), optimalAssignment.getFailures());
-  throw new HelixRebalanceException(errorMessage,
-  HelixRebalanceException.Type.FAILED_TO_CALCULATE);
-}
-maybeBestNode.ifPresent(node -> 
clusterModel.assign(replica.getResourceName(),
-replica.getPartitionName(), replica.getReplicaState(), 
node.getInstanceName()));
+// Sort the replicas so the input is stable for the greedy algorithm.
+// For the other algorithm implementation, this sorting could be 
unnecessary.
+for (AssignableReplica replica : 
getOrderedAssignableReplica(clusterModel)) {
 
 Review comment:
   This doesn't answer my question...


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:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@helix.apache.org
For additional commands, e-mail: reviews-h...@helix.apache.org



[GitHub] [helix] narendly commented on a change in pull request #466: Integrate the WAGED rebalancer with all the related components.

2019-09-18 Thread GitBox
narendly commented on a change in pull request #466: Integrate the WAGED 
rebalancer with all the related components.
URL: https://github.com/apache/helix/pull/466#discussion_r325871359
 
 

 ##
 File path: 
helix-core/src/main/java/org/apache/helix/HelixRebalanceException.java
 ##
 @@ -27,6 +27,7 @@
 INVALID_CLUSTER_STATUS,
 INVALID_REBALANCER_STATUS,
 FAILED_TO_CALCULATE,
+INVALID_INPUT,
 
 Review comment:
   I do not see a TODO...


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:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@helix.apache.org
For additional commands, e-mail: reviews-h...@helix.apache.org



[GitHub] [helix] narendly commented on a change in pull request #466: Integrate the WAGED rebalancer with all the related components.

2019-09-17 Thread GitBox
narendly commented on a change in pull request #466: Integrate the WAGED 
rebalancer with all the related components.
URL: https://github.com/apache/helix/pull/466#discussion_r325377580
 
 

 ##
 File path: 
helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
 ##
 @@ -235,6 +216,65 @@ private boolean validateOfflineInstancesLimit(final 
ResourceControllerDataProvid
 return true;
   }
 
+  /**
+   * Rebalance with the WAGED rebalancer
+   * The rebalancer only calculates the new ideal assignment for all the 
resources that are
+   * configured to use the WAGED rebalancer.
+   *
+   * @param cache  Cluster data cache.
+   * @param currentStateOutput The current state information.
+   * @param helixManager
+   * @param resourceMapThe complete resource map. The method will 
filter the map for the compatible resources.
+   * @param output The best possible state output.
+   * @param failureResources   The failure records that will be updated if any 
resource cannot be computed.
+   * @return The map of all the calculated resources.
+   */
+  private Map 
computeResourceBestPossibleStateWithWagedRebalancer(
+  ResourceControllerDataProvider cache, CurrentStateOutput 
currentStateOutput,
+  HelixManager helixManager, Map resourceMap, 
BestPossibleStateOutput output,
+  List failureResources) {
+// Find the compatible resources: 1. FULL_AUTO 2. Configured to use the 
WAGED rebalancer
+Map wagedRebalancedResourceMap =
+resourceMap.entrySet().stream().filter(resourceEntry -> {
+  IdealState is = cache.getIdealState(resourceEntry.getKey());
+  return is != null && 
is.getRebalanceMode().equals(IdealState.RebalanceMode.FULL_AUTO)
+  && 
WagedRebalancer.class.getName().equals(is.getRebalancerClassName());
+}).collect(Collectors.toMap(resourceEntry -> resourceEntry.getKey(),
+resourceEntry -> resourceEntry.getValue()));
+
+Map newIdealStates = new HashMap<>();
+// TODO avoid creating the rebalancer on every rebalance call for 
performance enhancement
+WagedRebalancer wagedRebalancer = new WagedRebalancer(helixManager);
+try {
+  newIdealStates.putAll(wagedRebalancer
+  .computeNewIdealStates(cache, wagedRebalancedResourceMap, 
currentStateOutput));
+} catch (HelixRebalanceException ex) {
+  // Note that unlike the legacy rebalancer, the WAGED rebalance won't 
return partial result.
+  // Since it calculates for all the eligible resources globally, a 
partial result is invalid.
+  // TODO propagate the rebalancer failure information to 
updateRebalanceStatus for monitoring.
+  LogUtil.logError(logger, _eventId, String
+  .format("Failed to calculate the new Ideal States using the 
rebalancer %s due to %s",
+  wagedRebalancer.getClass().getSimpleName(), 
ex.getFailureType()), ex);
+} finally {
+  wagedRebalancer.close();
+}
+Iterator itr = wagedRebalancedResourceMap.values().iterator();
+while (itr.hasNext()) {
+  Resource resource = itr.next();
+  IdealState is = newIdealStates.get(resource.getResourceName());
+  // Check if the WAGED rebalancer has calculated the result for this 
resource or not.
+  if (is != null && checkBestPossibleStateCalculation(is)) {
+// The WAGED rebalancer calculates a valid result, record in the output
+updateBestPossibleStateOutput(output, resource, is);
+  } else {
+failureResources.add(resource.getResourceName());
+LogUtil.logWarn(logger, _eventId,
+"Failed to calculate best possible states for " + 
resource.getResourceName());
 
 Review comment:
   Nit: String.format


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:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@helix.apache.org
For additional commands, e-mail: reviews-h...@helix.apache.org



[GitHub] [helix] narendly commented on a change in pull request #466: Integrate the WAGED rebalancer with all the related components.

2019-09-17 Thread GitBox
narendly commented on a change in pull request #466: Integrate the WAGED 
rebalancer with all the related components.
URL: https://github.com/apache/helix/pull/466#discussion_r325378185
 
 

 ##
 File path: 
helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/TestAssignmentMetadataStore.java
 ##
 @@ -20,6 +20,7 @@
  */
 
 import java.util.Map;
+
 
 Review comment:
   Remove


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:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@helix.apache.org
For additional commands, e-mail: reviews-h...@helix.apache.org



[GitHub] [helix] narendly commented on a change in pull request #466: Integrate the WAGED rebalancer with all the related components.

2019-09-17 Thread GitBox
narendly commented on a change in pull request #466: Integrate the WAGED 
rebalancer with all the related components.
URL: https://github.com/apache/helix/pull/466#discussion_r325300581
 
 

 ##
 File path: 
helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintBasedAlgorithmFactory.java
 ##
 @@ -19,22 +19,26 @@
  * under the License.
  */
 
-import java.util.ArrayList;
 import java.util.List;
 
+import com.google.common.collect.ImmutableList;
 import org.apache.helix.controller.rebalancer.waged.RebalanceAlgorithm;
-import org.apache.helix.model.ClusterConfig;
 
 public class ConstraintBasedAlgorithmFactory {
 
   // TODO: the parameter comes from cluster config, will tune how these 2 
integers will change the
   // soft constraint weight model
   public static RebalanceAlgorithm getInstance() {
-// TODO initialize constraints, depending on constraints implementations 
PRs
-List hardConstraints = new ArrayList<>();
-List softConstraints = new ArrayList<>();
+List hardConstraints = ImmutableList
 
 Review comment:
   Question: is this factory meant to be static? How do we modify the 
constraint based algorithm after-the-fact if the user wishes to use more 
constraints or if we want to add more?


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:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@helix.apache.org
For additional commands, e-mail: reviews-h...@helix.apache.org



[GitHub] [helix] narendly commented on a change in pull request #466: Integrate the WAGED rebalancer with all the related components.

2019-09-17 Thread GitBox
narendly commented on a change in pull request #466: Integrate the WAGED 
rebalancer with all the related components.
URL: https://github.com/apache/helix/pull/466#discussion_r325302467
 
 

 ##
 File path: 
helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java
 ##
 @@ -228,6 +232,30 @@ public static ClusterModel 
generateClusterModel(ResourceControllerDataProvider d
 return totalReplicaMap;
   }
 
+  // For backward compatibility, propagate the critical simple fields from the 
IdealState to
+  // the Resource Config.
+  // Eventually, Resource Config should be the only metadata node that 
contains the required information.
 
 Review comment:
   Use a block comment please
   
   /**
   
   */


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:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@helix.apache.org
For additional commands, e-mail: reviews-h...@helix.apache.org



[GitHub] [helix] narendly commented on a change in pull request #466: Integrate the WAGED rebalancer with all the related components.

2019-09-17 Thread GitBox
narendly commented on a change in pull request #466: Integrate the WAGED 
rebalancer with all the related components.
URL: https://github.com/apache/helix/pull/466#discussion_r325297593
 
 

 ##
 File path: 
helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintBasedAlgorithm.java
 ##
 @@ -66,29 +66,26 @@
   @Override
   public OptimalAssignment calculate(ClusterModel clusterModel) throws 
HelixRebalanceException {
 OptimalAssignment optimalAssignment = new OptimalAssignment();
-Map> replicasByResource = 
clusterModel.getAssignableReplicaMap();
 List nodes = new 
ArrayList<>(clusterModel.getAssignableNodes().values());
-
-// TODO: different orders of resource/replica could lead to different 
greedy assignments, will
-// revisit and improve the performance
-for (String resource : replicasByResource.keySet()) {
-  for (AssignableReplica replica : replicasByResource.get(resource)) {
-Optional maybeBestNode =
-getNodeWithHighestPoints(replica, nodes, 
clusterModel.getContext(), optimalAssignment);
-// stop immediately if any replica cannot find best assignable node
-if (optimalAssignment.hasAnyFailure()) {
-  String errorMessage = String.format(
-  "Unable to find any available candidate node for partition %s; 
Fail reasons: %s",
-  replica.getPartitionName(), optimalAssignment.getFailures());
-  throw new HelixRebalanceException(errorMessage,
-  HelixRebalanceException.Type.FAILED_TO_CALCULATE);
-}
-maybeBestNode.ifPresent(node -> 
clusterModel.assign(replica.getResourceName(),
-replica.getPartitionName(), replica.getReplicaState(), 
node.getInstanceName()));
+// Sort the replicas so the input is stable for the greedy algorithm.
+// For the other algorithm implementation, this sorting could be 
unnecessary.
+for (AssignableReplica replica : 
getOrderedAssignableReplica(clusterModel)) {
 
 Review comment:
   Should we add some pre-processing step that does whatever is necessary for 
the input? 
   For example, we may require different pre-processing. In this case, for the 
greedy algo, we need to sort the replicas. For other algos, we may need 
different types of pre-processing (such as sort by fault zones, etc.).


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:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@helix.apache.org
For additional commands, e-mail: reviews-h...@helix.apache.org



[GitHub] [helix] narendly commented on a change in pull request #466: Integrate the WAGED rebalancer with all the related components.

2019-09-17 Thread GitBox
narendly commented on a change in pull request #466: Integrate the WAGED 
rebalancer with all the related components.
URL: https://github.com/apache/helix/pull/466#discussion_r325288698
 
 

 ##
 File path: 
helix-core/src/main/java/org/apache/helix/HelixRebalanceException.java
 ##
 @@ -27,6 +27,7 @@
 INVALID_CLUSTER_STATUS,
 INVALID_REBALANCER_STATUS,
 FAILED_TO_CALCULATE,
+INVALID_INPUT,
 
 Review comment:
   @jiajunwang Could you please put a TODO with what you said in your comment?


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:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@helix.apache.org
For additional commands, e-mail: reviews-h...@helix.apache.org



[GitHub] [helix] narendly commented on a change in pull request #466: Integrate the WAGED rebalancer with all the related components.

2019-09-17 Thread GitBox
narendly commented on a change in pull request #466: Integrate the WAGED 
rebalancer with all the related components.
URL: https://github.com/apache/helix/pull/466#discussion_r325292549
 
 

 ##
 File path: 
helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/AssignmentMetadataStore.java
 ##
 @@ -124,7 +146,7 @@ private HelixProperty combineAssignments(String name,
 HelixProperty property = new HelixProperty(name);
 // Add each resource's assignment as a simple field in one ZNRecord
 assignmentMap.forEach((resource, assignment) -> 
property.getRecord().setSimpleField(resource,
-Arrays.toString(SERIALIZER.serialize(assignment.getRecord();
+new String(SERIALIZER.serialize(assignment.getRecord();
 
 Review comment:
   Could we add a small comment saying that Arrays.toString doesn't work 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:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@helix.apache.org
For additional commands, e-mail: reviews-h...@helix.apache.org



[GitHub] [helix] narendly commented on a change in pull request #466: Integrate the WAGED rebalancer with all the related components.

2019-09-17 Thread GitBox
narendly commented on a change in pull request #466: Integrate the WAGED 
rebalancer with all the related components.
URL: https://github.com/apache/helix/pull/466#discussion_r325292244
 
 

 ##
 File path: 
helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/AssignmentMetadataStore.java
 ##
 @@ -50,33 +52,43 @@
   private Map _globalBaseline;
   private Map _bestPossibleAssignment;
 
+  AssignmentMetadataStore(String zkServers, String clusterName) {
 
 Review comment:
   @jiajunwang Let's rename zkServers to metadataStoreAddrs or something. We do 
not want close coupling with ZK.


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:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@helix.apache.org
For additional commands, e-mail: reviews-h...@helix.apache.org



[GitHub] [helix] narendly commented on a change in pull request #466: Integrate the WAGED rebalancer with all the related components.

2019-09-17 Thread GitBox
narendly commented on a change in pull request #466: Integrate the WAGED 
rebalancer with all the related components.
URL: https://github.com/apache/helix/pull/466#discussion_r325306030
 
 

 ##
 File path: 
helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
 ##
 @@ -113,53 +114,33 @@ private BestPossibleStateOutput compute(ClusterEvent 
event, Map newIdealStates = new HashMap<>();
-WagedRebalancer wagedRebalancer = new WagedRebalancer(helixManager);
-try {
-  newIdealStates
-  .putAll(wagedRebalancer.computeNewIdealStates(cache, resourceMap, 
currentStateOutput));
-} catch (HelixRebalanceException ex) {
-  // Note that unlike the legacy rebalancer, the WAGED rebalance won't 
return partial result.
-  // Since it calculates for all the eligible resources globally, a 
partial result is invalid.
-  // TODO propagate the rebalancer failure information to 
updateRebalanceStatus for monitoring.
-  LogUtil.logError(logger, _eventId, String
-  .format("Failed to calculate the new Ideal States using the 
rebalancer %s due to %s",
-  wagedRebalancer.getClass().getSimpleName(), 
ex.getFailureType()), ex);
-}
+boolean isValid =
+validateOfflineInstancesLimit(cache, 
event.getAttribute(AttributeName.helixmanager.name()));
 
 final List failureResources = new ArrayList<>();
-Iterator itr = resourceMap.values().iterator();
+
+Map calculatedResourceMap =
+computeResourceBestPossibleStateWithWagedRebalancer(cache, 
currentStateOutput, helixManager,
+resourceMap, output, failureResources);
+
+HashMap remainingResourceMap = new 
HashMap<>(resourceMap);
 
 Review comment:
   HashMap -> 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:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@helix.apache.org
For additional commands, e-mail: reviews-h...@helix.apache.org



[GitHub] [helix] narendly commented on a change in pull request #466: Integrate the WAGED rebalancer with all the related components.

2019-09-17 Thread GitBox
narendly commented on a change in pull request #466: Integrate the WAGED 
rebalancer with all the related components.
URL: https://github.com/apache/helix/pull/466#discussion_r325302251
 
 

 ##
 File path: 
helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java
 ##
 @@ -218,6 +221,7 @@ public static ClusterModel 
generateClusterModel(ResourceControllerDataProvider d
 for (Map.Entry entry : stateCountMap.entrySet()) {
   String state = entry.getKey();
   for (int i = 0; i < entry.getValue(); i++) {
+mergeIdealStateToResourceConfig(resourceConfig, is);
 
 Review comment:
   Rename to mergeIdealStateWithResourceConfig


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:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@helix.apache.org
For additional commands, e-mail: reviews-h...@helix.apache.org



[GitHub] [helix] narendly commented on a change in pull request #466: Integrate the WAGED rebalancer with all the related components.

2019-09-17 Thread GitBox
narendly commented on a change in pull request #466: Integrate the WAGED 
rebalancer with all the related components.
URL: https://github.com/apache/helix/pull/466#discussion_r325300798
 
 

 ##
 File path: 
helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/NodeMaxPartitionLimitConstraint.java
 ##
 @@ -28,9 +28,12 @@
   @Override
   boolean isAssignmentValid(AssignableNode node, AssignableReplica replica,
   ClusterContext clusterContext) {
-return node.getAssignedReplicaCount() < node.getMaxPartition()
-&& 
node.getAssignedPartitionsByResource(replica.getResourceName()).size() < replica
-.getResourceMaxPartitionsPerInstance();
+boolean exceedMaxPartitionLimit =
 
 Review comment:
   Good separation for readability.


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:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@helix.apache.org
For additional commands, e-mail: reviews-h...@helix.apache.org



[GitHub] [helix] narendly commented on a change in pull request #466: Integrate the WAGED rebalancer with all the related components.

2019-09-17 Thread GitBox
narendly commented on a change in pull request #466: Integrate the WAGED 
rebalancer with all the related components.
URL: https://github.com/apache/helix/pull/466#discussion_r325291979
 
 

 ##
 File path: 
helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/AssignmentMetadataStore.java
 ##
 @@ -50,33 +52,43 @@
   private Map _globalBaseline;
   private Map _bestPossibleAssignment;
 
+  AssignmentMetadataStore(String zkServers, String clusterName) {
+this(new ZkBucketDataAccessor(zkServers), clusterName);
+  }
+
+  @VisibleForTesting
 
 Review comment:
   This is actually a legitimate usage. As you could imagine, very soon we may 
encounter performance (latency) related problems, and the direction is to use a 
different metadata store instead of ZK.
   
   That means we will have a different BucketDataAccessor implementation. 
Perhaps we should rename it to BlobDataAccessor or LargeDataAccessor.
   
   So we should remove VisibleForTesting tag.


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:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@helix.apache.org
For additional commands, e-mail: reviews-h...@helix.apache.org



[GitHub] [helix] narendly commented on a change in pull request #466: Integrate the WAGED rebalancer with all the related components.

2019-09-17 Thread GitBox
narendly commented on a change in pull request #466: Integrate the WAGED 
rebalancer with all the related components.
URL: https://github.com/apache/helix/pull/466#discussion_r325300078
 
 

 ##
 File path: 
helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintBasedAlgorithm.java
 ##
 @@ -126,4 +123,53 @@ public OptimalAssignment calculate(ClusterModel 
clusterModel) throws HelixRebala
 return hardConstraints.stream().map(HardConstraint::getDescription)
 .collect(Collectors.toList());
   }
+
+  // TODO investigate better ways to sort replicas. One option is sorting 
based on the creation time.
+  private List getOrderedAssignableReplica(ClusterModel 
clusterModel) {
 
 Review comment:
   As commented above, consider generalizing this to a pre-process step.


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:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@helix.apache.org
For additional commands, e-mail: reviews-h...@helix.apache.org



[GitHub] [helix] narendly commented on a change in pull request #466: Integrate the WAGED rebalancer with all the related components.

2019-09-13 Thread GitBox
narendly commented on a change in pull request #466: Integrate the WAGED 
rebalancer with all the related components.
URL: https://github.com/apache/helix/pull/466#discussion_r324049647
 
 

 ##
 File path: 
helix-core/src/main/java/org/apache/helix/HelixRebalanceException.java
 ##
 @@ -27,6 +27,7 @@
 INVALID_CLUSTER_STATUS,
 INVALID_REBALANCER_STATUS,
 FAILED_TO_CALCULATE,
+INVALID_INPUT,
 
 Review comment:
   I think it would be a good idea to actually add some description about each 
of these failure modes?


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:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@helix.apache.org
For additional commands, e-mail: reviews-h...@helix.apache.org



[GitHub] [helix] narendly commented on a change in pull request #466: Integrate the WAGED rebalancer with all the related components.

2019-09-13 Thread GitBox
narendly commented on a change in pull request #466: Integrate the WAGED 
rebalancer with all the related components.
URL: https://github.com/apache/helix/pull/466#discussion_r324051056
 
 

 ##
 File path: 
helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/WagedRebalancer.java
 ##
 @@ -118,27 +110,26 @@ protected WagedRebalancer(AssignmentMetadataStore 
assignmentMetadataStore,
   public Map 
computeNewIdealStates(ResourceControllerDataProvider clusterData,
   Map resourceMap, final CurrentStateOutput 
currentStateOutput)
   throws HelixRebalanceException {
-LOG.info("Start computing new ideal states for resources: {}", 
resourceMap.keySet().toString());
+if (resourceMap.isEmpty()) {
+  LOG.warn("There is no resource to be rebalanced by {}",
+  this.getClass().getSimpleName());
+  return Collections.emptyMap();
+}
 
-// Find the compatible resources: 1. FULL_AUTO 2. Configured to use the 
WAGED rebalancer
-resourceMap = resourceMap.entrySet().stream().filter(resourceEntry -> {
+LOG.info("Start computing new ideal states for resources: {}", 
resourceMap.keySet().toString());
+if (!resourceMap.entrySet().stream().allMatch(resourceEntry -> {
   IdealState is = clusterData.getIdealState(resourceEntry.getKey());
   return is != null && 
is.getRebalanceMode().equals(IdealState.RebalanceMode.FULL_AUTO)
   && getClass().getName().equals(is.getRebalancerClassName());
-}).collect(Collectors.toMap(resourceEntry -> resourceEntry.getKey(),
-resourceEntry -> resourceEntry.getValue()));
-
-if (resourceMap.isEmpty()) {
-  LOG.warn("There is no valid resource to be rebalanced by {}",
-  this.getClass().getSimpleName());
-  return Collections.emptyMap();
-} else {
-  LOG.info("Valid resources that will be rebalanced by {}: {}", 
this.getClass().getSimpleName(),
-  resourceMap.keySet().toString());
+})) {
+  throw new HelixRebalanceException(
+  "Input contains invalid resource(s) that cannot be rebalanced by the 
WAGED rebalancer.",
 
 Review comment:
   I think that the occurrence for this Exception would be low, so would it be 
a better idea to actually log the names of resources that cannot be rebalanced?
   
   From the operator's perspective, it would be very, very helpful. If I just 
saw this exception without the actual names of the resources, this error 
message is almost useless. Say you have 100 resources, and the rebalance failed 
due to 1 of them - then you would have to loop through all 100 of them just to 
find the problem child!


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:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@helix.apache.org
For additional commands, e-mail: reviews-h...@helix.apache.org



[GitHub] [helix] narendly commented on a change in pull request #466: Integrate the WAGED rebalancer with all the related components.

2019-09-13 Thread GitBox
narendly commented on a change in pull request #466: Integrate the WAGED 
rebalancer with all the related components.
URL: https://github.com/apache/helix/pull/466#discussion_r324050208
 
 

 ##
 File path: 
helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/AbstractTestClusterModel.java
 ##
 @@ -105,6 +105,7 @@ protected ResourceControllerDataProvider 
setupClusterDataCache() throws IOExcept
 testClusterConfig.setDisabledInstances(Collections.emptyMap());
 testClusterConfig.setTopologyAwareEnabled(false);
 testClusterConfig.setInstanceCapacityKeys(new 
ArrayList<>(_capacityDataMap.keySet()));
+testClusterConfig.setTopologyAwareEnabled(true);
 
 Review comment:
   Why do you set this field to false and back to true? Remove line 106?


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:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@helix.apache.org
For additional commands, e-mail: reviews-h...@helix.apache.org



[GitHub] [helix] narendly commented on a change in pull request #466: Integrate the WAGED rebalancer with all the related components.

2019-09-11 Thread GitBox
narendly commented on a change in pull request #466: Integrate the WAGED 
rebalancer with all the related components.
URL: https://github.com/apache/helix/pull/466#discussion_r323559731
 
 

 ##
 File path: 
helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/WagedRebalancer.java
 ##
 @@ -349,4 +338,61 @@ private IdealState 
generateIdealStateWithAssignment(String resourceName,
 }
 return preferenceList;
   }
+
+  private Map getBaseline(
+  AssignmentMetadataStore assignmentMetadataStore, CurrentStateOutput 
currentStateOutput,
+  Set resources) throws HelixRebalanceException {
+Map currentBaseline;
+try {
+  currentBaseline = assignmentMetadataStore.getBaseline();
+} catch (HelixException hex) {
+  LOG.warn("Failed to get the current baseline assignment. Use the current 
states instead",
+  hex);
+  currentBaseline = getCurrentStateAssingment(currentStateOutput, 
resources);
+} catch (Exception ex) {
+  throw new HelixRebalanceException(
+  "Failed to get the current baseline assignment because of unexpected 
error.",
+  HelixRebalanceException.Type.INVALID_REBALANCER_STATUS, ex);
+}
+return currentBaseline;
+  }
+
+  private Map getBestPossibleAssignment(
+  AssignmentMetadataStore assignmentMetadataStore, CurrentStateOutput 
currentStateOutput,
+  Set resources) throws HelixRebalanceException {
+Map currentBestAssignment;
+try {
+  // TODO fix the deserialize
 
 Review comment:
   That is exactly the reason I had it so that it returned an empty 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:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@helix.apache.org
For additional commands, e-mail: reviews-h...@helix.apache.org



[GitHub] [helix] narendly commented on a change in pull request #466: Integrate the WAGED rebalancer with all the related components.

2019-09-11 Thread GitBox
narendly commented on a change in pull request #466: Integrate the WAGED 
rebalancer with all the related components.
URL: https://github.com/apache/helix/pull/466#discussion_r323535310
 
 

 ##
 File path: 
helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/WagedRebalancer.java
 ##
 @@ -349,4 +338,61 @@ private IdealState 
generateIdealStateWithAssignment(String resourceName,
 }
 return preferenceList;
   }
+
+  private Map getBaseline(
+  AssignmentMetadataStore assignmentMetadataStore, CurrentStateOutput 
currentStateOutput,
+  Set resources) throws HelixRebalanceException {
+Map currentBaseline;
+try {
+  currentBaseline = assignmentMetadataStore.getBaseline();
+} catch (HelixException hex) {
+  LOG.warn("Failed to get the current baseline assignment. Use the current 
states instead",
+  hex);
+  currentBaseline = getCurrentStateAssingment(currentStateOutput, 
resources);
+} catch (Exception ex) {
+  throw new HelixRebalanceException(
+  "Failed to get the current baseline assignment because of unexpected 
error.",
+  HelixRebalanceException.Type.INVALID_REBALANCER_STATUS, ex);
+}
+return currentBaseline;
+  }
+
+  private Map getBestPossibleAssignment(
+  AssignmentMetadataStore assignmentMetadataStore, CurrentStateOutput 
currentStateOutput,
+  Set resources) throws HelixRebalanceException {
+Map currentBestAssignment;
+try {
+  // TODO fix the deserialize
 
 Review comment:
   Isn't it because you never wrote to bestPossibleAssignment? Didn't we agree 
that you'll handle it in the rebalancer?


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:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@helix.apache.org
For additional commands, e-mail: reviews-h...@helix.apache.org