This is an automated email from the ASF dual-hosted git repository.
scwhittle pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/beam.git
The following commit(s) were added to refs/heads/master by this push:
new eb5f9950c4a [Direct Runner] Fix test failure and logging configuration
by ensuring overridden Loggers are not garbaged collected (#37763)
eb5f9950c4a is described below
commit eb5f9950c4aeef58d36569156c827da945ec3d27
Author: Sam Whittle <[email protected]>
AuthorDate: Wed Mar 4 13:53:24 2026 +0000
[Direct Runner] Fix test failure and logging configuration by ensuring
overridden Loggers are not garbaged collected (#37763)
---
.../src/main/java/org/apache/beam/runners/direct/DirectRunner.java | 7 ++++++-
1 file changed, 6 insertions(+), 1 deletion(-)
diff --git
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
index 874cb96293c..7db6f61fbfd 100644
---
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
+++
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
@@ -27,6 +27,7 @@ import java.util.Map;
import java.util.Set;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
+import java.util.logging.Logger;
import org.apache.beam.runners.core.SplittableParDoViaKeyedWorkItems;
import org.apache.beam.runners.direct.DirectRunner.DirectPipelineResult;
import
org.apache.beam.runners.direct.TestStreamEvaluatorFactory.DirectTestStreamFactory;
@@ -185,7 +186,10 @@ public class DirectRunner extends
PipelineRunner<DirectPipelineResult> {
DisplayDataValidator.validatePipeline(pipeline);
DisplayDataValidator.validateOptions(options);
-
SdkHarnessOptions.getConfiguredLoggerFromOptions(options.as(SdkHarnessOptions.class));
+ // Ensure a reference is held on the configured loggers as otherwise
they are eligible for
+ // garbage collection as they are internally weak references.
+ List<Logger> loggers =
+
SdkHarnessOptions.getConfiguredLoggerFromOptions(options.as(SdkHarnessOptions.class));
ExecutorService metricsPool =
Executors.newCachedThreadPool(
new ThreadFactoryBuilder()
@@ -226,6 +230,7 @@ public class DirectRunner extends
PipelineRunner<DirectPipelineResult> {
throw new RuntimeException(t);
}
}
+ loggers.clear();
return result;
} finally {
MetricsEnvironment.setMetricsSupported(false);