[ 
https://issues.apache.org/jira/browse/YARN-11687?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17843013#comment-17843013
 ] 

ASF GitHub Bot commented on YARN-11687:
---------------------------------------

brumi1024 commented on code in PR #6780:
URL: https://github.com/apache/hadoop/pull/6780#discussion_r1587889935


##########
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsV2ResourceCalculator.java:
##########
@@ -0,0 +1,135 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package 
org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.classification.VisibleForTesting;
+import org.apache.hadoop.util.CpuTimeTracker;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+
+/**
+ * A CGroupV2 file-system based Resource calculator without the process tree 
features.
+ *
+ * The feature only works if cluster runs in pure V2 version, because when we 
read the
+ * /proc/{pid}/cgroup file currently we can not handle multiple lines.
+ */
+public class CGroupsV2ResourceCalculator extends 
AbstractCGroupsResourceCalculator {
+  private static final Logger LOG = 
LoggerFactory.getLogger(CGroupsV2ResourceCalculator.class);
+  private final Map<String, String> stats = new ConcurrentHashMap<>();
+
+  @VisibleForTesting
+  String root = "/";

Review Comment:
   Maybe `File.listRoots()`?



##########
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsV2ResourceCalculator.java:
##########
@@ -0,0 +1,135 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package 
org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.classification.VisibleForTesting;
+import org.apache.hadoop.util.CpuTimeTracker;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+
+/**
+ * A CGroupV2 file-system based Resource calculator without the process tree 
features.
+ *
+ * The feature only works if cluster runs in pure V2 version, because when we 
read the
+ * /proc/{pid}/cgroup file currently we can not handle multiple lines.
+ */
+public class CGroupsV2ResourceCalculator extends 
AbstractCGroupsResourceCalculator {
+  private static final Logger LOG = 
LoggerFactory.getLogger(CGroupsV2ResourceCalculator.class);
+  private final Map<String, String> stats = new ConcurrentHashMap<>();
+
+  @VisibleForTesting
+  String root = "/";
+
+  /**
+   * Create resource calculator for the container that has the specified pid.
+   * @param pid A pid from the cgroup or null for all containers
+   */
+  public CGroupsV2ResourceCalculator(String pid) {
+    super(pid);
+  }
+
+  @Override
+  public void initialize() throws YarnException {
+    cpuTimeTracker = new CpuTimeTracker(jiffyLengthMs);
+    cGroupsHandler = ResourceHandlerModule.getCGroupsHandler();
+  }
+
+  @Override
+  public long getCumulativeCpuTime() {
+    // https://docs.kernel.org/admin-guide/cgroup-v2.html#cpu-interface-files
+    return jiffyLengthMs < 0
+        ? UNAVAILABLE
+        : getStat("cpu.stat#usage_usec") * jiffyLengthMs;
+  }
+
+  @Override
+  public long getRssMemorySize(int olderThanAge) {
+    // 
https://docs.kernel.org/admin-guide/cgroup-v2.html#memory-interface-files
+    return 1 < olderThanAge
+        ? UNAVAILABLE
+        : getStat("memory.stat#anon");
+  }
+
+  @Override
+  public long getVirtualMemorySize(int olderThanAge) {
+    // 
https://docs.kernel.org/admin-guide/cgroup-v2.html#memory-interface-files
+    return 1 < olderThanAge
+        ? UNAVAILABLE
+        : getStat("memory.stat#vmalloc");
+  }
+
+  @Override
+  public void updateProcessTree() {
+    try (Stream<Path> cGroupFiles = Files.list(getCGroupPath())){
+      List<Path> statFiles = cGroupFiles
+          .filter(path -> path.toString().endsWith(".stat"))
+          .collect(Collectors.toList());
+      for (Path statFile : statFiles) {
+        String[] lines = fileToString(statFile).split(System.lineSeparator());
+        for (String line: lines) {
+          String[] parts = line.split(" ");
+          stats.put(statFile.getFileName() + "#" + parts[0], parts[1]);
+        }
+      }
+      cpuTimeTracker.updateElapsedJiffies(
+          BigInteger.valueOf(getStat("cpu.stat#usage_usec")), clock.getTime());

Review Comment:
   `cpu.stat#usage_usec` could be a final static string.



##########
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsV2ResourceCalculator.java:
##########
@@ -0,0 +1,135 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package 
org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.classification.VisibleForTesting;
+import org.apache.hadoop.util.CpuTimeTracker;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+
+/**
+ * A CGroupV2 file-system based Resource calculator without the process tree 
features.
+ *
+ * The feature only works if cluster runs in pure V2 version, because when we 
read the
+ * /proc/{pid}/cgroup file currently we can not handle multiple lines.
+ */
+public class CGroupsV2ResourceCalculator extends 
AbstractCGroupsResourceCalculator {
+  private static final Logger LOG = 
LoggerFactory.getLogger(CGroupsV2ResourceCalculator.class);
+  private final Map<String, String> stats = new ConcurrentHashMap<>();
+
+  @VisibleForTesting
+  String root = "/";
+
+  /**
+   * Create resource calculator for the container that has the specified pid.
+   * @param pid A pid from the cgroup or null for all containers
+   */
+  public CGroupsV2ResourceCalculator(String pid) {
+    super(pid);
+  }
+
+  @Override
+  public void initialize() throws YarnException {
+    cpuTimeTracker = new CpuTimeTracker(jiffyLengthMs);
+    cGroupsHandler = ResourceHandlerModule.getCGroupsHandler();
+  }
+
+  @Override
+  public long getCumulativeCpuTime() {
+    // https://docs.kernel.org/admin-guide/cgroup-v2.html#cpu-interface-files
+    return jiffyLengthMs < 0
+        ? UNAVAILABLE
+        : getStat("cpu.stat#usage_usec") * jiffyLengthMs;
+  }
+
+  @Override
+  public long getRssMemorySize(int olderThanAge) {
+    // 
https://docs.kernel.org/admin-guide/cgroup-v2.html#memory-interface-files
+    return 1 < olderThanAge
+        ? UNAVAILABLE
+        : getStat("memory.stat#anon");
+  }
+
+  @Override
+  public long getVirtualMemorySize(int olderThanAge) {
+    // 
https://docs.kernel.org/admin-guide/cgroup-v2.html#memory-interface-files
+    return 1 < olderThanAge
+        ? UNAVAILABLE
+        : getStat("memory.stat#vmalloc");

Review Comment:
   Same here.



##########
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsV2ResourceCalculator.java:
##########
@@ -0,0 +1,135 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package 
org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.classification.VisibleForTesting;
+import org.apache.hadoop.util.CpuTimeTracker;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+
+/**
+ * A CGroupV2 file-system based Resource calculator without the process tree 
features.
+ *
+ * The feature only works if cluster runs in pure V2 version, because when we 
read the
+ * /proc/{pid}/cgroup file currently we can not handle multiple lines.
+ */
+public class CGroupsV2ResourceCalculator extends 
AbstractCGroupsResourceCalculator {
+  private static final Logger LOG = 
LoggerFactory.getLogger(CGroupsV2ResourceCalculator.class);
+  private final Map<String, String> stats = new ConcurrentHashMap<>();
+
+  @VisibleForTesting
+  String root = "/";
+
+  /**
+   * Create resource calculator for the container that has the specified pid.
+   * @param pid A pid from the cgroup or null for all containers
+   */
+  public CGroupsV2ResourceCalculator(String pid) {
+    super(pid);
+  }
+
+  @Override
+  public void initialize() throws YarnException {
+    cpuTimeTracker = new CpuTimeTracker(jiffyLengthMs);

Review Comment:
   The v1 calculator has an isAvailable check here. Is this intentionally 
missed here?



##########
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsV2ResourceCalculator.java:
##########
@@ -0,0 +1,135 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package 
org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.classification.VisibleForTesting;
+import org.apache.hadoop.util.CpuTimeTracker;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+
+/**
+ * A CGroupV2 file-system based Resource calculator without the process tree 
features.
+ *
+ * The feature only works if cluster runs in pure V2 version, because when we 
read the
+ * /proc/{pid}/cgroup file currently we can not handle multiple lines.
+ */
+public class CGroupsV2ResourceCalculator extends 
AbstractCGroupsResourceCalculator {
+  private static final Logger LOG = 
LoggerFactory.getLogger(CGroupsV2ResourceCalculator.class);
+  private final Map<String, String> stats = new ConcurrentHashMap<>();
+
+  @VisibleForTesting
+  String root = "/";
+
+  /**
+   * Create resource calculator for the container that has the specified pid.
+   * @param pid A pid from the cgroup or null for all containers
+   */
+  public CGroupsV2ResourceCalculator(String pid) {
+    super(pid);
+  }
+
+  @Override
+  public void initialize() throws YarnException {
+    cpuTimeTracker = new CpuTimeTracker(jiffyLengthMs);
+    cGroupsHandler = ResourceHandlerModule.getCGroupsHandler();
+  }
+
+  @Override
+  public long getCumulativeCpuTime() {
+    // https://docs.kernel.org/admin-guide/cgroup-v2.html#cpu-interface-files
+    return jiffyLengthMs < 0
+        ? UNAVAILABLE
+        : getStat("cpu.stat#usage_usec") * jiffyLengthMs;

Review Comment:
   The file names could be final strings.
   
   Also getCumulativeCpuTime could be a common method, as the only thing 
different is how we get the number of jiffies.



##########
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsV2ResourceCalculator.java:
##########
@@ -0,0 +1,135 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package 
org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.classification.VisibleForTesting;
+import org.apache.hadoop.util.CpuTimeTracker;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+
+/**
+ * A CGroupV2 file-system based Resource calculator without the process tree 
features.
+ *
+ * The feature only works if cluster runs in pure V2 version, because when we 
read the
+ * /proc/{pid}/cgroup file currently we can not handle multiple lines.
+ */
+public class CGroupsV2ResourceCalculator extends 
AbstractCGroupsResourceCalculator {
+  private static final Logger LOG = 
LoggerFactory.getLogger(CGroupsV2ResourceCalculator.class);
+  private final Map<String, String> stats = new ConcurrentHashMap<>();
+
+  @VisibleForTesting
+  String root = "/";
+
+  /**
+   * Create resource calculator for the container that has the specified pid.
+   * @param pid A pid from the cgroup or null for all containers
+   */
+  public CGroupsV2ResourceCalculator(String pid) {
+    super(pid);
+  }
+
+  @Override
+  public void initialize() throws YarnException {
+    cpuTimeTracker = new CpuTimeTracker(jiffyLengthMs);
+    cGroupsHandler = ResourceHandlerModule.getCGroupsHandler();
+  }
+
+  @Override
+  public long getCumulativeCpuTime() {
+    // https://docs.kernel.org/admin-guide/cgroup-v2.html#cpu-interface-files
+    return jiffyLengthMs < 0
+        ? UNAVAILABLE
+        : getStat("cpu.stat#usage_usec") * jiffyLengthMs;
+  }
+
+  @Override
+  public long getRssMemorySize(int olderThanAge) {
+    // 
https://docs.kernel.org/admin-guide/cgroup-v2.html#memory-interface-files
+    return 1 < olderThanAge
+        ? UNAVAILABLE
+        : getStat("memory.stat#anon");
+  }
+
+  @Override
+  public long getVirtualMemorySize(int olderThanAge) {
+    // 
https://docs.kernel.org/admin-guide/cgroup-v2.html#memory-interface-files
+    return 1 < olderThanAge
+        ? UNAVAILABLE
+        : getStat("memory.stat#vmalloc");
+  }
+
+  @Override
+  public void updateProcessTree() {
+    try (Stream<Path> cGroupFiles = Files.list(getCGroupPath())){
+      List<Path> statFiles = cGroupFiles
+          .filter(path -> path.toString().endsWith(".stat"))
+          .collect(Collectors.toList());
+      for (Path statFile : statFiles) {
+        String[] lines = fileToString(statFile).split(System.lineSeparator());
+        for (String line: lines) {
+          String[] parts = line.split(" ");
+          stats.put(statFile.getFileName() + "#" + parts[0], parts[1]);
+        }
+      }
+      cpuTimeTracker.updateElapsedJiffies(
+          BigInteger.valueOf(getStat("cpu.stat#usage_usec")), clock.getTime());
+      LOG.debug("The {} process has the following stat properties updated: 
{}", pid, stats);
+    } catch (Exception e) {
+      LOG.warn("Failed to read CGroupV2 stats for process: " + pid, e);
+    }
+  }
+
+  private Path getCGroupPath() throws IOException {
+    String relativePath = pid == null
+        ? 
ResourceHandlerModule.getCGroupsHandler().getRelativePathForCGroup("")
+        : getCGroupRelativePathForPid();
+    return Paths.get(cGroupsHandler.getCGroupMountPath(), relativePath);
+  }
+
+  private String getCGroupRelativePathForPid() throws IOException {
+    // https://docs.kernel.org/admin-guide/cgroup-v2.html#processes
+    String pidCGroupsFile = fileToString(Paths.get(root, "proc", pid, 
"cgroup"));

Review Comment:
   The v1 resource calculator gets the proc fs dir as a parameter. Is there a 
reason why that couldn't be common thing?
   
   On a quick glance the 
`CGroupsResourceCalculator.getCGroupRelativePathForPid` method should work here 
as well, have you maybe tried it? 



##########
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsV2ResourceCalculator.java:
##########
@@ -0,0 +1,135 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package 
org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.classification.VisibleForTesting;
+import org.apache.hadoop.util.CpuTimeTracker;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+
+/**
+ * A CGroupV2 file-system based Resource calculator without the process tree 
features.
+ *
+ * The feature only works if cluster runs in pure V2 version, because when we 
read the
+ * /proc/{pid}/cgroup file currently we can not handle multiple lines.
+ */
+public class CGroupsV2ResourceCalculator extends 
AbstractCGroupsResourceCalculator {
+  private static final Logger LOG = 
LoggerFactory.getLogger(CGroupsV2ResourceCalculator.class);
+  private final Map<String, String> stats = new ConcurrentHashMap<>();
+
+  @VisibleForTesting
+  String root = "/";
+
+  /**
+   * Create resource calculator for the container that has the specified pid.
+   * @param pid A pid from the cgroup or null for all containers
+   */
+  public CGroupsV2ResourceCalculator(String pid) {
+    super(pid);
+  }
+
+  @Override
+  public void initialize() throws YarnException {
+    cpuTimeTracker = new CpuTimeTracker(jiffyLengthMs);
+    cGroupsHandler = ResourceHandlerModule.getCGroupsHandler();
+  }
+
+  @Override
+  public long getCumulativeCpuTime() {
+    // https://docs.kernel.org/admin-guide/cgroup-v2.html#cpu-interface-files
+    return jiffyLengthMs < 0
+        ? UNAVAILABLE
+        : getStat("cpu.stat#usage_usec") * jiffyLengthMs;
+  }
+
+  @Override
+  public long getRssMemorySize(int olderThanAge) {
+    // 
https://docs.kernel.org/admin-guide/cgroup-v2.html#memory-interface-files
+    return 1 < olderThanAge
+        ? UNAVAILABLE
+        : getStat("memory.stat#anon");
+  }
+
+  @Override
+  public long getVirtualMemorySize(int olderThanAge) {
+    // 
https://docs.kernel.org/admin-guide/cgroup-v2.html#memory-interface-files
+    return 1 < olderThanAge
+        ? UNAVAILABLE
+        : getStat("memory.stat#vmalloc");
+  }
+
+  @Override
+  public void updateProcessTree() {
+    try (Stream<Path> cGroupFiles = Files.list(getCGroupPath())){
+      List<Path> statFiles = cGroupFiles
+          .filter(path -> path.toString().endsWith(".stat"))
+          .collect(Collectors.toList());
+      for (Path statFile : statFiles) {
+        String[] lines = fileToString(statFile).split(System.lineSeparator());
+        for (String line: lines) {
+          String[] parts = line.split(" ");
+          stats.put(statFile.getFileName() + "#" + parts[0], parts[1]);
+        }
+      }
+      cpuTimeTracker.updateElapsedJiffies(
+          BigInteger.valueOf(getStat("cpu.stat#usage_usec")), clock.getTime());
+      LOG.debug("The {} process has the following stat properties updated: 
{}", pid, stats);
+    } catch (Exception e) {
+      LOG.warn("Failed to read CGroupV2 stats for process: " + pid, e);
+    }

Review Comment:
   For example: this is essentially the same method as 
`CGroupsResourceCalculator.updateProcessTree` just operates on different files. 
Basically read the jiffies, update the elapsed jiffies and then update the 
virtual/physical memory variables. We can edit the v1 method slightly but it 
would be good to keep one implementation only. 



##########
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsV2ResourceCalculator.java:
##########
@@ -0,0 +1,135 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package 
org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.classification.VisibleForTesting;
+import org.apache.hadoop.util.CpuTimeTracker;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+
+/**
+ * A CGroupV2 file-system based Resource calculator without the process tree 
features.
+ *
+ * The feature only works if cluster runs in pure V2 version, because when we 
read the
+ * /proc/{pid}/cgroup file currently we can not handle multiple lines.
+ */
+public class CGroupsV2ResourceCalculator extends 
AbstractCGroupsResourceCalculator {
+  private static final Logger LOG = 
LoggerFactory.getLogger(CGroupsV2ResourceCalculator.class);
+  private final Map<String, String> stats = new ConcurrentHashMap<>();
+
+  @VisibleForTesting
+  String root = "/";
+
+  /**
+   * Create resource calculator for the container that has the specified pid.
+   * @param pid A pid from the cgroup or null for all containers
+   */
+  public CGroupsV2ResourceCalculator(String pid) {
+    super(pid);
+  }
+
+  @Override
+  public void initialize() throws YarnException {
+    cpuTimeTracker = new CpuTimeTracker(jiffyLengthMs);
+    cGroupsHandler = ResourceHandlerModule.getCGroupsHandler();
+  }
+
+  @Override
+  public long getCumulativeCpuTime() {
+    // https://docs.kernel.org/admin-guide/cgroup-v2.html#cpu-interface-files
+    return jiffyLengthMs < 0
+        ? UNAVAILABLE
+        : getStat("cpu.stat#usage_usec") * jiffyLengthMs;
+  }
+
+  @Override
+  public long getRssMemorySize(int olderThanAge) {
+    // 
https://docs.kernel.org/admin-guide/cgroup-v2.html#memory-interface-files
+    return 1 < olderThanAge
+        ? UNAVAILABLE
+        : getStat("memory.stat#anon");

Review Comment:
   Same as above.





> Update CGroupsResourceCalculator to track usages using cgroupv2
> ---------------------------------------------------------------
>
>                 Key: YARN-11687
>                 URL: https://issues.apache.org/jira/browse/YARN-11687
>             Project: Hadoop YARN
>          Issue Type: Sub-task
>            Reporter: Benjamin Teke
>            Assignee: Bence Kosztolnik
>            Priority: Major
>              Labels: pull-request-available
>
> [CGroupsResourceCalculator|https://github.com/apache/hadoop/blob/f609460bda0c2bd87dd3580158e549e2f34f14d5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsResourceCalculator.java]
>  should also be updated to handle the cgroup v2 changes.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: yarn-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: yarn-issues-h...@hadoop.apache.org

Reply via email to