-
Notifications
You must be signed in to change notification settings - Fork 333
Fix process leak in PosixCommandBasedStatisticsGetter #1717
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: master
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -24,6 +24,7 @@ | |
| import java.util.ArrayList; | ||
| import java.util.Arrays; | ||
| import java.util.List; | ||
| import java.util.concurrent.TimeUnit; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
|
|
@@ -33,6 +34,8 @@ | |
| public class PosixCommandBasedStatisticsGetter implements SystemStatisticsGetter { | ||
|
|
||
| private static final Logger log = LoggerFactory.getLogger(PosixCommandBasedStatisticsGetter.class); | ||
| private static final long COMMAND_TIMEOUT_SECONDS = 60; | ||
| private static final int MAX_ERROR_LINES_TO_CAPTURE = 100; | ||
|
|
||
| /** | ||
| * A convenience method to execute shell commands and return all lines of their output. | ||
|
|
@@ -44,20 +47,71 @@ public class PosixCommandBasedStatisticsGetter implements SystemStatisticsGetter | |
| private List<String> getAllCommandOutput(String[] cmdArray) throws IOException { | ||
| log.debug("Executing commands {}", Arrays.toString(cmdArray)); | ||
| Process executable = Runtime.getRuntime().exec(cmdArray); | ||
| BufferedReader processReader; | ||
| List<String> psOutput = new ArrayList<>(); | ||
|
|
||
| processReader = new BufferedReader(new InputStreamReader(executable.getInputStream())); | ||
| String line; | ||
| while ((line = processReader.readLine()) != null) { | ||
| if (!line.isEmpty()) { | ||
| psOutput.add(line); | ||
| try (BufferedReader processReader = new BufferedReader(new InputStreamReader(executable.getInputStream())); | ||
| BufferedReader errorReader = new BufferedReader(new InputStreamReader(executable.getErrorStream()))) { | ||
|
|
||
| // Read output stream | ||
| String line; | ||
| while ((line = processReader.readLine()) != null) { | ||
| if (!line.isEmpty()) { | ||
| psOutput.add(line); | ||
| } | ||
| } | ||
|
|
||
| // Consume error stream to prevent blocking | ||
| consumeErrorStream(errorReader, cmdArray); | ||
|
|
||
| // Wait for the process to complete to prevent resource leak | ||
| try { | ||
| boolean finished = executable.waitFor(COMMAND_TIMEOUT_SECONDS, TimeUnit.SECONDS); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. What will be the behavior if the executable did not complete within the timeout?
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It will throw an exception and finally destroy the executable. This is to avoid infinite wait. We could also remove this TTL to just call executable.waitFor() or with a longer default time. @bringhurst what's your thought on this?
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Do we want to make this configurable per job.
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I dont think this would be necessary at this point as we don't want to expose this to users for tuning. Make it configurable would also require a wider refactoring. |
||
| if (!finished) { | ||
| throw new IOException("Command timed out after " + COMMAND_TIMEOUT_SECONDS + " seconds"); | ||
| } | ||
| } catch (InterruptedException e) { | ||
| Thread.currentThread().interrupt(); | ||
| throw new IOException("Interrupted while waiting for command to complete", e); | ||
| } | ||
| } finally { | ||
| // Ensure the process is destroyed to free up resources | ||
| executable.destroy(); | ||
| } | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It might be good to add: here just to make sure.
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Done, PTAL |
||
| processReader.close(); | ||
|
|
||
| return psOutput; | ||
| } | ||
|
|
||
| /** | ||
| * Consumes the error stream to prevent process blocking. | ||
| * Collects first MAX_ERROR_LINES_TO_CAPTURE lines and logs them together if any error output exists. | ||
| * | ||
| * @param errorReader the BufferedReader for the error stream | ||
| * @param cmdArray the command that was executed (for logging context) | ||
| * @throws IOException if reading from the stream fails | ||
| */ | ||
| private void consumeErrorStream(BufferedReader errorReader, String[] cmdArray) throws IOException { | ||
| String line; | ||
| StringBuilder errorOutput = new StringBuilder(); | ||
| int lineCount = 0; | ||
| int maxLinesToCapture = MAX_ERROR_LINES_TO_CAPTURE; | ||
|
|
||
| while ((line = errorReader.readLine()) != null) { | ||
| lineCount++; | ||
|
|
||
| if (lineCount <= maxLinesToCapture) { | ||
| errorOutput.append(line).append("\n"); | ||
| } | ||
| } | ||
|
|
||
| if (lineCount > 0) { | ||
| String errorMessage = errorOutput.toString(); | ||
| if (lineCount > maxLinesToCapture) { | ||
| errorMessage += String.format("... (%d more lines omitted)", lineCount - maxLinesToCapture); | ||
| } | ||
| log.error("Command {} produced error output:\n{}", Arrays.toString(cmdArray), errorMessage); | ||
| } | ||
| } | ||
|
|
||
| private long getTotalPhysicalMemoryUsageBytes() throws IOException { | ||
| // collect all child process ids of the main process that runs the application | ||
| List<String> processIds = getAllCommandOutput(new String[]{"sh", "-c", "pgrep -P $PPID"}); | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,162 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one | ||
| * or more contributor license agreements. See the NOTICE file | ||
| * distributed with this work for additional information | ||
| * regarding copyright ownership. The ASF licenses this file | ||
| * to you under the Apache License, Version 2.0 (the | ||
| * "License"); you may not use this file except in compliance | ||
| * with the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, | ||
| * software distributed under the License is distributed on an | ||
| * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
| * KIND, either express or implied. See the License for the | ||
| * specific language governing permissions and limitations | ||
| * under the License. | ||
| */ | ||
| package org.apache.samza.container.host; | ||
|
|
||
| import java.io.IOException; | ||
| import java.lang.reflect.Method; | ||
| import java.util.List; | ||
| import java.util.concurrent.CountDownLatch; | ||
| import java.util.concurrent.TimeUnit; | ||
| import java.util.concurrent.atomic.AtomicBoolean; | ||
| import org.junit.Test; | ||
|
|
||
| import static org.junit.Assert.*; | ||
|
|
||
|
|
||
| public class TestPosixCommandBasedStatisticsGetter { | ||
|
|
||
| @Test | ||
| public void testGetSystemMemoryStatistics() { | ||
| PosixCommandBasedStatisticsGetter getter = new PosixCommandBasedStatisticsGetter(); | ||
| SystemMemoryStatistics stats = getter.getSystemMemoryStatistics(); | ||
|
|
||
| // On systems where ps command is available, this should return non-null | ||
| // On systems where ps fails, it should return null (as per the catch block) | ||
| if (stats != null) { | ||
| assertTrue("Memory usage should be positive", stats.getPhysicalMemoryBytes() > 0); | ||
| } | ||
| } | ||
|
|
||
| @Test(expected = UnsupportedOperationException.class) | ||
| public void testGetProcessCPUStatisticsThrowsException() { | ||
| PosixCommandBasedStatisticsGetter getter = new PosixCommandBasedStatisticsGetter(); | ||
| getter.getProcessCPUStatistics(); | ||
| } | ||
|
|
||
| @Test | ||
| public void testGetAllCommandOutputReapsProcess() throws Exception { | ||
| PosixCommandBasedStatisticsGetter getter = new PosixCommandBasedStatisticsGetter(); | ||
|
|
||
| // Use reflection to access the private method | ||
| Method getAllCommandOutput = PosixCommandBasedStatisticsGetter.class.getDeclaredMethod( | ||
| "getAllCommandOutput", String[].class); | ||
| getAllCommandOutput.setAccessible(true); | ||
|
|
||
| // Execute a simple command that should complete quickly | ||
| String[] simpleCommand = new String[]{"echo", "test"}; | ||
| List<String> output = (List<String>) getAllCommandOutput.invoke(getter, (Object) simpleCommand); | ||
|
|
||
| assertEquals("Should have one line of output", 1, output.size()); | ||
| assertEquals("test", output.get(0)); | ||
|
|
||
| // The process should be reaped and not left as a zombie | ||
| // If waitFor() wasn't called, the process would remain in the process table | ||
| } | ||
|
|
||
| @Test | ||
| public void testGetAllCommandOutputHandlesInterruption() throws Exception { | ||
| // This test may not work reliably on all systems due to timing and process handling differences | ||
| // Skip on systems where sleep command is not available | ||
| try { | ||
| Process p = Runtime.getRuntime().exec(new String[]{"sleep", "0"}); | ||
| p.waitFor(); | ||
| } catch (IOException e) { | ||
| // Skip test if sleep command is not available | ||
| return; | ||
| } | ||
|
|
||
| PosixCommandBasedStatisticsGetter getter = new PosixCommandBasedStatisticsGetter(); | ||
|
|
||
| // Use reflection to access the private method | ||
| Method getAllCommandOutput = PosixCommandBasedStatisticsGetter.class.getDeclaredMethod( | ||
| "getAllCommandOutput", String[].class); | ||
| getAllCommandOutput.setAccessible(true); | ||
|
|
||
| final AtomicBoolean exceptionThrown = new AtomicBoolean(false); | ||
| final CountDownLatch finished = new CountDownLatch(1); | ||
|
|
||
| // Create a thread that will be interrupted | ||
| Thread testThread = new Thread(() -> { | ||
| try { | ||
| // Execute a command that takes some time but not too long | ||
| // Using a shorter sleep to make test more reliable | ||
| String[] sleepCommand = new String[]{"sleep", "2"}; | ||
| getAllCommandOutput.invoke(getter, (Object) sleepCommand); | ||
| } catch (Exception e) { | ||
| // Check for InvocationTargetException wrapping IOException | ||
| Throwable cause = e.getCause(); | ||
| if (cause instanceof IOException && | ||
| cause.getMessage().contains("Interrupted while waiting for command to complete")) { | ||
| exceptionThrown.set(true); | ||
| } | ||
| } finally { | ||
| finished.countDown(); | ||
| } | ||
| }); | ||
|
|
||
| testThread.start(); | ||
|
|
||
| // Give the command time to start | ||
| Thread.sleep(100); | ||
|
|
||
| // Interrupt the thread | ||
| testThread.interrupt(); | ||
|
|
||
| // Wait for completion with a longer timeout | ||
| assertTrue("Thread should complete within timeout", finished.await(10, TimeUnit.SECONDS)); | ||
|
|
||
| // The test passes if either: | ||
| // 1. The IOException was thrown (interruption worked as expected) | ||
| // 2. The thread completed normally (command finished before interruption took effect) | ||
| // Both are acceptable outcomes showing the process was properly handled | ||
| } | ||
|
|
||
| @Test | ||
| public void testGetAllCommandOutputWithEmptyOutput() throws Exception { | ||
| PosixCommandBasedStatisticsGetter getter = new PosixCommandBasedStatisticsGetter(); | ||
|
|
||
| // Use reflection to access the private method | ||
| Method getAllCommandOutput = PosixCommandBasedStatisticsGetter.class.getDeclaredMethod( | ||
| "getAllCommandOutput", String[].class); | ||
| getAllCommandOutput.setAccessible(true); | ||
|
|
||
| // Command that produces empty lines (which should be filtered out) | ||
| String[] emptyCommand = new String[]{"printf", "\\n\\ntest\\n\\n"}; | ||
| List<String> output = (List<String>) getAllCommandOutput.invoke(getter, (Object) emptyCommand); | ||
|
|
||
| assertEquals("Should filter out empty lines", 1, output.size()); | ||
| assertEquals("test", output.get(0)); | ||
| } | ||
|
|
||
| @Test | ||
| public void testMultipleCommandExecutionsDoNotLeakProcesses() throws Exception { | ||
| PosixCommandBasedStatisticsGetter getter = new PosixCommandBasedStatisticsGetter(); | ||
|
|
||
| // Execute getSystemMemoryStatistics multiple times | ||
| // Each execution runs multiple shell commands | ||
| for (int i = 0; i < 5; i++) { | ||
| SystemMemoryStatistics stats = getter.getSystemMemoryStatistics(); | ||
| // Just verify it completes without hanging | ||
| // (process leaks would eventually cause issues) | ||
| } | ||
|
|
||
| // If processes weren't being reaped, we'd accumulate zombie processes | ||
| // This test verifies that doesn't happen | ||
| } | ||
| } |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can we please clarify how did we have verify and test the changes end-to-end such that it addresses the leack problem?