You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@storm.apache.org by GitBox <gi...@apache.org> on 2020/03/05 20:20:54 UTC

[GitHub] [storm] Ethanlm commented on a change in pull request #3215: Storm3585 - New compact Constraint config including maxNodeCoLocationCnt and incompatibleComponents

Ethanlm commented on a change in pull request #3215: Storm3585 - New compact Constraint config including maxNodeCoLocationCnt and incompatibleComponents
URL: https://github.com/apache/storm/pull/3215#discussion_r388454006
 
 

 ##########
 File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/ConstraintSolverStrategy.java
 ##########
 @@ -38,71 +42,182 @@
 import org.apache.storm.shade.com.google.common.annotations.VisibleForTesting;
 import org.apache.storm.utils.ObjectReader;
 import org.apache.storm.utils.Time;
+import org.apache.storm.validation.ConfigValidation;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 public class ConstraintSolverStrategy extends BaseResourceAwareStrategy {
-    //hard coded max number of states to search
     private static final Logger LOG = LoggerFactory.getLogger(ConstraintSolverStrategy.class);
 
-    //constraints and spreads
-    private Map<String, Map<String, Integer>> constraintMatrix;
-    private HashSet<String> spreadComps = new HashSet<>();
+    public static final String CONSTRAINT_TYPE_MAX_NODE_CO_LOCATION_CNT = "maxNodeCoLocationCnt";
+    public static final String CONSTRAINT_TYPE_INCOMPATIBLE_COMPONENTS = "incompatibleComponents";
 
-    private Map<String, RasNode> nodes;
-    private Map<ExecutorDetails, String> execToComp;
-    private Map<String, Set<ExecutorDetails>> compToExecs;
-    private List<String> favoredNodeIds;
-    private List<String> unFavoredNodeIds;
+    /**
+     * Component constraint as derived from configuration.
+     * This is backward compatible and can parse old style Config.TOPOLOGY_RAS_CONSTRAINTS and Config.TOPOLOGY_SPREAD_COMPONENTS.
+     * New style Config.TOPOLOGY_RAS_CONSTRAINTS is map where each component has a list of other incompatible components
+     * and an optional number that specifies the maximum co-location count for the component on a node.
+     *
+     * <p>comp-1 cannot exist on same worker as comp-2 or comp-3, and at most "2" comp-1 on same node</p>
+     * <p>comp-2 and comp-4 cannot be on same node (missing comp-1 is implied from comp-1 constraint)</p>
+     *
+     *  <p>
+     *      { "comp-1": { "maxNodeCoLocationCnt": 2, "incompatibleComponents": ["comp-2", "comp-3" ] },
+     *        "comp-2": { "incompatibleComponents": [ "comp-4" ] }
+     *      }
+     *  </p>
+     */
+    public static final class ConstraintConfig {
+        private Map<String, Set<String>> incompatibleComponents = new HashMap<>();
+        private Map<String, Integer> maxCoLocationCnts = new HashMap<>(); // maximum node CoLocationCnt for restricted components
 
-    static Map<String, Map<String, Integer>> getConstraintMap(TopologyDetails topo, Set<String> comps) {
-        Map<String, Map<String, Integer>> matrix = new HashMap<>();
-        for (String comp : comps) {
-            matrix.put(comp, new HashMap<>());
-            for (String comp2 : comps) {
-                matrix.get(comp).put(comp2, 0);
-            }
+        ConstraintConfig(TopologyDetails topo) {
+            // getExecutorToComponent().values() also contains system components
+            this(topo.getConf(), Sets.union(topo.getComponents().keySet(), new HashSet(topo.getExecutorToComponent().values())));
         }
-        List<List<String>> constraints = (List<List<String>>) topo.getConf().get(Config.TOPOLOGY_RAS_CONSTRAINTS);
-        if (constraints != null) {
-            for (List<String> constraintPair : constraints) {
-                String comp1 = constraintPair.get(0);
-                String comp2 = constraintPair.get(1);
-                if (!matrix.containsKey(comp1)) {
-                    LOG.warn("Comp: {} declared in constraints is not valid!", comp1);
-                    continue;
+
+        ConstraintConfig(Map<String, Object> conf, Set<String> comps) {
+            // spread constraints, old style may come from Config.TOPOLOGY_RAS_CONSTRAINTS where the target
+            Object rasConstraints = conf.get(Config.TOPOLOGY_RAS_CONSTRAINTS);
+            comps.forEach(k -> incompatibleComponents.computeIfAbsent(k, x -> new HashSet<>()));
+            if (rasConstraints instanceof List) {
+                // old style
+                List<List<String>> constraints = (List<List<String>>) rasConstraints;
+                for (List<String> constraintPair : constraints) {
+                    String comp1 = constraintPair.get(0);
+                    String comp2 = constraintPair.get(1);
+                    if (!comps.contains(comp1)) {
+                        LOG.warn("Comp: {} declared in constraints is not valid!", comp1);
+                        continue;
+                    }
+                    if (!comps.contains(comp2)) {
+                        LOG.warn("Comp: {} declared in constraints is not valid!", comp2);
+                        continue;
+                    }
+                    if (comp1.equals(comp2)) {
+                        maxCoLocationCnts.put(comp1, 1);
 
 Review comment:
   `incompatibleComponents` means the component cannot co-exist on the same worker. It does not necessary mean that there can only be at most one of this component on the same node. 
   
   Here we don't need to (or shouldn't) differentiate whether `comp1.equals(comp2)` or not and we can deal with it just like others.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please 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


With regards,
Apache Git Services