|
49 | 49 | import java.util.concurrent.TimeoutException; |
50 | 50 | import java.util.concurrent.atomic.AtomicInteger; |
51 | 51 | import java.util.concurrent.atomic.AtomicLong; |
| 52 | +import java.util.logging.Level; |
| 53 | +import java.util.logging.LogManager; |
52 | 54 | import org.apache.beam.runners.direct.DirectRunner.DirectPipelineResult; |
53 | 55 | import org.apache.beam.sdk.Pipeline; |
54 | 56 | import org.apache.beam.sdk.PipelineResult; |
|
75 | 77 | import org.apache.beam.sdk.transforms.Create; |
76 | 78 | import org.apache.beam.sdk.transforms.DoFn; |
77 | 79 | import org.apache.beam.sdk.transforms.Flatten; |
| 80 | +import org.apache.beam.sdk.transforms.Impulse; |
78 | 81 | import org.apache.beam.sdk.transforms.MapElements; |
79 | 82 | import org.apache.beam.sdk.transforms.PTransform; |
80 | 83 | import org.apache.beam.sdk.transforms.ParDo; |
@@ -744,6 +747,50 @@ public interface TestSerializationOfOptions extends PipelineOptions { |
744 | 747 | void setIgnoredField(String value); |
745 | 748 | } |
746 | 749 |
|
| 750 | + @Test |
| 751 | + public void testLogLevel() { |
| 752 | + PipelineOptions options = |
| 753 | + PipelineOptionsFactory.fromArgs( |
| 754 | + new String[] { |
| 755 | + "--runner=DirectRunner", |
| 756 | + "--defaultSdkHarnessLogLevel=ERROR", |
| 757 | + "--sdkHarnessLogLevelOverrides={\"org.apache.beam.runners.direct.DirectRunnerTest\":\"INFO\"}" |
| 758 | + }) |
| 759 | + .create(); |
| 760 | + Pipeline pipeline = Pipeline.create(options); |
| 761 | + |
| 762 | + LogManager logManager = LogManager.getLogManager(); |
| 763 | + // use full name to avoid conflicts with org.slf4j.Logger |
| 764 | + java.util.logging.Logger rootLogger = logManager.getLogger(""); |
| 765 | + Level originalLevel = rootLogger.getLevel(); |
| 766 | + |
| 767 | + try { |
| 768 | + pipeline |
| 769 | + .apply(Impulse.create()) |
| 770 | + .apply( |
| 771 | + ParDo.of( |
| 772 | + new DoFn<byte[], byte[]>() { |
| 773 | + @ProcessElement |
| 774 | + public void process(@Element byte[] element, OutputReceiver<byte[]> o) { |
| 775 | + LogManager logManager = LogManager.getLogManager(); |
| 776 | + java.util.logging.Logger rootLogger = logManager.getLogger(""); |
| 777 | + // check loglevel here. Whether actual logs are rendered depends on slf4j impl |
| 778 | + // and upstream configs. |
| 779 | + assertEquals(Level.SEVERE, rootLogger.getLevel()); |
| 780 | + assertEquals( |
| 781 | + Level.INFO, |
| 782 | + java.util.logging.Logger.getLogger( |
| 783 | + "org.apache.beam.runners.direct.DirectRunnerTest") |
| 784 | + .getLevel()); |
| 785 | + } |
| 786 | + })); |
| 787 | + pipeline.run(); |
| 788 | + } finally { |
| 789 | + // resume original log level |
| 790 | + rootLogger.setLevel(originalLevel); |
| 791 | + } |
| 792 | + } |
| 793 | + |
747 | 794 | private static class LongNoDecodeCoder extends AtomicCoder<Long> { |
748 | 795 | @Override |
749 | 796 | public void encode(Long value, OutputStream outStream) throws IOException {} |
|
0 commit comments