You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@storm.apache.org by et...@apache.org on 2020/06/08 14:17:10 UTC

[storm] branch master updated: [STORM-3638] Add fast process liveness check (#3273)

This is an automated email from the ASF dual-hosted git repository.

ethanli pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
     new 5c8d2fb  [STORM-3638] Add fast process liveness check (#3273)
5c8d2fb is described below

commit 5c8d2fb85d7a1e86b0f1f31246d6376f835f9e86
Author: Bipin Prasad <bi...@yahoo.com>
AuthorDate: Mon Jun 8 09:16:52 2020 -0500

    [STORM-3638] Add fast process liveness check (#3273)
---
 .../apache/storm/daemon/supervisor/Container.java  | 408 ++++++++++++++++++---
 .../storm/daemon/supervisor/ContainerTest.java     |  95 +++++
 2 files changed, 462 insertions(+), 41 deletions(-)

diff --git a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/Container.java b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/Container.java
index 966ea54..83e5a84 100644
--- a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/Container.java
+++ b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/Container.java
@@ -25,16 +25,19 @@ import java.io.File;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.io.Writer;
-import java.lang.ProcessBuilder.Redirect;
 import java.nio.file.Files;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.ConcurrentHashMap;
+
 import org.apache.commons.lang.StringUtils;
 import org.apache.storm.Config;
 import org.apache.storm.DaemonConfig;
@@ -62,12 +65,12 @@ import org.yaml.snakeyaml.Yaml;
  * Represents a container that a worker will run in.
  */
 public abstract class Container implements Killable {
-
     private static final Logger LOG = LoggerFactory.getLogger(Container.class);
     private static final String MEMORY_USED_METRIC = "UsedMemory";
     private static final String SYSTEM_COMPONENT_ID = "System";
     private static final String INVALID_EXECUTOR_ID = "-1";
     private static final String INVALID_STREAM_ID = "None";
+    private static final Map<String, Integer> cachedUserToUidMap = new ConcurrentHashMap<>();
 
     private final Meter numCleanupExceptions;
     private final Meter numKillExceptions;
@@ -252,28 +255,32 @@ public abstract class Container implements Killable {
      *
      * @throws IOException on any error
      */
-    protected boolean isProcessAlive(long pid, String user) throws IOException {
+    public static boolean isProcessAlive(long pid, String user) throws IOException {
         if (ServerUtils.IS_ON_WINDOWS) {
             return isWindowsProcessAlive(pid, user);
         }
         return isPosixProcessAlive(pid, user);
     }
 
-    private boolean isWindowsProcessAlive(long pid, String user) throws IOException {
+    private static boolean isWindowsProcessAlive(long pid, String user) throws IOException {
         boolean ret = false;
+        LOG.debug("CMD: tasklist /fo list /fi \"pid eq {}\" /v", pid);
         ProcessBuilder pb = new ProcessBuilder("tasklist", "/fo", "list", "/fi", "pid eq " + pid, "/v");
-        pb.redirectError(Redirect.INHERIT);
-        Process p = pb.start();
-        try (BufferedReader in = new BufferedReader(new InputStreamReader(p.getInputStream()))) {
-            String read;
-            while ((read = in.readLine()) != null) {
-                if (read.contains("User Name:")) { //Check for : in case someone called their user "User Name"
+        pb.redirectError(ProcessBuilder.Redirect.INHERIT);
+        try (BufferedReader in = new BufferedReader(new InputStreamReader(pb.start().getInputStream()))) {
+            int lineNo = 0;
+            String line;
+            while ((line = in.readLine()) != null) {
+                lineNo++;
+                LOG.debug("CMD=LINE#{}: {}", lineNo, line);
+                if (line.contains("User Name:")) { //Check for : in case someone called their user "User Name"
                     //This line contains the user name for the pid we're looking up
                     //Example line: "User Name:    exampleDomain\exampleUser"
-                    List<String> userNameLineSplitOnWhitespace = Arrays.asList(read.split(":"));
+                    List<String> userNameLineSplitOnWhitespace = Arrays.asList(line.split(":"));
                     if (userNameLineSplitOnWhitespace.size() == 2) {
                         List<String> userAndMaybeDomain = Arrays.asList(userNameLineSplitOnWhitespace.get(1).trim().split("\\\\"));
                         String processUser = userAndMaybeDomain.size() == 2 ? userAndMaybeDomain.get(1) : userAndMaybeDomain.get(0);
+                        processUser = processUser.trim();
                         if (user.equals(processUser)) {
                             ret = true;
                         } else {
@@ -281,7 +288,7 @@ public abstract class Container implements Killable {
                         }
                     } else {
                         LOG.error("Received unexpected output from tasklist command. Expected one colon in user name line. Line was {}",
-                            read);
+                                line);
                     }
                     break;
                 }
@@ -290,49 +297,368 @@ public abstract class Container implements Killable {
         return ret;
     }
 
-    private boolean isPosixProcessAlive(long pid, String user) throws IOException {
-        boolean ret = false;
+    private static boolean isPosixProcessAlive(long pid, String user) throws IOException {
+        LOG.debug("CMD: ps -o user -p {}", pid);
         ProcessBuilder pb = new ProcessBuilder("ps", "-o", "user", "-p", String.valueOf(pid));
-        pb.redirectError(Redirect.INHERIT);
-        Process p = pb.start();
-        try (BufferedReader in = new BufferedReader(new InputStreamReader(p.getInputStream()))) {
-            String first = in.readLine();
-            assert ("USER".equals(first));
-            String processUser;
-            while ((processUser = in.readLine()) != null) {
-                if (user.equals(processUser)) {
-                    ret = true;
-                    break;
-                } else {
-                    LOG.info("Found {} running as {}, but expected it to be {}", pid, processUser, user);
+        pb.redirectError(ProcessBuilder.Redirect.INHERIT);
+        try (BufferedReader in = new BufferedReader(new InputStreamReader(pb.start().getInputStream()))) {
+            int lineNo = 1;
+            String line = in.readLine();
+            LOG.debug("CMD-LINE#{}: {}", lineNo, line);
+            if (!"USER".equals(line.trim())) {
+                LOG.error("Expecting first line to contain USER, found \"{}\"", line);
+                return false;
+            }
+            while ((line = in.readLine()) != null) {
+                lineNo++;
+                LOG.debug("CMD-LINE#{}: {}", lineNo, line);
+                line = line.trim();
+                if (user.equals(line)) {
+                    return true;
                 }
+                LOG.info("Found {} running as {}, but expected it to be {}", pid, line, user);
             }
+        } catch (IOException ex) {
+            String err = String.format("Cannot read output of command \"ps -o user -p %d\"", pid);
+            throw new IOException(err, ex);
         }
-        return ret;
+        return false;
+    }
+
+    /**
+     * Are any of the processes alive and running for the specified user. If collection is empty or null
+     * then the return value is trivially false.
+     *
+     * @param pids the PIDs of the running processes
+     * @param user the user that is expected to own that process
+     * @return true if any one of the processes is owned by user and alive, else false
+     * @throws IOException on I/O exception
+     */
+    public static boolean isAnyProcessAlive(Collection<Long> pids, String user) throws IOException {
+        if (pids == null || pids.isEmpty()) {
+            return false;
+        }
+
+        if (ServerUtils.IS_ON_WINDOWS) {
+            return isAnyWindowsProcessAlive(pids, user);
+        }
+        return isAnyPosixProcessAlive(pids, user);
+    }
+
+    /**
+     * Are any of the processes alive and running for the specified userId. If collection is empty or null
+     * then the return value is trivially false.
+     *
+     * @param pids the PIDs of the running processes
+     * @param uid the user that is expected to own that process
+     * @return true if any one of the processes is owned by user and alive, else false
+     * @throws IOException on I/O exception
+     */
+    public static boolean isAnyProcessAlive(Collection<Long> pids, int uid) throws IOException {
+        if (pids == null || pids.isEmpty()) {
+            return false;
+        }
+        if (ServerUtils.IS_ON_WINDOWS) {
+            return isAnyWindowsProcessAlive(pids, uid);
+        }
+        return isAnyPosixProcessAlive(pids, uid);
+    }
+
+    /**
+     * Find if any of the Windows processes are alive and owned by the specified user.
+     * Command reference https://docs.microsoft.com/en-us/windows-server/administration/windows-commands/tasklist.
+     *
+     * @param pids the PIDs of the running processes
+     * @param user the user that is expected to own that process
+     * @return true if any one of the processes is owned by user and alive, else false
+     * @throws IOException on I/O exception
+     */
+    private static boolean isAnyWindowsProcessAlive(Collection<Long> pids, String user) throws IOException {
+        List<String> cmdArgs = new ArrayList<>();
+        cmdArgs.add("tasklist");
+        cmdArgs.add("/fo");
+        cmdArgs.add("list");
+        pids.forEach(pid -> {
+            cmdArgs.add("/fi");
+            cmdArgs.add("pid eq " + pid);
+        });
+        cmdArgs.add("/v");
+        LOG.debug("CMD: {}", String.join(" ", cmdArgs));
+        ProcessBuilder pb = new ProcessBuilder(cmdArgs);
+        pb.redirectError(ProcessBuilder.Redirect.INHERIT);
+        List<String> unexpectedUsers = new ArrayList<>();
+        try (BufferedReader in = new BufferedReader(new InputStreamReader(pb.start().getInputStream()))) {
+            int lineNo = 0;
+            String line;
+            while ((line = in.readLine()) != null) {
+                lineNo++;
+                LOG.debug("CMD-LINE#{}: {}", lineNo, line);
+                if (line.contains("User Name:")) { //Check for : in case someone called their user "User Name"
+                    //This line contains the user name for the pid we're looking up
+                    //Example line: "User Name:    exampleDomain\exampleUser"
+                    List<String> userNameLineSplitOnWhitespace = Arrays.asList(line.split(":"));
+                    if (userNameLineSplitOnWhitespace.size() == 2) {
+                        List<String> userAndMaybeDomain = Arrays.asList(userNameLineSplitOnWhitespace.get(1).trim().split("\\\\"));
+                        String processUser = userAndMaybeDomain.size() == 2 ? userAndMaybeDomain.get(1) : userAndMaybeDomain.get(0);
+                        processUser = processUser.trim();
+                        if (user.equals(processUser)) {
+                            return true;
+                        }
+                        unexpectedUsers.add(processUser);
+                    } else {
+                        LOG.error("Received unexpected output from tasklist command. Expected one colon in user name line. Line was {}",
+                                line);
+                    }
+                }
+            }
+        } catch (IOException ex) {
+            String err = String.format("Cannot read output of command \"%s\"", String.join(" ", cmdArgs));
+            throw new IOException(err, ex);
+        }
+        String pidsAsStr = StringUtils.join(pids, ",");
+        if (unexpectedUsers.isEmpty()) {
+            LOG.info("None of the processes {} are alive", pidsAsStr);
+        } else {
+            LOG.info("{} of the Processes {} are running as user(s) {}: but expected user is {}",
+                    unexpectedUsers.size(), pidsAsStr, String.join(",", new TreeSet<>(unexpectedUsers)), user);
+        }
+        return false;
+    }
+
+    /**
+     * Find if any of the Windows processes are alive and owned by the specified userId.
+     * This overridden method is provided for symmetry, but is not implemented.
+     *
+     * @param pids the PIDs of the running processes
+     * @param uid the user that is expected to own that process
+     * @return true if any one of the processes is owned by user and alive, else false
+     * @throws IOException on I/O exception
+     */
+    private static boolean isAnyWindowsProcessAlive(Collection<Long> pids, int uid) throws IOException {
+        throw new IllegalArgumentException("UID is not supported on Windows");
+    }
+
+    /**
+     * Are any of the processes alive and running for the specified user.
+     *
+     * @param pids the PIDs of the running processes
+     * @param user the user that is expected to own that process
+     * @return true if any one of the processes is owned by user and alive, else false
+     * @throws IOException on I/O exception
+     */
+    private static boolean isAnyPosixProcessAlive(Collection<Long> pids, String user) throws IOException {
+        String pidParams = StringUtils.join(pids, ",");
+        LOG.debug("CMD: ps -o user -p {}", pidParams);
+        ProcessBuilder pb = new ProcessBuilder("ps", "-o", "user", "-p", pidParams);
+        pb.redirectError(ProcessBuilder.Redirect.INHERIT);
+        List<String> unexpectedUsers = new ArrayList<>();
+        try (BufferedReader in = new BufferedReader(new InputStreamReader(pb.start().getInputStream()))) {
+            int lineNo = 1;
+            String line = in.readLine();
+            LOG.debug("CMD-LINE#{}: {}", lineNo, line);
+            if (!"USER".equals(line.trim())) {
+                LOG.error("Expecting first line to contain USER, found \"{}\"", line);
+                return false;
+            }
+            while ((line = in.readLine()) != null) {
+                lineNo++;
+                LOG.debug("CMD-LINE#{}: {}", lineNo, line);
+                line = line.trim();
+                if (user.equals(line)) {
+                    return true;
+                }
+                unexpectedUsers.add(line);
+            }
+        } catch (IOException ex) {
+            String err = String.format("Cannot read output of command \"ps -o user -p %s\"", pidParams);
+            throw new IOException(err, ex);
+        }
+        if (unexpectedUsers.isEmpty()) {
+            LOG.info("None of the processes {} are alive", pidParams);
+        } else {
+            LOG.info("{} of {} Processes {} are running as user(s) {}: but expected user is {}",
+                    unexpectedUsers.size(), pids.size(), pidParams, String.join(",", new TreeSet<>(unexpectedUsers)), user);
+        }
+        return false;
+    }
+
+    /**
+     * Are any of the processes alive and running for the specified UID.
+     *
+     * @param pids the PIDs of the running processes
+     * @param uid the userId that is expected to own that process
+     * @return true if any one of the processes is owned by user and alive, else false
+     * @throws IOException on I/O exception
+     */
+    private static boolean isAnyPosixProcessAlive(Collection<Long> pids, int uid) throws IOException {
+        String pidParams = StringUtils.join(pids, ",");
+        LOG.debug("CMD: ps -o uid -p {}", pidParams);
+        ProcessBuilder pb = new ProcessBuilder("ps", "-o", "uid", "-p", pidParams);
+        pb.redirectError(ProcessBuilder.Redirect.INHERIT);
+        List<String> unexpectedUsers = new ArrayList<>();
+        try (BufferedReader in = new BufferedReader(new InputStreamReader(pb.start().getInputStream()))) {
+            int lineNo = 1;
+            String line = in.readLine();
+            LOG.debug("CMD-LINE#{}: {}", lineNo, line);
+            if (!"UID".equals(line.trim())) {
+                LOG.error("Expecting first line to contain UID, found \"{}\"", line);
+                return false;
+            }
+            while ((line = in.readLine()) != null) {
+                lineNo++;
+                LOG.debug("CMD-LINE#{}: {}", lineNo, line);
+                line = line.trim();
+                try {
+                    if (uid == Integer.parseInt(line)) {
+                        return true;
+                    }
+                } catch (Exception ex) {
+                    LOG.warn("Expecting UID integer but got {} in output of ps command", line);
+                }
+                unexpectedUsers.add(line);
+            }
+        } catch (IOException ex) {
+            String err = String.format("Cannot read output of command \"ps -o uid -p %s\"", pidParams);
+            throw new IOException(err, ex);
+        }
+        if (unexpectedUsers.isEmpty()) {
+            LOG.info("None of the processes {} are alive", pidParams);
+        } else {
+            LOG.info("{} of {} Processes {} are running as UIDs {}: but expected userId is {}",
+                    unexpectedUsers.size(), pids.size(), pidParams, String.join(",", new TreeSet<>(unexpectedUsers)), uid);
+        }
+        return false;
+    }
+
+    /**
+     * Get the userId for a user name. This works on Posix systems by using "id -u" command.
+     * Throw IllegalArgumentException on Windows.
+     *
+     * @param user username to be converted to UID. This is optional, in which case current user is returned.
+     * @return UID for the specified user (if supplied), else UID of current user, -1 upon Exception.
+     */
+    public static int getUserId(String user) {
+        if (ServerUtils.IS_ON_WINDOWS) {
+            throw new IllegalArgumentException("Not supported in Windows platform");
+        }
+        List<String> cmdArgs = new ArrayList<>();
+        cmdArgs.add("id");
+        cmdArgs.add("-u");
+        if (user != null && !user.isEmpty()) {
+            cmdArgs.add(user);
+        }
+        LOG.debug("CMD: {}", String.join(" ", cmdArgs));
+        ProcessBuilder pb = new ProcessBuilder(cmdArgs);
+        pb.redirectError(ProcessBuilder.Redirect.INHERIT);
+        try (BufferedReader in = new BufferedReader(new InputStreamReader(pb.start().getInputStream()))) {
+            String line = in.readLine();
+            LOG.debug("CMD-LINE#1: {}", line);
+            try {
+                return Integer.parseInt(line.trim());
+            } catch (NumberFormatException ex) {
+                LOG.error("Expecting UID integer but got {} in output of \"id -u {}\" command", line, user);
+                return -1;
+            }
+        } catch (IOException ex) {
+            LOG.error(String.format("Cannot read output of command \"%s\"", String.join(" ", cmdArgs)), ex);
+            return -1;
+        }
+    }
+
+    /**
+     * Get the userId of the onwer of the path by running "ls -dn path" command.
+     * This command works on Posix systems only.
+     *
+     * @param fpath full path to the file or directory.
+     * @return UID for the specified if successful, -1 upon failure.
+     */
+    public static int getPathOwnerUid(String fpath) {
+        if (ServerUtils.IS_ON_WINDOWS) {
+            throw new IllegalArgumentException("Not supported in Windows platform");
+        }
+        File f = new File(fpath);
+        if (!f.exists()) {
+            LOG.error("Cannot determine owner of non-existent file {}", fpath);
+            return -1;
+        }
+        LOG.debug("CMD: ls -dn {}", fpath);
+        ProcessBuilder pb = new ProcessBuilder("ls", "-dn", fpath);
+        pb.redirectError(ProcessBuilder.Redirect.INHERIT);
+        try (BufferedReader in = new BufferedReader(new InputStreamReader(pb.start().getInputStream()))) {
+            String line = in.readLine();
+            LOG.debug("CMD-OUTLINE: {}", line);
+            line = line.trim();
+            String[] parts = line.split("\\s+");
+            if (parts.length < 3) {
+                LOG.error("Expecting at least 3 space separated fields in \"ls -dn {}\" output, got {}", fpath, line);
+                return -1;
+            }
+            try {
+                return Integer.parseInt(parts[2]);
+            } catch (NumberFormatException ex) {
+                LOG.error("Expecting at third field {} to be numeric UID \"ls -dn {}\" output, got {}", parts[2], fpath, line);
+                return -1;
+            }
+        } catch (IOException ex) {
+            LOG.error(String.format("Cannot read output of command \"ls -dn %s\"", fpath), ex);
+            return -1;
+        }
+    }
+
+    /**
+     * Get UID of the owner to the workerId Root directory.
+     *
+     * @return User ID (UID) of owner of the workerId root directory, -1 if directory is missing.
+     */
+    private int getWorkerPathOwnerUid(String workerId) {
+        return getPathOwnerUid(ConfigUtils.workerRoot(conf, workerId));
     }
-    
+
+    /**
+     * Find if all processes for the user on workId are dead.
+     * This method attempts to optimize the calls by:
+     * <p>
+     *     <li>creating a collection of ProcessIds and checking all of them at once</li>
+     *     <li>using userId one Posix systems instead of user</li>
+     * </p>
+     *
+     * @return true if all processes for the user are dead on the worker
+     * @throws IOException if external commands have exception.
+     */
     @Override
     public boolean areAllProcessesDead() throws IOException {
         Set<Long> pids = getAllPids();
         String user = getRunWorkerAsUser();
 
         boolean allDead = true;
-        for (Long pid : pids) {
-            LOG.debug("Checking if pid {} owner {} is alive", pid, user);
-            if (!isProcessAlive(pid, user)) {
-                LOG.debug("{}: PID {} is dead", workerId, pid);
+        try {
+            if (pids.isEmpty()) {
+                return true;
+            }
+            if (ServerUtils.IS_ON_WINDOWS) {
+                return allDead = !isAnyProcessAlive(pids, user);
+            }
+            // optimized for Posix - try to use uid
+            if (!cachedUserToUidMap.containsKey(user)) {
+                int uid = getWorkerPathOwnerUid(workerId);
+                if (uid < 0) {
+                    uid = getUserId(user);
+                }
+                if (uid >= 0) {
+                    cachedUserToUidMap.put(user, uid);
+                }
+            }
+            if (cachedUserToUidMap.containsKey(user)) {
+                return allDead = !isAnyProcessAlive(pids, cachedUserToUidMap.get(user));
             } else {
-                allDead = false;
-                break;
+                return allDead = !isAnyProcessAlive(pids, user);
+            }
+        } finally {
+            if (allDead && shutdownTimer != null) {
+                shutdownTimer.stop();
+                shutdownTimer = null;
             }
         }
-        
-        if (allDead && shutdownTimer != null) {
-            shutdownTimer.stop();
-            shutdownTimer = null;
-        }
-
-        return allDead;
     }
 
     @Override
diff --git a/storm-server/src/test/java/org/apache/storm/daemon/supervisor/ContainerTest.java b/storm-server/src/test/java/org/apache/storm/daemon/supervisor/ContainerTest.java
index 198d03f..5c4483d 100644
--- a/storm-server/src/test/java/org/apache/storm/daemon/supervisor/ContainerTest.java
+++ b/storm-server/src/test/java/org/apache/storm/daemon/supervisor/ContainerTest.java
@@ -13,17 +13,25 @@
 package org.apache.storm.daemon.supervisor;
 
 import com.google.common.base.Joiner;
+
+import java.io.BufferedReader;
 import java.io.File;
 import java.io.IOException;
+import java.io.InputStreamReader;
 import java.io.StringWriter;
+import java.nio.file.Files;
+import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+
+import org.apache.commons.lang.RandomStringUtils;
 import org.apache.storm.Config;
 import org.apache.storm.DaemonConfig;
 import org.apache.storm.container.ResourceIsolationInterface;
@@ -31,10 +39,13 @@ import org.apache.storm.daemon.supervisor.Container.ContainerType;
 import org.apache.storm.generated.LocalAssignment;
 import org.apache.storm.generated.ProfileRequest;
 import org.apache.storm.utils.ObjectReader;
+import org.apache.storm.utils.ServerUtils;
 import org.junit.Test;
 import org.yaml.snakeyaml.Yaml;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.mockito.Mockito.any;
 import static org.mockito.Mockito.eq;
@@ -271,4 +282,88 @@ public class ContainerTest {
             return false;
         }
     }
+
+    private Collection<Long> getRunningProcessIds() throws IOException {
+        // get list of few running processes
+        Collection<Long> pids = new ArrayList<>();
+        Process p = Runtime.getRuntime().exec(ServerUtils.IS_ON_WINDOWS ? "tasklist" : "ps -e");
+        try (BufferedReader input = new BufferedReader(new InputStreamReader(p.getInputStream()))) {
+            String line;
+            while ((line = input.readLine()) != null) {
+                line = line.trim();
+                if (line.isEmpty()) {
+                    continue;
+                }
+                try {
+                    pids.add(Long.parseLong(line.split("\\s")[0]));
+                } catch (Exception ex) {
+                    ex.printStackTrace();
+                }
+            }
+        }
+        return pids;
+    }
+
+    @Test
+    public void testIsProcessAlive() throws Exception {
+        // specific selected process should not be alive for a randomly generated user
+        String randomUser = RandomStringUtils.randomAlphanumeric(12);
+
+        // get list of few running processes
+        Collection<Long> pids = getRunningProcessIds();
+        assertFalse(pids.isEmpty());
+        for (long pid: pids) {
+            boolean status = Container.isProcessAlive(pid, randomUser);
+            assertFalse("Random user " + randomUser + " is not expected to own any process", status);
+        }
+
+        boolean status = false;
+        String currentUser = System.getProperty("user.name");
+        for (long pid: pids) {
+            // at least one pid will be owned by the current user (doing the testing)
+            if (Container.isProcessAlive(pid, currentUser)) {
+                status = true;
+                break;
+            }
+        }
+        assertTrue("Expecting user " + currentUser + " to own at least one process", status);
+    }
+
+    @Test
+    public void testIsAnyProcessAlive() throws Exception {
+        // no process should be alive for a randomly generated user
+        String randomUser = RandomStringUtils.randomAlphanumeric(12);
+        Collection<Long> pids = getRunningProcessIds();
+
+        assertFalse(pids.isEmpty());
+        boolean status = Container.isAnyProcessAlive(pids, randomUser);
+        assertFalse("Random user " + randomUser + " is not expected to own any process", status);
+
+        // at least one pid will be owned by the current user (doing the testing)
+        String currentUser = System.getProperty("user.name");
+        status = Container.isAnyProcessAlive(pids, currentUser);
+        assertTrue("Expecting user " + currentUser + " to own at least one process", status);
+
+        if (!ServerUtils.IS_ON_WINDOWS) {
+            // userid test is valid only on Posix platforms
+            int inValidUserId = -1;
+            status = Container.isAnyProcessAlive(pids, inValidUserId);
+            assertFalse("Invalid userId " + randomUser + " is not expected to own any process", status);
+
+            int currentUid = Container.getUserId(null);
+            status = Container.isAnyProcessAlive(pids, currentUid);
+            assertTrue("Expecting uid " + currentUid + " to own at least one process", status);
+        }
+    }
+
+    @Test
+    public void testGetUserId() throws Exception {
+        if (ServerUtils.IS_ON_WINDOWS) {
+            return; // trivially succeed on Windows, since this test is not for Windows platform
+        }
+        int uid1 = Container.getUserId(null);
+        Path p = Files.createTempFile("testGetUser", ".txt");
+        int uid2 = Container.getPathOwnerUid(p.toString());
+        assertEquals("User UID " + uid1 + " is not same as file " + p.toString() + " owner UID of " + uid2, uid1, uid2);
+    }
 }