Skip to content

Commit eb5f995

Browse files
authored
[Direct Runner] Fix test failure and logging configuration by ensuring overridden Loggers are not garbaged collected (#37763)
1 parent e1e0637 commit eb5f995

1 file changed

Lines changed: 6 additions & 1 deletion

File tree

runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java

Lines changed: 6 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,7 @@
2727
import java.util.Set;
2828
import java.util.concurrent.ExecutorService;
2929
import java.util.concurrent.Executors;
30+
import java.util.logging.Logger;
3031
import org.apache.beam.runners.core.SplittableParDoViaKeyedWorkItems;
3132
import org.apache.beam.runners.direct.DirectRunner.DirectPipelineResult;
3233
import org.apache.beam.runners.direct.TestStreamEvaluatorFactory.DirectTestStreamFactory;
@@ -185,7 +186,10 @@ public DirectPipelineResult run(Pipeline pipeline) {
185186

186187
DisplayDataValidator.validatePipeline(pipeline);
187188
DisplayDataValidator.validateOptions(options);
188-
SdkHarnessOptions.getConfiguredLoggerFromOptions(options.as(SdkHarnessOptions.class));
189+
// Ensure a reference is held on the configured loggers as otherwise they are eligible for
190+
// garbage collection as they are internally weak references.
191+
List<Logger> loggers =
192+
SdkHarnessOptions.getConfiguredLoggerFromOptions(options.as(SdkHarnessOptions.class));
189193
ExecutorService metricsPool =
190194
Executors.newCachedThreadPool(
191195
new ThreadFactoryBuilder()
@@ -226,6 +230,7 @@ public DirectPipelineResult run(Pipeline pipeline) {
226230
throw new RuntimeException(t);
227231
}
228232
}
233+
loggers.clear();
229234
return result;
230235
} finally {
231236
MetricsEnvironment.setMetricsSupported(false);

0 commit comments

Comments
 (0)