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/08/28 20:35:52 UTC

[GitHub] [storm] bipinprasad opened a new pull request #3326: [STORM-3692] Handle UID return from Files.getOwner()

bipinprasad opened a new pull request #3326:
URL: https://github.com/apache/storm/pull/3326


   ## What is the purpose of the change
   
   *(Pushback to Community) Owner on /proc/pid is sometimes returned as UID instead of username in production. Handle this special case.*
   
   ## How was the change tested
   
   *New test was written to mock this condition and test the correct behavior of ServerUtils.isAnyPosixProcessPidDirAlive()*


----------------------------------------------------------------
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



[GitHub] [storm] Ethanlm merged pull request #3326: [STORM-3692] Handle UID return from Files.getOwner()

Posted by GitBox <gi...@apache.org>.
Ethanlm merged pull request #3326:
URL: https://github.com/apache/storm/pull/3326


   


----------------------------------------------------------------
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



[GitHub] [storm] bipinprasad commented on a change in pull request #3326: [STORM-3692] Handle UID return from Files.getOwner()

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3326:
URL: https://github.com/apache/storm/pull/3326#discussion_r481327827



##########
File path: storm-server/src/test/java/org/apache/storm/utils/ServerUtilsTest.java
##########
@@ -162,6 +185,191 @@ public void testGetUserId() throws Exception {
         int uid1 = ServerUtils.getUserId(null);
         Path p = Files.createTempFile("testGetUser", ".txt");
         int uid2 = ServerUtils.getPathOwnerUid(p.toString());
+        if (!p.toFile().delete()) {
+            LOG.warn("Could not delete tempoary file {}", p);
+        }
         assertEquals("User UID " + uid1 + " is not same as file " + p.toString() + " owner UID of " + uid2, uid1, uid2);
     }
+
+    @Test
+    public void testIsAnyProcessPosixProcessPidDirAlive() throws IOException {
+        final String testName = "testIsAnyProcessPosixProcessPidDirAlive";
+        List<String> errors = new ArrayList<>();
+        int maxPidCnt = 5;
+        if (ServerUtils.IS_ON_WINDOWS) {
+            LOG.info("{}: test cannot be run on Windows. Marked as successful", testName);
+            return;
+        }
+        final Path parentDir = Paths.get("/proc");
+        if (!parentDir.toFile().exists()) {
+            LOG.info("{}: test cannot be run on system without process directory {}, os.name={}",
+                    testName, parentDir, System.getProperty("os.name"));
+            // check if we can get process id on this Posix system - testing test code, useful on Mac
+            String cmd = "/bin/sleep 10";
+            if (getPidOfPosixProcess(Runtime.getRuntime().exec(cmd), errors) < 0) {
+                fail(String.format("%s: Cannot obtain process id for executed command \"%s\"\n%s",
+                        testName, cmd, String.join("\n\t", errors)));
+            }
+            return;
+        }
+        // Create processes and wait for their termination
+        Set<Long> observables = new HashSet<>();
+
+        for (int i = 0 ; i < maxPidCnt ; i++) {
+            String cmd = "sleep 2000";
+            Process process = Runtime.getRuntime().exec(cmd);
+            long pid = getPidOfPosixProcess(process, errors);
+            LOG.info("{}: ({}) ran process \"{}\" with pid={}", testName, i, cmd, pid);
+            if (pid < 0) {
+                String e = String.format("%s: (%d) Cannot obtain process id for executed command \"%s\"", testName, i, cmd);
+                errors.add(e);
+                LOG.error(e);
+                continue;
+            }
+            observables.add(pid);
+        }
+        String userName = System.getProperty("user.name");
+        // now kill processes one by one
+        List<Long> pidList = new ArrayList<>(observables);
+        final long processKillIntervalMs = 2000;
+        for (int i = 0; i < pidList.size(); i++) {
+            long pid = pidList.get(i);
+            LOG.info("{}: ({}) Sleeping for {} milliseconds before kill", testName, i, processKillIntervalMs);
+            if (sleepInterrupted(processKillIntervalMs)) {
+                return;
+            }
+            Runtime.getRuntime().exec("kill -9 " + pid);
+            LOG.info("{}: ({}) Sleeping for {} milliseconds after kill", testName, i, processKillIntervalMs);
+            if (sleepInterrupted(processKillIntervalMs)) {
+                return;
+            }
+            boolean pidDirsAvailable = ServerUtils.isAnyPosixProcessPidDirAlive(observables, userName);
+            if (i < pidList.size() - 1) {
+                if (pidDirsAvailable) {
+                    LOG.info("{}: ({}) Found existing process directories before killing last process", testName, i);
+                } else {
+                    String e = String.format("%s: (%d) Found no existing process directories before killing last process", testName, i);
+                    errors.add(e);
+                    LOG.error(e);
+                }
+            } else {
+                if (pidDirsAvailable) {
+                    String e = String.format("%s: (%d) Found existing process directories after killing last process", testName, i);
+                    errors.add(e);
+                    LOG.error(e);
+                } else {
+                    LOG.info("{}: ({}) Found no existing process directories after killing last process", testName, i);
+                }
+            }
+        }
+        if (!errors.isEmpty()) {
+            fail(String.format("There are %d failures in test:\n\t%s", errors.size(), String.join("\n\t", errors)));
+        }
+    }
+
+

Review comment:
       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



[GitHub] [storm] Ethanlm commented on a change in pull request #3326: [STORM-3692] Handle UID return from Files.getOwner()

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3326:
URL: https://github.com/apache/storm/pull/3326#discussion_r481367315



##########
File path: storm-server/src/test/java/org/apache/storm/utils/ServerUtilsTest.java
##########
@@ -162,6 +185,191 @@ public void testGetUserId() throws Exception {
         int uid1 = ServerUtils.getUserId(null);
         Path p = Files.createTempFile("testGetUser", ".txt");
         int uid2 = ServerUtils.getPathOwnerUid(p.toString());
+        if (!p.toFile().delete()) {
+            LOG.warn("Could not delete tempoary file {}", p);
+        }
         assertEquals("User UID " + uid1 + " is not same as file " + p.toString() + " owner UID of " + uid2, uid1, uid2);
     }
+
+    @Test
+    public void testIsAnyProcessPosixProcessPidDirAlive() throws IOException {
+        final String testName = "testIsAnyProcessPosixProcessPidDirAlive";
+        List<String> errors = new ArrayList<>();
+        int maxPidCnt = 5;
+        if (ServerUtils.IS_ON_WINDOWS) {
+            LOG.info("{}: test cannot be run on Windows. Marked as successful", testName);
+            return;
+        }
+        final Path parentDir = Paths.get("/proc");
+        if (!parentDir.toFile().exists()) {
+            LOG.info("{}: test cannot be run on system without process directory {}, os.name={}",
+                    testName, parentDir, System.getProperty("os.name"));
+            // check if we can get process id on this Posix system - testing test code, useful on Mac
+            String cmd = "/bin/sleep 10";
+            if (getPidOfPosixProcess(Runtime.getRuntime().exec(cmd), errors) < 0) {
+                fail(String.format("%s: Cannot obtain process id for executed command \"%s\"\n%s",
+                        testName, cmd, String.join("\n\t", errors)));
+            }
+            return;
+        }
+        // Create processes and wait for their termination
+        Set<Long> observables = new HashSet<>();
+
+        for (int i = 0 ; i < maxPidCnt ; i++) {
+            String cmd = "sleep 2000";
+            Process process = Runtime.getRuntime().exec(cmd);
+            long pid = getPidOfPosixProcess(process, errors);
+            LOG.info("{}: ({}) ran process \"{}\" with pid={}", testName, i, cmd, pid);
+            if (pid < 0) {
+                String e = String.format("%s: (%d) Cannot obtain process id for executed command \"%s\"", testName, i, cmd);
+                errors.add(e);
+                LOG.error(e);
+                continue;
+            }
+            observables.add(pid);
+        }
+        String userName = System.getProperty("user.name");
+        // now kill processes one by one
+        List<Long> pidList = new ArrayList<>(observables);
+        final long processKillIntervalMs = 2000;
+        for (int i = 0; i < pidList.size(); i++) {
+            long pid = pidList.get(i);
+            LOG.info("{}: ({}) Sleeping for {} milliseconds before kill", testName, i, processKillIntervalMs);
+            if (sleepInterrupted(processKillIntervalMs)) {
+                return;
+            }
+            Runtime.getRuntime().exec("kill -9 " + pid);
+            LOG.info("{}: ({}) Sleeping for {} milliseconds after kill", testName, i, processKillIntervalMs);
+            if (sleepInterrupted(processKillIntervalMs)) {
+                return;
+            }
+            boolean pidDirsAvailable = ServerUtils.isAnyPosixProcessPidDirAlive(observables, userName);
+            if (i < pidList.size() - 1) {
+                if (pidDirsAvailable) {
+                    LOG.info("{}: ({}) Found existing process directories before killing last process", testName, i);
+                } else {
+                    String e = String.format("%s: (%d) Found no existing process directories before killing last process", testName, i);
+                    errors.add(e);
+                    LOG.error(e);
+                }
+            } else {
+                if (pidDirsAvailable) {
+                    String e = String.format("%s: (%d) Found existing process directories after killing last process", testName, i);
+                    errors.add(e);
+                    LOG.error(e);
+                } else {
+                    LOG.info("{}: ({}) Found no existing process directories after killing last process", testName, i);
+                }
+            }
+        }
+        if (!errors.isEmpty()) {
+            fail(String.format("There are %d failures in test:\n\t%s", errors.size(), String.join("\n\t", errors)));
+        }
+    }
+
+
+    /**
+     * Simulate the production scenario where the owner of the process directory is sometimes returned as the
+     * UID instead of user. This scenario is simulated by calling
+     * {@link ServerUtils#isAnyPosixProcessPidDirAlive(Collection, String, boolean)} with the last parameter
+     * set to true as well as false.
+     *
+     * @throws Exception on I/O exception
+     */
+    @Test
+    public void testIsAnyPosixProcessPidDirAliveMockingFileOwnerUid() throws Exception {
+        File procDir = new File("/proc");
+        if (!procDir.exists()) {
+            LOG.info("Test testIsAnyPosixProcessPidDirAlive is designed to run on systems with /proc directory only, marking as success");
+            return;
+        }
+        Collection<Long> pids = getRunningProcessIds();
+        assertFalse(pids.isEmpty());
+
+        for (int i = 0 ; i < 2 ; i++) {
+            boolean mockFileOwnerToUid = (i % 2 == 0);
+            // at least one pid will be owned by the current user (doing the testing)
+            String currentUser = System.getProperty("user.name");
+            boolean status = ServerUtils.isAnyPosixProcessPidDirAlive(pids, currentUser, mockFileOwnerToUid);
+            String err = String.format("(mockFileOwnerToUid=%s) Expecting user %s to own at least one process",
+                    mockFileOwnerToUid, currentUser);
+            assertTrue(err, status);
+        }
+    }
+
+    /**
+     * Make the best effort to obtain the Process ID from the Process object. Thus staying entirely with the JVM.
+     *
+     * @param p Process instance returned upon executing {@link Runtime#exec(String)}.
+     * @param errors Populate errors when PID is a negative number.
+     * @return positive PID upon success, otherwise negative.
+     */
+    private synchronized long getPidOfPosixProcess(Process p, List<String> errors) {

Review comment:
       https://github.com/apache/storm/blob/master/storm-server/src/test/java/org/apache/storm/utils/ServerUtilsTest.java#L266
   This method already exists in ServerUtilsTest (from your another merged PR)




----------------------------------------------------------------
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



[GitHub] [storm] Ethanlm commented on a change in pull request #3326: [STORM-3692] Handle UID return from Files.getOwner()

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3326:
URL: https://github.com/apache/storm/pull/3326#discussion_r481367538



##########
File path: storm-server/src/test/java/org/apache/storm/utils/ServerUtilsTest.java
##########
@@ -162,6 +185,191 @@ public void testGetUserId() throws Exception {
         int uid1 = ServerUtils.getUserId(null);
         Path p = Files.createTempFile("testGetUser", ".txt");
         int uid2 = ServerUtils.getPathOwnerUid(p.toString());
+        if (!p.toFile().delete()) {
+            LOG.warn("Could not delete tempoary file {}", p);
+        }
         assertEquals("User UID " + uid1 + " is not same as file " + p.toString() + " owner UID of " + uid2, uid1, uid2);
     }
+
+    @Test
+    public void testIsAnyProcessPosixProcessPidDirAlive() throws IOException {

Review comment:
       You need to rebase on latest master. Then you should be able to see the conflicts/duplicates




----------------------------------------------------------------
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



[GitHub] [storm] Ethanlm commented on a change in pull request #3326: [STORM-3692] Handle UID return from Files.getOwner()

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3326:
URL: https://github.com/apache/storm/pull/3326#discussion_r481167830



##########
File path: storm-server/src/main/java/org/apache/storm/utils/ServerUtils.java
##########
@@ -1217,9 +1199,127 @@ public static boolean areAllProcessesDead(Map<String, Object> conf, String user,
             }
         }
         if (cachedUserToUidMap.containsKey(user)) {
-            return allDead = !ServerUtils.isAnyProcessAlive(pids, cachedUserToUidMap.get(user));
+            return !ServerUtils.isAnyProcessAlive(pids, cachedUserToUidMap.get(user));
         } else {
-            return allDead = !ServerUtils.isAnyProcessAlive(pids, user);
+            return !ServerUtils.isAnyProcessAlive(pids, user);
         }
     }
+
+    /**
+     * Find if the process is alive using the existence of /proc/&lt;pid&gt; directory
+     * owned by the supplied user. This is an alternative to "ps -p pid -u uid" command
+     * used in {@link #isAnyPosixProcessAlive(Collection, int)}
+     *
+     * <p>
+     * Processes are tracked using the existence of the directory "/proc/&lt;pid&gt;
+     * For each of the supplied PIDs, their PID directory is checked for existence and ownership
+     * by the specified uid.
+     * </p>
+     *
+     * @param pids Process IDs that need to be monitored for liveness
+     * @param user 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
+     */
+    public static boolean isAnyPosixProcessPidDirAlive(Collection<Long> pids, String user) throws IOException {
+        return isAnyPosixProcessPidDirAlive(pids, user, false);
+    }
+
+    /**
+     * Find if the process is alive using the existence of /proc/&lt;pid&gt; directory
+     * owned by the supplied user. This is an alternative to "ps -p pid -u uid" command
+     * used in {@link #isAnyPosixProcessAlive(Collection, int)}
+     *
+     * <p>
+     * Processes are tracked using the existence of the directory "/proc/&lt;pid&gt;
+     * For each of the supplied PIDs, their PID directory is checked for existence and ownership
+     * by the specified uid.
+     * </p>
+     *
+     * @param pids Process IDs that need to be monitored for liveness
+     * @param user the userId that is expected to own that process
+     * @param mockFileOwnerToUid if true (used for testing), then convert File.owner to UID
+     * @return true if any one of the processes is owned by user and alive, else false
+     * @throws IOException on I/O exception
+     */
+    @VisibleForTesting
+    public static boolean isAnyPosixProcessPidDirAlive(Collection<Long> pids, String user, boolean mockFileOwnerToUid)
+            throws IOException {
+        File procDir = new File("/proc");
+        if (!procDir.exists()) {
+            throw new IOException("Missing process directory " + procDir.getAbsolutePath() + ": method not supported on "
+                    + "os.name=" + System.getProperty("os.name"));
+        }
+        for (long pid: pids) {
+            File pidDir = new File(procDir, String.valueOf(pid));
+            if (!pidDir.exists()) {
+                continue;
+            }
+            // check if existing process is owned by the specified user, if not, the process is dead
+            String pidUser;
+            try {
+                pidUser = Files.getOwner(pidDir.toPath()).getName();
+            } catch (NoSuchFileException ex) {
+                continue; // process died before the user can be checked
+            }
+            if (mockFileOwnerToUid) {
+                // code activated in testing to simulate Files.getOwner returning UID (which sometimes happens in runtime)
+                if (StringUtils.isNumeric(pidUser)) {
+                    LOG.info("Skip mocking, since owner {} of pidDir {} is already numeric", pidUser, pidDir);
+                } else {
+                    Integer uid = cachedUserToUidMap.get(pidUser);
+                    if (uid == null) {
+                        uid = ServerUtils.getUserId(pidUser);
+                        if (uid < 0) {
+                            String err = String.format("Cannot get UID for %s, while mocking the owner of pidDir %s",
+                                    pidUser, pidDir.getAbsolutePath());
+                            throw new IOException(err);
+                        }
+                        cachedUserToUidMap.put(pidUser, uid);
+                        LOG.info("Found UID {} for {}, while mocking the owner of pidDir {}", uid, pidUser, pidDir);
+                    } else {
+                        LOG.info("Found cached UID {} for {}, while mocking the owner of pidDir {}", uid, pidUser, pidDir);
+                    }
+                    pidUser = String.valueOf(uid);
+                }
+            }
+            //sometimes uid is returned instead of username - if so, try to convert and compare with uid
+            if (StringUtils.isNumeric(pidUser)) {
+                // numeric pidUser - this is UID
+                LOG.debug("Process directory {} owner is uid={}", pidDir, pidUser);
+                int pidUid = Integer.parseInt(pidUser);
+                Integer uid = cachedUserToUidMap.get(user);
+                if (uid == null) {
+                    uid = ServerUtils.getUserId(user);
+                    if (uid < 0) {
+                        String err = String.format("Cannot get uid for %s to compare with owner id=%d of process directory %s",
+                                user, pidUid, pidDir.getAbsolutePath());
+                        throw new IOException(err);
+                    }
+                    cachedUserToUidMap.put(user, uid);
+                }
+                if (uid == pidUid) {
+                    LOG.debug("Process {} is alive and owned by user {}/{}", pid, user, uid);
+                    return true;
+                } else {
+                    LOG.info("Prior process is dead, since directory {} owner {} is not same as expected user {}/{}, "
+                            + "likely pid {} was reused for a new process for uid {}", pidDir, pidUser, user, uid, pid, uid);

Review comment:
       Should the last "uid" be "pidUid"?
   
   It is not easy to distinguish "user" vs "pidUser", "uid" vs "pidUid". I suggest to change them to be 
   `actualUser vs expectedUser, actualUid vs expectedUid`

##########
File path: storm-server/src/test/java/org/apache/storm/utils/ServerUtilsTest.java
##########
@@ -162,6 +185,191 @@ public void testGetUserId() throws Exception {
         int uid1 = ServerUtils.getUserId(null);
         Path p = Files.createTempFile("testGetUser", ".txt");
         int uid2 = ServerUtils.getPathOwnerUid(p.toString());
+        if (!p.toFile().delete()) {
+            LOG.warn("Could not delete tempoary file {}", p);
+        }
         assertEquals("User UID " + uid1 + " is not same as file " + p.toString() + " owner UID of " + uid2, uid1, uid2);
     }
+
+    @Test
+    public void testIsAnyProcessPosixProcessPidDirAlive() throws IOException {
+        final String testName = "testIsAnyProcessPosixProcessPidDirAlive";
+        List<String> errors = new ArrayList<>();
+        int maxPidCnt = 5;
+        if (ServerUtils.IS_ON_WINDOWS) {
+            LOG.info("{}: test cannot be run on Windows. Marked as successful", testName);
+            return;
+        }
+        final Path parentDir = Paths.get("/proc");
+        if (!parentDir.toFile().exists()) {
+            LOG.info("{}: test cannot be run on system without process directory {}, os.name={}",
+                    testName, parentDir, System.getProperty("os.name"));
+            // check if we can get process id on this Posix system - testing test code, useful on Mac
+            String cmd = "/bin/sleep 10";
+            if (getPidOfPosixProcess(Runtime.getRuntime().exec(cmd), errors) < 0) {
+                fail(String.format("%s: Cannot obtain process id for executed command \"%s\"\n%s",
+                        testName, cmd, String.join("\n\t", errors)));
+            }
+            return;
+        }
+        // Create processes and wait for their termination
+        Set<Long> observables = new HashSet<>();
+
+        for (int i = 0 ; i < maxPidCnt ; i++) {
+            String cmd = "sleep 2000";
+            Process process = Runtime.getRuntime().exec(cmd);
+            long pid = getPidOfPosixProcess(process, errors);
+            LOG.info("{}: ({}) ran process \"{}\" with pid={}", testName, i, cmd, pid);
+            if (pid < 0) {
+                String e = String.format("%s: (%d) Cannot obtain process id for executed command \"%s\"", testName, i, cmd);
+                errors.add(e);
+                LOG.error(e);
+                continue;
+            }
+            observables.add(pid);
+        }
+        String userName = System.getProperty("user.name");
+        // now kill processes one by one
+        List<Long> pidList = new ArrayList<>(observables);
+        final long processKillIntervalMs = 2000;
+        for (int i = 0; i < pidList.size(); i++) {
+            long pid = pidList.get(i);
+            LOG.info("{}: ({}) Sleeping for {} milliseconds before kill", testName, i, processKillIntervalMs);
+            if (sleepInterrupted(processKillIntervalMs)) {
+                return;
+            }
+            Runtime.getRuntime().exec("kill -9 " + pid);
+            LOG.info("{}: ({}) Sleeping for {} milliseconds after kill", testName, i, processKillIntervalMs);
+            if (sleepInterrupted(processKillIntervalMs)) {
+                return;
+            }
+            boolean pidDirsAvailable = ServerUtils.isAnyPosixProcessPidDirAlive(observables, userName);
+            if (i < pidList.size() - 1) {
+                if (pidDirsAvailable) {
+                    LOG.info("{}: ({}) Found existing process directories before killing last process", testName, i);
+                } else {
+                    String e = String.format("%s: (%d) Found no existing process directories before killing last process", testName, i);
+                    errors.add(e);
+                    LOG.error(e);
+                }
+            } else {
+                if (pidDirsAvailable) {
+                    String e = String.format("%s: (%d) Found existing process directories after killing last process", testName, i);
+                    errors.add(e);
+                    LOG.error(e);
+                } else {
+                    LOG.info("{}: ({}) Found no existing process directories after killing last process", testName, i);
+                }
+            }
+        }
+        if (!errors.isEmpty()) {
+            fail(String.format("There are %d failures in test:\n\t%s", errors.size(), String.join("\n\t", errors)));
+        }
+    }
+
+
+    /**
+     * Simulate the production scenario where the owner of the process directory is sometimes returned as the
+     * UID instead of user. This scenario is simulated by calling
+     * {@link ServerUtils#isAnyPosixProcessPidDirAlive(Collection, String, boolean)} with the last parameter
+     * set to true as well as false.
+     *
+     * @throws Exception on I/O exception
+     */
+    @Test
+    public void testIsAnyPosixProcessPidDirAliveMockingFileOwnerUid() throws Exception {
+        File procDir = new File("/proc");
+        if (!procDir.exists()) {
+            LOG.info("Test testIsAnyPosixProcessPidDirAlive is designed to run on systems with /proc directory only, marking as success");
+            return;
+        }
+        Collection<Long> pids = getRunningProcessIds();
+        assertFalse(pids.isEmpty());
+
+        for (int i = 0 ; i < 2 ; i++) {
+            boolean mockFileOwnerToUid = (i % 2 == 0);

Review comment:
       I would suggest to use Parameterized Test here.

##########
File path: storm-server/src/test/java/org/apache/storm/utils/ServerUtilsTest.java
##########
@@ -162,6 +185,191 @@ public void testGetUserId() throws Exception {
         int uid1 = ServerUtils.getUserId(null);
         Path p = Files.createTempFile("testGetUser", ".txt");
         int uid2 = ServerUtils.getPathOwnerUid(p.toString());
+        if (!p.toFile().delete()) {
+            LOG.warn("Could not delete tempoary file {}", p);
+        }
         assertEquals("User UID " + uid1 + " is not same as file " + p.toString() + " owner UID of " + uid2, uid1, uid2);
     }
+
+    @Test
+    public void testIsAnyProcessPosixProcessPidDirAlive() throws IOException {
+        final String testName = "testIsAnyProcessPosixProcessPidDirAlive";
+        List<String> errors = new ArrayList<>();
+        int maxPidCnt = 5;
+        if (ServerUtils.IS_ON_WINDOWS) {
+            LOG.info("{}: test cannot be run on Windows. Marked as successful", testName);
+            return;
+        }
+        final Path parentDir = Paths.get("/proc");
+        if (!parentDir.toFile().exists()) {
+            LOG.info("{}: test cannot be run on system without process directory {}, os.name={}",
+                    testName, parentDir, System.getProperty("os.name"));
+            // check if we can get process id on this Posix system - testing test code, useful on Mac
+            String cmd = "/bin/sleep 10";
+            if (getPidOfPosixProcess(Runtime.getRuntime().exec(cmd), errors) < 0) {
+                fail(String.format("%s: Cannot obtain process id for executed command \"%s\"\n%s",
+                        testName, cmd, String.join("\n\t", errors)));
+            }
+            return;
+        }
+        // Create processes and wait for their termination
+        Set<Long> observables = new HashSet<>();
+
+        for (int i = 0 ; i < maxPidCnt ; i++) {
+            String cmd = "sleep 2000";
+            Process process = Runtime.getRuntime().exec(cmd);
+            long pid = getPidOfPosixProcess(process, errors);
+            LOG.info("{}: ({}) ran process \"{}\" with pid={}", testName, i, cmd, pid);
+            if (pid < 0) {
+                String e = String.format("%s: (%d) Cannot obtain process id for executed command \"%s\"", testName, i, cmd);
+                errors.add(e);
+                LOG.error(e);
+                continue;
+            }
+            observables.add(pid);
+        }
+        String userName = System.getProperty("user.name");
+        // now kill processes one by one
+        List<Long> pidList = new ArrayList<>(observables);
+        final long processKillIntervalMs = 2000;
+        for (int i = 0; i < pidList.size(); i++) {
+            long pid = pidList.get(i);
+            LOG.info("{}: ({}) Sleeping for {} milliseconds before kill", testName, i, processKillIntervalMs);
+            if (sleepInterrupted(processKillIntervalMs)) {
+                return;
+            }
+            Runtime.getRuntime().exec("kill -9 " + pid);
+            LOG.info("{}: ({}) Sleeping for {} milliseconds after kill", testName, i, processKillIntervalMs);
+            if (sleepInterrupted(processKillIntervalMs)) {
+                return;
+            }
+            boolean pidDirsAvailable = ServerUtils.isAnyPosixProcessPidDirAlive(observables, userName);
+            if (i < pidList.size() - 1) {
+                if (pidDirsAvailable) {
+                    LOG.info("{}: ({}) Found existing process directories before killing last process", testName, i);
+                } else {
+                    String e = String.format("%s: (%d) Found no existing process directories before killing last process", testName, i);
+                    errors.add(e);
+                    LOG.error(e);
+                }
+            } else {
+                if (pidDirsAvailable) {
+                    String e = String.format("%s: (%d) Found existing process directories after killing last process", testName, i);
+                    errors.add(e);
+                    LOG.error(e);
+                } else {
+                    LOG.info("{}: ({}) Found no existing process directories after killing last process", testName, i);
+                }
+            }
+        }
+        if (!errors.isEmpty()) {
+            fail(String.format("There are %d failures in test:\n\t%s", errors.size(), String.join("\n\t", errors)));
+        }
+    }
+
+

Review comment:
       nit: more than 1 new line

##########
File path: storm-server/src/test/java/org/apache/storm/utils/ServerUtilsTest.java
##########
@@ -162,6 +185,191 @@ public void testGetUserId() throws Exception {
         int uid1 = ServerUtils.getUserId(null);
         Path p = Files.createTempFile("testGetUser", ".txt");
         int uid2 = ServerUtils.getPathOwnerUid(p.toString());
+        if (!p.toFile().delete()) {
+            LOG.warn("Could not delete tempoary file {}", p);
+        }
         assertEquals("User UID " + uid1 + " is not same as file " + p.toString() + " owner UID of " + uid2, uid1, uid2);
     }
+
+    @Test
+    public void testIsAnyProcessPosixProcessPidDirAlive() throws IOException {

Review comment:
       Isn't this already present? 
   https://github.com/apache/storm/blob/master/storm-server/src/test/java/org/apache/storm/utils/ServerUtilsTest.java#L184

##########
File path: storm-server/src/test/java/org/apache/storm/utils/ServerUtilsTest.java
##########
@@ -162,6 +185,191 @@ public void testGetUserId() throws Exception {
         int uid1 = ServerUtils.getUserId(null);
         Path p = Files.createTempFile("testGetUser", ".txt");
         int uid2 = ServerUtils.getPathOwnerUid(p.toString());
+        if (!p.toFile().delete()) {
+            LOG.warn("Could not delete tempoary file {}", p);
+        }
         assertEquals("User UID " + uid1 + " is not same as file " + p.toString() + " owner UID of " + uid2, uid1, uid2);
     }
+
+    @Test
+    public void testIsAnyProcessPosixProcessPidDirAlive() throws IOException {
+        final String testName = "testIsAnyProcessPosixProcessPidDirAlive";
+        List<String> errors = new ArrayList<>();
+        int maxPidCnt = 5;
+        if (ServerUtils.IS_ON_WINDOWS) {
+            LOG.info("{}: test cannot be run on Windows. Marked as successful", testName);
+            return;
+        }
+        final Path parentDir = Paths.get("/proc");
+        if (!parentDir.toFile().exists()) {
+            LOG.info("{}: test cannot be run on system without process directory {}, os.name={}",
+                    testName, parentDir, System.getProperty("os.name"));
+            // check if we can get process id on this Posix system - testing test code, useful on Mac
+            String cmd = "/bin/sleep 10";
+            if (getPidOfPosixProcess(Runtime.getRuntime().exec(cmd), errors) < 0) {
+                fail(String.format("%s: Cannot obtain process id for executed command \"%s\"\n%s",
+                        testName, cmd, String.join("\n\t", errors)));
+            }
+            return;
+        }
+        // Create processes and wait for their termination
+        Set<Long> observables = new HashSet<>();
+
+        for (int i = 0 ; i < maxPidCnt ; i++) {
+            String cmd = "sleep 2000";
+            Process process = Runtime.getRuntime().exec(cmd);
+            long pid = getPidOfPosixProcess(process, errors);
+            LOG.info("{}: ({}) ran process \"{}\" with pid={}", testName, i, cmd, pid);
+            if (pid < 0) {
+                String e = String.format("%s: (%d) Cannot obtain process id for executed command \"%s\"", testName, i, cmd);
+                errors.add(e);
+                LOG.error(e);
+                continue;
+            }
+            observables.add(pid);
+        }
+        String userName = System.getProperty("user.name");
+        // now kill processes one by one
+        List<Long> pidList = new ArrayList<>(observables);
+        final long processKillIntervalMs = 2000;
+        for (int i = 0; i < pidList.size(); i++) {
+            long pid = pidList.get(i);
+            LOG.info("{}: ({}) Sleeping for {} milliseconds before kill", testName, i, processKillIntervalMs);
+            if (sleepInterrupted(processKillIntervalMs)) {
+                return;
+            }
+            Runtime.getRuntime().exec("kill -9 " + pid);
+            LOG.info("{}: ({}) Sleeping for {} milliseconds after kill", testName, i, processKillIntervalMs);
+            if (sleepInterrupted(processKillIntervalMs)) {
+                return;
+            }
+            boolean pidDirsAvailable = ServerUtils.isAnyPosixProcessPidDirAlive(observables, userName);
+            if (i < pidList.size() - 1) {
+                if (pidDirsAvailable) {
+                    LOG.info("{}: ({}) Found existing process directories before killing last process", testName, i);
+                } else {
+                    String e = String.format("%s: (%d) Found no existing process directories before killing last process", testName, i);
+                    errors.add(e);
+                    LOG.error(e);
+                }
+            } else {
+                if (pidDirsAvailable) {
+                    String e = String.format("%s: (%d) Found existing process directories after killing last process", testName, i);
+                    errors.add(e);
+                    LOG.error(e);
+                } else {
+                    LOG.info("{}: ({}) Found no existing process directories after killing last process", testName, i);
+                }
+            }
+        }
+        if (!errors.isEmpty()) {
+            fail(String.format("There are %d failures in test:\n\t%s", errors.size(), String.join("\n\t", errors)));
+        }
+    }
+
+
+    /**
+     * Simulate the production scenario where the owner of the process directory is sometimes returned as the
+     * UID instead of user. This scenario is simulated by calling
+     * {@link ServerUtils#isAnyPosixProcessPidDirAlive(Collection, String, boolean)} with the last parameter
+     * set to true as well as false.
+     *
+     * @throws Exception on I/O exception
+     */
+    @Test
+    public void testIsAnyPosixProcessPidDirAliveMockingFileOwnerUid() throws Exception {
+        File procDir = new File("/proc");
+        if (!procDir.exists()) {
+            LOG.info("Test testIsAnyPosixProcessPidDirAlive is designed to run on systems with /proc directory only, marking as success");
+            return;
+        }
+        Collection<Long> pids = getRunningProcessIds();
+        assertFalse(pids.isEmpty());
+
+        for (int i = 0 ; i < 2 ; i++) {
+            boolean mockFileOwnerToUid = (i % 2 == 0);
+            // at least one pid will be owned by the current user (doing the testing)
+            String currentUser = System.getProperty("user.name");
+            boolean status = ServerUtils.isAnyPosixProcessPidDirAlive(pids, currentUser, mockFileOwnerToUid);
+            String err = String.format("(mockFileOwnerToUid=%s) Expecting user %s to own at least one process",
+                    mockFileOwnerToUid, currentUser);
+            assertTrue(err, status);
+        }
+    }
+
+    /**
+     * Make the best effort to obtain the Process ID from the Process object. Thus staying entirely with the JVM.
+     *
+     * @param p Process instance returned upon executing {@link Runtime#exec(String)}.
+     * @param errors Populate errors when PID is a negative number.
+     * @return positive PID upon success, otherwise negative.
+     */
+    private synchronized long getPidOfPosixProcess(Process p, List<String> errors) {

Review comment:
       this method already exist?

##########
File path: storm-server/src/test/java/org/apache/storm/utils/ServerUtilsTest.java
##########
@@ -162,6 +185,191 @@ public void testGetUserId() throws Exception {
         int uid1 = ServerUtils.getUserId(null);
         Path p = Files.createTempFile("testGetUser", ".txt");
         int uid2 = ServerUtils.getPathOwnerUid(p.toString());
+        if (!p.toFile().delete()) {
+            LOG.warn("Could not delete tempoary file {}", p);
+        }
         assertEquals("User UID " + uid1 + " is not same as file " + p.toString() + " owner UID of " + uid2, uid1, uid2);
     }
+
+    @Test
+    public void testIsAnyProcessPosixProcessPidDirAlive() throws IOException {
+        final String testName = "testIsAnyProcessPosixProcessPidDirAlive";
+        List<String> errors = new ArrayList<>();
+        int maxPidCnt = 5;
+        if (ServerUtils.IS_ON_WINDOWS) {
+            LOG.info("{}: test cannot be run on Windows. Marked as successful", testName);
+            return;
+        }
+        final Path parentDir = Paths.get("/proc");
+        if (!parentDir.toFile().exists()) {
+            LOG.info("{}: test cannot be run on system without process directory {}, os.name={}",
+                    testName, parentDir, System.getProperty("os.name"));
+            // check if we can get process id on this Posix system - testing test code, useful on Mac
+            String cmd = "/bin/sleep 10";
+            if (getPidOfPosixProcess(Runtime.getRuntime().exec(cmd), errors) < 0) {
+                fail(String.format("%s: Cannot obtain process id for executed command \"%s\"\n%s",
+                        testName, cmd, String.join("\n\t", errors)));
+            }
+            return;
+        }
+        // Create processes and wait for their termination
+        Set<Long> observables = new HashSet<>();
+
+        for (int i = 0 ; i < maxPidCnt ; i++) {
+            String cmd = "sleep 2000";
+            Process process = Runtime.getRuntime().exec(cmd);
+            long pid = getPidOfPosixProcess(process, errors);
+            LOG.info("{}: ({}) ran process \"{}\" with pid={}", testName, i, cmd, pid);
+            if (pid < 0) {
+                String e = String.format("%s: (%d) Cannot obtain process id for executed command \"%s\"", testName, i, cmd);
+                errors.add(e);
+                LOG.error(e);
+                continue;
+            }
+            observables.add(pid);
+        }
+        String userName = System.getProperty("user.name");
+        // now kill processes one by one
+        List<Long> pidList = new ArrayList<>(observables);
+        final long processKillIntervalMs = 2000;
+        for (int i = 0; i < pidList.size(); i++) {
+            long pid = pidList.get(i);
+            LOG.info("{}: ({}) Sleeping for {} milliseconds before kill", testName, i, processKillIntervalMs);
+            if (sleepInterrupted(processKillIntervalMs)) {
+                return;
+            }
+            Runtime.getRuntime().exec("kill -9 " + pid);
+            LOG.info("{}: ({}) Sleeping for {} milliseconds after kill", testName, i, processKillIntervalMs);
+            if (sleepInterrupted(processKillIntervalMs)) {
+                return;
+            }
+            boolean pidDirsAvailable = ServerUtils.isAnyPosixProcessPidDirAlive(observables, userName);
+            if (i < pidList.size() - 1) {
+                if (pidDirsAvailable) {
+                    LOG.info("{}: ({}) Found existing process directories before killing last process", testName, i);
+                } else {
+                    String e = String.format("%s: (%d) Found no existing process directories before killing last process", testName, i);
+                    errors.add(e);
+                    LOG.error(e);
+                }
+            } else {
+                if (pidDirsAvailable) {
+                    String e = String.format("%s: (%d) Found existing process directories after killing last process", testName, i);
+                    errors.add(e);
+                    LOG.error(e);
+                } else {
+                    LOG.info("{}: ({}) Found no existing process directories after killing last process", testName, i);
+                }
+            }
+        }
+        if (!errors.isEmpty()) {
+            fail(String.format("There are %d failures in test:\n\t%s", errors.size(), String.join("\n\t", errors)));
+        }
+    }
+
+
+    /**
+     * Simulate the production scenario where the owner of the process directory is sometimes returned as the
+     * UID instead of user. This scenario is simulated by calling
+     * {@link ServerUtils#isAnyPosixProcessPidDirAlive(Collection, String, boolean)} with the last parameter
+     * set to true as well as false.
+     *
+     * @throws Exception on I/O exception
+     */
+    @Test
+    public void testIsAnyPosixProcessPidDirAliveMockingFileOwnerUid() throws Exception {
+        File procDir = new File("/proc");
+        if (!procDir.exists()) {
+            LOG.info("Test testIsAnyPosixProcessPidDirAlive is designed to run on systems with /proc directory only, marking as success");
+            return;
+        }
+        Collection<Long> pids = getRunningProcessIds();
+        assertFalse(pids.isEmpty());
+
+        for (int i = 0 ; i < 2 ; i++) {
+            boolean mockFileOwnerToUid = (i % 2 == 0);
+            // at least one pid will be owned by the current user (doing the testing)
+            String currentUser = System.getProperty("user.name");
+            boolean status = ServerUtils.isAnyPosixProcessPidDirAlive(pids, currentUser, mockFileOwnerToUid);
+            String err = String.format("(mockFileOwnerToUid=%s) Expecting user %s to own at least one process",
+                    mockFileOwnerToUid, currentUser);
+            assertTrue(err, status);
+        }
+    }
+
+    /**
+     * Make the best effort to obtain the Process ID from the Process object. Thus staying entirely with the JVM.
+     *
+     * @param p Process instance returned upon executing {@link Runtime#exec(String)}.
+     * @param errors Populate errors when PID is a negative number.
+     * @return positive PID upon success, otherwise negative.
+     */
+    private synchronized long getPidOfPosixProcess(Process p, List<String> errors) {
+        Class<? extends Process> pClass = p.getClass();
+        String pObjStr = ToStringBuilder.reflectionToString(p, ToStringStyle.SHORT_PREFIX_STYLE);
+        String pclassName = pClass.getName();
+        try {
+            if (pclassName.equals("java.lang.UNIXProcess")) {
+                Field f = pClass.getDeclaredField("pid");
+                f.setAccessible(true);
+                long pid = f.getLong(p);
+                f.setAccessible(false);
+                if (pid < 0) {
+                    errors.add("\t \"pid\" attribute in Process class " + pclassName + " returned -1, process=" + pObjStr);
+                }
+                return pid;
+            }
+            for (Field f : pClass.getDeclaredFields()) {
+                if (!f.getName().equalsIgnoreCase("pid")) {
+                    continue;
+                }
+                LOG.info("ServerUtilsTest.getPidOfPosixProcess(): found attribute {}#{}", pclassName, f.getName());
+                f.setAccessible(true);
+                long pid = f.getLong(p);
+                f.setAccessible(false);
+                if (pid < 0) {
+                    errors.add("\t \"pid\" attribute in Process class " + pclassName + " returned -1, process=" + pObjStr);
+                }
+                return pid;
+            }
+            // post JDK 9 there should be getPid() - future JDK-11 compatibility only for the sake of Travis test in community
+            try {
+                Method m = pClass.getDeclaredMethod("getPid");
+                LOG.info("ServerUtilsTest.getPidOfPosixProcess(): found method {}#getPid()\n", pclassName);
+                long pid = (Long)m.invoke(p);
+                if (pid < 0) {
+                    errors.add("\t \"getPid()\" method in Process class " + pclassName + " returned -1, process=" + pObjStr);
+                }
+                return pid;
+            } catch (SecurityException e) {
+                errors.add("\t getPid() method in Process class " + pclassName + " cannot be called: " + e.getMessage() + ", process=" + pObjStr);
+                return -1;
+            } catch (NoSuchMethodException e) {
+                // ignore and try something else
+            }
+            errors.add("\t Process class " + pclassName + " missing field \"pid\" and missing method \"getPid()\", process=" + pObjStr);
+            return -1;
+        } catch (Exception e) {
+            errors.add("\t Exception in Process class " + pclassName + ": " + e.getMessage() + ", process=" + pObjStr);
+            e.printStackTrace();
+            return -1;
+        }
+    }
+
+    /**
+     * Sleep for specified milliseconds and return true if sleep was interrupted.
+     *
+     * @param milliSeconds number of milliseconds to sleep
+     * @return true if sleep was interrupted, false otherwise.
+     */
+    private boolean sleepInterrupted(long milliSeconds) {

Review comment:
       this method already exist?




----------------------------------------------------------------
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



[GitHub] [storm] bipinprasad commented on a change in pull request #3326: [STORM-3692] Handle UID return from Files.getOwner()

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3326:
URL: https://github.com/apache/storm/pull/3326#discussion_r481326434



##########
File path: storm-server/src/main/java/org/apache/storm/utils/ServerUtils.java
##########
@@ -1217,9 +1199,127 @@ public static boolean areAllProcessesDead(Map<String, Object> conf, String user,
             }
         }
         if (cachedUserToUidMap.containsKey(user)) {
-            return allDead = !ServerUtils.isAnyProcessAlive(pids, cachedUserToUidMap.get(user));
+            return !ServerUtils.isAnyProcessAlive(pids, cachedUserToUidMap.get(user));
         } else {
-            return allDead = !ServerUtils.isAnyProcessAlive(pids, user);
+            return !ServerUtils.isAnyProcessAlive(pids, user);
         }
     }
+
+    /**
+     * Find if the process is alive using the existence of /proc/&lt;pid&gt; directory
+     * owned by the supplied user. This is an alternative to "ps -p pid -u uid" command
+     * used in {@link #isAnyPosixProcessAlive(Collection, int)}
+     *
+     * <p>
+     * Processes are tracked using the existence of the directory "/proc/&lt;pid&gt;
+     * For each of the supplied PIDs, their PID directory is checked for existence and ownership
+     * by the specified uid.
+     * </p>
+     *
+     * @param pids Process IDs that need to be monitored for liveness
+     * @param user 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
+     */
+    public static boolean isAnyPosixProcessPidDirAlive(Collection<Long> pids, String user) throws IOException {
+        return isAnyPosixProcessPidDirAlive(pids, user, false);
+    }
+
+    /**
+     * Find if the process is alive using the existence of /proc/&lt;pid&gt; directory
+     * owned by the supplied user. This is an alternative to "ps -p pid -u uid" command
+     * used in {@link #isAnyPosixProcessAlive(Collection, int)}
+     *
+     * <p>
+     * Processes are tracked using the existence of the directory "/proc/&lt;pid&gt;
+     * For each of the supplied PIDs, their PID directory is checked for existence and ownership
+     * by the specified uid.
+     * </p>
+     *
+     * @param pids Process IDs that need to be monitored for liveness
+     * @param user the userId that is expected to own that process
+     * @param mockFileOwnerToUid if true (used for testing), then convert File.owner to UID
+     * @return true if any one of the processes is owned by user and alive, else false
+     * @throws IOException on I/O exception
+     */
+    @VisibleForTesting
+    public static boolean isAnyPosixProcessPidDirAlive(Collection<Long> pids, String user, boolean mockFileOwnerToUid)
+            throws IOException {
+        File procDir = new File("/proc");
+        if (!procDir.exists()) {
+            throw new IOException("Missing process directory " + procDir.getAbsolutePath() + ": method not supported on "
+                    + "os.name=" + System.getProperty("os.name"));
+        }
+        for (long pid: pids) {
+            File pidDir = new File(procDir, String.valueOf(pid));
+            if (!pidDir.exists()) {
+                continue;
+            }
+            // check if existing process is owned by the specified user, if not, the process is dead
+            String pidUser;
+            try {
+                pidUser = Files.getOwner(pidDir.toPath()).getName();
+            } catch (NoSuchFileException ex) {
+                continue; // process died before the user can be checked
+            }
+            if (mockFileOwnerToUid) {
+                // code activated in testing to simulate Files.getOwner returning UID (which sometimes happens in runtime)
+                if (StringUtils.isNumeric(pidUser)) {
+                    LOG.info("Skip mocking, since owner {} of pidDir {} is already numeric", pidUser, pidDir);
+                } else {
+                    Integer uid = cachedUserToUidMap.get(pidUser);
+                    if (uid == null) {
+                        uid = ServerUtils.getUserId(pidUser);
+                        if (uid < 0) {
+                            String err = String.format("Cannot get UID for %s, while mocking the owner of pidDir %s",
+                                    pidUser, pidDir.getAbsolutePath());
+                            throw new IOException(err);
+                        }
+                        cachedUserToUidMap.put(pidUser, uid);
+                        LOG.info("Found UID {} for {}, while mocking the owner of pidDir {}", uid, pidUser, pidDir);
+                    } else {
+                        LOG.info("Found cached UID {} for {}, while mocking the owner of pidDir {}", uid, pidUser, pidDir);
+                    }
+                    pidUser = String.valueOf(uid);
+                }
+            }
+            //sometimes uid is returned instead of username - if so, try to convert and compare with uid
+            if (StringUtils.isNumeric(pidUser)) {
+                // numeric pidUser - this is UID
+                LOG.debug("Process directory {} owner is uid={}", pidDir, pidUser);
+                int pidUid = Integer.parseInt(pidUser);
+                Integer uid = cachedUserToUidMap.get(user);
+                if (uid == null) {
+                    uid = ServerUtils.getUserId(user);
+                    if (uid < 0) {
+                        String err = String.format("Cannot get uid for %s to compare with owner id=%d of process directory %s",
+                                user, pidUid, pidDir.getAbsolutePath());
+                        throw new IOException(err);
+                    }
+                    cachedUserToUidMap.put(user, uid);
+                }
+                if (uid == pidUid) {
+                    LOG.debug("Process {} is alive and owned by user {}/{}", pid, user, uid);
+                    return true;
+                } else {
+                    LOG.info("Prior process is dead, since directory {} owner {} is not same as expected user {}/{}, "
+                            + "likely pid {} was reused for a new process for uid {}", pidDir, pidUser, user, uid, pid, uid);

Review comment:
       renamed variables




----------------------------------------------------------------
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



[GitHub] [storm] bipinprasad commented on a change in pull request #3326: [STORM-3692] Handle UID return from Files.getOwner()

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3326:
URL: https://github.com/apache/storm/pull/3326#discussion_r481366176



##########
File path: storm-server/src/test/java/org/apache/storm/utils/ServerUtilsTest.java
##########
@@ -162,6 +185,191 @@ public void testGetUserId() throws Exception {
         int uid1 = ServerUtils.getUserId(null);
         Path p = Files.createTempFile("testGetUser", ".txt");
         int uid2 = ServerUtils.getPathOwnerUid(p.toString());
+        if (!p.toFile().delete()) {
+            LOG.warn("Could not delete tempoary file {}", p);
+        }
         assertEquals("User UID " + uid1 + " is not same as file " + p.toString() + " owner UID of " + uid2, uid1, uid2);
     }
+
+    @Test
+    public void testIsAnyProcessPosixProcessPidDirAlive() throws IOException {

Review comment:
       Yes. Something is not right. I will do a force push with all changes.




----------------------------------------------------------------
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



[GitHub] [storm] bipinprasad commented on a change in pull request #3326: [STORM-3692] Handle UID return from Files.getOwner()

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3326:
URL: https://github.com/apache/storm/pull/3326#discussion_r481340117



##########
File path: storm-server/src/test/java/org/apache/storm/utils/ServerUtilsTest.java
##########
@@ -162,6 +185,191 @@ public void testGetUserId() throws Exception {
         int uid1 = ServerUtils.getUserId(null);
         Path p = Files.createTempFile("testGetUser", ".txt");
         int uid2 = ServerUtils.getPathOwnerUid(p.toString());
+        if (!p.toFile().delete()) {
+            LOG.warn("Could not delete tempoary file {}", p);
+        }
         assertEquals("User UID " + uid1 + " is not same as file " + p.toString() + " owner UID of " + uid2, uid1, uid2);
     }
+
+    @Test
+    public void testIsAnyProcessPosixProcessPidDirAlive() throws IOException {
+        final String testName = "testIsAnyProcessPosixProcessPidDirAlive";
+        List<String> errors = new ArrayList<>();
+        int maxPidCnt = 5;
+        if (ServerUtils.IS_ON_WINDOWS) {
+            LOG.info("{}: test cannot be run on Windows. Marked as successful", testName);
+            return;
+        }
+        final Path parentDir = Paths.get("/proc");
+        if (!parentDir.toFile().exists()) {
+            LOG.info("{}: test cannot be run on system without process directory {}, os.name={}",
+                    testName, parentDir, System.getProperty("os.name"));
+            // check if we can get process id on this Posix system - testing test code, useful on Mac
+            String cmd = "/bin/sleep 10";
+            if (getPidOfPosixProcess(Runtime.getRuntime().exec(cmd), errors) < 0) {
+                fail(String.format("%s: Cannot obtain process id for executed command \"%s\"\n%s",
+                        testName, cmd, String.join("\n\t", errors)));
+            }
+            return;
+        }
+        // Create processes and wait for their termination
+        Set<Long> observables = new HashSet<>();
+
+        for (int i = 0 ; i < maxPidCnt ; i++) {
+            String cmd = "sleep 2000";
+            Process process = Runtime.getRuntime().exec(cmd);
+            long pid = getPidOfPosixProcess(process, errors);
+            LOG.info("{}: ({}) ran process \"{}\" with pid={}", testName, i, cmd, pid);
+            if (pid < 0) {
+                String e = String.format("%s: (%d) Cannot obtain process id for executed command \"%s\"", testName, i, cmd);
+                errors.add(e);
+                LOG.error(e);
+                continue;
+            }
+            observables.add(pid);
+        }
+        String userName = System.getProperty("user.name");
+        // now kill processes one by one
+        List<Long> pidList = new ArrayList<>(observables);
+        final long processKillIntervalMs = 2000;
+        for (int i = 0; i < pidList.size(); i++) {
+            long pid = pidList.get(i);
+            LOG.info("{}: ({}) Sleeping for {} milliseconds before kill", testName, i, processKillIntervalMs);
+            if (sleepInterrupted(processKillIntervalMs)) {
+                return;
+            }
+            Runtime.getRuntime().exec("kill -9 " + pid);
+            LOG.info("{}: ({}) Sleeping for {} milliseconds after kill", testName, i, processKillIntervalMs);
+            if (sleepInterrupted(processKillIntervalMs)) {
+                return;
+            }
+            boolean pidDirsAvailable = ServerUtils.isAnyPosixProcessPidDirAlive(observables, userName);
+            if (i < pidList.size() - 1) {
+                if (pidDirsAvailable) {
+                    LOG.info("{}: ({}) Found existing process directories before killing last process", testName, i);
+                } else {
+                    String e = String.format("%s: (%d) Found no existing process directories before killing last process", testName, i);
+                    errors.add(e);
+                    LOG.error(e);
+                }
+            } else {
+                if (pidDirsAvailable) {
+                    String e = String.format("%s: (%d) Found existing process directories after killing last process", testName, i);
+                    errors.add(e);
+                    LOG.error(e);
+                } else {
+                    LOG.info("{}: ({}) Found no existing process directories after killing last process", testName, i);
+                }
+            }
+        }
+        if (!errors.isEmpty()) {
+            fail(String.format("There are %d failures in test:\n\t%s", errors.size(), String.join("\n\t", errors)));
+        }
+    }
+
+
+    /**
+     * Simulate the production scenario where the owner of the process directory is sometimes returned as the
+     * UID instead of user. This scenario is simulated by calling
+     * {@link ServerUtils#isAnyPosixProcessPidDirAlive(Collection, String, boolean)} with the last parameter
+     * set to true as well as false.
+     *
+     * @throws Exception on I/O exception
+     */
+    @Test
+    public void testIsAnyPosixProcessPidDirAliveMockingFileOwnerUid() throws Exception {
+        File procDir = new File("/proc");
+        if (!procDir.exists()) {
+            LOG.info("Test testIsAnyPosixProcessPidDirAlive is designed to run on systems with /proc directory only, marking as success");
+            return;
+        }
+        Collection<Long> pids = getRunningProcessIds();
+        assertFalse(pids.isEmpty());
+
+        for (int i = 0 ; i < 2 ; i++) {
+            boolean mockFileOwnerToUid = (i % 2 == 0);
+            // at least one pid will be owned by the current user (doing the testing)
+            String currentUser = System.getProperty("user.name");
+            boolean status = ServerUtils.isAnyPosixProcessPidDirAlive(pids, currentUser, mockFileOwnerToUid);
+            String err = String.format("(mockFileOwnerToUid=%s) Expecting user %s to own at least one process",
+                    mockFileOwnerToUid, currentUser);
+            assertTrue(err, status);
+        }
+    }
+
+    /**
+     * Make the best effort to obtain the Process ID from the Process object. Thus staying entirely with the JVM.
+     *
+     * @param p Process instance returned upon executing {@link Runtime#exec(String)}.
+     * @param errors Populate errors when PID is a negative number.
+     * @return positive PID upon success, otherwise negative.
+     */
+    private synchronized long getPidOfPosixProcess(Process p, List<String> errors) {
+        Class<? extends Process> pClass = p.getClass();
+        String pObjStr = ToStringBuilder.reflectionToString(p, ToStringStyle.SHORT_PREFIX_STYLE);
+        String pclassName = pClass.getName();
+        try {
+            if (pclassName.equals("java.lang.UNIXProcess")) {
+                Field f = pClass.getDeclaredField("pid");
+                f.setAccessible(true);
+                long pid = f.getLong(p);
+                f.setAccessible(false);
+                if (pid < 0) {
+                    errors.add("\t \"pid\" attribute in Process class " + pclassName + " returned -1, process=" + pObjStr);
+                }
+                return pid;
+            }
+            for (Field f : pClass.getDeclaredFields()) {
+                if (!f.getName().equalsIgnoreCase("pid")) {
+                    continue;
+                }
+                LOG.info("ServerUtilsTest.getPidOfPosixProcess(): found attribute {}#{}", pclassName, f.getName());
+                f.setAccessible(true);
+                long pid = f.getLong(p);
+                f.setAccessible(false);
+                if (pid < 0) {
+                    errors.add("\t \"pid\" attribute in Process class " + pclassName + " returned -1, process=" + pObjStr);
+                }
+                return pid;
+            }
+            // post JDK 9 there should be getPid() - future JDK-11 compatibility only for the sake of Travis test in community
+            try {
+                Method m = pClass.getDeclaredMethod("getPid");
+                LOG.info("ServerUtilsTest.getPidOfPosixProcess(): found method {}#getPid()\n", pclassName);
+                long pid = (Long)m.invoke(p);
+                if (pid < 0) {
+                    errors.add("\t \"getPid()\" method in Process class " + pclassName + " returned -1, process=" + pObjStr);
+                }
+                return pid;
+            } catch (SecurityException e) {
+                errors.add("\t getPid() method in Process class " + pclassName + " cannot be called: " + e.getMessage() + ", process=" + pObjStr);
+                return -1;
+            } catch (NoSuchMethodException e) {
+                // ignore and try something else
+            }
+            errors.add("\t Process class " + pclassName + " missing field \"pid\" and missing method \"getPid()\", process=" + pObjStr);
+            return -1;
+        } catch (Exception e) {
+            errors.add("\t Exception in Process class " + pclassName + ": " + e.getMessage() + ", process=" + pObjStr);
+            e.printStackTrace();
+            return -1;
+        }
+    }
+
+    /**
+     * Sleep for specified milliseconds and return true if sleep was interrupted.
+     *
+     * @param milliSeconds number of milliseconds to sleep
+     * @return true if sleep was interrupted, false otherwise.
+     */
+    private boolean sleepInterrupted(long milliSeconds) {

Review comment:
       I am not aware of any existing method with similar functionality.
   Or did you mean to ask why this is showing up in this pull request, since it should already exist.




----------------------------------------------------------------
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



[GitHub] [storm] Ethanlm commented on a change in pull request #3326: [STORM-3692] Handle UID return from Files.getOwner()

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3326:
URL: https://github.com/apache/storm/pull/3326#discussion_r481367174



##########
File path: storm-server/src/test/java/org/apache/storm/utils/ServerUtilsTest.java
##########
@@ -162,6 +185,191 @@ public void testGetUserId() throws Exception {
         int uid1 = ServerUtils.getUserId(null);
         Path p = Files.createTempFile("testGetUser", ".txt");
         int uid2 = ServerUtils.getPathOwnerUid(p.toString());
+        if (!p.toFile().delete()) {
+            LOG.warn("Could not delete tempoary file {}", p);
+        }
         assertEquals("User UID " + uid1 + " is not same as file " + p.toString() + " owner UID of " + uid2, uid1, uid2);
     }
+
+    @Test
+    public void testIsAnyProcessPosixProcessPidDirAlive() throws IOException {
+        final String testName = "testIsAnyProcessPosixProcessPidDirAlive";
+        List<String> errors = new ArrayList<>();
+        int maxPidCnt = 5;
+        if (ServerUtils.IS_ON_WINDOWS) {
+            LOG.info("{}: test cannot be run on Windows. Marked as successful", testName);
+            return;
+        }
+        final Path parentDir = Paths.get("/proc");
+        if (!parentDir.toFile().exists()) {
+            LOG.info("{}: test cannot be run on system without process directory {}, os.name={}",
+                    testName, parentDir, System.getProperty("os.name"));
+            // check if we can get process id on this Posix system - testing test code, useful on Mac
+            String cmd = "/bin/sleep 10";
+            if (getPidOfPosixProcess(Runtime.getRuntime().exec(cmd), errors) < 0) {
+                fail(String.format("%s: Cannot obtain process id for executed command \"%s\"\n%s",
+                        testName, cmd, String.join("\n\t", errors)));
+            }
+            return;
+        }
+        // Create processes and wait for their termination
+        Set<Long> observables = new HashSet<>();
+
+        for (int i = 0 ; i < maxPidCnt ; i++) {
+            String cmd = "sleep 2000";
+            Process process = Runtime.getRuntime().exec(cmd);
+            long pid = getPidOfPosixProcess(process, errors);
+            LOG.info("{}: ({}) ran process \"{}\" with pid={}", testName, i, cmd, pid);
+            if (pid < 0) {
+                String e = String.format("%s: (%d) Cannot obtain process id for executed command \"%s\"", testName, i, cmd);
+                errors.add(e);
+                LOG.error(e);
+                continue;
+            }
+            observables.add(pid);
+        }
+        String userName = System.getProperty("user.name");
+        // now kill processes one by one
+        List<Long> pidList = new ArrayList<>(observables);
+        final long processKillIntervalMs = 2000;
+        for (int i = 0; i < pidList.size(); i++) {
+            long pid = pidList.get(i);
+            LOG.info("{}: ({}) Sleeping for {} milliseconds before kill", testName, i, processKillIntervalMs);
+            if (sleepInterrupted(processKillIntervalMs)) {
+                return;
+            }
+            Runtime.getRuntime().exec("kill -9 " + pid);
+            LOG.info("{}: ({}) Sleeping for {} milliseconds after kill", testName, i, processKillIntervalMs);
+            if (sleepInterrupted(processKillIntervalMs)) {
+                return;
+            }
+            boolean pidDirsAvailable = ServerUtils.isAnyPosixProcessPidDirAlive(observables, userName);
+            if (i < pidList.size() - 1) {
+                if (pidDirsAvailable) {
+                    LOG.info("{}: ({}) Found existing process directories before killing last process", testName, i);
+                } else {
+                    String e = String.format("%s: (%d) Found no existing process directories before killing last process", testName, i);
+                    errors.add(e);
+                    LOG.error(e);
+                }
+            } else {
+                if (pidDirsAvailable) {
+                    String e = String.format("%s: (%d) Found existing process directories after killing last process", testName, i);
+                    errors.add(e);
+                    LOG.error(e);
+                } else {
+                    LOG.info("{}: ({}) Found no existing process directories after killing last process", testName, i);
+                }
+            }
+        }
+        if (!errors.isEmpty()) {
+            fail(String.format("There are %d failures in test:\n\t%s", errors.size(), String.join("\n\t", errors)));
+        }
+    }
+
+
+    /**
+     * Simulate the production scenario where the owner of the process directory is sometimes returned as the
+     * UID instead of user. This scenario is simulated by calling
+     * {@link ServerUtils#isAnyPosixProcessPidDirAlive(Collection, String, boolean)} with the last parameter
+     * set to true as well as false.
+     *
+     * @throws Exception on I/O exception
+     */
+    @Test
+    public void testIsAnyPosixProcessPidDirAliveMockingFileOwnerUid() throws Exception {
+        File procDir = new File("/proc");
+        if (!procDir.exists()) {
+            LOG.info("Test testIsAnyPosixProcessPidDirAlive is designed to run on systems with /proc directory only, marking as success");
+            return;
+        }
+        Collection<Long> pids = getRunningProcessIds();
+        assertFalse(pids.isEmpty());
+
+        for (int i = 0 ; i < 2 ; i++) {
+            boolean mockFileOwnerToUid = (i % 2 == 0);
+            // at least one pid will be owned by the current user (doing the testing)
+            String currentUser = System.getProperty("user.name");
+            boolean status = ServerUtils.isAnyPosixProcessPidDirAlive(pids, currentUser, mockFileOwnerToUid);
+            String err = String.format("(mockFileOwnerToUid=%s) Expecting user %s to own at least one process",
+                    mockFileOwnerToUid, currentUser);
+            assertTrue(err, status);
+        }
+    }
+
+    /**
+     * Make the best effort to obtain the Process ID from the Process object. Thus staying entirely with the JVM.
+     *
+     * @param p Process instance returned upon executing {@link Runtime#exec(String)}.
+     * @param errors Populate errors when PID is a negative number.
+     * @return positive PID upon success, otherwise negative.
+     */
+    private synchronized long getPidOfPosixProcess(Process p, List<String> errors) {
+        Class<? extends Process> pClass = p.getClass();
+        String pObjStr = ToStringBuilder.reflectionToString(p, ToStringStyle.SHORT_PREFIX_STYLE);
+        String pclassName = pClass.getName();
+        try {
+            if (pclassName.equals("java.lang.UNIXProcess")) {
+                Field f = pClass.getDeclaredField("pid");
+                f.setAccessible(true);
+                long pid = f.getLong(p);
+                f.setAccessible(false);
+                if (pid < 0) {
+                    errors.add("\t \"pid\" attribute in Process class " + pclassName + " returned -1, process=" + pObjStr);
+                }
+                return pid;
+            }
+            for (Field f : pClass.getDeclaredFields()) {
+                if (!f.getName().equalsIgnoreCase("pid")) {
+                    continue;
+                }
+                LOG.info("ServerUtilsTest.getPidOfPosixProcess(): found attribute {}#{}", pclassName, f.getName());
+                f.setAccessible(true);
+                long pid = f.getLong(p);
+                f.setAccessible(false);
+                if (pid < 0) {
+                    errors.add("\t \"pid\" attribute in Process class " + pclassName + " returned -1, process=" + pObjStr);
+                }
+                return pid;
+            }
+            // post JDK 9 there should be getPid() - future JDK-11 compatibility only for the sake of Travis test in community
+            try {
+                Method m = pClass.getDeclaredMethod("getPid");
+                LOG.info("ServerUtilsTest.getPidOfPosixProcess(): found method {}#getPid()\n", pclassName);
+                long pid = (Long)m.invoke(p);
+                if (pid < 0) {
+                    errors.add("\t \"getPid()\" method in Process class " + pclassName + " returned -1, process=" + pObjStr);
+                }
+                return pid;
+            } catch (SecurityException e) {
+                errors.add("\t getPid() method in Process class " + pclassName + " cannot be called: " + e.getMessage() + ", process=" + pObjStr);
+                return -1;
+            } catch (NoSuchMethodException e) {
+                // ignore and try something else
+            }
+            errors.add("\t Process class " + pclassName + " missing field \"pid\" and missing method \"getPid()\", process=" + pObjStr);
+            return -1;
+        } catch (Exception e) {
+            errors.add("\t Exception in Process class " + pclassName + ": " + e.getMessage() + ", process=" + pObjStr);
+            e.printStackTrace();
+            return -1;
+        }
+    }
+
+    /**
+     * Sleep for specified milliseconds and return true if sleep was interrupted.
+     *
+     * @param milliSeconds number of milliseconds to sleep
+     * @return true if sleep was interrupted, false otherwise.
+     */
+    private boolean sleepInterrupted(long milliSeconds) {

Review comment:
       https://github.com/apache/storm/blob/master/storm-server/src/test/java/org/apache/storm/utils/ServerUtilsTest.java#L324
   This method already exists in ServerUtilsTest (from your another merged PR)




----------------------------------------------------------------
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



[GitHub] [storm] bipinprasad commented on a change in pull request #3326: [STORM-3692] Handle UID return from Files.getOwner()

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3326:
URL: https://github.com/apache/storm/pull/3326#discussion_r481340048



##########
File path: storm-server/src/test/java/org/apache/storm/utils/ServerUtilsTest.java
##########
@@ -162,6 +185,191 @@ public void testGetUserId() throws Exception {
         int uid1 = ServerUtils.getUserId(null);
         Path p = Files.createTempFile("testGetUser", ".txt");
         int uid2 = ServerUtils.getPathOwnerUid(p.toString());
+        if (!p.toFile().delete()) {
+            LOG.warn("Could not delete tempoary file {}", p);
+        }
         assertEquals("User UID " + uid1 + " is not same as file " + p.toString() + " owner UID of " + uid2, uid1, uid2);
     }
+
+    @Test
+    public void testIsAnyProcessPosixProcessPidDirAlive() throws IOException {
+        final String testName = "testIsAnyProcessPosixProcessPidDirAlive";
+        List<String> errors = new ArrayList<>();
+        int maxPidCnt = 5;
+        if (ServerUtils.IS_ON_WINDOWS) {
+            LOG.info("{}: test cannot be run on Windows. Marked as successful", testName);
+            return;
+        }
+        final Path parentDir = Paths.get("/proc");
+        if (!parentDir.toFile().exists()) {
+            LOG.info("{}: test cannot be run on system without process directory {}, os.name={}",
+                    testName, parentDir, System.getProperty("os.name"));
+            // check if we can get process id on this Posix system - testing test code, useful on Mac
+            String cmd = "/bin/sleep 10";
+            if (getPidOfPosixProcess(Runtime.getRuntime().exec(cmd), errors) < 0) {
+                fail(String.format("%s: Cannot obtain process id for executed command \"%s\"\n%s",
+                        testName, cmd, String.join("\n\t", errors)));
+            }
+            return;
+        }
+        // Create processes and wait for their termination
+        Set<Long> observables = new HashSet<>();
+
+        for (int i = 0 ; i < maxPidCnt ; i++) {
+            String cmd = "sleep 2000";
+            Process process = Runtime.getRuntime().exec(cmd);
+            long pid = getPidOfPosixProcess(process, errors);
+            LOG.info("{}: ({}) ran process \"{}\" with pid={}", testName, i, cmd, pid);
+            if (pid < 0) {
+                String e = String.format("%s: (%d) Cannot obtain process id for executed command \"%s\"", testName, i, cmd);
+                errors.add(e);
+                LOG.error(e);
+                continue;
+            }
+            observables.add(pid);
+        }
+        String userName = System.getProperty("user.name");
+        // now kill processes one by one
+        List<Long> pidList = new ArrayList<>(observables);
+        final long processKillIntervalMs = 2000;
+        for (int i = 0; i < pidList.size(); i++) {
+            long pid = pidList.get(i);
+            LOG.info("{}: ({}) Sleeping for {} milliseconds before kill", testName, i, processKillIntervalMs);
+            if (sleepInterrupted(processKillIntervalMs)) {
+                return;
+            }
+            Runtime.getRuntime().exec("kill -9 " + pid);
+            LOG.info("{}: ({}) Sleeping for {} milliseconds after kill", testName, i, processKillIntervalMs);
+            if (sleepInterrupted(processKillIntervalMs)) {
+                return;
+            }
+            boolean pidDirsAvailable = ServerUtils.isAnyPosixProcessPidDirAlive(observables, userName);
+            if (i < pidList.size() - 1) {
+                if (pidDirsAvailable) {
+                    LOG.info("{}: ({}) Found existing process directories before killing last process", testName, i);
+                } else {
+                    String e = String.format("%s: (%d) Found no existing process directories before killing last process", testName, i);
+                    errors.add(e);
+                    LOG.error(e);
+                }
+            } else {
+                if (pidDirsAvailable) {
+                    String e = String.format("%s: (%d) Found existing process directories after killing last process", testName, i);
+                    errors.add(e);
+                    LOG.error(e);
+                } else {
+                    LOG.info("{}: ({}) Found no existing process directories after killing last process", testName, i);
+                }
+            }
+        }
+        if (!errors.isEmpty()) {
+            fail(String.format("There are %d failures in test:\n\t%s", errors.size(), String.join("\n\t", errors)));
+        }
+    }
+
+
+    /**
+     * Simulate the production scenario where the owner of the process directory is sometimes returned as the
+     * UID instead of user. This scenario is simulated by calling
+     * {@link ServerUtils#isAnyPosixProcessPidDirAlive(Collection, String, boolean)} with the last parameter
+     * set to true as well as false.
+     *
+     * @throws Exception on I/O exception
+     */
+    @Test
+    public void testIsAnyPosixProcessPidDirAliveMockingFileOwnerUid() throws Exception {
+        File procDir = new File("/proc");
+        if (!procDir.exists()) {
+            LOG.info("Test testIsAnyPosixProcessPidDirAlive is designed to run on systems with /proc directory only, marking as success");
+            return;
+        }
+        Collection<Long> pids = getRunningProcessIds();
+        assertFalse(pids.isEmpty());
+
+        for (int i = 0 ; i < 2 ; i++) {
+            boolean mockFileOwnerToUid = (i % 2 == 0);
+            // at least one pid will be owned by the current user (doing the testing)
+            String currentUser = System.getProperty("user.name");
+            boolean status = ServerUtils.isAnyPosixProcessPidDirAlive(pids, currentUser, mockFileOwnerToUid);
+            String err = String.format("(mockFileOwnerToUid=%s) Expecting user %s to own at least one process",
+                    mockFileOwnerToUid, currentUser);
+            assertTrue(err, status);
+        }
+    }
+
+    /**
+     * Make the best effort to obtain the Process ID from the Process object. Thus staying entirely with the JVM.
+     *
+     * @param p Process instance returned upon executing {@link Runtime#exec(String)}.
+     * @param errors Populate errors when PID is a negative number.
+     * @return positive PID upon success, otherwise negative.
+     */
+    private synchronized long getPidOfPosixProcess(Process p, List<String> errors) {

Review comment:
       I am not aware of any existing method with similar functionality.

##########
File path: storm-server/src/test/java/org/apache/storm/utils/ServerUtilsTest.java
##########
@@ -162,6 +185,191 @@ public void testGetUserId() throws Exception {
         int uid1 = ServerUtils.getUserId(null);
         Path p = Files.createTempFile("testGetUser", ".txt");
         int uid2 = ServerUtils.getPathOwnerUid(p.toString());
+        if (!p.toFile().delete()) {
+            LOG.warn("Could not delete tempoary file {}", p);
+        }
         assertEquals("User UID " + uid1 + " is not same as file " + p.toString() + " owner UID of " + uid2, uid1, uid2);
     }
+
+    @Test
+    public void testIsAnyProcessPosixProcessPidDirAlive() throws IOException {
+        final String testName = "testIsAnyProcessPosixProcessPidDirAlive";
+        List<String> errors = new ArrayList<>();
+        int maxPidCnt = 5;
+        if (ServerUtils.IS_ON_WINDOWS) {
+            LOG.info("{}: test cannot be run on Windows. Marked as successful", testName);
+            return;
+        }
+        final Path parentDir = Paths.get("/proc");
+        if (!parentDir.toFile().exists()) {
+            LOG.info("{}: test cannot be run on system without process directory {}, os.name={}",
+                    testName, parentDir, System.getProperty("os.name"));
+            // check if we can get process id on this Posix system - testing test code, useful on Mac
+            String cmd = "/bin/sleep 10";
+            if (getPidOfPosixProcess(Runtime.getRuntime().exec(cmd), errors) < 0) {
+                fail(String.format("%s: Cannot obtain process id for executed command \"%s\"\n%s",
+                        testName, cmd, String.join("\n\t", errors)));
+            }
+            return;
+        }
+        // Create processes and wait for their termination
+        Set<Long> observables = new HashSet<>();
+
+        for (int i = 0 ; i < maxPidCnt ; i++) {
+            String cmd = "sleep 2000";
+            Process process = Runtime.getRuntime().exec(cmd);
+            long pid = getPidOfPosixProcess(process, errors);
+            LOG.info("{}: ({}) ran process \"{}\" with pid={}", testName, i, cmd, pid);
+            if (pid < 0) {
+                String e = String.format("%s: (%d) Cannot obtain process id for executed command \"%s\"", testName, i, cmd);
+                errors.add(e);
+                LOG.error(e);
+                continue;
+            }
+            observables.add(pid);
+        }
+        String userName = System.getProperty("user.name");
+        // now kill processes one by one
+        List<Long> pidList = new ArrayList<>(observables);
+        final long processKillIntervalMs = 2000;
+        for (int i = 0; i < pidList.size(); i++) {
+            long pid = pidList.get(i);
+            LOG.info("{}: ({}) Sleeping for {} milliseconds before kill", testName, i, processKillIntervalMs);
+            if (sleepInterrupted(processKillIntervalMs)) {
+                return;
+            }
+            Runtime.getRuntime().exec("kill -9 " + pid);
+            LOG.info("{}: ({}) Sleeping for {} milliseconds after kill", testName, i, processKillIntervalMs);
+            if (sleepInterrupted(processKillIntervalMs)) {
+                return;
+            }
+            boolean pidDirsAvailable = ServerUtils.isAnyPosixProcessPidDirAlive(observables, userName);
+            if (i < pidList.size() - 1) {
+                if (pidDirsAvailable) {
+                    LOG.info("{}: ({}) Found existing process directories before killing last process", testName, i);
+                } else {
+                    String e = String.format("%s: (%d) Found no existing process directories before killing last process", testName, i);
+                    errors.add(e);
+                    LOG.error(e);
+                }
+            } else {
+                if (pidDirsAvailable) {
+                    String e = String.format("%s: (%d) Found existing process directories after killing last process", testName, i);
+                    errors.add(e);
+                    LOG.error(e);
+                } else {
+                    LOG.info("{}: ({}) Found no existing process directories after killing last process", testName, i);
+                }
+            }
+        }
+        if (!errors.isEmpty()) {
+            fail(String.format("There are %d failures in test:\n\t%s", errors.size(), String.join("\n\t", errors)));
+        }
+    }
+
+
+    /**
+     * Simulate the production scenario where the owner of the process directory is sometimes returned as the
+     * UID instead of user. This scenario is simulated by calling
+     * {@link ServerUtils#isAnyPosixProcessPidDirAlive(Collection, String, boolean)} with the last parameter
+     * set to true as well as false.
+     *
+     * @throws Exception on I/O exception
+     */
+    @Test
+    public void testIsAnyPosixProcessPidDirAliveMockingFileOwnerUid() throws Exception {
+        File procDir = new File("/proc");
+        if (!procDir.exists()) {
+            LOG.info("Test testIsAnyPosixProcessPidDirAlive is designed to run on systems with /proc directory only, marking as success");
+            return;
+        }
+        Collection<Long> pids = getRunningProcessIds();
+        assertFalse(pids.isEmpty());
+
+        for (int i = 0 ; i < 2 ; i++) {
+            boolean mockFileOwnerToUid = (i % 2 == 0);
+            // at least one pid will be owned by the current user (doing the testing)
+            String currentUser = System.getProperty("user.name");
+            boolean status = ServerUtils.isAnyPosixProcessPidDirAlive(pids, currentUser, mockFileOwnerToUid);
+            String err = String.format("(mockFileOwnerToUid=%s) Expecting user %s to own at least one process",
+                    mockFileOwnerToUid, currentUser);
+            assertTrue(err, status);
+        }
+    }
+
+    /**
+     * Make the best effort to obtain the Process ID from the Process object. Thus staying entirely with the JVM.
+     *
+     * @param p Process instance returned upon executing {@link Runtime#exec(String)}.
+     * @param errors Populate errors when PID is a negative number.
+     * @return positive PID upon success, otherwise negative.
+     */
+    private synchronized long getPidOfPosixProcess(Process p, List<String> errors) {
+        Class<? extends Process> pClass = p.getClass();
+        String pObjStr = ToStringBuilder.reflectionToString(p, ToStringStyle.SHORT_PREFIX_STYLE);
+        String pclassName = pClass.getName();
+        try {
+            if (pclassName.equals("java.lang.UNIXProcess")) {
+                Field f = pClass.getDeclaredField("pid");
+                f.setAccessible(true);
+                long pid = f.getLong(p);
+                f.setAccessible(false);
+                if (pid < 0) {
+                    errors.add("\t \"pid\" attribute in Process class " + pclassName + " returned -1, process=" + pObjStr);
+                }
+                return pid;
+            }
+            for (Field f : pClass.getDeclaredFields()) {
+                if (!f.getName().equalsIgnoreCase("pid")) {
+                    continue;
+                }
+                LOG.info("ServerUtilsTest.getPidOfPosixProcess(): found attribute {}#{}", pclassName, f.getName());
+                f.setAccessible(true);
+                long pid = f.getLong(p);
+                f.setAccessible(false);
+                if (pid < 0) {
+                    errors.add("\t \"pid\" attribute in Process class " + pclassName + " returned -1, process=" + pObjStr);
+                }
+                return pid;
+            }
+            // post JDK 9 there should be getPid() - future JDK-11 compatibility only for the sake of Travis test in community
+            try {
+                Method m = pClass.getDeclaredMethod("getPid");
+                LOG.info("ServerUtilsTest.getPidOfPosixProcess(): found method {}#getPid()\n", pclassName);
+                long pid = (Long)m.invoke(p);
+                if (pid < 0) {
+                    errors.add("\t \"getPid()\" method in Process class " + pclassName + " returned -1, process=" + pObjStr);
+                }
+                return pid;
+            } catch (SecurityException e) {
+                errors.add("\t getPid() method in Process class " + pclassName + " cannot be called: " + e.getMessage() + ", process=" + pObjStr);
+                return -1;
+            } catch (NoSuchMethodException e) {
+                // ignore and try something else
+            }
+            errors.add("\t Process class " + pclassName + " missing field \"pid\" and missing method \"getPid()\", process=" + pObjStr);
+            return -1;
+        } catch (Exception e) {
+            errors.add("\t Exception in Process class " + pclassName + ": " + e.getMessage() + ", process=" + pObjStr);
+            e.printStackTrace();
+            return -1;
+        }
+    }
+
+    /**
+     * Sleep for specified milliseconds and return true if sleep was interrupted.
+     *
+     * @param milliSeconds number of milliseconds to sleep
+     * @return true if sleep was interrupted, false otherwise.
+     */
+    private boolean sleepInterrupted(long milliSeconds) {

Review comment:
       I am not aware of any existing method with similar functionality.




----------------------------------------------------------------
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