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/25 22:54:56 UTC

[GitHub] [storm] govind-menon opened a new pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning

govind-menon opened a new pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning
URL: https://github.com/apache/storm/pull/3237
 
 
   

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

[GitHub] [storm] govind-menon commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning

Posted by GitBox <gi...@apache.org>.
govind-menon commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning
URL: https://github.com/apache/storm/pull/3237#discussion_r401227537
 
 

 ##########
 File path: storm-server/src/main/java/org/apache/storm/daemon/supervisor/BasicContainer.java
 ##########
 @@ -665,8 +666,10 @@ protected String javaCmd(String cmd) {
         commandList.addAll(classPathParams);
         commandList.add(getWorkerMain(topoVersion));
         commandList.add(topologyId);
+        if (numaId != null) {
+            supervisorId += Constants.NUMA_ID_SEPARATOR + numaId;
 
 Review comment:
   The mkLaunchCommand is in BasicContainer - is there a situation where it wouldn't be called?

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

[GitHub] [storm] bipinprasad commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning
URL: https://github.com/apache/storm/pull/3237#discussion_r405091436
 
 

 ##########
 File path: storm-server/src/main/java/org/apache/storm/daemon/supervisor/timer/SupervisorHeartbeat.java
 ##########
 @@ -87,12 +153,16 @@ private SupervisorInfo buildSupervisorInfo(Map<String, Object> conf, Supervisor
             ret.put(stringNumberEntry.getKey(), stringNumberEntry.getValue().doubleValue());
         }
 
+        LOG.debug(NormalizedResources.RESOURCE_NAME_NORMALIZER.normalizedResourceMap(ret).toString());
         return NormalizedResources.RESOURCE_NAME_NORMALIZER.normalizedResourceMap(ret);
     }
 
     @Override
     public void run() {
-        SupervisorInfo supervisorInfo = buildSupervisorInfo(conf, supervisor);
-        stormClusterState.supervisorHeartbeat(supervisorId, supervisorInfo);
+        Map<String, Object> validatedNumaMap = Utils.getNumaMap(conf);
 
 Review comment:
   I think line should be pulled into buildSupervisorInfo(). It appears that the intent is to create as many supervisorInfo records as there are Config.SUPERVISOR_NUMA_META entries with any leftover creating another SupervisorInfo(). And private buildSupervisorInfo() method is never called with validatedNumaMap of null. Each of these supervisorInfos creating a heartbeat-er.

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

[GitHub] [storm] govind-menon commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning

Posted by GitBox <gi...@apache.org>.
govind-menon commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning
URL: https://github.com/apache/storm/pull/3237#discussion_r403085070
 
 

 ##########
 File path: storm-server/src/main/java/org/apache/storm/daemon/supervisor/BasicContainer.java
 ##########
 @@ -665,8 +666,10 @@ protected String javaCmd(String cmd) {
         commandList.addAll(classPathParams);
         commandList.add(getWorkerMain(topoVersion));
         commandList.add(topologyId);
+        if (numaId != null) {
+            supervisorId += Constants.NUMA_ID_SEPARATOR + numaId;
 
 Review comment:
   I'm putting this back here - there could be/are multiple implementations for how a ResourceIsolation manager could implement NUMA pinning but the supervisor ID MUST contain the numa id in the current implementation. But this supervisor id is effectively only used when the worker is launched. I feel it is appropriate to keep it here so that it's explicitly used as opposed to a super constructor where it can be missed when future changes are maded

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

[GitHub] [storm] Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning
URL: https://github.com/apache/storm/pull/3237#discussion_r400992849
 
 

 ##########
 File path: storm-server/src/test/java/org/apache/storm/TestCgroups.java
 ##########
 @@ -43,6 +45,16 @@
     public void testSetupAndTearDown() throws IOException {
         Config config = new Config();
         config.putAll(Utils.readDefaultConfig());
+
+        Map numaNode = new HashMap();
 
 Review comment:
   Should be `Map<String, Object>`,  `HashMap<>`

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

[GitHub] [storm] agresch commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning

Posted by GitBox <gi...@apache.org>.
agresch commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning
URL: https://github.com/apache/storm/pull/3237#discussion_r401648633
 
 

 ##########
 File path: storm-client/src/jvm/org/apache/storm/Config.java
 ##########
 @@ -1096,6 +1096,24 @@
     @IsPositiveNumber
     @NotNull
     public static final String SUPERVISOR_WORKER_TIMEOUT_SECS = "supervisor.worker.timeout.secs";
+
+    /**
+     * A map with keys mapped to each NUMA Node on the supervisor that will be used
 
 Review comment:
   Can we add a tracking JIRA?

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

[GitHub] [storm] govind-menon merged pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning

Posted by GitBox <gi...@apache.org>.
govind-menon merged pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning
URL: https://github.com/apache/storm/pull/3237
 
 
   

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

[GitHub] [storm] agresch commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning

Posted by GitBox <gi...@apache.org>.
agresch commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning
URL: https://github.com/apache/storm/pull/3237#discussion_r401650052
 
 

 ##########
 File path: storm-client/src/jvm/org/apache/storm/utils/Utils.java
 ##########
 @@ -124,11 +124,47 @@
     private static String memoizedLocalHostnameString = null;
     public static final Pattern TOPOLOGY_KEY_PATTERN = Pattern.compile("^[\\w \\t\\._-]+$", Pattern.UNICODE_CHARACTER_CLASS);
 
+    public static final String NUMA_MEMORY_IN_MB = "numa.memory.mb";
+    public static final String NUMA_CORES = "numa.cores";
+    public static final String NUMA_PORTS = "numa.ports";
+    public static final String NUMA_GENERIC_RESOURCES_MAP = "numa.generic.resources.map";
+
     static {
         localConf = readStormConfig();
         serializationDelegate = getSerializationDelegate(localConf);
     }
 
+    /**
+     * Return supervisor numa configuration.
+     * @param stormConf stormConf
+     * @return getNumaMap
+     */
+    public static Map<String, Object> getNumaMap(Map<String, Object> stormConf) {
 
 Review comment:
   That makes sense.

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

[GitHub] [storm] Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning
URL: https://github.com/apache/storm/pull/3237#discussion_r400994230
 
 

 ##########
 File path: storm-server/src/test/java/org/apache/storm/TestCgroups.java
 ##########
 @@ -43,6 +45,16 @@
     public void testSetupAndTearDown() throws IOException {
         Config config = new Config();
         config.putAll(Utils.readDefaultConfig());
+
+        Map numaNode = new HashMap();
+        numaNode.put(Utils.NUMA_CORES, Collections.singletonList("0"));
+        numaNode.put(Utils.NUMA_PORTS, Collections.singletonList(8081));
+        numaNode.put(Utils.NUMA_MEMORY_IN_MB, 2048);
+        Map numaMap = new HashMap();
 
 Review comment:
   This should include key/value type too.

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

[GitHub] [storm] Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning
URL: https://github.com/apache/storm/pull/3237#discussion_r400950752
 
 

 ##########
 File path: storm-client/src/jvm/org/apache/storm/validation/ConfigValidation.java
 ##########
 @@ -416,6 +423,41 @@ public void validateInteger(String name, Object o) {
         }
     }
 
+    public static class NumaEntryValidator extends Validator {
+
+        @Override
+        public void validateField(String name, Object o) {
+            if (o == null) {
+                return;
+            }
+            Map numa = (Map<String, Object>) o;
+            for (String key : new String[]{NUMA_CORES, NUMA_MEMORY_IN_MB, NUMA_PORTS}) {
+                if (!numa.containsKey(key)) {
+                    throw new IllegalArgumentException(
+                            "The numa configuration key [" + key + "] is missing!"
+                    );
+                }
+            }
+
+            List<Integer> cores = (List<Integer>) numa.get(NUMA_CORES);
+            Set coreSet = new HashSet();
 
 Review comment:
   `Set` should be `Set<Integer>`;  `HashSet` should be `HashSet<>`

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

[GitHub] [storm] Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning
URL: https://github.com/apache/storm/pull/3237#discussion_r402617904
 
 

 ##########
 File path: storm-server/src/main/java/org/apache/storm/daemon/supervisor/BasicContainer.java
 ##########
 @@ -615,7 +616,7 @@ protected String javaCmd(String cmd) {
      * @throws IOException on any error.
      */
     private List<String> mkLaunchCommand(final int memOnheap, final String stormRoot,
-                                         final String jlp) throws IOException {
+                                         final String jlp, final String numaId) throws IOException {
 
 Review comment:
   `numaId` not used anywhere?

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

[GitHub] [storm] Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning
URL: https://github.com/apache/storm/pull/3237#discussion_r400992238
 
 

 ##########
 File path: storm-server/src/main/java/org/apache/storm/container/cgroup/CgroupManager.java
 ##########
 @@ -203,6 +209,30 @@ public void reserveResourcesForWorker(String workerId, Integer totalMem, Integer
             }
         }
 
+        if (numaId != null) {
 
 Review comment:
   Is this a different implementation than what we have internally? 
   Internally you used `numactl`. Here seems to be using cgroup directly. Is this well tested?

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

[GitHub] [storm] Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning
URL: https://github.com/apache/storm/pull/3237#discussion_r400952542
 
 

 ##########
 File path: storm-server/src/main/java/org/apache/storm/container/cgroup/CgroupManager.java
 ##########
 @@ -50,6 +54,7 @@
     private CgroupCommon rootCgroup;
     private String rootDir;
     private Map<String, Object> conf;
+    protected Map<String, Object> validatedNumaMap = new ConcurrentHashMap();
 
 Review comment:
   initialization can be removed 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


With regards,
Apache Git Services

[GitHub] [storm] govind-menon commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning

Posted by GitBox <gi...@apache.org>.
govind-menon commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning
URL: https://github.com/apache/storm/pull/3237#discussion_r403074841
 
 

 ##########
 File path: storm-server/src/main/java/org/apache/storm/container/ResourceIsolationInterface.java
 ##########
 @@ -37,7 +37,7 @@
      * @param workerMemory the amount of memory for the worker or null if not enforced
      * @param workerCpu the amount of cpu for the worker or null if not enforced
      */
-    void reserveResourcesForWorker(String workerId, Integer workerMemory, Integer workerCpu);
+    void reserveResourcesForWorker(String workerId, Integer workerMemory, Integer workerCpu, String numaId);
 
 Review comment:
   Added

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

[GitHub] [storm] Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning
URL: https://github.com/apache/storm/pull/3237#discussion_r402615193
 
 

 ##########
 File path: storm-server/src/main/java/org/apache/storm/container/ResourceIsolationInterface.java
 ##########
 @@ -37,7 +37,7 @@
      * @param workerMemory the amount of memory for the worker or null if not enforced
      * @param workerCpu the amount of cpu for the worker or null if not enforced
      */
-    void reserveResourcesForWorker(String workerId, Integer workerMemory, Integer workerCpu);
+    void reserveResourcesForWorker(String workerId, Integer workerMemory, Integer workerCpu, String numaId);
 
 Review comment:
   missing `@param numaId`

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

[GitHub] [storm] agresch commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning

Posted by GitBox <gi...@apache.org>.
agresch commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning
URL: https://github.com/apache/storm/pull/3237#discussion_r401651869
 
 

 ##########
 File path: storm-client/src/jvm/org/apache/storm/utils/Utils.java
 ##########
 @@ -124,11 +124,47 @@
     private static String memoizedLocalHostnameString = null;
     public static final Pattern TOPOLOGY_KEY_PATTERN = Pattern.compile("^[\\w \\t\\._-]+$", Pattern.UNICODE_CHARACTER_CLASS);
 
+    public static final String NUMA_MEMORY_IN_MB = "numa.memory.mb";
+    public static final String NUMA_CORES = "numa.cores";
+    public static final String NUMA_PORTS = "numa.ports";
+    public static final String NUMA_GENERIC_RESOURCES_MAP = "numa.generic.resources.map";
+
     static {
         localConf = readStormConfig();
         serializationDelegate = getSerializationDelegate(localConf);
     }
 
+    /**
+     * Return supervisor numa configuration.
+     * @param stormConf stormConf
+     * @return getNumaMap
+     */
+    public static Map<String, Object> getNumaMap(Map<String, Object> stormConf) {
+        Object numa = stormConf.get(Config.SUPERVISOR_NUMA_META);
+        if (numa == null) {
+            return Collections.emptyMap();
+        }
+        return (Map<String, Object>) numa;
+    }
+
+    /**
+     * getNumaIdForPort.
 
 Review comment:
   This one seems to be used only in storm-server.  Can we make this one SupervisorUtils?
   
   Please update the java doc to indicate this is specific to a single supervisor.  stormConf could be renamed supervisorConf for clarity on 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


With regards,
Apache Git Services

[GitHub] [storm] Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning
URL: https://github.com/apache/storm/pull/3237#discussion_r401240239
 
 

 ##########
 File path: storm-server/src/main/java/org/apache/storm/daemon/supervisor/BasicContainer.java
 ##########
 @@ -665,8 +666,10 @@ protected String javaCmd(String cmd) {
         commandList.addAll(classPathParams);
         commandList.add(getWorkerMain(topoVersion));
         commandList.add(topologyId);
+        if (numaId != null) {
+            supervisorId += Constants.NUMA_ID_SEPARATOR + numaId;
 
 Review comment:
   I mostly worried about the order. If it's not called yet but accessed somewhere else, the result can be unexpected. We shouldn't implicitly change the meaning of the attribute `supervisorId`. If we want to attach "numa" to it, do it earlier as earliest as possible so we know everywhere we see a `supervisorId` in Container, it has "numa"

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

[GitHub] [storm] govind-menon commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning

Posted by GitBox <gi...@apache.org>.
govind-menon commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning
URL: https://github.com/apache/storm/pull/3237#discussion_r401893100
 
 

 ##########
 File path: storm-client/src/jvm/org/apache/storm/Config.java
 ##########
 @@ -1096,6 +1096,24 @@
     @IsPositiveNumber
     @NotNull
     public static final String SUPERVISOR_WORKER_TIMEOUT_SECS = "supervisor.worker.timeout.secs";
+
+    /**
+     * A map with keys mapped to each NUMA Node on the supervisor that will be used
 
 Review comment:
   https://issues.apache.org/jira/browse/STORM-3615

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

[GitHub] [storm] govind-menon commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning

Posted by GitBox <gi...@apache.org>.
govind-menon commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning
URL: https://github.com/apache/storm/pull/3237#discussion_r406264259
 
 

 ##########
 File path: storm-server/src/main/java/org/apache/storm/daemon/supervisor/timer/SupervisorHeartbeat.java
 ##########
 @@ -87,12 +153,16 @@ private SupervisorInfo buildSupervisorInfo(Map<String, Object> conf, Supervisor
             ret.put(stringNumberEntry.getKey(), stringNumberEntry.getValue().doubleValue());
         }
 
+        LOG.debug(NormalizedResources.RESOURCE_NAME_NORMALIZER.normalizedResourceMap(ret).toString());
         return NormalizedResources.RESOURCE_NAME_NORMALIZER.normalizedResourceMap(ret);
     }
 
     @Override
     public void run() {
-        SupervisorInfo supervisorInfo = buildSupervisorInfo(conf, supervisor);
-        stormClusterState.supervisorHeartbeat(supervisorId, supervisorInfo);
+        Map<String, Object> validatedNumaMap = Utils.getNumaMap(conf);
 
 Review comment:
   The supervisor info does one thing - it creates an object with information about the actual supervisor. I think it's better to not add something like the NUMA heartbeating to its purview.

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

[GitHub] [storm] Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning
URL: https://github.com/apache/storm/pull/3237#discussion_r400949079
 
 

 ##########
 File path: storm-client/src/jvm/org/apache/storm/utils/Utils.java
 ##########
 @@ -124,11 +124,47 @@
     private static String memoizedLocalHostnameString = null;
     public static final Pattern TOPOLOGY_KEY_PATTERN = Pattern.compile("^[\\w \\t\\._-]+$", Pattern.UNICODE_CHARACTER_CLASS);
 
+    public static final String NUMA_MEMORY_IN_MB = "numa.memory.mb";
+    public static final String NUMA_CORES = "numa.cores";
+    public static final String NUMA_PORTS = "numa.ports";
+    public static final String NUMA_GENERIC_RESOURCES_MAP = "numa.generic.resources.map";
+
     static {
         localConf = readStormConfig();
         serializationDelegate = getSerializationDelegate(localConf);
     }
 
+    /**
+     * Return supervisor numa configuration.
+     * @param stormConf stormConf
+     * @return getNumaMap
+     */
+    public static Map<String, Object> getNumaMap(Map<String, Object> stormConf) {
+        Object numa = stormConf.get(Config.SUPERVISOR_NUMA_META);
+        if (numa == null) {
+            return Collections.emptyMap();
+        }
+        return (Map<String, Object>) numa;
+    }
+
+    /**
+     * getNumaIdForPort.
+     * @param port port
+     * @param stormConf stormConf
+     * @return getNumaIdForPort
+     */
+    public static String getNumaIdForPort(Number port, Map<String, Object> stormConf) {
+        Map<String, Object> validatedNumaMap = getNumaMap(stormConf);
+        for (Entry<String, Object> numaEntry : validatedNumaMap.entrySet()) {
+            Map numaMap  = (Map<String, Object>) numaEntry.getValue();
 
 Review comment:
   `Map` should be `Map<String, Object>`

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

[GitHub] [storm] Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning
URL: https://github.com/apache/storm/pull/3237#discussion_r400992849
 
 

 ##########
 File path: storm-server/src/test/java/org/apache/storm/TestCgroups.java
 ##########
 @@ -43,6 +45,16 @@
     public void testSetupAndTearDown() throws IOException {
         Config config = new Config();
         config.putAll(Utils.readDefaultConfig());
+
+        Map numaNode = new HashMap();
 
 Review comment:
   Should be `Map<String, Object>`

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

[GitHub] [storm] Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning
URL: https://github.com/apache/storm/pull/3237#discussion_r404851228
 
 

 ##########
 File path: storm-server/src/main/java/org/apache/storm/daemon/supervisor/Container.java
 ##########
 @@ -130,6 +131,10 @@ protected Container(ContainerType type, Map<String, Object> conf, String supervi
         this.ops = ops;
         this.conf = conf;
         this.supervisorId = supervisorId;
+        String numaId = SupervisorUtils.getNumaIdForPort(port, conf);
+        if (numaId != null) {
+            this.supervisorId +=  Constants.NUMA_ID_SEPARATOR + numaId;
 
 Review comment:
   This should be removed according to the change in BasicContainer. mkLaunchCommand

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

[GitHub] [storm] govind-menon commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning

Posted by GitBox <gi...@apache.org>.
govind-menon commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning
URL: https://github.com/apache/storm/pull/3237#discussion_r401222719
 
 

 ##########
 File path: storm-client/src/jvm/org/apache/storm/Config.java
 ##########
 @@ -1096,6 +1096,24 @@
     @IsPositiveNumber
     @NotNull
     public static final String SUPERVISOR_WORKER_TIMEOUT_SECS = "supervisor.worker.timeout.secs";
+
+    /**
+     * A map with keys mapped to each NUMA Node on the supervisor that will be used
 
 Review comment:
   I'll put up a different PR for the documentation

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

[GitHub] [storm] Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning
URL: https://github.com/apache/storm/pull/3237#discussion_r401240239
 
 

 ##########
 File path: storm-server/src/main/java/org/apache/storm/daemon/supervisor/BasicContainer.java
 ##########
 @@ -665,8 +666,10 @@ protected String javaCmd(String cmd) {
         commandList.addAll(classPathParams);
         commandList.add(getWorkerMain(topoVersion));
         commandList.add(topologyId);
+        if (numaId != null) {
+            supervisorId += Constants.NUMA_ID_SEPARATOR + numaId;
 
 Review comment:
   I mostly worried about the order. If it's not called yet but accessed somewhere else, the result can be unexpected. We shouldn't implicitly change the meaning of the attribute `supervisorId`. If we want to attach "numa" to it, do it as earliest as possible so we know everywhere we see a `supervisorId` in Container, it has "numa"

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

[GitHub] [storm] Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning
URL: https://github.com/apache/storm/pull/3237#discussion_r402614589
 
 

 ##########
 File path: storm-server/src/main/java/org/apache/storm/container/cgroup/CgroupManager.java
 ##########
 @@ -102,6 +109,9 @@ public void prepare(Map<String, Object> conf) throws IOException {
             throw new RuntimeException("Cgroup error, please check /proc/cgroups");
         }
         this.prepareSubSystem(this.conf);
+        validatedNumaMap = Utils.getNumaMap(conf);
+        workerToNumaId = new ConcurrentHashMap();
+        validatedNumaMap = new ConcurrentHashMap();
 
 Review comment:
   L112 and L114 conflict.  And `validatedNumaMap` doesn't seem to be used anywhere

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

[GitHub] [storm] Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning
URL: https://github.com/apache/storm/pull/3237#discussion_r405004649
 
 

 ##########
 File path: storm-server/src/main/java/org/apache/storm/container/cgroup/CgroupManager.java
 ##########
 @@ -102,6 +109,8 @@ public void prepare(Map<String, Object> conf) throws IOException {
             throw new RuntimeException("Cgroup error, please check /proc/cgroups");
         }
         this.prepareSubSystem(this.conf);
+        validatedNumaMap = Utils.getNumaMap(conf);
 
 Review comment:
   Is this used anywhere?

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

[GitHub] [storm] agresch commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning

Posted by GitBox <gi...@apache.org>.
agresch commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning
URL: https://github.com/apache/storm/pull/3237#discussion_r400976278
 
 

 ##########
 File path: storm-client/src/jvm/org/apache/storm/Config.java
 ##########
 @@ -1096,6 +1096,24 @@
     @IsPositiveNumber
     @NotNull
     public static final String SUPERVISOR_WORKER_TIMEOUT_SECS = "supervisor.worker.timeout.secs";
+
+    /**
+     * A map with keys mapped to each NUMA Node on the supervisor that will be used
 
 Review comment:
   I think we should add a doc describing this feature.

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

[GitHub] [storm] Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning
URL: https://github.com/apache/storm/pull/3237#discussion_r404841435
 
 

 ##########
 File path: storm-server/src/main/java/org/apache/storm/daemon/supervisor/BasicContainer.java
 ##########
 @@ -665,8 +666,10 @@ protected String javaCmd(String cmd) {
         commandList.addAll(classPathParams);
         commandList.add(getWorkerMain(topoVersion));
         commandList.add(topologyId);
+        if (numaId != null) {
+            supervisorId += Constants.NUMA_ID_SEPARATOR + numaId;
 
 Review comment:
   This way is also fine.  Originally it was
   
   ```
   if (numaId != null) {
              this.supervisorId += Constants.NUMA_ID_SEPARATOR + numaId;
   }
   ```
   and it changed `this.supervisorId`. 
   
   Currently `this.supervisorId` is not changed, so it is fine.

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

[GitHub] [storm] Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning
URL: https://github.com/apache/storm/pull/3237#discussion_r406321386
 
 

 ##########
 File path: storm-client/src/jvm/org/apache/storm/utils/Utils.java
 ##########
 @@ -124,11 +124,47 @@
     private static String memoizedLocalHostnameString = null;
     public static final Pattern TOPOLOGY_KEY_PATTERN = Pattern.compile("^[\\w \\t\\._-]+$", Pattern.UNICODE_CHARACTER_CLASS);
 
+    public static final String NUMA_MEMORY_IN_MB = "numa.memory.mb";
+    public static final String NUMA_CORES = "numa.cores";
+    public static final String NUMA_PORTS = "numa.ports";
+    public static final String NUMA_GENERIC_RESOURCES_MAP = "numa.generic.resources.map";
+
     static {
         localConf = readStormConfig();
         serializationDelegate = getSerializationDelegate(localConf);
     }
 
+    /**
+     * Return supervisor numa configuration.
+     * @param stormConf stormConf
+     * @return getNumaMap
+     */
+    public static Map<String, Object> getNumaMap(Map<String, Object> stormConf) {
 
 Review comment:
   As I looked again, I realized `getNumaMap` is only used in storm-server and Aaron's previous makes sense  

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

[GitHub] [storm] govind-menon commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning

Posted by GitBox <gi...@apache.org>.
govind-menon commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning
URL: https://github.com/apache/storm/pull/3237#discussion_r401847401
 
 

 ##########
 File path: storm-client/src/jvm/org/apache/storm/utils/Utils.java
 ##########
 @@ -124,11 +124,47 @@
     private static String memoizedLocalHostnameString = null;
     public static final Pattern TOPOLOGY_KEY_PATTERN = Pattern.compile("^[\\w \\t\\._-]+$", Pattern.UNICODE_CHARACTER_CLASS);
 
+    public static final String NUMA_MEMORY_IN_MB = "numa.memory.mb";
+    public static final String NUMA_CORES = "numa.cores";
+    public static final String NUMA_PORTS = "numa.ports";
+    public static final String NUMA_GENERIC_RESOURCES_MAP = "numa.generic.resources.map";
+
     static {
         localConf = readStormConfig();
         serializationDelegate = getSerializationDelegate(localConf);
     }
 
+    /**
+     * Return supervisor numa configuration.
+     * @param stormConf stormConf
+     * @return getNumaMap
+     */
+    public static Map<String, Object> getNumaMap(Map<String, Object> stormConf) {
+        Object numa = stormConf.get(Config.SUPERVISOR_NUMA_META);
+        if (numa == null) {
+            return Collections.emptyMap();
+        }
+        return (Map<String, Object>) numa;
+    }
+
+    /**
+     * getNumaIdForPort.
 
 Review comment:
   https://issues.apache.org/jira/browse/STORM-3615

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

[GitHub] [storm] govind-menon commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning

Posted by GitBox <gi...@apache.org>.
govind-menon commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning
URL: https://github.com/apache/storm/pull/3237#discussion_r401225553
 
 

 ##########
 File path: storm-client/src/jvm/org/apache/storm/utils/Utils.java
 ##########
 @@ -124,11 +124,47 @@
     private static String memoizedLocalHostnameString = null;
     public static final Pattern TOPOLOGY_KEY_PATTERN = Pattern.compile("^[\\w \\t\\._-]+$", Pattern.UNICODE_CHARACTER_CLASS);
 
+    public static final String NUMA_MEMORY_IN_MB = "numa.memory.mb";
+    public static final String NUMA_CORES = "numa.cores";
+    public static final String NUMA_PORTS = "numa.ports";
+    public static final String NUMA_GENERIC_RESOURCES_MAP = "numa.generic.resources.map";
+
     static {
         localConf = readStormConfig();
         serializationDelegate = getSerializationDelegate(localConf);
     }
 
+    /**
+     * Return supervisor numa configuration.
+     * @param stormConf stormConf
+     * @return getNumaMap
+     */
+    public static Map<String, Object> getNumaMap(Map<String, Object> stormConf) {
 
 Review comment:
   These are used in the client side to do topology configuration so I think it's better to keep it here

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [storm] Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning
URL: https://github.com/apache/storm/pull/3237#discussion_r400948900
 
 

 ##########
 File path: storm-client/src/jvm/org/apache/storm/utils/Utils.java
 ##########
 @@ -124,11 +124,47 @@
     private static String memoizedLocalHostnameString = null;
     public static final Pattern TOPOLOGY_KEY_PATTERN = Pattern.compile("^[\\w \\t\\._-]+$", Pattern.UNICODE_CHARACTER_CLASS);
 
+    public static final String NUMA_MEMORY_IN_MB = "numa.memory.mb";
+    public static final String NUMA_CORES = "numa.cores";
+    public static final String NUMA_PORTS = "numa.ports";
+    public static final String NUMA_GENERIC_RESOURCES_MAP = "numa.generic.resources.map";
+
     static {
         localConf = readStormConfig();
         serializationDelegate = getSerializationDelegate(localConf);
     }
 
+    /**
+     * Return supervisor numa configuration.
+     * @param stormConf stormConf
+     * @return getNumaMap
+     */
+    public static Map<String, Object> getNumaMap(Map<String, Object> stormConf) {
+        Object numa = stormConf.get(Config.SUPERVISOR_NUMA_META);
+        if (numa == null) {
+            return Collections.emptyMap();
+        }
+        return (Map<String, Object>) numa;
+    }
+
+    /**
+     * getNumaIdForPort.
+     * @param port port
+     * @param stormConf stormConf
+     * @return getNumaIdForPort
+     */
+    public static String getNumaIdForPort(Number port, Map<String, Object> stormConf) {
 
 Review comment:
   `port` should just be `int` or `Integer`

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

[GitHub] [storm] kishorvpatil commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning

Posted by GitBox <gi...@apache.org>.
kishorvpatil commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning
URL: https://github.com/apache/storm/pull/3237#discussion_r407715241
 
 

 ##########
 File path: storm-server/src/main/java/org/apache/storm/daemon/supervisor/Container.java
 ##########
 @@ -37,6 +37,7 @@
 import java.util.Set;
 import org.apache.commons.lang.StringUtils;
 import org.apache.storm.Config;
+import org.apache.storm.Constants;
 
 Review comment:
   remove unused import

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

[GitHub] [storm] Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning
URL: https://github.com/apache/storm/pull/3237#discussion_r406324039
 
 

 ##########
 File path: storm-server/src/main/java/org/apache/storm/container/cgroup/CgroupManager.java
 ##########
 @@ -50,6 +55,8 @@
     private CgroupCommon rootCgroup;
     private String rootDir;
     private Map<String, Object> conf;
+    private Map<String, String> workerToNumaId;
+    protected Map<String, Object> validatedNumaMap;
 
 Review comment:
   can this be removed?

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

[GitHub] [storm] Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning
URL: https://github.com/apache/storm/pull/3237#discussion_r401240239
 
 

 ##########
 File path: storm-server/src/main/java/org/apache/storm/daemon/supervisor/BasicContainer.java
 ##########
 @@ -665,8 +666,10 @@ protected String javaCmd(String cmd) {
         commandList.addAll(classPathParams);
         commandList.add(getWorkerMain(topoVersion));
         commandList.add(topologyId);
+        if (numaId != null) {
+            supervisorId += Constants.NUMA_ID_SEPARATOR + numaId;
 
 Review comment:
   I mostly worried about the order. If it's not called yet but accessed somewhere else, the result can be unexpected. We shouldn't implicitly change the meaning of the attribute `supervisorId`

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

[GitHub] [storm] Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning
URL: https://github.com/apache/storm/pull/3237#discussion_r400949845
 
 

 ##########
 File path: storm-client/src/jvm/org/apache/storm/validation/ConfigValidation.java
 ##########
 @@ -416,6 +423,41 @@ public void validateInteger(String name, Object o) {
         }
     }
 
+    public static class NumaEntryValidator extends Validator {
+
+        @Override
+        public void validateField(String name, Object o) {
+            if (o == null) {
+                return;
+            }
+            Map numa = (Map<String, Object>) o;
 
 Review comment:
   `Map` should be `Map<String, Object>`

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

[GitHub] [storm] kishorvpatil commented on issue #3237: STORM-3259: Adds NUMA awareness to enable worker pinning

Posted by GitBox <gi...@apache.org>.
kishorvpatil commented on issue #3237: STORM-3259: Adds NUMA awareness to enable worker pinning
URL: https://github.com/apache/storm/pull/3237#issuecomment-613112370
 
 
   Thank you for addressing all comments @govind-menon 

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

[GitHub] [storm] agresch commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning

Posted by GitBox <gi...@apache.org>.
agresch commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning
URL: https://github.com/apache/storm/pull/3237#discussion_r400971846
 
 

 ##########
 File path: storm-client/src/jvm/org/apache/storm/utils/Utils.java
 ##########
 @@ -124,11 +124,47 @@
     private static String memoizedLocalHostnameString = null;
     public static final Pattern TOPOLOGY_KEY_PATTERN = Pattern.compile("^[\\w \\t\\._-]+$", Pattern.UNICODE_CHARACTER_CLASS);
 
+    public static final String NUMA_MEMORY_IN_MB = "numa.memory.mb";
+    public static final String NUMA_CORES = "numa.cores";
+    public static final String NUMA_PORTS = "numa.ports";
+    public static final String NUMA_GENERIC_RESOURCES_MAP = "numa.generic.resources.map";
+
     static {
         localConf = readStormConfig();
         serializationDelegate = getSerializationDelegate(localConf);
     }
 
+    /**
+     * Return supervisor numa configuration.
+     * @param stormConf stormConf
+     * @return getNumaMap
+     */
+    public static Map<String, Object> getNumaMap(Map<String, Object> stormConf) {
 
 Review comment:
   Seems like all these file changes might be better added to SupervisorUtils?

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

[GitHub] [storm] Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3237: STORM-3259: Adds NUMA awareness to enable worker pinning
URL: https://github.com/apache/storm/pull/3237#discussion_r400967075
 
 

 ##########
 File path: storm-server/src/main/java/org/apache/storm/daemon/supervisor/BasicContainer.java
 ##########
 @@ -665,8 +666,10 @@ protected String javaCmd(String cmd) {
         commandList.addAll(classPathParams);
         commandList.add(getWorkerMain(topoVersion));
         commandList.add(topologyId);
+        if (numaId != null) {
+            supervisorId += Constants.NUMA_ID_SEPARATOR + numaId;
 
 Review comment:
   This changes `Container` attribute `supervisorId` when it `mkLaunchCommand`, which means the `supervisorId` is different depending on whether it calles `mkLaunchCommand` or not. I am afraid it might introduce confusions. 

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