Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@
import org.apache.beam.sdk.metrics.MetricResults;
import org.apache.beam.sdk.metrics.MetricsEnvironment;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.options.SdkHarnessOptions;
import org.apache.beam.sdk.runners.PTransformOverride;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.util.UserCodeException;
Expand Down Expand Up @@ -184,7 +185,7 @@ public DirectPipelineResult run(Pipeline pipeline) {

DisplayDataValidator.validatePipeline(pipeline);
DisplayDataValidator.validateOptions(options);

SdkHarnessOptions.getConfiguredLoggerFromOptions(options.as(SdkHarnessOptions.class));
ExecutorService metricsPool =
Executors.newCachedThreadPool(
new ThreadFactoryBuilder()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,8 @@
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.logging.Level;
import java.util.logging.LogManager;
import org.apache.beam.runners.direct.DirectRunner.DirectPipelineResult;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.PipelineResult;
Expand All @@ -75,6 +77,7 @@
import org.apache.beam.sdk.transforms.Create;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.Flatten;
import org.apache.beam.sdk.transforms.Impulse;
import org.apache.beam.sdk.transforms.MapElements;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.ParDo;
Expand Down Expand Up @@ -744,6 +747,50 @@ public interface TestSerializationOfOptions extends PipelineOptions {
void setIgnoredField(String value);
}

@Test
public void testLogLevel() {
PipelineOptions options =
PipelineOptionsFactory.fromArgs(
new String[] {
"--runner=DirectRunner",
"--defaultSdkHarnessLogLevel=ERROR",
"--sdkHarnessLogLevelOverrides={\"org.apache.beam.runners.direct.DirectRunnerTest\":\"INFO\"}"
})
.create();
Pipeline pipeline = Pipeline.create(options);

LogManager logManager = LogManager.getLogManager();
// use full name to avoid conflicts with org.slf4j.Logger
java.util.logging.Logger rootLogger = logManager.getLogger("");
Level originalLevel = rootLogger.getLevel();

try {
pipeline
.apply(Impulse.create())
.apply(
ParDo.of(
new DoFn<byte[], byte[]>() {
@ProcessElement
public void process(@Element byte[] element, OutputReceiver<byte[]> o) {
LogManager logManager = LogManager.getLogManager();
java.util.logging.Logger rootLogger = logManager.getLogger("");
// check loglevel here. Whether actual logs are rendered depends on slf4j impl
// and upstream configs.
assertEquals(Level.SEVERE, rootLogger.getLevel());
assertEquals(
Level.INFO,
java.util.logging.Logger.getLogger(
"org.apache.beam.runners.direct.DirectRunnerTest")
.getLevel());
}
}));
pipeline.run();
} finally {
// resume original log level
rootLogger.setLevel(originalLevel);
}
}

private static class LongNoDecodeCoder extends AtomicCoder<Long> {
@Override
public void encode(Long value, OutputStream outStream) throws IOException {}
Expand Down
Loading