Skip to content

Commit 0b5a014

Browse files
committed
Improve stderr drain performance by only record the first 100 lines
1 parent d0311c8 commit 0b5a014

1 file changed

Lines changed: 33 additions & 8 deletions

File tree

samza-core/src/main/java/org/apache/samza/container/host/PosixCommandBasedStatisticsGetter.java

Lines changed: 33 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -35,6 +35,7 @@ public class PosixCommandBasedStatisticsGetter implements SystemStatisticsGetter
3535

3636
private static final Logger log = LoggerFactory.getLogger(PosixCommandBasedStatisticsGetter.class);
3737
private static final long COMMAND_TIMEOUT_SECONDS = 10;
38+
private static final int MAX_ERROR_LINES_TO_CAPTURE = 100;
3839

3940
/**
4041
* A convenience method to execute shell commands and return all lines of their output.
@@ -60,14 +61,7 @@ private List<String> getAllCommandOutput(String[] cmdArray) throws IOException {
6061
}
6162

6263
// Consume error stream to prevent blocking
63-
StringBuilder errorOutput = new StringBuilder();
64-
while ((line = errorReader.readLine()) != null) {
65-
errorOutput.append(line).append("\n");
66-
}
67-
68-
if (!line.isEmpty()) {
69-
log.error("stderr while running {}: {}", cmdArray, errorOutput);
70-
}
64+
consumeErrorStream(errorReader, cmdArray);
7165

7266
// Wait for the process to complete to prevent resource leak
7367
try {
@@ -87,6 +81,37 @@ private List<String> getAllCommandOutput(String[] cmdArray) throws IOException {
8781
return psOutput;
8882
}
8983

84+
/**
85+
* Consumes the error stream to prevent process blocking.
86+
* Collects first MAX_ERROR_LINES_TO_CAPTURE lines and logs them together if any error output exists.
87+
*
88+
* @param errorReader the BufferedReader for the error stream
89+
* @param cmdArray the command that was executed (for logging context)
90+
* @throws IOException if reading from the stream fails
91+
*/
92+
private void consumeErrorStream(BufferedReader errorReader, String[] cmdArray) throws IOException {
93+
String line;
94+
StringBuilder errorOutput = new StringBuilder();
95+
int lineCount = 0;
96+
int maxLinesToCapture = MAX_ERROR_LINES_TO_CAPTURE;
97+
98+
while ((line = errorReader.readLine()) != null) {
99+
lineCount++;
100+
101+
if (lineCount <= maxLinesToCapture) {
102+
errorOutput.append(line).append("\n");
103+
}
104+
}
105+
106+
if (lineCount > 0) {
107+
String errorMessage = errorOutput.toString();
108+
if (lineCount > maxLinesToCapture) {
109+
errorMessage += String.format("... (%d more lines omitted)", lineCount - maxLinesToCapture);
110+
}
111+
log.error("Command {} produced error output:\n{}", Arrays.toString(cmdArray), errorMessage);
112+
}
113+
}
114+
90115
private long getTotalPhysicalMemoryUsageBytes() throws IOException {
91116
// collect all child process ids of the main process that runs the application
92117
List<String> processIds = getAllCommandOutput(new String[]{"sh", "-c", "pgrep -P $PPID"});

0 commit comments

Comments
 (0)