From 4e7814f716bba02a2eb1d71f61a95d081b035346 Mon Sep 17 00:00:00 2001 From: gaurav gupta Date: Wed, 10 Aug 2016 16:43:03 -0700 Subject: [PATCH 001/112] Made byteArrayCoder final static --- .../build-tools/src/main/resources/beam/findbugs-filter.xml | 6 ------ .../src/main/java/org/apache/beam/sdk/util/BitSetCoder.java | 2 +- 2 files changed, 1 insertion(+), 7 deletions(-) diff --git a/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml b/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml index 1ad79880772b..d135cbb671d9 100644 --- a/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml +++ b/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml @@ -109,12 +109,6 @@ - - - - - - diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java index 82c5f51c63d6..b406b1a56cc5 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java @@ -32,7 +32,7 @@ class BitSetCoder extends AtomicCoder { private static final BitSetCoder INSTANCE = new BitSetCoder(); - private transient ByteArrayCoder byteArrayCoder = ByteArrayCoder.of(); + private static final ByteArrayCoder byteArrayCoder = ByteArrayCoder.of(); private BitSetCoder() {} From 2ae1a7478df037cf558a808816216e7002b33b47 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Wed, 10 Aug 2016 17:58:09 -0700 Subject: [PATCH 002/112] CompressedSource: CompressedReader is never splittable The only way it's safe to split a compressed file is if the file is not compressed. This can only happen when the source itself is splittable, and that in turn will result in the inner source's reader being returned. A CompressedReader will only be created in the event that the file is NOT splittable. So remove all the logic handling splittable compressed readers, and instead go with the logic when we know/assume the file is compressed. * TextIO: test compression with larger files It is important for correctness that we test with large files because otherwise the compressed file may be larger than the uncompressed file, which could mask bugs * TextIOTest: flesh out more * TextIOTest: add large uncompressed file --- .../apache/beam/sdk/io/CompressedSource.java | 80 ++---- .../beam/sdk/io/CompressedSourceTest.java | 2 +- .../org/apache/beam/sdk/io/TextIOTest.java | 251 ++++++++++++++---- 3 files changed, 227 insertions(+), 106 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CompressedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CompressedSource.java index ee4b84b52717..11ff90f66af9 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CompressedSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CompressedSource.java @@ -430,7 +430,6 @@ public static class CompressedReader extends FileBasedReader { private final FileBasedReader readerDelegate; private final CompressedSource source; - private final boolean splittable; private final Object progressLock = new Object(); @GuardedBy("progressLock") private int numRecordsRead; @@ -443,13 +442,6 @@ public static class CompressedReader extends FileBasedReader { public CompressedReader(CompressedSource source, FileBasedReader readerDelegate) { super(source); this.source = source; - boolean splittable; - try { - splittable = source.isSplittable(); - } catch (Exception e) { - throw new RuntimeException("Unable to tell whether source " + source + " is splittable", e); - } - this.splittable = splittable; this.readerDelegate = readerDelegate; } @@ -463,27 +455,19 @@ public T getCurrent() throws NoSuchElementException { @Override public boolean allowsDynamicSplitting() { - return splittable; + return false; } @Override public final long getSplitPointsConsumed() { - if (splittable) { - return readerDelegate.getSplitPointsConsumed(); - } else { - synchronized (progressLock) { - return (isDone() && numRecordsRead > 0) ? 1 : 0; - } + synchronized (progressLock) { + return (isDone() && numRecordsRead > 0) ? 1 : 0; } } @Override public final long getSplitPointsRemaining() { - if (splittable) { - return readerDelegate.getSplitPointsRemaining(); - } else { - return isDone() ? 0 : 1; - } + return isDone() ? 0 : 1; } /** @@ -491,18 +475,14 @@ public final long getSplitPointsRemaining() { */ @Override protected final boolean isAtSplitPoint() { - if (splittable) { - return readerDelegate.isAtSplitPoint(); - } else { - // We have to return true for the first record, but not for the state before reading it, - // and not for the state after reading any other record. Hence == rather than >= or <=. - // This is required because FileBasedReader is intended for readers that can read a range - // of offsets in a file and where the range can be split in parts. CompressedReader, - // however, is a degenerate case because it cannot be split, but it has to satisfy the - // semantics of offsets and split points anyway. - synchronized (progressLock) { - return numRecordsRead == 1; - } + // We have to return true for the first record, but not for the state before reading it, + // and not for the state after reading any other record. Hence == rather than >= or <=. + // This is required because FileBasedReader is intended for readers that can read a range + // of offsets in a file and where the range can be split in parts. CompressedReader, + // however, is a degenerate case because it cannot be split, but it has to satisfy the + // semantics of offsets and split points anyway. + synchronized (progressLock) { + return numRecordsRead == 1; } } @@ -546,14 +526,9 @@ public void close() throws IOException { */ @Override protected final void startReading(ReadableByteChannel channel) throws IOException { - if (splittable) { - // No-op. We will always delegate to the inner reader, so this.channel and this.progressLock - // will never be used. - } else { - synchronized (progressLock) { - this.channel = new CountingChannel(channel, getCurrentSource().getStartOffset()); - channel = this.channel; - } + synchronized (progressLock) { + this.channel = new CountingChannel(channel, getCurrentSource().getStartOffset()); + channel = this.channel; } if (source.getChannelFactory() instanceof FileNameBasedDecompressingChannelFactory) { @@ -582,30 +557,21 @@ protected final boolean readNextRecord() throws IOException { return true; } - // Splittable: simply delegates to the inner reader. - // // Unsplittable: returns the offset in the input stream that has been read by the input. // these positions are likely to be coarse-grained (in the event of buffering) and // over-estimates (because they reflect the number of bytes read to produce an element, not its // start) but both of these provide better data than e.g., reporting the start of the file. @Override protected final long getCurrentOffset() throws NoSuchElementException { - if (splittable) { - return readerDelegate.getCurrentOffset(); - } else { - synchronized (progressLock) { - if (numRecordsRead <= 1) { - // Since the first record is at a split point, it should start at the beginning of the - // file. This avoids the bad case where the decompressor read the entire file, which - // would cause the file to be treated as empty when returning channel.getCount() as it - // is outside the valid range. - return 0; - } - if (channel == null) { - throw new NoSuchElementException(); - } - return channel.getCount(); + synchronized (progressLock) { + if (numRecordsRead <= 1) { + // Since the first record is at a split point, it should start at the beginning of the + // file. This avoids the bad case where the decompressor read the entire file, which + // would cause the file to be treated as empty when returning channel.getCount() as it + // is outside the valid range. + return 0; } + return channel.getCount(); } } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CompressedSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CompressedSourceTest.java index 01e5fe5e9c6c..4a9f95088804 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CompressedSourceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CompressedSourceTest.java @@ -563,7 +563,7 @@ protected boolean readNextRecord() throws IOException { if (channel.read(buff) != 1) { return false; } - current = new Byte(buff.get(0)); + current = buff.get(0); offset += 1; return true; } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java index 6ec3a717af6b..6fd3093f2155 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java @@ -25,6 +25,7 @@ import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasValue; import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.startsWith; import static org.junit.Assert.assertEquals; @@ -61,6 +62,7 @@ import com.google.common.collect.ImmutableList; +import org.apache.commons.compress.compressors.bzip2.BZip2CompressorOutputStream; import org.junit.BeforeClass; import org.junit.Ignore; import org.junit.Rule; @@ -79,6 +81,7 @@ import java.io.FileOutputStream; import java.io.FileReader; import java.io.IOException; +import java.io.OutputStream; import java.io.PrintStream; import java.nio.channels.FileChannel; import java.nio.channels.SeekableByteChannel; @@ -102,8 +105,11 @@ @RunWith(JUnit4.class) @SuppressWarnings("unchecked") public class TextIOTest { - @Rule public TemporaryFolder tmpFolder = new TemporaryFolder(); - @Rule public ExpectedException expectedException = ExpectedException.none(); + + @Rule + public TemporaryFolder tmpFolder = new TemporaryFolder(); + @Rule + public ExpectedException expectedException = ExpectedException.none(); @BeforeClass public static void setupClass() { @@ -166,7 +172,7 @@ public void testReadEmptyInts() throws Exception { @Test @Category(NeedsRunner.class) public void testReadNulls() throws Exception { - runTestRead(new Void[]{ null, null, null }, VoidCoder.of()); + runTestRead(new Void[]{null, null, null}, VoidCoder.of()); } @Test @@ -419,32 +425,205 @@ public void testCompressionTypeIsSet() throws Exception { assertEquals(CompressionType.GZIP, read.getCompressionType()); } - @Test - @Category(NeedsRunner.class) - public void testCompressedRead() throws Exception { - String[] lines = {"Irritable eagle", "Optimistic jay", "Fanciful hawk"}; - File tmpFile = tmpFolder.newFile(); - String filename = tmpFile.getPath(); - - List expected = new ArrayList<>(); - try (PrintStream writer = - new PrintStream(new GZIPOutputStream(new FileOutputStream(tmpFile)))) { + /** + * Helper that writes the given lines (adding a newline in between) to a stream, then closes the + * stream. + */ + private static void writeToStreamAndClose(String[] lines, OutputStream outputStream) { + try (PrintStream writer = new PrintStream(outputStream)) { for (String line : lines) { writer.println(line); - expected.add(line); } } + } + /** + * Helper method that runs TextIO.Read.from(filename).withCompressionType(compressionType) + * and asserts that the results match the given expected output. + */ + private static void assertReadingCompressedFileMatchesExpected( + String filename, CompressionType compressionType, String[] expected) { Pipeline p = TestPipeline.create(); - TextIO.Read.Bound read = - TextIO.Read.from(filename).withCompressionType(CompressionType.GZIP); + TextIO.Read.from(filename).withCompressionType(compressionType); PCollection output = p.apply(read); PAssert.that(output).containsInAnyOrder(expected); p.run(); } + /** + * Helper to make an array of compressible strings. Returns ["word"i] for i in range(0,n). + */ + private static String[] makeLines(int n) { + String[] ret = new String[n]; + for (int i = 0; i < n; ++i) { + ret[i] = "word" + i; + } + return ret; + } + + /** + * Tests reading from a small, gzipped file with no .gz extension but GZIP compression set. + */ + @Test + @Category(NeedsRunner.class) + public void testSmallCompressedGzipReadNoExtension() throws Exception { + String[] lines = {"Irritable eagle", "Optimistic jay", "Fanciful hawk"}; + File tmpFile = tmpFolder.newFile(); // no GZ extension + String filename = tmpFile.getPath(); + + writeToStreamAndClose(lines, new GZIPOutputStream(new FileOutputStream(tmpFile))); + assertReadingCompressedFileMatchesExpected(filename, CompressionType.GZIP, lines); + } + + /** + * Tests reading from a small, gzipped file with .gz extension and AUTO or GZIP compression set. + */ + @Test + @Category(NeedsRunner.class) + public void testSmallCompressedGzipRead() throws Exception { + String[] lines = {"Irritable eagle", "Optimistic jay", "Fanciful hawk"}; + File tmpFile = tmpFolder.newFile("small_gzip.gz"); + String filename = tmpFile.getPath(); + + writeToStreamAndClose(lines, new GZIPOutputStream(new FileOutputStream(tmpFile))); + // Should work in AUTO mode. + assertReadingCompressedFileMatchesExpected(filename, CompressionType.AUTO, lines); + // Should work in GZIP mode. + assertReadingCompressedFileMatchesExpected(filename, CompressionType.GZIP, lines); + } + + /** + * Tests reading from a small, uncompressed file with .gz extension. + * This must work in AUTO or GZIP modes. This is needed because some network file systems / HTTP + * clients will transparently decompress gzipped content. + */ + @Test + @Category(NeedsRunner.class) + public void testSmallCompressedGzipReadActuallyUncompressed() throws Exception { + String[] lines = {"Irritable eagle", "Optimistic jay", "Fanciful hawk"}; + File tmpFile = tmpFolder.newFile("not_really_gzipped.gz"); // GZ file extension lies + String filename = tmpFile.getPath(); + + writeToStreamAndClose(lines, new FileOutputStream(tmpFile)); + // Should work with GZIP compression set. + assertReadingCompressedFileMatchesExpected(filename, CompressionType.GZIP, lines); + // Should also work with AUTO mode set. + assertReadingCompressedFileMatchesExpected(filename, CompressionType.AUTO, lines); + } + + /** + * Tests reading from a small, bzip2ed file with no .bz2 extension but BZIP2 compression set. + */ + @Test + @Category(NeedsRunner.class) + public void testSmallCompressedBzip2ReadNoExtension() throws Exception { + String[] lines = {"Irritable eagle", "Optimistic jay", "Fanciful hawk"}; + File tmpFile = tmpFolder.newFile(); // no BZ2 extension + String filename = tmpFile.getPath(); + + writeToStreamAndClose(lines, new BZip2CompressorOutputStream(new FileOutputStream(tmpFile))); + assertReadingCompressedFileMatchesExpected(filename, CompressionType.BZIP2, lines); + } + + /** + * Tests reading from a small, bzip2ed file with .bz2 extension and AUTO or BZIP2 compression set. + */ + @Test + @Category(NeedsRunner.class) + public void testSmallCompressedBzipRead() throws Exception { + String[] lines = {"Irritable eagle", "Optimistic jay", "Fanciful hawk"}; + File tmpFile = tmpFolder.newFile("small_bzip2.bz2"); + String filename = tmpFile.getPath(); + + writeToStreamAndClose(lines, new BZip2CompressorOutputStream(new FileOutputStream(tmpFile))); + // Should work in AUTO mode. + assertReadingCompressedFileMatchesExpected(filename, CompressionType.AUTO, lines); + // Should work in BZIP2 mode. + assertReadingCompressedFileMatchesExpected(filename, CompressionType.BZIP2, lines); + } + + /** + * Tests reading from a large, bzip2ed file with .bz2 extension and AUTO or BZIP2 compression set. + * It is important to test a large compressible file because using only small files may mask bugs + * from range tracking that can only occur if the file compression ratio is high -- small + * compressed files are usually as big as the uncompressed ones or bigger. + */ + @Test + @Category(NeedsRunner.class) + public void testLargeCompressedBzipRead() throws Exception { + String[] lines = makeLines(5000); + File bz2File = tmpFolder.newFile("large_bzip2.bz2"); + String bz2Filename = bz2File.getPath(); + + writeToStreamAndClose(lines, new BZip2CompressorOutputStream(new FileOutputStream(bz2File))); + // Should work in AUTO mode. + assertReadingCompressedFileMatchesExpected(bz2Filename, CompressionType.AUTO, lines); + // Should work in BZIP2 mode. + assertReadingCompressedFileMatchesExpected(bz2Filename, CompressionType.BZIP2, lines); + + // Confirm that the compressed file is smaller than the uncompressed file. + File txtFile = tmpFolder.newFile("large_bzip2.txt"); + writeToStreamAndClose(lines, new FileOutputStream(txtFile)); + assertThat(Files.size(txtFile.toPath()), greaterThan(Files.size(bz2File.toPath()))); + } + + /** + * Tests reading from a large, gzipped file with .gz extension and AUTO or GZIP compression set. + * It is important to test a large compressible file because using only small files may mask bugs + * from range tracking that can only occur if the file compression ratio is high -- small + * compressed files are usually as big as the uncompressed ones or bigger. + */ + @Test + @Category(NeedsRunner.class) + public void testLargeCompressedGzipRead() throws Exception { + String[] lines = makeLines(5000); + File gzFile = tmpFolder.newFile("large_gzip.gz"); + String gzFilename = gzFile.getPath(); + + writeToStreamAndClose(lines, new GZIPOutputStream(new FileOutputStream(gzFile))); + // Should work in AUTO mode. + assertReadingCompressedFileMatchesExpected(gzFilename, CompressionType.AUTO, lines); + // Should work in BZIP2 mode. + assertReadingCompressedFileMatchesExpected(gzFilename, CompressionType.GZIP, lines); + + // Confirm that the compressed file is smaller than the uncompressed file. + File txtFile = tmpFolder.newFile("large_gzip.txt"); + writeToStreamAndClose(lines, new FileOutputStream(txtFile)); + assertThat(Files.size(txtFile.toPath()), greaterThan(Files.size(gzFile.toPath()))); + } + + /** + * Tests reading from a large, uncompressed file. + */ + @Test + @Category(NeedsRunner.class) + public void testLargeUncompressedReadTxt() throws Exception { + String[] lines = makeLines(5000); + File txtFile = tmpFolder.newFile("large_file.txt"); + String txtFilename = txtFile.getPath(); + + writeToStreamAndClose(lines, new FileOutputStream(txtFile)); + // Should work in AUTO mode. + assertReadingCompressedFileMatchesExpected(txtFilename, CompressionType.AUTO, lines); + } + + /** + * Tests reading from a large, uncompressed file with a weird file extension. + */ + @Test + @Category(NeedsRunner.class) + public void testLargeUncompressedReadWeirdExtension() throws Exception { + String[] lines = makeLines(5000); + File txtFile = tmpFolder.newFile("large_file.bin.data.foo"); + String txtFilename = txtFile.getPath(); + + writeToStreamAndClose(lines, new FileOutputStream(txtFile)); + // Should work in AUTO mode. + assertReadingCompressedFileMatchesExpected(txtFilename, CompressionType.AUTO, lines); + } + /** * Create a zip file with the given lines. * @@ -553,7 +732,7 @@ public void testZipCompressedReadWithEmptyFile() throws Exception { @Test @Category(NeedsRunner.class) public void testZipCompressedReadWithEmptyEntry() throws Exception { - String filename = createZipFile(new ArrayList(), null, new String[]{ }); + String filename = createZipFile(new ArrayList(), null, new String[]{}); Pipeline p = TestPipeline.create(); @@ -571,9 +750,9 @@ public void testZipCompressedReadWithEmptyEntry() throws Exception { @Test @Category(NeedsRunner.class) public void testZipCompressedReadWithMultiEntriesFile() throws Exception { - String[] entry0 = new String[]{ "first", "second", "three" }; - String[] entry1 = new String[]{ "four", "five", "six" }; - String[] entry2 = new String[]{ "seven", "eight", "nine" }; + String[] entry0 = new String[]{"first", "second", "three"}; + String[] entry1 = new String[]{"four", "five", "six"}; + String[] entry2 = new String[]{"seven", "eight", "nine"}; List expected = new ArrayList<>(); @@ -599,10 +778,10 @@ public void testZipCompressedReadWithComplexEmptyAndPresentEntries() throws Exce String filename = createZipFile( new ArrayList(), null, - new String[] {"cat"}, - new String[] {}, - new String[] {}, - new String[] {"dog"}); + new String[]{"cat"}, + new String[]{}, + new String[]{}, + new String[]{"dog"}); List expected = ImmutableList.of("cat", "dog"); Pipeline p = TestPipeline.create(); @@ -614,30 +793,6 @@ public void testZipCompressedReadWithComplexEmptyAndPresentEntries() throws Exce p.run(); } - @Test - @Category(NeedsRunner.class) - public void testGZIPReadWhenUncompressed() throws Exception { - String[] lines = {"Meritorious condor", "Obnoxious duck"}; - File tmpFile = tmpFolder.newFile(); - String filename = tmpFile.getPath(); - - List expected = new ArrayList<>(); - try (PrintStream writer = new PrintStream(new FileOutputStream(tmpFile))) { - for (String line : lines) { - writer.println(line); - expected.add(line); - } - } - - Pipeline p = TestPipeline.create(); - TextIO.Read.Bound read = - TextIO.Read.from(filename).withCompressionType(CompressionType.GZIP); - PCollection output = p.apply(read); - - PAssert.that(output).containsInAnyOrder(expected); - p.run(); - } - @Test public void testTextIOGetName() { assertEquals("TextIO.Read", TextIO.Read.from("somefile").getName()); From 169f7a21336001f423ea2a741b15361bb01de3dc Mon Sep 17 00:00:00 2001 From: David Rieber Date: Tue, 9 Aug 2016 14:05:25 -0700 Subject: [PATCH 003/112] Do not add DataDisks to windmill service jobs. --- .../beam/runners/dataflow/DataflowPipelineTranslator.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java index ac06b5275d4d..0ca92c849f43 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java @@ -428,7 +428,9 @@ public Job translate(List packages) { workerPool.setPackages(packages); workerPool.setNumWorkers(options.getNumWorkers()); - if (options.isStreaming()) { + if (options.isStreaming() + && (options.getExperiments() == null + || !options.getExperiments().contains("enable_windmill_service"))) { // Use separate data disk for streaming. Disk disk = new Disk(); disk.setDiskType(options.getWorkerDiskType()); From 8d4e91009555cacf2e2badc94475fb7655c26438 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Thu, 11 Aug 2016 09:16:55 -0700 Subject: [PATCH 004/112] Remove timeout in DirectRunnerTest If the test hangs due to bugs, the infrastructure should kill it. --- .../java/org/apache/beam/runners/direct/DirectRunnerTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java index ddce4583a5d4..9739adb0f366 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java @@ -155,7 +155,7 @@ public void updateChanged(ProcessContext c) { assertThat("Each element should have been processed twice", changed.get(), equalTo(6)); } - @Test(timeout = 5000L) + @Test public void byteArrayCountShouldSucceed() { Pipeline p = getPipeline(); From d20a7ada7eb3ee714917e7c334e1b15ecc2c3b03 Mon Sep 17 00:00:00 2001 From: bchambers Date: Fri, 29 Jul 2016 09:41:17 -0700 Subject: [PATCH 005/112] Remove Counter and associated code Aggregator is the model level concept. Counter was specific to the Dataflow Runner, and is now not needed as part of Beam. --- .../org/apache/beam/sdk/util/DoFnRunners.java | 78 - .../beam/runners/dataflow/DataflowRunner.java | 4 +- .../apache/beam/sdk/transforms/Combine.java | 13 - .../org/apache/beam/sdk/transforms/Max.java | 27 +- .../org/apache/beam/sdk/transforms/Min.java | 28 +- .../org/apache/beam/sdk/transforms/Sum.java | 27 +- .../beam/sdk/util/CounterAggregator.java | 128 -- .../apache/beam/sdk/util/common/Counter.java | 1287 ----------------- .../beam/sdk/util/common/CounterName.java | 153 -- .../beam/sdk/util/common/CounterProvider.java | 27 - .../beam/sdk/util/common/CounterSet.java | 179 --- .../util/common/ElementByteSizeObserver.java | 24 +- .../beam/sdk/util/CounterAggregatorTest.java | 256 ---- .../beam/sdk/util/common/CounterSetTest.java | 227 --- .../beam/sdk/util/common/CounterTest.java | 736 ---------- 15 files changed, 15 insertions(+), 3179 deletions(-) delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/util/CounterAggregator.java delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/Counter.java delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/CounterName.java delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/CounterProvider.java delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/CounterSet.java delete mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/util/CounterAggregatorTest.java delete mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/CounterSetTest.java delete mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/CounterTest.java diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunners.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunners.java index a9f3cf4304a1..60892281c7b7 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunners.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunners.java @@ -23,8 +23,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.DoFnRunner.ReduceFnExecutor; import org.apache.beam.sdk.util.ExecutionContext.StepContext; -import org.apache.beam.sdk.util.common.CounterSet; -import org.apache.beam.sdk.util.common.CounterSet.AddCounterMutator; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; @@ -71,33 +69,6 @@ public static DoFnRunner simpleRunner( windowingStrategy); } - /** - * Returns a basic implementation of {@link DoFnRunner} that works for most - * {@link OldDoFn OldDoFns}. - * - *

It invokes {@link OldDoFn#processElement} for each input. - */ - public static DoFnRunner simpleRunner( - PipelineOptions options, - OldDoFn fn, - SideInputReader sideInputReader, - OutputManager outputManager, - TupleTag mainOutputTag, - List> sideOutputTags, - StepContext stepContext, - CounterSet.AddCounterMutator addCounterMutator, - WindowingStrategy windowingStrategy) { - return simpleRunner(options, - fn, - sideInputReader, - outputManager, - mainOutputTag, - sideOutputTags, - stepContext, - CounterAggregator.factoryFor(addCounterMutator), - windowingStrategy); - } - /** * Returns an implementation of {@link DoFnRunner} that handles late data dropping. * @@ -132,33 +103,6 @@ DoFnRunner, KV> lateDataDroppingRunner( reduceFnExecutor.getDroppedDueToLatenessAggregator()); } - /** - * Returns an implementation of {@link DoFnRunner} that handles late data dropping. - * - *

It drops elements from expired windows before they reach the underlying {@link OldDoFn}. - */ - public static - DoFnRunner, KV> lateDataDroppingRunner( - PipelineOptions options, - ReduceFnExecutor reduceFnExecutor, - SideInputReader sideInputReader, - OutputManager outputManager, - TupleTag> mainOutputTag, - List> sideOutputTags, - StepContext stepContext, - CounterSet.AddCounterMutator addCounterMutator, - WindowingStrategy windowingStrategy) { - return lateDataDroppingRunner( - options, - reduceFnExecutor, - sideInputReader, - outputManager, - mainOutputTag, - sideOutputTags, - stepContext, - CounterAggregator.factoryFor(addCounterMutator), - windowingStrategy); - } public static DoFnRunner createDefault( PipelineOptions options, @@ -197,26 +141,4 @@ public static DoFnRunner createDefault( aggregatorFactory, windowingStrategy); } - - public static DoFnRunner createDefault( - PipelineOptions options, - OldDoFn doFn, - SideInputReader sideInputReader, - OutputManager outputManager, - TupleTag mainOutputTag, - List> sideOutputTags, - StepContext stepContext, - AddCounterMutator addCounterMutator, - WindowingStrategy windowingStrategy) { - return createDefault( - options, - doFn, - sideInputReader, - outputManager, - mainOutputTag, - sideOutputTags, - stepContext, - CounterAggregator.factoryFor(addCounterMutator), - windowingStrategy); - } } diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index bea6264bd36f..667a63b50ffa 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -212,9 +212,9 @@ public class DataflowRunner extends PipelineRunner { // Default Docker container images that execute Dataflow worker harness, residing in Google // Container Registry, separately for Batch and Streaming. public static final String BATCH_WORKER_HARNESS_CONTAINER_IMAGE = - "dataflow.gcr.io/v1beta3/beam-java-batch:beam-master-20160804-dofn"; + "dataflow.gcr.io/v1beta3/beam-java-batch:beam-master-20160810"; public static final String STREAMING_WORKER_HARNESS_CONTAINER_IMAGE = - "dataflow.gcr.io/v1beta3/beam-java-streaming:beam-master-20160804-dofn"; + "dataflow.gcr.io/v1beta3/beam-java-streaming:beam-master-20160810"; // The limit of CreateJob request size. private static final int CREATE_JOB_REQUEST_LIMIT_BYTES = 10 * 1024 * 1024; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java index a8258000b614..6ba3f8a8964d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java @@ -49,7 +49,6 @@ import org.apache.beam.sdk.util.PropertyNames; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.WindowingStrategy; -import org.apache.beam.sdk.util.common.Counter; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; @@ -736,10 +735,6 @@ private static int[] wrap(int value) { return new int[] { value }; } - public Counter getCounter(@SuppressWarnings("unused") String name) { - throw new UnsupportedOperationException("BinaryCombineIntegerFn does not support getCounter"); - } - private static final class ToIntegerCodingFunction implements DelegateCoder.CodingFunction { @Override @@ -839,10 +834,6 @@ private static long[] wrap(long value) { return new long[] { value }; } - public Counter getCounter(@SuppressWarnings("unused") String name) { - throw new UnsupportedOperationException("BinaryCombineLongFn does not support getCounter"); - } - private static final class ToLongCodingFunction implements DelegateCoder.CodingFunction { @Override @@ -944,10 +935,6 @@ private static double[] wrap(double value) { return new double[] { value }; } - public Counter getCounter(@SuppressWarnings("unused") String name) { - throw new UnsupportedOperationException("BinaryCombineDoubleFn does not support getCounter"); - } - private static final class ToDoubleCodingFunction implements DelegateCoder.CodingFunction { @Override diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Max.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Max.java index 52617b65705e..eed13fbb70b4 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Max.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Max.java @@ -19,9 +19,6 @@ import org.apache.beam.sdk.transforms.Combine.BinaryCombineFn; import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.util.common.Counter; -import org.apache.beam.sdk.util.common.Counter.AggregationKind; -import org.apache.beam.sdk.util.common.CounterProvider; import java.io.Serializable; import java.util.Comparator; @@ -218,8 +215,7 @@ public void populateDisplayData(DisplayData.Builder builder) { * A {@code CombineFn} that computes the maximum of a collection of {@code Integer}s, useful as an * argument to {@link Combine#globally} or {@link Combine#perKey}. */ - public static class MaxIntegerFn extends Combine.BinaryCombineIntegerFn - implements CounterProvider { + public static class MaxIntegerFn extends Combine.BinaryCombineIntegerFn { @Override public int apply(int left, int right) { return left >= right ? left : right; @@ -229,19 +225,13 @@ public int apply(int left, int right) { public int identity() { return Integer.MIN_VALUE; } - - @Override - public Counter getCounter(String name) { - return Counter.ints(name, AggregationKind.MAX); - } } /** * A {@code CombineFn} that computes the maximum of a collection of {@code Long}s, useful as an * argument to {@link Combine#globally} or {@link Combine#perKey}. */ - public static class MaxLongFn extends Combine.BinaryCombineLongFn - implements CounterProvider { + public static class MaxLongFn extends Combine.BinaryCombineLongFn { @Override public long apply(long left, long right) { return left >= right ? left : right; @@ -251,19 +241,13 @@ public long apply(long left, long right) { public long identity() { return Long.MIN_VALUE; } - - @Override - public Counter getCounter(String name) { - return Counter.longs(name, AggregationKind.MAX); - } } /** * A {@code CombineFn} that computes the maximum of a collection of {@code Double}s, useful as an * argument to {@link Combine#globally} or {@link Combine#perKey}. */ - public static class MaxDoubleFn extends Combine.BinaryCombineDoubleFn - implements CounterProvider { + public static class MaxDoubleFn extends Combine.BinaryCombineDoubleFn { @Override public double apply(double left, double right) { return left >= right ? left : right; @@ -273,10 +257,5 @@ public double apply(double left, double right) { public double identity() { return Double.NEGATIVE_INFINITY; } - - @Override - public Counter getCounter(String name) { - return Counter.doubles(name, AggregationKind.MAX); - } } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Min.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Min.java index 31591348a56d..9c9d14f3ad01 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Min.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Min.java @@ -19,9 +19,6 @@ import org.apache.beam.sdk.transforms.Combine.BinaryCombineFn; import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.util.common.Counter; -import org.apache.beam.sdk.util.common.Counter.AggregationKind; -import org.apache.beam.sdk.util.common.CounterProvider; import java.io.Serializable; import java.util.Comparator; @@ -218,8 +215,7 @@ public void populateDisplayData(DisplayData.Builder builder) { * A {@code CombineFn} that computes the minimum of a collection of {@code Integer}s, useful as an * argument to {@link Combine#globally} or {@link Combine#perKey}. */ - public static class MinIntegerFn extends Combine.BinaryCombineIntegerFn - implements CounterProvider { + public static class MinIntegerFn extends Combine.BinaryCombineIntegerFn { @Override public int apply(int left, int right) { @@ -230,20 +226,13 @@ public int apply(int left, int right) { public int identity() { return Integer.MAX_VALUE; } - - @Override - public Counter getCounter(String name) { - return Counter.ints(name, AggregationKind.MIN); - } } /** * A {@code CombineFn} that computes the minimum of a collection of {@code Long}s, useful as an * argument to {@link Combine#globally} or {@link Combine#perKey}. */ - public static class MinLongFn extends Combine.BinaryCombineLongFn - implements CounterProvider { - + public static class MinLongFn extends Combine.BinaryCombineLongFn { @Override public long apply(long left, long right) { return left <= right ? left : right; @@ -253,19 +242,13 @@ public long apply(long left, long right) { public long identity() { return Long.MAX_VALUE; } - - @Override - public Counter getCounter(String name) { - return Counter.longs(name, AggregationKind.MIN); - } } /** * A {@code CombineFn} that computes the minimum of a collection of {@code Double}s, useful as an * argument to {@link Combine#globally} or {@link Combine#perKey}. */ - public static class MinDoubleFn extends Combine.BinaryCombineDoubleFn - implements CounterProvider { + public static class MinDoubleFn extends Combine.BinaryCombineDoubleFn { @Override public double apply(double left, double right) { @@ -276,10 +259,5 @@ public double apply(double left, double right) { public double identity() { return Double.POSITIVE_INFINITY; } - - @Override - public Counter getCounter(String name) { - return Counter.doubles(name, AggregationKind.MIN); - } } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sum.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sum.java index 07f78b50655a..27c5ced7a9d3 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sum.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sum.java @@ -17,10 +17,6 @@ */ package org.apache.beam.sdk.transforms; -import org.apache.beam.sdk.util.common.Counter; -import org.apache.beam.sdk.util.common.Counter.AggregationKind; -import org.apache.beam.sdk.util.common.CounterProvider; - /** * {@code PTransform}s for computing the sum of the elements in a * {@code PCollection}, or the sum of the values associated with @@ -123,8 +119,7 @@ public static Combine.PerKey doublesPerKey() { * {@code Iterable} of {@code Integer}s, useful as an argument to * {@link Combine#globally} or {@link Combine#perKey}. */ - public static class SumIntegerFn - extends Combine.BinaryCombineIntegerFn implements CounterProvider { + public static class SumIntegerFn extends Combine.BinaryCombineIntegerFn { @Override public int apply(int a, int b) { return a + b; @@ -134,11 +129,6 @@ public int apply(int a, int b) { public int identity() { return 0; } - - @Override - public Counter getCounter(String name) { - return Counter.ints(name, AggregationKind.SUM); - } } /** @@ -147,7 +137,7 @@ public Counter getCounter(String name) { * {@link Combine#globally} or {@link Combine#perKey}. */ public static class SumLongFn - extends Combine.BinaryCombineLongFn implements CounterProvider { + extends Combine.BinaryCombineLongFn { @Override public long apply(long a, long b) { return a + b; @@ -157,11 +147,6 @@ public long apply(long a, long b) { public long identity() { return 0; } - - @Override - public Counter getCounter(String name) { - return Counter.longs(name, AggregationKind.SUM); - } } /** @@ -169,8 +154,7 @@ public Counter getCounter(String name) { * {@code Iterable} of {@code Double}s, useful as an argument to * {@link Combine#globally} or {@link Combine#perKey}. */ - public static class SumDoubleFn - extends Combine.BinaryCombineDoubleFn implements CounterProvider { + public static class SumDoubleFn extends Combine.BinaryCombineDoubleFn { @Override public double apply(double a, double b) { return a + b; @@ -180,10 +164,5 @@ public double apply(double a, double b) { public double identity() { return 0; } - - @Override - public Counter getCounter(String name) { - return Counter.doubles(name, AggregationKind.SUM); - } } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CounterAggregator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CounterAggregator.java deleted file mode 100644 index 5bde8ef457bb..000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CounterAggregator.java +++ /dev/null @@ -1,128 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.util; - -import org.apache.beam.sdk.transforms.Aggregator; -import org.apache.beam.sdk.transforms.Combine.CombineFn; -import org.apache.beam.sdk.transforms.Max; -import org.apache.beam.sdk.transforms.Min; -import org.apache.beam.sdk.transforms.Sum; -import org.apache.beam.sdk.util.common.Counter; -import org.apache.beam.sdk.util.common.CounterProvider; -import org.apache.beam.sdk.util.common.CounterSet; -import org.apache.beam.sdk.util.common.CounterSet.AddCounterMutator; -import com.google.common.annotations.VisibleForTesting; - -/** - * An implementation of the {@code Aggregator} interface that uses a - * {@link Counter} as the underlying representation. Supports {@link CombineFn}s - * from the {@link Sum}, {@link Min} and {@link Max} classes. - * - * @param the type of input values - * @param the type of accumulator values - * @param the type of output value - */ -public class CounterAggregator - implements Aggregator { - - private static class CounterAggregatorFactory implements AggregatorFactory { - private final AddCounterMutator addCounterMutator; - - private CounterAggregatorFactory(CounterSet.AddCounterMutator addCounterMutator) { - this.addCounterMutator = addCounterMutator; - } - - @Override - public Aggregator createAggregatorForDoFn( - Class fnClass, ExecutionContext.StepContext stepContext, - String userName, CombineFn combine) { - boolean isSystem = fnClass.isAnnotationPresent(SystemDoFnInternal.class); - String mangledName = (isSystem ? "" : "user-") + stepContext.getStepName() + "-" + userName; - - return new CounterAggregator<>(mangledName, combine, addCounterMutator); - } - } - - private final Counter counter; - private final CombineFn combiner; - - /** - * Create a factory for producing {@link CounterAggregator CounterAggregators} backed by the given - * {@link CounterSet.AddCounterMutator}. - */ - public static AggregatorFactory factoryFor( - CounterSet.AddCounterMutator addCounterMutator) { - return new CounterAggregatorFactory(addCounterMutator); - } - - /** - * Constructs a new aggregator with the given name and aggregation logic - * specified in the CombineFn argument. The underlying counter is - * automatically added into the provided CounterSet. - * - *

If a counter with the same name already exists, it will be reused, as - * long as it has the same type. - */ - @VisibleForTesting CounterAggregator( - String name, CombineFn combiner, - CounterSet.AddCounterMutator addCounterMutator) { - // Safe contravariant cast - this(constructCounter(name, combiner), addCounterMutator, - (CombineFn) combiner); - } - - private CounterAggregator(Counter counter, - CounterSet.AddCounterMutator addCounterMutator, - CombineFn combiner) { - try { - this.counter = addCounterMutator.addCounter(counter); - } catch (IllegalArgumentException ex) { - throw new IllegalArgumentException( - "aggregator's name collides with an existing aggregator " - + "or system-provided counter of an incompatible type"); - } - this.combiner = combiner; - } - - private static Counter constructCounter(String name, - CombineFn combiner) { - if (combiner instanceof CounterProvider) { - @SuppressWarnings("unchecked") - CounterProvider counterProvider = (CounterProvider) combiner; - return counterProvider.getCounter(name); - } else { - throw new IllegalArgumentException("unsupported combiner in Aggregator: " - + combiner.getClass().getName()); - } - } - - @Override - public void addValue(InputT value) { - counter.addValue(value); - } - - @Override - public String getName() { - return counter.getFlatName(); - } - - @Override - public CombineFn getCombineFn() { - return combiner; - } -} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/Counter.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/Counter.java deleted file mode 100644 index 550c648beb38..000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/Counter.java +++ /dev/null @@ -1,1287 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.util.common; - -import static org.apache.beam.sdk.util.common.Counter.AggregationKind.AND; -import static org.apache.beam.sdk.util.common.Counter.AggregationKind.MEAN; -import static org.apache.beam.sdk.util.common.Counter.AggregationKind.OR; - -import static com.google.common.base.Preconditions.checkArgument; - -import org.apache.beam.sdk.values.TypeDescriptor; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.util.concurrent.AtomicDouble; - -import java.util.Objects; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.atomic.AtomicReference; - -import javax.annotation.Nullable; - -/** - * A Counter enables the aggregation of a stream of values over time. The - * cumulative aggregate value is updated as new values are added, or it can be - * reset to a new value. Multiple kinds of aggregation are supported depending - * on the type of the counter. - * - *

Counters compare using value equality of their name, kind, and - * cumulative value. Equal counters should have equal toString()s. - * - *

After all possible mutations have completed, the reader should check - * {@link #isDirty} for every counter, otherwise updates may be lost. - * - *

A counter may become dirty without a corresponding update to the value. - * This generally will occur when the calls to {@code addValue()}, {@code committing()}, - * and {@code committed()} are interleaved such that the value is updated - * between the calls to committing and the read of the value. - * - * @param the type of values aggregated by this counter - */ -public abstract class Counter { - /** - * Possible kinds of counter aggregation. - */ - public static enum AggregationKind { - - /** - * Computes the sum of all added values. - * Applicable to {@link Integer}, {@link Long}, and {@link Double} values. - */ - SUM, - - /** - * Computes the maximum value of all added values. - * Applicable to {@link Integer}, {@link Long}, and {@link Double} values. - */ - MAX, - - /** - * Computes the minimum value of all added values. - * Applicable to {@link Integer}, {@link Long}, and {@link Double} values. - */ - MIN, - - /** - * Computes the arithmetic mean of all added values. Applicable to - * {@link Integer}, {@link Long}, and {@link Double} values. - */ - MEAN, - - /** - * Computes boolean AND over all added values. - * Applicable only to {@link Boolean} values. - */ - AND, - - /** - * Computes boolean OR over all added values. Applicable only to - * {@link Boolean} values. - */ - OR - // TODO: consider adding VECTOR_SUM, HISTOGRAM, KV_SET, PRODUCT, TOP. - } - - /** - * Constructs a new {@link Counter} that aggregates {@link Integer}, values - * according to the desired aggregation kind. The supported aggregation kinds - * are {@link AggregationKind#SUM}, {@link AggregationKind#MIN}, - * {@link AggregationKind#MAX}, and {@link AggregationKind#MEAN}. - * This is a convenience wrapper over a - * {@link Counter} implementation that aggregates {@link Long} values. This is - * useful when the application handles (boxed) {@link Integer} values that - * are not readily convertible to the (boxed) {@link Long} values otherwise - * expected by the {@link Counter} implementation aggregating {@link Long} - * values. - * - * @param name the name of the new counter - * @param kind the new counter's aggregation kind - * @return the newly constructed Counter - * @throws IllegalArgumentException if the aggregation kind is not supported - */ - public static Counter ints(CounterName name, AggregationKind kind) { - return new IntegerCounter(name, kind); - } - - /** - * Constructs a new {@code Counter} with an unstructured name. - * - * @deprecated use {@link #ints(CounterName, AggregationKind)}. - */ - @Deprecated - public static Counter ints(String name, AggregationKind kind) { - return new IntegerCounter(CounterName.named(name), kind); - } - - /** - * Constructs a new {@link Counter} that aggregates {@link Long} values - * according to the desired aggregation kind. The supported aggregation kinds - * are {@link AggregationKind#SUM}, {@link AggregationKind#MIN}, - * {@link AggregationKind#MAX}, and {@link AggregationKind#MEAN}. - * - * @param name the name of the new counter - * @param kind the new counter's aggregation kind - * @return the newly constructed Counter - * @throws IllegalArgumentException if the aggregation kind is not supported - */ - public static Counter longs(CounterName name, AggregationKind kind) { - return new LongCounter(name, kind); - } - - /** - * Constructs a new {@code Counter} with an unstructured name. - * - * @deprecated use {@link #longs(CounterName, AggregationKind)}. - */ - @Deprecated - public static Counter longs(String name, AggregationKind kind) { - return new LongCounter(CounterName.named(name), kind); - } - - /** - * Constructs a new {@link Counter} that aggregates {@link Double} values - * according to the desired aggregation kind. The supported aggregation kinds - * are {@link AggregationKind#SUM}, {@link AggregationKind#MIN}, - * {@link AggregationKind#MAX}, and {@link AggregationKind#MEAN}. - * - * @param name the name of the new counter - * @param kind the new counter's aggregation kind - * @return the newly constructed Counter - * @throws IllegalArgumentException if the aggregation kind is not supported - */ - public static Counter doubles(CounterName name, AggregationKind kind) { - return new DoubleCounter(name, kind); - } - - /** - * Constructs a new {@code Counter} with an unstructured name. - * - * @deprecated use {@link #doubles(CounterName, AggregationKind)}. - */ - @Deprecated - public static Counter doubles(String name, AggregationKind kind) { - return new DoubleCounter(CounterName.named(name), kind); - } - - /** - * Constructs a new {@link Counter} that aggregates {@link Boolean} values - * according to the desired aggregation kind. The only supported aggregation - * kinds are {@link AggregationKind#AND} and {@link AggregationKind#OR}. - * - * @param name the name of the new counter - * @param kind the new counter's aggregation kind - * @return the newly constructed Counter - * @throws IllegalArgumentException if the aggregation kind is not supported - */ - public static Counter booleans(CounterName name, AggregationKind kind) { - return new BooleanCounter(name, kind); - } - - /** - * Constructs a new {@code Counter} with an unstructured name. - * - * @deprecated use {@link #booleans(CounterName, AggregationKind)}. - */ - @Deprecated - public static Counter booleans(String name, AggregationKind kind) { - return new BooleanCounter(CounterName.named(name), kind); - } - - ////////////////////////////////////////////////////////////////////////////// - - /** - * Adds a new value to the aggregation stream. Returns this (to allow method - * chaining). - */ - public abstract Counter addValue(T value); - - /** - * Resets the aggregation stream to this new value. This aggregator must not - * be a MEAN aggregator. Returns this (to allow method chaining). - */ - public abstract Counter resetToValue(T value); - - /** - * Resets the aggregation stream to this new value. Returns this (to allow - * method chaining). The value of elementCount must be non-negative, and this - * aggregator must be a MEAN aggregator. - */ - public abstract Counter resetMeanToValue(long elementCount, T value); - - /** - * Resets the counter's delta value to have no values accumulated and returns - * the value of the delta prior to the reset. - * - * @return the aggregate delta at the time this method is called - */ - public abstract T getAndResetDelta(); - - /** - * Resets the counter's delta value to have no values accumulated and returns - * the value of the delta prior to the reset, for a MEAN counter. - * - * @return the mean delta t the time this method is called - */ - public abstract CounterMean getAndResetMeanDelta(); - - /** - * Returns the counter's flat name. - */ - public String getFlatName() { - return name.getFlatName(); - } - - /** - * Returns the counter's name. - * - * @deprecated use {@link #getFlatName}. - */ - @Deprecated - public String getName() { - return name.getFlatName(); - } - - /** - * Returns the counter's aggregation kind. - */ - public AggregationKind getKind() { - return kind; - } - - /** - * Returns the counter's type. - */ - public Class getType() { - return new TypeDescriptor(getClass()) {}.getRawType(); - } - - /** - * Returns the aggregated value, or the sum for MEAN aggregation, either - * total or, if delta, since the last update extraction or resetDelta. - */ - public abstract T getAggregate(); - - /** - * The mean value of a {@code Counter}, represented as an aggregate value and - * a count. - * - * @param the type of the aggregate - */ - public static interface CounterMean { - /** - * Gets the aggregate value of this {@code CounterMean}. - */ - T getAggregate(); - - /** - * Gets the count of this {@code CounterMean}. - */ - long getCount(); - } - - /** - * Returns the mean in the form of a CounterMean, or null if this is not a - * MEAN counter. - */ - @Nullable - public abstract CounterMean getMean(); - - /** - * Represents whether counters' values have been committed to the backend. - * - *

Runners can use this information to optimize counters updates. - * For example, if counters are committed, runners may choose to skip the updates. - * - *

Counters' state transition table: - * {@code - * Action\Current State COMMITTED DIRTY COMMITTING - * addValue() DIRTY DIRTY DIRTY - * committing() None COMMITTING None - * committed() None None COMMITTED - * } - */ - @VisibleForTesting - enum CommitState { - /** - * There are no local updates that haven't been committed to the backend. - */ - COMMITTED, - /** - * There are local updates that haven't been committed to the backend. - */ - DIRTY, - /** - * Local updates are committing to the backend, but are still pending. - */ - COMMITTING, - } - - /** - * Returns if the counter contains non-committed aggregate. - */ - public boolean isDirty() { - return commitState.get() != CommitState.COMMITTED; - } - - /** - * Changes the counter from {@code CommitState.DIRTY} to {@code CommitState.COMMITTING}. - * - * @return true if successful. False return indicates that the commit state - * was not in {@code CommitState.DIRTY}. - */ - public boolean committing() { - return commitState.compareAndSet(CommitState.DIRTY, CommitState.COMMITTING); - } - - /** - * Changes the counter from {@code CommitState.COMMITTING} to {@code CommitState.COMMITTED}. - * - * @return true if successful. - * - *

False return indicates that the counter was updated while the committing is pending. - * That counter update might or might not has been committed. The {@code commitState} has to - * stay in {@code CommitState.DIRTY}. - */ - public boolean committed() { - return commitState.compareAndSet(CommitState.COMMITTING, CommitState.COMMITTED); - } - - /** - * Sets the counter to {@code CommitState.DIRTY}. - * - *

Must be called at the end of {@link #addValue}, {@link #resetToValue}, - * {@link #resetMeanToValue}, and {@link #merge}. - */ - protected void setDirty() { - commitState.set(CommitState.DIRTY); - } - - /** - * Returns a string representation of the Counter. Useful for debugging logs. - * Example return value: "ElementCount:SUM(15)". - */ - @Override - public String toString() { - StringBuilder sb = new StringBuilder(); - sb.append(getFlatName()); - sb.append(":"); - sb.append(getKind()); - sb.append("("); - switch (kind) { - case SUM: - case MAX: - case MIN: - case AND: - case OR: - sb.append(getAggregate()); - break; - case MEAN: - sb.append(getMean()); - break; - default: - throw illegalArgumentException(); - } - sb.append(")"); - - return sb.toString(); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } else if (o instanceof Counter) { - Counter that = (Counter) o; - if (this.name.equals(that.name) && this.kind == that.kind - && this.getClass().equals(that.getClass())) { - if (kind == MEAN) { - CounterMean thisMean = this.getMean(); - CounterMean thatMean = that.getMean(); - return thisMean == thatMean - || (Objects.equals(thisMean.getAggregate(), thatMean.getAggregate()) - && thisMean.getCount() == thatMean.getCount()); - } else { - return Objects.equals(this.getAggregate(), that.getAggregate()); - } - } - } - return false; - } - - @Override - public int hashCode() { - if (kind == MEAN) { - CounterMean mean = getMean(); - return Objects.hash(getClass(), name, kind, mean.getAggregate(), mean.getCount()); - } else { - return Objects.hash(getClass(), name, kind, getAggregate()); - } - } - - /** - * Returns whether this Counter is compatible with that Counter. If - * so, they can be merged into a single Counter. - */ - public boolean isCompatibleWith(Counter that) { - return this.name.equals(that.name) - && this.kind == that.kind - && this.getClass().equals(that.getClass()); - } - - /** - * Merges this counter with the provided counter, returning this counter with the combined value - * of both counters. This may reset the delta of this counter. - * - * @throws IllegalArgumentException if the provided Counter is not compatible with this Counter - */ - public abstract Counter merge(Counter that); - - ////////////////////////////////////////////////////////////////////////////// - - /** The name and metadata of this counter. */ - protected final CounterName name; - - /** The kind of aggregation function to apply to this counter. */ - protected final AggregationKind kind; - - /** The commit state of this counter. */ - protected final AtomicReference commitState; - - protected Counter(CounterName name, AggregationKind kind) { - this.name = name; - this.kind = kind; - this.commitState = new AtomicReference<>(CommitState.COMMITTED); - } - - ////////////////////////////////////////////////////////////////////////////// - - /** - * Implements a {@link Counter} for {@link Long} values. - */ - private static class LongCounter extends Counter { - private final AtomicLong aggregate; - private final AtomicLong deltaAggregate; - private final AtomicReference mean; - private final AtomicReference deltaMean; - - /** Initializes a new {@link Counter} for {@link Long} values. */ - private LongCounter(CounterName name, AggregationKind kind) { - super(name, kind); - switch (kind) { - case MEAN: - mean = new AtomicReference<>(); - deltaMean = new AtomicReference<>(); - getAndResetMeanDelta(); - mean.set(deltaMean.get()); - aggregate = deltaAggregate = null; - break; - case SUM: - case MAX: - case MIN: - aggregate = new AtomicLong(); - deltaAggregate = new AtomicLong(); - getAndResetDelta(); - aggregate.set(deltaAggregate.get()); - mean = deltaMean = null; - break; - default: - throw illegalArgumentException(); - } - } - - @Override - public LongCounter addValue(Long value) { - try { - switch (kind) { - case SUM: - aggregate.addAndGet(value); - deltaAggregate.addAndGet(value); - break; - case MEAN: - addToMeanAndSet(value, mean); - addToMeanAndSet(value, deltaMean); - break; - case MAX: - maxAndSet(value, aggregate); - maxAndSet(value, deltaAggregate); - break; - case MIN: - minAndSet(value, aggregate); - minAndSet(value, deltaAggregate); - break; - default: - throw illegalArgumentException(); - } - return this; - } finally { - setDirty(); - } - } - - private void minAndSet(Long value, AtomicLong target) { - long current; - long update; - do { - current = target.get(); - update = Math.min(value, current); - } while (update < current && !target.compareAndSet(current, update)); - } - - private void maxAndSet(Long value, AtomicLong target) { - long current; - long update; - do { - current = target.get(); - update = Math.max(value, current); - } while (update > current && !target.compareAndSet(current, update)); - } - - private void addToMeanAndSet(Long value, AtomicReference target) { - LongCounterMean current; - LongCounterMean update; - do { - current = target.get(); - update = new LongCounterMean(current.getAggregate() + value, current.getCount() + 1L); - } while (!target.compareAndSet(current, update)); - } - - @Override - public Long getAggregate() { - if (kind != MEAN) { - return aggregate.get(); - } else { - return getMean().getAggregate(); - } - } - - @Override - public Long getAndResetDelta() { - switch (kind) { - case SUM: - return deltaAggregate.getAndSet(0L); - case MAX: - return deltaAggregate.getAndSet(Long.MIN_VALUE); - case MIN: - return deltaAggregate.getAndSet(Long.MAX_VALUE); - default: - throw illegalArgumentException(); - } - } - - @Override - public Counter resetToValue(Long value) { - try { - if (kind == MEAN) { - throw illegalArgumentException(); - } - aggregate.set(value); - deltaAggregate.set(value); - return this; - } finally { - setDirty(); - } - } - - @Override - public Counter resetMeanToValue(long elementCount, Long value) { - try { - if (kind != MEAN) { - throw illegalArgumentException(); - } - if (elementCount < 0) { - throw new IllegalArgumentException("elementCount must be non-negative"); - } - LongCounterMean counterMean = new LongCounterMean(value, elementCount); - mean.set(counterMean); - deltaMean.set(counterMean); - return this; - } finally { - setDirty(); - } - } - - @Override - public CounterMean getAndResetMeanDelta() { - if (kind != MEAN) { - throw illegalArgumentException(); - } - return deltaMean.getAndSet(new LongCounterMean(0L, 0L)); - } - - @Override - @Nullable - public CounterMean getMean() { - if (kind != MEAN) { - throw illegalArgumentException(); - } - return mean.get(); - } - - @Override - public Counter merge(Counter that) { - try { - checkArgument( - this.isCompatibleWith(that), "Counters %s and %s are incompatible", this, that); - switch (kind) { - case SUM: - case MIN: - case MAX: - return addValue(that.getAggregate()); - case MEAN: - CounterMean thisCounterMean = this.getMean(); - CounterMean thatCounterMean = that.getMean(); - return resetMeanToValue( - thisCounterMean.getCount() + thatCounterMean.getCount(), - thisCounterMean.getAggregate() + thatCounterMean.getAggregate()); - default: - throw illegalArgumentException(); - } - } finally { - setDirty(); - } - } - - private static class LongCounterMean implements CounterMean { - private final long aggregate; - private final long count; - - public LongCounterMean(long aggregate, long count) { - this.aggregate = aggregate; - this.count = count; - } - - @Override - public Long getAggregate() { - return aggregate; - } - - @Override - public long getCount() { - return count; - } - - @Override - public String toString() { - return aggregate + "/" + count; - } - } - } - - /** - * Implements a {@link Counter} for {@link Double} values. - */ - private static class DoubleCounter extends Counter { - AtomicDouble aggregate; - AtomicDouble deltaAggregate; - AtomicReference mean; - AtomicReference deltaMean; - - /** Initializes a new {@link Counter} for {@link Double} values. */ - private DoubleCounter(CounterName name, AggregationKind kind) { - super(name, kind); - switch (kind) { - case MEAN: - aggregate = deltaAggregate = null; - mean = new AtomicReference<>(); - deltaMean = new AtomicReference<>(); - getAndResetMeanDelta(); - mean.set(deltaMean.get()); - break; - case SUM: - case MAX: - case MIN: - mean = deltaMean = null; - aggregate = new AtomicDouble(); - deltaAggregate = new AtomicDouble(); - getAndResetDelta(); - aggregate.set(deltaAggregate.get()); - break; - default: - throw illegalArgumentException(); - } - } - - @Override - public DoubleCounter addValue(Double value) { - try { - switch (kind) { - case SUM: - aggregate.addAndGet(value); - deltaAggregate.addAndGet(value); - break; - case MEAN: - addToMeanAndSet(value, mean); - addToMeanAndSet(value, deltaMean); - break; - case MAX: - maxAndSet(value, aggregate); - maxAndSet(value, deltaAggregate); - break; - case MIN: - minAndSet(value, aggregate); - minAndSet(value, deltaAggregate); - break; - default: - throw illegalArgumentException(); - } - return this; - } finally { - setDirty(); - } - } - - private void addToMeanAndSet(Double value, AtomicReference target) { - DoubleCounterMean current; - DoubleCounterMean update; - do { - current = target.get(); - update = new DoubleCounterMean(current.getAggregate() + value, current.getCount() + 1); - } while (!target.compareAndSet(current, update)); - } - - private void maxAndSet(Double value, AtomicDouble target) { - double current; - double update; - do { - current = target.get(); - update = Math.max(current, value); - } while (update > current && !target.compareAndSet(current, update)); - } - - private void minAndSet(Double value, AtomicDouble target) { - double current; - double update; - do { - current = target.get(); - update = Math.min(current, value); - } while (update < current && !target.compareAndSet(current, update)); - } - - @Override - public Double getAndResetDelta() { - switch (kind) { - case SUM: - return deltaAggregate.getAndSet(0.0); - case MAX: - return deltaAggregate.getAndSet(Double.NEGATIVE_INFINITY); - case MIN: - return deltaAggregate.getAndSet(Double.POSITIVE_INFINITY); - default: - throw illegalArgumentException(); - } - } - - @Override - public Counter resetToValue(Double value) { - try { - if (kind == MEAN) { - throw illegalArgumentException(); - } - aggregate.set(value); - deltaAggregate.set(value); - return this; - } finally { - setDirty(); - } - } - - @Override - public Counter resetMeanToValue(long elementCount, Double value) { - try { - if (kind != MEAN) { - throw illegalArgumentException(); - } - if (elementCount < 0) { - throw new IllegalArgumentException("elementCount must be non-negative"); - } - DoubleCounterMean counterMean = new DoubleCounterMean(value, elementCount); - mean.set(counterMean); - deltaMean.set(counterMean); - return this; - } finally { - setDirty(); - } - } - - @Override - public CounterMean getAndResetMeanDelta() { - if (kind != MEAN) { - throw illegalArgumentException(); - } - return deltaMean.getAndSet(new DoubleCounterMean(0.0, 0L)); - } - - @Override - public Double getAggregate() { - if (kind != MEAN) { - return aggregate.get(); - } else { - return getMean().getAggregate(); - } - } - - @Override - @Nullable - public CounterMean getMean() { - if (kind != MEAN) { - throw illegalArgumentException(); - } - return mean.get(); - } - - @Override - public Counter merge(Counter that) { - try { - checkArgument( - this.isCompatibleWith(that), "Counters %s and %s are incompatible", this, that); - switch (kind) { - case SUM: - case MIN: - case MAX: - return addValue(that.getAggregate()); - case MEAN: - CounterMean thisCounterMean = this.getMean(); - CounterMean thatCounterMean = that.getMean(); - return resetMeanToValue( - thisCounterMean.getCount() + thatCounterMean.getCount(), - thisCounterMean.getAggregate() + thatCounterMean.getAggregate()); - default: - throw illegalArgumentException(); - } - } finally { - setDirty(); - } - } - - private static class DoubleCounterMean implements CounterMean { - private final double aggregate; - private final long count; - - public DoubleCounterMean(double aggregate, long count) { - this.aggregate = aggregate; - this.count = count; - } - - @Override - public Double getAggregate() { - return aggregate; - } - - @Override - public long getCount() { - return count; - } - - @Override - public String toString() { - return aggregate + "/" + count; - } - } - } - - /** - * Implements a {@link Counter} for {@link Boolean} values. - */ - private static class BooleanCounter extends Counter { - private final AtomicBoolean aggregate; - private final AtomicBoolean deltaAggregate; - - /** Initializes a new {@link Counter} for {@link Boolean} values. */ - private BooleanCounter(CounterName name, AggregationKind kind) { - super(name, kind); - aggregate = new AtomicBoolean(); - deltaAggregate = new AtomicBoolean(); - getAndResetDelta(); - aggregate.set(deltaAggregate.get()); - } - - @Override - public BooleanCounter addValue(Boolean value) { - try { - if (kind.equals(AND) && !value) { - aggregate.set(value); - deltaAggregate.set(value); - } else if (kind.equals(OR) && value) { - aggregate.set(value); - deltaAggregate.set(value); - } - return this; - } finally { - setDirty(); - } - } - - @Override - public Boolean getAndResetDelta() { - switch (kind) { - case AND: - return deltaAggregate.getAndSet(true); - case OR: - return deltaAggregate.getAndSet(false); - default: - throw illegalArgumentException(); - } - } - - @Override - public Counter resetToValue(Boolean value) { - try { - aggregate.set(value); - deltaAggregate.set(value); - return this; - } finally { - setDirty(); - } - } - - @Override - public Counter resetMeanToValue(long elementCount, Boolean value) { - throw illegalArgumentException(); - } - - @Override - public CounterMean getAndResetMeanDelta() { - throw illegalArgumentException(); - } - - @Override - public Boolean getAggregate() { - return aggregate.get(); - } - - @Override - @Nullable - public CounterMean getMean() { - throw illegalArgumentException(); - } - - @Override - public Counter merge(Counter that) { - try { - checkArgument( - this.isCompatibleWith(that), "Counters %s and %s are incompatible", this, that); - return addValue(that.getAggregate()); - } finally { - setDirty(); - } - } - } - - /** - * Implements a {@link Counter} for {@link String} values. - */ - private static class StringCounter extends Counter { - /** Initializes a new {@link Counter} for {@link String} values. */ - private StringCounter(CounterName name, AggregationKind kind) { - super(name, kind); - // TODO: Support MIN, MAX of Strings. - throw illegalArgumentException(); - } - - @Override - public StringCounter addValue(String value) { - switch (kind) { - default: - throw illegalArgumentException(); - } - } - - @Override - public Counter resetToValue(String value) { - switch (kind) { - default: - throw illegalArgumentException(); - } - } - - @Override - public Counter resetMeanToValue(long elementCount, String value) { - switch (kind) { - default: - throw illegalArgumentException(); - } - } - - @Override - public String getAndResetDelta() { - switch (kind) { - default: - throw illegalArgumentException(); - } - } - - @Override - public CounterMean getAndResetMeanDelta() { - switch (kind) { - default: - throw illegalArgumentException(); - } - } - - @Override - public String getAggregate() { - switch (kind) { - default: - throw illegalArgumentException(); - } - } - - @Override - @Nullable - public CounterMean getMean() { - switch (kind) { - default: - throw illegalArgumentException(); - } - } - - @Override - public Counter merge(Counter that) { - checkArgument(this.isCompatibleWith(that), "Counters %s and %s are incompatible", this, that); - switch (kind) { - default: - throw illegalArgumentException(); - } - } - } - - /** - * Implements a {@link Counter} for {@link Integer} values. - */ - private static class IntegerCounter extends Counter { - private final AtomicInteger aggregate; - private final AtomicInteger deltaAggregate; - private final AtomicReference mean; - private final AtomicReference deltaMean; - - /** Initializes a new {@link Counter} for {@link Integer} values. */ - private IntegerCounter(CounterName name, AggregationKind kind) { - super(name, kind); - switch (kind) { - case MEAN: - aggregate = deltaAggregate = null; - mean = new AtomicReference<>(); - deltaMean = new AtomicReference<>(); - getAndResetMeanDelta(); - mean.set(deltaMean.get()); - break; - case SUM: - case MAX: - case MIN: - mean = deltaMean = null; - aggregate = new AtomicInteger(); - deltaAggregate = new AtomicInteger(); - getAndResetDelta(); - aggregate.set(deltaAggregate.get()); - break; - default: - throw illegalArgumentException(); - } - } - - @Override - public IntegerCounter addValue(Integer value) { - try { - switch (kind) { - case SUM: - aggregate.getAndAdd(value); - deltaAggregate.getAndAdd(value); - break; - case MEAN: - addToMeanAndSet(value, mean); - addToMeanAndSet(value, deltaMean); - break; - case MAX: - maxAndSet(value, aggregate); - maxAndSet(value, deltaAggregate); - break; - case MIN: - minAndSet(value, aggregate); - minAndSet(value, deltaAggregate); - break; - default: - throw illegalArgumentException(); - } - return this; - } finally { - setDirty(); - } - } - - private void addToMeanAndSet(int value, AtomicReference target) { - IntegerCounterMean current; - IntegerCounterMean update; - do { - current = target.get(); - update = new IntegerCounterMean(current.getAggregate() + value, current.getCount() + 1); - } while (!target.compareAndSet(current, update)); - } - - private void maxAndSet(int value, AtomicInteger target) { - int current; - int update; - do { - current = target.get(); - update = Math.max(value, current); - } while (update > current && !target.compareAndSet(current, update)); - } - - private void minAndSet(int value, AtomicInteger target) { - int current; - int update; - do { - current = target.get(); - update = Math.min(value, current); - } while (update < current && !target.compareAndSet(current, update)); - } - - @Override - public Integer getAndResetDelta() { - switch (kind) { - case SUM: - return deltaAggregate.getAndSet(0); - case MAX: - return deltaAggregate.getAndSet(Integer.MIN_VALUE); - case MIN: - return deltaAggregate.getAndSet(Integer.MAX_VALUE); - default: - throw illegalArgumentException(); - } - } - - @Override - public Counter resetToValue(Integer value) { - try { - if (kind == MEAN) { - throw illegalArgumentException(); - } - aggregate.set(value); - deltaAggregate.set(value); - return this; - } finally { - setDirty(); - } - } - - @Override - public Counter resetMeanToValue(long elementCount, Integer value) { - try { - if (kind != MEAN) { - throw illegalArgumentException(); - } - if (elementCount < 0) { - throw new IllegalArgumentException("elementCount must be non-negative"); - } - IntegerCounterMean counterMean = new IntegerCounterMean(value, elementCount); - mean.set(counterMean); - deltaMean.set(counterMean); - return this; - } finally { - setDirty(); - } - } - - @Override - public CounterMean getAndResetMeanDelta() { - if (kind != MEAN) { - throw illegalArgumentException(); - } - return deltaMean.getAndSet(new IntegerCounterMean(0, 0L)); - } - - @Override - public Integer getAggregate() { - if (kind != MEAN) { - return aggregate.get(); - } else { - return getMean().getAggregate(); - } - } - - @Override - @Nullable - public CounterMean getMean() { - if (kind != MEAN) { - throw illegalArgumentException(); - } - return mean.get(); - } - - @Override - public Counter merge(Counter that) { - try { - checkArgument( - this.isCompatibleWith(that), "Counters %s and %s are incompatible", this, that); - switch (kind) { - case SUM: - case MIN: - case MAX: - return addValue(that.getAggregate()); - case MEAN: - CounterMean thisCounterMean = this.getMean(); - CounterMean thatCounterMean = that.getMean(); - return resetMeanToValue( - thisCounterMean.getCount() + thatCounterMean.getCount(), - thisCounterMean.getAggregate() + thatCounterMean.getAggregate()); - default: - throw illegalArgumentException(); - } - } finally { - setDirty(); - } - } - - private static class IntegerCounterMean implements CounterMean { - private final int aggregate; - private final long count; - - public IntegerCounterMean(int aggregate, long count) { - this.aggregate = aggregate; - this.count = count; - } - - @Override - public Integer getAggregate() { - return aggregate; - } - - @Override - public long getCount() { - return count; - } - - @Override - public String toString() { - return aggregate + "/" + count; - } - } - } - - ////////////////////////////////////////////////////////////////////////////// - - /** - * Constructs an {@link IllegalArgumentException} explaining that this - * {@link Counter}'s aggregation kind is not supported by its value type. - */ - protected IllegalArgumentException illegalArgumentException() { - return new IllegalArgumentException("Cannot compute " + kind - + " aggregation over " + getType().getSimpleName() + " values."); - } -} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/CounterName.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/CounterName.java deleted file mode 100644 index b46be98af0ed..000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/CounterName.java +++ /dev/null @@ -1,153 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.util.common; - -import static com.google.common.base.Preconditions.checkNotNull; - -import com.google.common.base.Strings; - -import java.util.concurrent.atomic.AtomicReference; - -/** - * The name of a counter identifies the user-specified name, as well as the origin, - * the step the counter is associated with, and a prefix to add to the name. - * - *

For backwards compatibility, the {@link CounterName} will be converted to - * a flat name (string) during the migration. - */ -public class CounterName { - /** - * Returns a {@link CounterName} with the given name. - */ - public static CounterName named(String name) { - return new CounterName(name, "", "", ""); - } - - /** - * Returns a msecs {@link CounterName}. - */ - public static CounterName msecs(String name) { - return named(name + "-msecs"); - } - - /** - * Returns a {@link CounterName} identical to this, but with the given origin. - */ - public CounterName withOrigin(String origin) { - return new CounterName(this.name, origin, this.stepName, this.prefix); - } - - /** - * Returns a {@link CounterName} identical to this, but with the given step name. - */ - public CounterName withStepName(String stepName) { - return new CounterName(this.name, this.origin, stepName, this.prefix); - } - - /** - * Returns a {@link CounterName} identical to this, but with the given prefix. - */ - public CounterName withPrefix(String prefix) { - return new CounterName(this.name, this.origin, this.stepName, prefix); - } - - /** - * Name of the counter. - * - *

For example, process-msecs, ElementCount. - */ - private final String name; - - /** - * Origin (namespace) of counter name. - * - *

For example, "user" for user-defined counters. - * It is empty for counters defined by the SDK or the runner. - */ - private final String origin; - - /** - * System defined step name or the named-output of a step. - * - *

For example, {@code s1} or {@code s2.out}. - * It may be empty when counters don't associate with step names. - */ - private final String stepName; - - /** - * Prefix of group of counters. - * - *

It is empty when counters don't have general prefixes. - */ - private final String prefix; - - /** - * Flat name is the equivalent unstructured name. - * - *

It is null before {@link #getFlatName()} is called. - */ - private AtomicReference flatName; - - private CounterName(String name, String origin, String stepName, String prefix) { - this.name = checkNotNull(name, "name"); - this.origin = checkNotNull(origin, "origin"); - this.stepName = checkNotNull(stepName, "stepName"); - this.prefix = checkNotNull(prefix, "prefix"); - this.flatName = new AtomicReference(); - } - - /** - * Returns the flat name of a structured counter. - */ - public String getFlatName() { - String ret = flatName.get(); - if (ret == null) { - StringBuilder sb = new StringBuilder(); - if (!Strings.isNullOrEmpty(prefix)) { - // Not all runner versions use "-" to concatenate prefix, it may already have it in it. - sb.append(prefix); - } - if (!Strings.isNullOrEmpty(origin)) { - sb.append(origin + "-"); - } - if (!Strings.isNullOrEmpty(stepName)) { - sb.append(stepName + "-"); - } - sb.append(name); - flatName.compareAndSet(null, sb.toString()); - ret = flatName.get(); - } - return ret; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } else if (o instanceof CounterName) { - CounterName that = (CounterName) o; - return this.getFlatName().equals(that.getFlatName()); - } - return false; - } - - @Override - public int hashCode() { - return getFlatName().hashCode(); - } -} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/CounterProvider.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/CounterProvider.java deleted file mode 100644 index c2550cd21649..000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/CounterProvider.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.util.common; - -/** - * A counter provider can provide {@link Counter} instances. - * - * @param the input type of the counter. - */ -public interface CounterProvider { - Counter getCounter(String name); -} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/CounterSet.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/CounterSet.java deleted file mode 100644 index cb0ffe5aa7f3..000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/CounterSet.java +++ /dev/null @@ -1,179 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.util.common; - -import static com.google.common.base.Preconditions.checkArgument; - -import java.util.AbstractSet; -import java.util.HashMap; -import java.util.Iterator; - -/** - * A CounterSet maintains a set of {@link Counter}s. - * - *

Thread-safe. - */ -public class CounterSet extends AbstractSet> { - - /** Registered counters. */ - private final HashMap> counters = new HashMap<>(); - - private final AddCounterMutator addCounterMutator = new AddCounterMutator(); - - /** - * Constructs a CounterSet containing the given Counters. - */ - public CounterSet(Counter... counters) { - for (Counter counter : counters) { - addNewCounter(counter); - } - } - - /** - * Returns an object that supports adding additional counters into - * this CounterSet. - */ - public AddCounterMutator getAddCounterMutator() { - return addCounterMutator; - } - - /** - * Adds a new counter, throwing an exception if a counter of the - * same name already exists. - */ - public void addNewCounter(Counter counter) { - if (!addCounter(counter)) { - throw new IllegalArgumentException( - "Counter " + counter + " duplicates an existing counter in " + this); - } - } - - /** - * Adds the given Counter to this CounterSet. - * - *

If a counter with the same name already exists, it will be - * reused, as long as it is compatible. - * - * @return the Counter that was reused, or added - * @throws IllegalArgumentException if a counter with the same - * name but an incompatible kind had already been added - */ - public synchronized Counter addOrReuseCounter(Counter counter) { - String flatName = counter.getFlatName(); - Counter oldCounter = counters.get(flatName); - if (oldCounter == null) { - // A new counter. - counters.put(flatName, counter); - return counter; - } - if (counter.isCompatibleWith(oldCounter)) { - // Return the counter to reuse. - @SuppressWarnings("unchecked") - Counter compatibleCounter = (Counter) oldCounter; - return compatibleCounter; - } - throw new IllegalArgumentException( - "Counter " + counter + " duplicates incompatible counter " - + oldCounter + " in " + this); - } - - /** - * Adds a counter. Returns {@code true} if the counter was added to the set - * and false if the given counter was {@code null} or it already existed in - * the set. - * - * @param counter to register - */ - public boolean addCounter(Counter counter) { - return add(counter); - } - - /** - * Returns the Counter with the given name in this CounterSet; - * returns null if no such Counter exists. - */ - public synchronized Counter getExistingCounter(String name) { - return counters.get(name); - } - - @Override - public synchronized Iterator> iterator() { - return counters.values().iterator(); - } - - @Override - public synchronized int size() { - return counters.size(); - } - - @Override - public synchronized boolean add(Counter e) { - if (null == e) { - return false; - } - if (counters.containsKey(e.getFlatName())) { - return false; - } - counters.put(e.getFlatName(), e); - return true; - } - - public synchronized void merge(CounterSet that) { - for (Counter theirCounter : that) { - Counter myCounter = counters.get(theirCounter.getFlatName()); - if (myCounter != null) { - mergeCounters(myCounter, theirCounter); - } else { - addCounter(theirCounter); - } - } - } - - private void mergeCounters(Counter mine, Counter theirCounter) { - checkArgument( - mine.isCompatibleWith(theirCounter), - "Can't merge CounterSets containing incompatible counters with the same name: " - + "%s (existing) and %s (merged)", - mine, - theirCounter); - @SuppressWarnings("unchecked") - Counter theirs = (Counter) theirCounter; - mine.merge(theirs); - } - - /** - * A nested class that supports adding additional counters into the - * enclosing CounterSet. This is useful as a mutator, hiding other - * public methods of the CounterSet. - */ - public class AddCounterMutator { - /** - * Adds the given Counter into the enclosing CounterSet. - * - *

If a counter with the same name already exists, it will be - * reused, as long as it has the same type. - * - * @return the Counter that was reused, or added - * @throws IllegalArgumentException if a counter with the same - * name but an incompatible kind had already been added - */ - public Counter addCounter(Counter counter) { - return addOrReuseCounter(counter); - } - } -} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObserver.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObserver.java index 3e7011bb22bd..388355e8a899 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObserver.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObserver.java @@ -19,37 +19,21 @@ import java.util.Observable; import java.util.Observer; -import javax.annotation.Nullable; /** - * An observer that gets notified when additional bytes are read - * and/or used. It adds all bytes into a local counter. When the - * observer gets advanced via the next() call, it adds the total byte - * count to the specified counter, and prepares for the next element. + * An observer that gets notified when additional bytes are read and/or used. */ -public class ElementByteSizeObserver implements Observer { - @Nullable - private final Counter counter; +public abstract class ElementByteSizeObserver implements Observer { private boolean isLazy = false; private long totalSize = 0; private double scalingFactor = 1.0; - public ElementByteSizeObserver() { - this.counter = null; - } - - public ElementByteSizeObserver(Counter counter) { - this.counter = counter; - } + public ElementByteSizeObserver() {} /** * Called to report element byte size. */ - protected void reportElementSize(long elementByteSize) { - if (counter != null) { - counter.addValue(elementByteSize); - } - } + protected abstract void reportElementSize(long elementByteSize); /** * Sets byte counting for the current element as lazy. That is, the diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CounterAggregatorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CounterAggregatorTest.java deleted file mode 100644 index 3f96cf2095ed..000000000000 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CounterAggregatorTest.java +++ /dev/null @@ -1,256 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.util; - -import static org.apache.beam.sdk.util.common.Counter.AggregationKind.MAX; -import static org.apache.beam.sdk.util.common.Counter.AggregationKind.MIN; -import static org.apache.beam.sdk.util.common.Counter.AggregationKind.SUM; - -import static org.junit.Assert.assertEquals; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; -import static org.mockito.Mockito.withSettings; - -import org.apache.beam.sdk.transforms.Aggregator; -import org.apache.beam.sdk.transforms.Combine; -import org.apache.beam.sdk.transforms.Combine.CombineFn; -import org.apache.beam.sdk.transforms.Combine.IterableCombineFn; -import org.apache.beam.sdk.transforms.Max; -import org.apache.beam.sdk.transforms.Min; -import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.sdk.transforms.Sum; -import org.apache.beam.sdk.util.common.Counter; -import org.apache.beam.sdk.util.common.CounterProvider; -import org.apache.beam.sdk.util.common.CounterSet; -import org.apache.beam.sdk.util.common.CounterSet.AddCounterMutator; - -import com.google.common.collect.Iterables; - -import org.hamcrest.Matchers; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -import java.util.Arrays; -import java.util.List; - -/** - * Unit tests for the {@link Aggregator} API. - */ -@RunWith(JUnit4.class) -public class CounterAggregatorTest { - @Rule - public final ExpectedException expectedEx = ExpectedException.none(); - - private static final String AGGREGATOR_NAME = "aggregator_name"; - - @SuppressWarnings("rawtypes") - private void testAggregator(List items, - Combine.CombineFn combiner, - Counter expectedCounter) { - CounterSet counters = new CounterSet(); - Aggregator aggregator = new CounterAggregator<>( - AGGREGATOR_NAME, combiner, counters.getAddCounterMutator()); - for (V item : items) { - aggregator.addValue(item); - } - - assertEquals(Iterables.getOnlyElement(counters), expectedCounter); - } - - @Test - public void testGetName() { - String name = "testAgg"; - CounterAggregator aggregator = new CounterAggregator<>( - name, new Sum.SumLongFn(), - new CounterSet().getAddCounterMutator()); - - assertEquals(name, aggregator.getName()); - } - - @Test - public void testGetCombineFn() { - CombineFn combineFn = new Min.MinLongFn(); - - CounterAggregator aggregator = new CounterAggregator<>("foo", - combineFn, new CounterSet().getAddCounterMutator()); - - assertEquals(combineFn, aggregator.getCombineFn()); - } - - @Test - - public void testSumInteger() throws Exception { - testAggregator(Arrays.asList(2, 4, 1, 3), new Sum.SumIntegerFn(), - Counter.ints(AGGREGATOR_NAME, SUM).resetToValue(10)); - } - - @Test - public void testSumLong() throws Exception { - testAggregator(Arrays.asList(2L, 4L, 1L, 3L), new Sum.SumLongFn(), - Counter.longs(AGGREGATOR_NAME, SUM).resetToValue(10L)); - } - - @Test - public void testSumDouble() throws Exception { - testAggregator(Arrays.asList(2.0, 4.1, 1.0, 3.1), new Sum.SumDoubleFn(), - Counter.doubles(AGGREGATOR_NAME, SUM).resetToValue(10.2)); - } - - @Test - public void testMinInteger() throws Exception { - testAggregator(Arrays.asList(2, 4, 1, 3), new Min.MinIntegerFn(), - Counter.ints(AGGREGATOR_NAME, MIN).resetToValue(1)); - } - - @Test - public void testMinLong() throws Exception { - testAggregator(Arrays.asList(2L, 4L, 1L, 3L), new Min.MinLongFn(), - Counter.longs(AGGREGATOR_NAME, MIN).resetToValue(1L)); - } - - @Test - public void testMinDouble() throws Exception { - testAggregator(Arrays.asList(2.0, 4.1, 1.0, 3.1), new Min.MinDoubleFn(), - Counter.doubles(AGGREGATOR_NAME, MIN).resetToValue(1.0)); - } - - @Test - public void testMaxInteger() throws Exception { - testAggregator(Arrays.asList(2, 4, 1, 3), new Max.MaxIntegerFn(), - Counter.ints(AGGREGATOR_NAME, MAX).resetToValue(4)); - } - - @Test - public void testMaxLong() throws Exception { - testAggregator(Arrays.asList(2L, 4L, 1L, 3L), new Max.MaxLongFn(), - Counter.longs(AGGREGATOR_NAME, MAX).resetToValue(4L)); - } - - @Test - public void testMaxDouble() throws Exception { - testAggregator(Arrays.asList(2.0, 4.1, 1.0, 3.1), new Max.MaxDoubleFn(), - Counter.doubles(AGGREGATOR_NAME, MAX).resetToValue(4.1)); - } - - @Test - public void testCounterProviderCallsProvidedCounterAddValue() { - @SuppressWarnings("unchecked") - CombineFn combiner = mock(CombineFn.class, - withSettings().extraInterfaces(CounterProvider.class)); - @SuppressWarnings("unchecked") - CounterProvider provider = (CounterProvider) combiner; - - @SuppressWarnings("unchecked") - Counter mockCounter = mock(Counter.class); - String name = "foo"; - when(provider.getCounter(name)).thenReturn(mockCounter); - - AddCounterMutator addCounterMutator = mock(AddCounterMutator.class); - when(addCounterMutator.addCounter(mockCounter)).thenReturn(mockCounter); - - Aggregator aggregator = - new CounterAggregator<>(name, combiner, addCounterMutator); - - aggregator.addValue("bar_baz"); - - verify(mockCounter).addValue("bar_baz"); - verify(addCounterMutator).addCounter(mockCounter); - } - - - @Test - public void testCompatibleDuplicateNames() throws Exception { - CounterSet counters = new CounterSet(); - Aggregator aggregator1 = new CounterAggregator<>( - AGGREGATOR_NAME, new Sum.SumIntegerFn(), - counters.getAddCounterMutator()); - - Aggregator aggregator2 = new CounterAggregator<>( - AGGREGATOR_NAME, new Sum.SumIntegerFn(), - counters.getAddCounterMutator()); - - // The duplicate aggregators should update the same counter. - aggregator1.addValue(3); - aggregator2.addValue(4); - Assert.assertEquals( - new CounterSet(Counter.ints(AGGREGATOR_NAME, SUM).resetToValue(7)), - counters); - } - - @Test - public void testIncompatibleDuplicateNames() throws Exception { - CounterSet counters = new CounterSet(); - new CounterAggregator<>( - AGGREGATOR_NAME, new Sum.SumIntegerFn(), - counters.getAddCounterMutator()); - - expectedEx.expect(IllegalArgumentException.class); - expectedEx.expectMessage(Matchers.containsString( - "aggregator's name collides with an existing aggregator or " - + "system-provided counter of an incompatible type")); - new CounterAggregator<>( - AGGREGATOR_NAME, new Sum.SumLongFn(), - counters.getAddCounterMutator()); - } - - @Test - public void testUnsupportedCombineFn() throws Exception { - expectedEx.expect(IllegalArgumentException.class); - expectedEx.expectMessage(Matchers.containsString("unsupported combiner")); - new CounterAggregator<>( - AGGREGATOR_NAME, - new Combine.CombineFn, Integer>() { - @Override - public List createAccumulator() { - return null; - } - @Override - public List addInput(List accumulator, Integer input) { - return null; - } - @Override - public List mergeAccumulators(Iterable> accumulators) { - return null; - } - @Override - public Integer extractOutput(List accumulator) { - return null; - } - }, (new CounterSet()).getAddCounterMutator()); - } - - @Test - public void testUnsupportedSerializableFunction() throws Exception { - expectedEx.expect(IllegalArgumentException.class); - expectedEx.expectMessage(Matchers.containsString("unsupported combiner")); - CombineFn, Integer> combiner = IterableCombineFn - .of(new SerializableFunction, Integer>() { - @Override - public Integer apply(Iterable input) { - return null; - } - }); - new CounterAggregator<>(AGGREGATOR_NAME, combiner, - (new CounterSet()).getAddCounterMutator()); - } -} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/CounterSetTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/CounterSetTest.java deleted file mode 100644 index bce0a888ff4f..000000000000 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/CounterSetTest.java +++ /dev/null @@ -1,227 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.util.common; - -import static org.apache.beam.sdk.util.common.Counter.AggregationKind.MAX; -import static org.apache.beam.sdk.util.common.Counter.AggregationKind.SUM; - -import static org.hamcrest.Matchers.containsInAnyOrder; -import static org.hamcrest.Matchers.equalTo; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertSame; -import static org.junit.Assert.assertThat; -import static org.junit.Assert.assertTrue; - -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** - * Unit tests for {@link CounterSet}. - */ -@RunWith(JUnit4.class) -public class CounterSetTest { - private CounterSet set; - - @Rule - public ExpectedException thrown = ExpectedException.none(); - - @Before - public void setup() { - set = new CounterSet(); - } - - @Test - public void testAddWithDifferentNamesAddsAll() { - Counter c1 = Counter.longs("c1", SUM); - Counter c2 = Counter.ints("c2", MAX); - - boolean c1Add = set.add(c1); - boolean c2Add = set.add(c2); - - assertTrue(c1Add); - assertTrue(c2Add); - assertThat(set, containsInAnyOrder(c1, c2)); - } - - @Test - public void testAddWithAlreadyPresentNameReturnsFalse() { - Counter c1 = Counter.longs("c1", SUM); - Counter c1Dup = Counter.longs("c1", SUM); - - boolean c1Add = set.add(c1); - boolean c1DupAdd = set.add(c1Dup); - - assertTrue(c1Add); - assertFalse(c1DupAdd); - assertThat(set, containsInAnyOrder((Counter) c1)); - } - - @Test - public void testAddOrReuseWithAlreadyPresentReturnsPresent() { - Counter c1 = Counter.longs("c1", SUM); - Counter c1Dup = Counter.longs("c1", SUM); - - Counter c1AddResult = set.addOrReuseCounter(c1); - Counter c1DupAddResult = set.addOrReuseCounter(c1Dup); - - assertSame(c1, c1AddResult); - assertSame(c1AddResult, c1DupAddResult); - assertThat(set, containsInAnyOrder((Counter) c1)); - } - - @Test - public void testAddOrReuseWithNoCounterReturnsProvided() { - Counter c1 = Counter.longs("c1", SUM); - - Counter c1AddResult = set.addOrReuseCounter(c1); - - assertSame(c1, c1AddResult); - assertThat(set, containsInAnyOrder((Counter) c1)); - } - - @Test - public void testAddOrReuseWithIncompatibleTypesThrowsException() { - Counter c1 = Counter.longs("c1", SUM); - Counter c1Incompatible = Counter.ints("c1", MAX); - - set.addOrReuseCounter(c1); - - thrown.expect(IllegalArgumentException.class); - thrown.expectMessage("Counter " + c1Incompatible - + " duplicates incompatible counter " + c1 + " in " + set); - - set.addOrReuseCounter(c1Incompatible); - } - - @Test - public void testMergeWithDifferentNamesAddsAll() { - Counter c1 = Counter.longs("c1", SUM); - Counter c2 = Counter.ints("c2", MAX); - - set.add(c1); - set.add(c2); - - CounterSet newSet = new CounterSet(); - newSet.merge(set); - - assertThat(newSet, containsInAnyOrder(c1, c2)); - } - - @SuppressWarnings("unchecked") - @Test - public void testMergeWithSameNamesMerges() { - Counter c1 = Counter.longs("c1", SUM); - Counter c2 = Counter.ints("c2", MAX); - - set.add(c1); - set.add(c2); - - c1.addValue(3L); - c2.addValue(22); - - CounterSet newSet = new CounterSet(); - Counter c1Prime = Counter.longs("c1", SUM); - Counter c2Prime = Counter.ints("c2", MAX); - - c1Prime.addValue(7L); - c2Prime.addValue(14); - - newSet.add(c1Prime); - newSet.add(c2Prime); - - newSet.merge(set); - - assertThat((Counter) newSet.getExistingCounter("c1"), equalTo(c1Prime)); - assertThat((Long) newSet.getExistingCounter("c1").getAggregate(), equalTo(10L)); - - assertThat((Counter) newSet.getExistingCounter("c2"), equalTo(c2Prime)); - assertThat((Integer) newSet.getExistingCounter("c2").getAggregate(), equalTo(22)); - } - - @SuppressWarnings("unchecked") - @Test - public void testMergeWithIncompatibleTypesThrowsException() { - Counter c1 = Counter.longs("c1", SUM); - - set.add(c1); - - CounterSet newSet = new CounterSet(); - Counter c1Prime = Counter.longs("c1", MAX); - - newSet.add(c1Prime); - - thrown.expect(IllegalArgumentException.class); - thrown.expectMessage("c1"); - thrown.expectMessage("incompatible counters with the same name"); - - newSet.merge(set); - } - - @Test - public void testAddCounterMutatorAddCounterAddsCounter() { - Counter c1 = Counter.longs("c1", SUM); - - Counter addC1Result = set.getAddCounterMutator().addCounter(c1); - - assertSame(c1, addC1Result); - assertThat(set, containsInAnyOrder((Counter) c1)); - } - - @Test - public void testAddCounterMutatorAddEqualCounterReusesCounter() { - Counter c1 = Counter.longs("c1", SUM); - Counter c1dup = Counter.longs("c1", SUM); - - Counter addC1Result = set.getAddCounterMutator().addCounter(c1); - Counter addC1DupResult = set.getAddCounterMutator().addCounter(c1dup); - - assertThat(set, containsInAnyOrder((Counter) c1)); - assertSame(c1, addC1Result); - assertSame(c1, addC1DupResult); - } - - @Test - public void testAddCounterMutatorIncompatibleTypesThrowsException() { - Counter c1 = Counter.longs("c1", SUM); - Counter c1Incompatible = Counter.longs("c1", MAX); - - set.getAddCounterMutator().addCounter(c1); - - thrown.expect(IllegalArgumentException.class); - thrown.expectMessage("Counter " + c1Incompatible - + " duplicates incompatible counter " + c1 + " in " + set); - - set.getAddCounterMutator().addCounter(c1Incompatible); - } - - @Test - public void testAddCounterMutatorAddMultipleCounters() { - Counter c1 = Counter.longs("c1", SUM); - Counter c2 = Counter.longs("c2", MAX); - - set.getAddCounterMutator().addCounter(c1); - set.getAddCounterMutator().addCounter(c2); - - assertThat(set, containsInAnyOrder(c1, c2)); - } - -} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/CounterTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/CounterTest.java deleted file mode 100644 index 79f0cb78b3bd..000000000000 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/CounterTest.java +++ /dev/null @@ -1,736 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.util.common; - -import static org.apache.beam.sdk.util.Values.asDouble; -import static org.apache.beam.sdk.util.common.Counter.AggregationKind.AND; -import static org.apache.beam.sdk.util.common.Counter.AggregationKind.MAX; -import static org.apache.beam.sdk.util.common.Counter.AggregationKind.MEAN; -import static org.apache.beam.sdk.util.common.Counter.AggregationKind.MIN; -import static org.apache.beam.sdk.util.common.Counter.AggregationKind.OR; -import static org.apache.beam.sdk.util.common.Counter.AggregationKind.SUM; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -import org.apache.beam.sdk.util.common.Counter.CommitState; -import org.apache.beam.sdk.util.common.Counter.CounterMean; - -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -import java.util.Arrays; -import java.util.List; - -/** - * Unit tests for the {@link Counter} API. - */ -@RunWith(JUnit4.class) -public class CounterTest { - - @Rule - public ExpectedException thrown = ExpectedException.none(); - - private static void flush(Counter c) { - switch (c.getKind()) { - case SUM: - case MAX: - case MIN: - case AND: - case OR: - c.getAndResetDelta(); - break; - case MEAN: - c.getAndResetMeanDelta(); - break; - default: - throw new IllegalArgumentException("Unknown counter kind " + c.getKind()); - } - } - - private static final double EPSILON = 0.00000000001; - - @Test - public void testCompatibility() { - // Equal counters are compatible, of all kinds. - assertTrue( - Counter.longs("c", SUM).isCompatibleWith(Counter.longs("c", SUM))); - assertTrue( - Counter.ints("c", SUM).isCompatibleWith(Counter.ints("c", SUM))); - assertTrue( - Counter.doubles("c", SUM).isCompatibleWith(Counter.doubles("c", SUM))); - assertTrue( - Counter.booleans("c", OR).isCompatibleWith( - Counter.booleans("c", OR))); - - // The name, kind, and type of the counter must match. - assertFalse( - Counter.longs("c", SUM).isCompatibleWith(Counter.longs("c2", SUM))); - assertFalse( - Counter.longs("c", SUM).isCompatibleWith(Counter.longs("c", MAX))); - assertFalse( - Counter.longs("c", SUM).isCompatibleWith(Counter.ints("c", SUM))); - - // The value of the counters are ignored. - assertTrue( - Counter.longs("c", SUM).resetToValue(666L).isCompatibleWith( - Counter.longs("c", SUM).resetToValue(42L))); - } - - - private void assertOK(long total, long delta, Counter c) { - assertEquals(total, c.getAggregate().longValue()); - assertEquals(delta, c.getAndResetDelta().longValue()); - } - - private void assertOK(double total, double delta, Counter c) { - assertEquals(total, asDouble(c.getAggregate()), EPSILON); - assertEquals(delta, asDouble(c.getAndResetDelta()), EPSILON); - } - - - // Tests for SUM. - - @Test - public void testSumLong() { - Counter c = Counter.longs("sum-long", SUM); - long expectedTotal = 0; - long expectedDelta = 0; - assertOK(expectedTotal, expectedDelta, c); - - c.addValue(13L).addValue(42L).addValue(0L); - expectedTotal += 55; - expectedDelta += 55; - assertOK(expectedTotal, expectedDelta, c); - - c.resetToValue(120L).addValue(17L).addValue(37L); - expectedTotal = expectedDelta = 174; - assertOK(expectedTotal, expectedDelta, c); - - flush(c); - expectedDelta = 0; - assertOK(expectedTotal, expectedDelta, c); - - c.addValue(15L).addValue(42L); - expectedTotal += 57; - expectedDelta += 57; - assertOK(expectedTotal, expectedDelta, c); - - c.resetToValue(100L).addValue(17L).addValue(49L); - expectedTotal = expectedDelta = 166; - assertOK(expectedTotal, expectedDelta, c); - - Counter other = Counter.longs("sum-long", SUM); - other.addValue(12L); - expectedDelta = 12L; - expectedTotal += 12L; - c.merge(other); - assertOK(expectedTotal, expectedDelta, c); - } - - @Test - public void testSumDouble() { - Counter c = Counter.doubles("sum-double", SUM); - double expectedTotal = 0.0; - double expectedDelta = 0.0; - assertOK(expectedTotal, expectedDelta, c); - - c.addValue(Math.E).addValue(Math.PI).addValue(0.0); - expectedTotal += Math.E + Math.PI; - expectedDelta += Math.E + Math.PI; - assertOK(expectedTotal, expectedDelta, c); - - c.resetToValue(Math.sqrt(2)).addValue(2 * Math.PI).addValue(3 * Math.E); - expectedTotal = expectedDelta = Math.sqrt(2) + 2 * Math.PI + 3 * Math.E; - assertOK(expectedTotal, expectedDelta, c); - - flush(c); - expectedDelta = 0.0; - assertOK(expectedTotal, expectedDelta, c); - - c.addValue(7 * Math.PI).addValue(5 * Math.E); - expectedTotal += 7 * Math.PI + 5 * Math.E; - expectedDelta += 7 * Math.PI + 5 * Math.E; - assertOK(expectedTotal, expectedDelta, c); - - c.resetToValue(Math.sqrt(17)).addValue(17.0).addValue(49.0); - expectedTotal = expectedDelta = Math.sqrt(17.0) + 17.0 + 49.0; - assertOK(expectedTotal, expectedDelta, c); - - Counter other = Counter.doubles("sum-double", SUM); - other.addValue(12 * Math.PI); - expectedDelta = 12 * Math.PI; - expectedTotal += 12 * Math.PI; - c.merge(other); - assertOK(expectedTotal, expectedDelta, c); - } - - - // Tests for MAX. - - @Test - public void testMaxLong() { - Counter c = Counter.longs("max-long", MAX); - long expectedTotal = Long.MIN_VALUE; - long expectedDelta = Long.MIN_VALUE; - assertOK(expectedTotal, expectedDelta, c); - - c.addValue(13L).addValue(42L).addValue(Long.MIN_VALUE); - expectedTotal = expectedDelta = 42; - assertOK(expectedTotal, expectedDelta, c); - - c.resetToValue(120L).addValue(17L).addValue(37L); - expectedTotal = expectedDelta = 120; - assertOK(expectedTotal, expectedDelta, c); - - flush(c); - expectedDelta = Long.MIN_VALUE; - assertOK(expectedTotal, expectedDelta, c); - - c.addValue(42L).addValue(15L); - expectedDelta = 42; - assertOK(expectedTotal, expectedDelta, c); - - c.resetToValue(100L).addValue(171L).addValue(49L); - expectedTotal = expectedDelta = 171; - assertOK(expectedTotal, expectedDelta, c); - - Counter other = Counter.longs("max-long", MAX); - other.addValue(12L); - expectedDelta = 12L; - c.merge(other); - assertOK(expectedTotal, expectedDelta, c); - } - - @Test - public void testMaxDouble() { - Counter c = Counter.doubles("max-double", MAX); - double expectedTotal = Double.NEGATIVE_INFINITY; - double expectedDelta = Double.NEGATIVE_INFINITY; - assertOK(expectedTotal, expectedDelta, c); - - c.addValue(Math.E).addValue(Math.PI).addValue(Double.NEGATIVE_INFINITY); - expectedTotal = expectedDelta = Math.PI; - assertOK(expectedTotal, expectedDelta, c); - - c.resetToValue(Math.sqrt(12345)).addValue(2 * Math.PI).addValue(3 * Math.E); - expectedTotal = expectedDelta = Math.sqrt(12345); - assertOK(expectedTotal, expectedDelta, c); - - flush(c); - expectedDelta = Double.NEGATIVE_INFINITY; - assertOK(expectedTotal, expectedDelta, c); - - c.addValue(7 * Math.PI).addValue(5 * Math.E); - expectedDelta = 7 * Math.PI; - assertOK(expectedTotal, expectedDelta, c); - - c.resetToValue(Math.sqrt(17)).addValue(171.0).addValue(49.0); - expectedTotal = expectedDelta = 171.0; - assertOK(expectedTotal, expectedDelta, c); - - Counter other = Counter.doubles("max-double", MAX); - other.addValue(12 * Math.PI); - expectedDelta = 12 * Math.PI; - c.merge(other); - assertOK(expectedTotal, expectedDelta, c); - } - - - // Tests for MIN. - - @Test - public void testMinLong() { - Counter c = Counter.longs("min-long", MIN); - long expectedTotal = Long.MAX_VALUE; - long expectedDelta = Long.MAX_VALUE; - assertOK(expectedTotal, expectedDelta, c); - - c.addValue(13L).addValue(42L).addValue(Long.MAX_VALUE); - expectedTotal = expectedDelta = 13; - assertOK(expectedTotal, expectedDelta, c); - - c.resetToValue(120L).addValue(17L).addValue(37L); - expectedTotal = expectedDelta = 17; - assertOK(expectedTotal, expectedDelta, c); - - flush(c); - expectedDelta = Long.MAX_VALUE; - assertOK(expectedTotal, expectedDelta, c); - - c.addValue(42L).addValue(18L); - expectedDelta = 18; - assertOK(expectedTotal, expectedDelta, c); - - c.resetToValue(100L).addValue(171L).addValue(49L); - expectedTotal = expectedDelta = 49; - assertOK(expectedTotal, expectedDelta, c); - - Counter other = Counter.longs("min-long", MIN); - other.addValue(42L); - expectedTotal = expectedDelta = 42L; - c.merge(other); - assertOK(expectedTotal, expectedDelta, c); - } - - @Test - public void testMinDouble() { - Counter c = Counter.doubles("min-double", MIN); - double expectedTotal = Double.POSITIVE_INFINITY; - double expectedDelta = Double.POSITIVE_INFINITY; - assertOK(expectedTotal, expectedDelta, c); - - c.addValue(Math.E).addValue(Math.PI).addValue(Double.POSITIVE_INFINITY); - expectedTotal = expectedDelta = Math.E; - assertOK(expectedTotal, expectedDelta, c); - - c.resetToValue(Math.sqrt(12345)).addValue(2 * Math.PI).addValue(3 * Math.E); - expectedTotal = expectedDelta = 2 * Math.PI; - assertOK(expectedTotal, expectedDelta, c); - - flush(c); - expectedDelta = Double.POSITIVE_INFINITY; - assertOK(expectedTotal, expectedDelta, c); - - c.addValue(7 * Math.PI).addValue(5 * Math.E); - expectedDelta = 5 * Math.E; - assertOK(expectedTotal, expectedDelta, c); - - c.resetToValue(Math.sqrt(17)).addValue(171.0).addValue(0.0); - expectedTotal = expectedDelta = 0.0; - assertOK(expectedTotal, expectedDelta, c); - - Counter other = Counter.doubles("min-double", MIN); - other.addValue(42 * Math.E); - expectedDelta = 42 * Math.E; - c.merge(other); - assertOK(expectedTotal, expectedDelta, c); - } - - - // Tests for MEAN. - - private void assertMean(long s, long sd, long c, long cd, Counter cn) { - CounterMean mean = cn.getMean(); - CounterMean deltaMean = cn.getAndResetMeanDelta(); - assertEquals(s, mean.getAggregate().longValue()); - assertEquals(sd, deltaMean.getAggregate().longValue()); - assertEquals(c, mean.getCount()); - assertEquals(cd, deltaMean.getCount()); - } - - private void assertMean(double s, double sd, long c, long cd, - Counter cn) { - CounterMean mean = cn.getMean(); - CounterMean deltaMean = cn.getAndResetMeanDelta(); - assertEquals(s, mean.getAggregate().doubleValue(), EPSILON); - assertEquals(sd, deltaMean.getAggregate().doubleValue(), EPSILON); - assertEquals(c, mean.getCount()); - assertEquals(cd, deltaMean.getCount()); - } - - @Test - public void testMeanLong() { - Counter c = Counter.longs("mean-long", MEAN); - long expTotal = 0; - long expDelta = 0; - long expCountTotal = 0; - long expCountDelta = 0; - assertMean(expTotal, expDelta, expCountTotal, expCountDelta, c); - - c.addValue(13L).addValue(42L).addValue(0L); - expTotal += 55; - expDelta += 55; - expCountTotal += 3; - expCountDelta += 3; - assertMean(expTotal, expDelta, expCountTotal, expCountDelta, c); - - c.resetMeanToValue(1L, 120L).addValue(17L).addValue(37L); - expTotal = expDelta = 174; - assertMean(expTotal, expDelta, expCountTotal, expCountDelta, c); - - flush(c); - expDelta = 0; - expCountDelta = 0; - assertMean(expTotal, expDelta, expCountTotal, expCountDelta, c); - - c.addValue(15L).addValue(42L); - expTotal += 57; - expDelta += 57; - expCountTotal += 2; - expCountDelta += 2; - assertMean(expTotal, expDelta, expCountTotal, expCountDelta, c); - - c.resetMeanToValue(3L, 100L).addValue(17L).addValue(49L); - expTotal = expDelta = 166; - expCountTotal = expCountDelta = 5; - assertMean(expTotal, expDelta, expCountTotal, expCountDelta, c); - - Counter other = Counter.longs("mean-long", MEAN); - other.addValue(12L).addValue(44L).addValue(-5L); - expTotal += 12L + 44L - 5L; - expDelta += 12L + 44L - 5L; - expCountTotal += 3; - expCountDelta += 3; - c.merge(other); - assertMean(expTotal, expDelta, expCountTotal, expCountDelta, c); - } - - @Test - public void testMeanDouble() { - Counter c = Counter.doubles("mean-double", MEAN); - double expTotal = 0.0; - double expDelta = 0.0; - long expCountTotal = 0; - long expCountDelta = 0; - assertMean(expTotal, expDelta, expCountTotal, expCountDelta, c); - - c.addValue(Math.E).addValue(Math.PI).addValue(0.0); - expTotal += Math.E + Math.PI; - expDelta += Math.E + Math.PI; - expCountTotal += 3; - expCountDelta += 3; - assertMean(expTotal, expDelta, expCountTotal, expCountDelta, c); - - c.resetMeanToValue(1L, Math.sqrt(2)).addValue(2 * Math.PI) - .addValue(3 * Math.E); - expTotal = expDelta = Math.sqrt(2) + 2 * Math.PI + 3 * Math.E; - assertMean(expTotal, expDelta, expCountTotal, expCountDelta, c); - - flush(c); - expDelta = 0.0; - expCountDelta = 0; - assertMean(expTotal, expDelta, expCountTotal, expCountDelta, c); - - c.addValue(7 * Math.PI).addValue(5 * Math.E); - expTotal += 7 * Math.PI + 5 * Math.E; - expDelta += 7 * Math.PI + 5 * Math.E; - expCountTotal += 2; - expCountDelta += 2; - assertMean(expTotal, expDelta, expCountTotal, expCountDelta, c); - - c.resetMeanToValue(3L, Math.sqrt(17)).addValue(17.0).addValue(49.0); - expTotal = expDelta = Math.sqrt(17.0) + 17.0 + 49.0; - expCountTotal = expCountDelta = 5; - assertMean(expTotal, expDelta, expCountTotal, expCountDelta, c); - - Counter other = Counter.doubles("mean-double", MEAN); - other.addValue(3 * Math.PI).addValue(12 * Math.E); - expTotal += 3 * Math.PI + 12 * Math.E; - expDelta += 3 * Math.PI + 12 * Math.E; - expCountTotal += 2; - expCountDelta += 2; - c.merge(other); - assertMean(expTotal, expDelta, expCountTotal, expCountDelta, c); - } - - - // Test for AND and OR. - private void assertBool(boolean total, boolean delta, Counter c) { - assertEquals(total, c.getAggregate().booleanValue()); - assertEquals(delta, c.getAndResetDelta().booleanValue()); - } - - @Test - public void testBoolAnd() { - Counter c = Counter.booleans("bool-and", AND); - boolean expectedTotal = true; - boolean expectedDelta = true; - assertBool(expectedTotal, expectedDelta, c); - - c.addValue(true); - assertBool(expectedTotal, expectedDelta, c); - - c.addValue(false); - expectedTotal = expectedDelta = false; - assertBool(expectedTotal, expectedDelta, c); - - c.resetToValue(true).addValue(true); - expectedTotal = expectedDelta = true; - assertBool(expectedTotal, expectedDelta, c); - - c.addValue(false); - expectedTotal = expectedDelta = false; - assertBool(expectedTotal, expectedDelta, c); - - flush(c); - expectedDelta = true; - assertBool(expectedTotal, expectedDelta, c); - - c.addValue(false); - expectedDelta = false; - assertBool(expectedTotal, expectedDelta, c); - } - - @Test - public void testBoolOr() { - Counter c = Counter.booleans("bool-or", OR); - boolean expectedTotal = false; - boolean expectedDelta = false; - assertBool(expectedTotal, expectedDelta, c); - - c.addValue(false); - assertBool(expectedTotal, expectedDelta, c); - - c.addValue(true); - expectedTotal = expectedDelta = true; - assertBool(expectedTotal, expectedDelta, c); - - c.resetToValue(false).addValue(false); - expectedTotal = expectedDelta = false; - assertBool(expectedTotal, expectedDelta, c); - - c.addValue(true); - expectedTotal = expectedDelta = true; - assertBool(expectedTotal, expectedDelta, c); - - flush(c); - expectedDelta = false; - assertBool(expectedTotal, expectedDelta, c); - - c.addValue(true); - expectedDelta = true; - assertBool(expectedTotal, expectedDelta, c); - } - - // Incompatibility tests. - - @Test(expected = IllegalArgumentException.class) - public void testSumBool() { - Counter.booleans("counter", SUM); - } - - @Test(expected = IllegalArgumentException.class) - public void testMinBool() { - Counter.booleans("counter", MIN); - } - - @Test(expected = IllegalArgumentException.class) - public void testMaxBool() { - Counter.booleans("counter", MAX); - } - - @Test(expected = IllegalArgumentException.class) - public void testMeanBool() { - Counter.booleans("counter", MEAN); - } - - @Test(expected = IllegalArgumentException.class) - public void testAndLong() { - Counter.longs("counter", AND); - } - - @Test(expected = IllegalArgumentException.class) - public void testAndDouble() { - Counter.doubles("counter", AND); - } - - @Test(expected = IllegalArgumentException.class) - public void testOrLong() { - Counter.longs("counter", OR); - } - - @Test(expected = IllegalArgumentException.class) - public void testOrDouble() { - Counter.doubles("counter", OR); - } - - @Test - public void testMergeIncompatibleCounters() { - Counter longSums = Counter.longs("longsums", SUM); - Counter longMean = Counter.longs("longmean", MEAN); - Counter longMin = Counter.longs("longmin", MIN); - - Counter otherLongSums = Counter.longs("othersums", SUM); - Counter otherLongMean = Counter.longs("otherlongmean", MEAN); - - Counter doubleSums = Counter.doubles("doublesums", SUM); - Counter doubleMean = Counter.doubles("doublemean", MEAN); - - Counter boolAnd = Counter.booleans("and", AND); - Counter boolOr = Counter.booleans("or", OR); - - List> longCounters = - Arrays.asList(longSums, longMean, longMin, otherLongSums, otherLongMean); - for (Counter left : longCounters) { - for (Counter right : longCounters) { - if (left != right) { - assertIncompatibleMerge(left, right); - } - } - } - - assertIncompatibleMerge(doubleSums, doubleMean); - assertIncompatibleMerge(boolAnd, boolOr); - } - - private void assertIncompatibleMerge(Counter left, Counter right) { - thrown.expect(IllegalArgumentException.class); - thrown.expectMessage("Counters"); - thrown.expectMessage("are incompatible"); - left.merge(right); - } - - @Test - public void testDirtyBit() { - Counter longSum = Counter.longs("long-sum", SUM); - Counter longMean = Counter.longs("long-mean", MEAN); - Counter doubleSum = Counter.doubles("double-sum", SUM); - Counter doubleMean = Counter.doubles("double-sum", MEAN); - Counter intSum = Counter.ints("int-sum", SUM); - Counter intMean = Counter.ints("int-sum", MEAN); - Counter boolAnd = Counter.booleans("and", AND); - - // Test counters are not dirty and are COMMITTED initially. - assertFalse(longSum.isDirty()); - assertFalse(longMean.isDirty()); - assertFalse(doubleSum.isDirty()); - assertFalse(doubleMean.isDirty()); - assertFalse(intSum.isDirty()); - assertFalse(intMean.isDirty()); - assertFalse(boolAnd.isDirty()); - - assertEquals(CommitState.COMMITTED, longSum.commitState.get()); - assertEquals(CommitState.COMMITTED, longMean.commitState.get()); - assertEquals(CommitState.COMMITTED, doubleSum.commitState.get()); - assertEquals(CommitState.COMMITTED, doubleMean.commitState.get()); - assertEquals(CommitState.COMMITTED, intSum.commitState.get()); - assertEquals(CommitState.COMMITTED, intMean.commitState.get()); - assertEquals(CommitState.COMMITTED, boolAnd.commitState.get()); - - // Test counters are dirty after mutating. - longSum.addValue(1L); - longMean.resetMeanToValue(1L, 1L); - doubleSum.addValue(1.0); - doubleMean.resetMeanToValue(1L, 1.0); - intSum.addValue(1); - intMean.resetMeanToValue(1, 1); - boolAnd.addValue(true); - - assertTrue(longSum.isDirty()); - assertTrue(longMean.isDirty()); - assertTrue(doubleSum.isDirty()); - assertTrue(doubleMean.isDirty()); - assertTrue(intSum.isDirty()); - assertTrue(intMean.isDirty()); - assertTrue(boolAnd.isDirty()); - - assertEquals(CommitState.DIRTY, longSum.commitState.get()); - assertEquals(CommitState.DIRTY, longMean.commitState.get()); - assertEquals(CommitState.DIRTY, doubleSum.commitState.get()); - assertEquals(CommitState.DIRTY, doubleMean.commitState.get()); - assertEquals(CommitState.DIRTY, intSum.commitState.get()); - assertEquals(CommitState.DIRTY, intMean.commitState.get()); - assertEquals(CommitState.DIRTY, boolAnd.commitState.get()); - - // Test counters are dirty and are COMMITTING. - assertTrue(longSum.committing()); - assertTrue(longMean.committing()); - assertTrue(doubleSum.committing()); - assertTrue(doubleMean.committing()); - assertTrue(intSum.committing()); - assertTrue(intMean.committing()); - assertTrue(boolAnd.committing()); - - assertTrue(longSum.isDirty()); - assertTrue(longMean.isDirty()); - assertTrue(doubleSum.isDirty()); - assertTrue(doubleMean.isDirty()); - assertTrue(intSum.isDirty()); - assertTrue(intMean.isDirty()); - assertTrue(boolAnd.isDirty()); - - assertEquals(CommitState.COMMITTING, longSum.commitState.get()); - assertEquals(CommitState.COMMITTING, longMean.commitState.get()); - assertEquals(CommitState.COMMITTING, doubleSum.commitState.get()); - assertEquals(CommitState.COMMITTING, doubleMean.commitState.get()); - assertEquals(CommitState.COMMITTING, intSum.commitState.get()); - assertEquals(CommitState.COMMITTING, intMean.commitState.get()); - assertEquals(CommitState.COMMITTING, boolAnd.commitState.get()); - - // Test counters are dirty again after mutating. - longSum.addValue(1L); - longMean.resetMeanToValue(1L, 1L); - doubleSum.addValue(1.0); - doubleMean.resetMeanToValue(1L, 1.0); - intSum.addValue(1); - intMean.resetMeanToValue(1, 1); - boolAnd.addValue(true); - - assertFalse(longSum.committed()); - assertFalse(longMean.committed()); - assertFalse(doubleSum.committed()); - assertFalse(doubleMean.committed()); - assertFalse(intSum.committed()); - assertFalse(intMean.committed()); - assertFalse(boolAnd.committed()); - - assertTrue(longSum.isDirty()); - assertTrue(longMean.isDirty()); - assertTrue(doubleSum.isDirty()); - assertTrue(doubleMean.isDirty()); - assertTrue(intSum.isDirty()); - assertTrue(intMean.isDirty()); - assertTrue(boolAnd.isDirty()); - - assertEquals(CommitState.DIRTY, longSum.commitState.get()); - assertEquals(CommitState.DIRTY, longMean.commitState.get()); - assertEquals(CommitState.DIRTY, doubleSum.commitState.get()); - assertEquals(CommitState.DIRTY, doubleMean.commitState.get()); - assertEquals(CommitState.DIRTY, intSum.commitState.get()); - assertEquals(CommitState.DIRTY, intMean.commitState.get()); - assertEquals(CommitState.DIRTY, boolAnd.commitState.get()); - - // Test counters are not dirty and are COMMITTED. - assertTrue(longSum.committing()); - assertTrue(longMean.committing()); - assertTrue(doubleSum.committing()); - assertTrue(doubleMean.committing()); - assertTrue(intSum.committing()); - assertTrue(intMean.committing()); - assertTrue(boolAnd.committing()); - - assertTrue(longSum.committed()); - assertTrue(longMean.committed()); - assertTrue(doubleSum.committed()); - assertTrue(doubleMean.committed()); - assertTrue(intSum.committed()); - assertTrue(intMean.committed()); - assertTrue(boolAnd.committed()); - - assertFalse(longSum.isDirty()); - assertFalse(longMean.isDirty()); - assertFalse(doubleSum.isDirty()); - assertFalse(doubleMean.isDirty()); - assertFalse(intSum.isDirty()); - assertFalse(intMean.isDirty()); - assertFalse(boolAnd.isDirty()); - - assertEquals(CommitState.COMMITTED, longSum.commitState.get()); - assertEquals(CommitState.COMMITTED, longMean.commitState.get()); - assertEquals(CommitState.COMMITTED, doubleSum.commitState.get()); - assertEquals(CommitState.COMMITTED, doubleMean.commitState.get()); - assertEquals(CommitState.COMMITTED, intSum.commitState.get()); - assertEquals(CommitState.COMMITTED, intMean.commitState.get()); - assertEquals(CommitState.COMMITTED, boolAnd.commitState.get()); - } -} From 0e35a9b5e2e7e3c064ffe0beae7176923d1b9679 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Mon, 8 Aug 2016 19:09:58 -0700 Subject: [PATCH 006/112] Improve Write Error Message If provided with an Unbounded PCollection, Write will fail due to restriction of calling finalize only once. This error message fails in a deep stack trace based on it not being possible to apply a GroupByKey. Instead, throw immediately on application with a specific error message. --- .../direct/WriteWithShardingFactory.java | 4 ++++ .../java/org/apache/beam/sdk/io/Write.java | 5 ++++ .../org/apache/beam/sdk/io/WriteTest.java | 24 +++++++++++++++---- 3 files changed, 28 insertions(+), 5 deletions(-) diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java index cee400102ba5..c2157b8634ce 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java @@ -34,6 +34,7 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.PInput; @@ -74,6 +75,9 @@ private DynamicallyReshardedWrite(Bound original) { @Override public PDone apply(PCollection input) { + checkArgument(IsBounded.BOUNDED == input.isBounded(), + "%s can only be applied to a Bounded PCollection", + getClass().getSimpleName()); PCollection records = input.apply("RewindowInputs", Window.into(new GlobalWindows()).triggering(DefaultTrigger.of()) .withAllowedLateness(Duration.ZERO) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java index a846b7c33ba0..fea65ef23398 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.io; +import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; import org.apache.beam.sdk.Pipeline; @@ -40,6 +41,7 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PDone; @@ -106,6 +108,9 @@ private Bound(Sink sink, int numShards) { @Override public PDone apply(PCollection input) { + checkArgument(IsBounded.BOUNDED == input.isBounded(), + "%s can only be applied to a Bounded PCollection", + Write.class.getSimpleName()); PipelineOptions options = input.getPipeline().getOptions(); sink.validate(options); return createWrite(input, sink.createWriteOperation(options)); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java index 705b77cb3986..6b44b6a02741 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java @@ -19,8 +19,6 @@ import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; -import static org.apache.beam.sdk.values.KV.of; - import static org.hamcrest.Matchers.anyOf; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; @@ -30,8 +28,6 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; -import static java.util.concurrent.ThreadLocalRandom.current; - import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.SerializableCoder; @@ -56,14 +52,17 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollection.IsBounded; import com.google.common.base.MoreObjects; import com.google.common.base.Optional; import org.hamcrest.Matchers; import org.joda.time.Duration; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -76,6 +75,7 @@ import java.util.Objects; import java.util.Set; import java.util.UUID; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicInteger; /** @@ -83,6 +83,8 @@ */ @RunWith(JUnit4.class) public class WriteTest { + @Rule public ExpectedException thrown = ExpectedException.none(); + // Static store that can be accessed within the writer private static List sinkContents = new ArrayList<>(); // Static count of output shards @@ -108,7 +110,7 @@ private static class AddArbitraryKey extends DoFn> { @ProcessElement public void processElement(ProcessContext c) { - c.output(of(current().nextInt(), c.element())); + c.output(KV.of(ThreadLocalRandom.current().nextInt(), c.element())); } } @@ -289,6 +291,18 @@ public void populateDisplayData(DisplayData.Builder builder) { assertThat(displayData, hasDisplayItem("numShards", 1)); } + @Test + public void testWriteUnbounded() { + TestPipeline p = TestPipeline.create(); + PCollection unbounded = + p.apply(Create.of("foo")).setIsBoundedInternal(IsBounded.UNBOUNDED); + + TestSink sink = new TestSink(); + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("Write can only be applied to a Bounded PCollection"); + unbounded.apply(Write.to(sink)); + } + /** * Performs a Write transform and verifies the Write transform calls the appropriate methods on * a test sink in the correct order, as well as verifies that the elements of a PCollection are From aa380d87d4cc429277482ee67118c0515633f8cb Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Tue, 9 Aug 2016 10:47:09 -0700 Subject: [PATCH 007/112] Remove Streaming Write Overrides in DataflowRunner These writes should be forbidden based on the boundedness of the input PCollection. As Write explicitly forbids the application of the transform to an Unbounded PCollection, this will be equivalent in most cases; In cases where the input PCollection is Bounded, due to an UnboundedReadFromBoundedSource, the write will function as expected and does not need to be forbidden. --- .../beam/runners/dataflow/DataflowRunner.java | 72 +------------------ .../runners/dataflow/DataflowRunnerTest.java | 24 ------- .../org/apache/beam/sdk/io/WriteTest.java | 12 +++- 3 files changed, 11 insertions(+), 97 deletions(-) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index bea6264bd36f..76dbecf433b2 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -20,7 +20,6 @@ import static org.apache.beam.sdk.util.StringUtils.approximatePTransformName; import static org.apache.beam.sdk.util.StringUtils.approximateSimpleName; import static org.apache.beam.sdk.util.WindowedValue.valueInEmptyWindows; - import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; import static com.google.common.base.Strings.isNullOrEmpty; @@ -58,14 +57,12 @@ import org.apache.beam.sdk.coders.StandardCoder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.coders.VarLongCoder; -import org.apache.beam.sdk.io.AvroIO; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.FileBasedSink; import org.apache.beam.sdk.io.PubsubIO; import org.apache.beam.sdk.io.PubsubUnboundedSink; import org.apache.beam.sdk.io.PubsubUnboundedSource; import org.apache.beam.sdk.io.Read; -import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.io.Write; import org.apache.beam.sdk.options.PipelineOptions; @@ -143,6 +140,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; + import org.joda.time.DateTimeUtils; import org.joda.time.DateTimeZone; import org.joda.time.format.DateTimeFormat; @@ -172,6 +170,7 @@ import java.util.Set; import java.util.SortedSet; import java.util.TreeSet; + import javax.annotation.Nullable; /** @@ -329,11 +328,8 @@ public static DataflowRunner fromOptions(PipelineOptions options) { builder.put(View.AsSingleton.class, StreamingViewAsSingleton.class); builder.put(View.AsList.class, StreamingViewAsList.class); builder.put(View.AsIterable.class, StreamingViewAsIterable.class); - builder.put(Write.Bound.class, StreamingWrite.class); builder.put(Read.Unbounded.class, StreamingUnboundedRead.class); builder.put(Read.Bounded.class, StreamingBoundedRead.class); - builder.put(AvroIO.Write.Bound.class, UnsupportedIO.class); - builder.put(TextIO.Write.Bound.class, UnsupportedIO.class); builder.put(Window.Bound.class, AssignWindows.class); // In streaming mode must use either the custom Pubsub unbounded source/sink or // defer to Windmill's built-in implementation. @@ -2045,30 +2041,6 @@ public PDone apply(PCollection input) { } } - /** - * Specialized (non-)implementation for - * {@link org.apache.beam.sdk.io.Write.Bound Write.Bound} - * for the Dataflow runner in streaming mode. - */ - private static class StreamingWrite extends PTransform, PDone> { - /** - * Builds an instance of this class from the overridden transform. - */ - @SuppressWarnings("unused") // used via reflection in DataflowRunner#apply() - public StreamingWrite(DataflowRunner runner, Write.Bound transform) { } - - @Override - public PDone apply(PCollection input) { - throw new UnsupportedOperationException( - "The Write transform is not supported by the Dataflow streaming runner."); - } - - @Override - protected String getKindString() { - return "StreamingWrite"; - } - } - // ================================================================================ // PubsubIO translations // ================================================================================ @@ -2723,30 +2695,6 @@ private static class UnsupportedIO doFn; - /** - * Builds an instance of this class from the overridden transform. - */ - @SuppressWarnings("unused") // used via reflection in DataflowRunner#apply() - public UnsupportedIO(DataflowRunner runner, AvroIO.Read.Bound transform) { - this.transform = transform; - } - - /** - * Builds an instance of this class from the overridden transform. - */ - @SuppressWarnings("unused") // used via reflection in DataflowRunner#apply() - public UnsupportedIO(DataflowRunner runner, TextIO.Read.Bound transform) { - this.transform = transform; - } - - /** - * Builds an instance of this class from the overridden transform. - */ - @SuppressWarnings("unused") // used via reflection in DataflowRunner#apply() - public UnsupportedIO(DataflowRunner runner, Read.Bounded transform) { - this.transform = transform; - } - /** * Builds an instance of this class from the overridden transform. */ @@ -2755,22 +2703,6 @@ public UnsupportedIO(DataflowRunner runner, Read.Unbounded transform) { this.transform = transform; } - /** - * Builds an instance of this class from the overridden transform. - */ - @SuppressWarnings("unused") // used via reflection in DataflowRunner#apply() - public UnsupportedIO(DataflowRunner runner, AvroIO.Write.Bound transform) { - this.transform = transform; - } - - /** - * Builds an instance of this class from the overridden transform. - */ - @SuppressWarnings("unused") // used via reflection in DataflowRunner#apply() - public UnsupportedIO(DataflowRunner runner, TextIO.Write.Bound transform) { - this.transform = transform; - } - /** * Builds an instance of this class from the overridden doFn. */ diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java index 704410def77c..d7deffdb0323 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java @@ -55,7 +55,6 @@ import org.apache.beam.sdk.coders.BigEndianLongCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.VarLongCoder; -import org.apache.beam.sdk.io.AvroIO; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.options.PipelineOptions; @@ -82,7 +81,6 @@ import org.apache.beam.sdk.util.gcsfs.GcsPath; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; @@ -956,28 +954,6 @@ public void testReadUnboundedUnsupportedInBatch() throws Exception { testUnsupportedSource(Read.from(new TestCountingSource(1)), "Read.Unbounded", false); } - private void testUnsupportedSink( - PTransform, PDone> sink, String name, boolean streaming) - throws Exception { - thrown.expect(UnsupportedOperationException.class); - thrown.expectMessage( - "The DataflowRunner in streaming mode does not support " + name); - - Pipeline p = Pipeline.create(makeOptions(streaming)); - p.apply(Create.of("foo")).apply(sink); - p.run(); - } - - @Test - public void testAvroIOSinkUnsupportedInStreaming() throws Exception { - testUnsupportedSink(AvroIO.Write.to("foo").withSchema(String.class), "AvroIO.Write", true); - } - - @Test - public void testTextIOSinkUnsupportedInStreaming() throws Exception { - testUnsupportedSink(TextIO.Write.to("foo"), "TextIO.Write", true); - } - @Test public void testBatchViewAsSingletonToIsmRecord() throws Exception { DoFnTester>>>, diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java index 6b44b6a02741..f9bf472eab9d 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java @@ -52,7 +52,6 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PCollection.IsBounded; import com.google.common.base.MoreObjects; import com.google.common.base.Optional; @@ -294,8 +293,8 @@ public void populateDisplayData(DisplayData.Builder builder) { @Test public void testWriteUnbounded() { TestPipeline p = TestPipeline.create(); - PCollection unbounded = - p.apply(Create.of("foo")).setIsBoundedInternal(IsBounded.UNBOUNDED); + PCollection unbounded = p.apply(CountingInput.unbounded()) + .apply(MapElements.via(new ToStringFn())); TestSink sink = new TestSink(); thrown.expect(IllegalArgumentException.class); @@ -303,6 +302,13 @@ public void testWriteUnbounded() { unbounded.apply(Write.to(sink)); } + private static class ToStringFn extends SimpleFunction { + @Override + public String apply(Long input) { + return Long.toString(input); + } + } + /** * Performs a Write transform and verifies the Write transform calls the appropriate methods on * a test sink in the correct order, as well as verifies that the elements of a PCollection are From c9a32e8b8b4ca182721bf81639bd2a28e53f9525 Mon Sep 17 00:00:00 2001 From: Mark Liu Date: Tue, 2 Aug 2016 17:25:14 -0700 Subject: [PATCH 008/112] [BEAM-495] Create General Verifier for File Checksum --- .../org/apache/beam/examples/WordCountIT.java | 112 +------------- .../beam/sdk/testing/FileChecksumMatcher.java | 137 ++++++++++++++++++ .../sdk/testing/FileChecksumMatcherTest.java | 78 ++++++++++ 3 files changed, 219 insertions(+), 108 deletions(-) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FileChecksumMatcher.java create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java diff --git a/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java b/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java index c087c67d8391..de6615c95ea5 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java +++ b/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java @@ -18,38 +18,18 @@ package org.apache.beam.examples; -import static com.google.common.base.Preconditions.checkArgument; - import org.apache.beam.examples.WordCount.WordCountOptions; -import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.testing.SerializableMatcher; +import org.apache.beam.sdk.testing.FileChecksumMatcher; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.testing.TestPipelineOptions; -import org.apache.beam.sdk.util.IOChannelFactory; import org.apache.beam.sdk.util.IOChannelUtils; -import com.google.common.base.Strings; -import com.google.common.hash.HashCode; -import com.google.common.hash.Hashing; -import com.google.common.io.CharStreams; - -import org.hamcrest.Description; -import org.hamcrest.TypeSafeMatcher; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.io.Reader; -import java.nio.channels.Channels; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.Collection; import java.util.Date; -import java.util.List; /** * End-to-end tests of WordCount. @@ -73,94 +53,10 @@ public void testE2EWordCount() throws Exception { String.format("WordCountIT-%tF-% - implements SerializableMatcher { - - private static final Logger LOG = LoggerFactory.getLogger(WordCountOnSuccessMatcher.class); - - private static final String EXPECTED_CHECKSUM = "c04722202dee29c442b55ead54c6000693e85e77"; - private String actualChecksum; - - private final String outputPath; - - WordCountOnSuccessMatcher(String outputPath) { - checkArgument( - !Strings.isNullOrEmpty(outputPath), - "Expected valid output path, but received %s", outputPath); - - this.outputPath = outputPath; - } - - @Override - protected boolean matchesSafely(PipelineResult pResult) { - try { - // Load output data - List outputs = readLines(outputPath); - - // Verify outputs. Checksum is computed using SHA-1 algorithm - actualChecksum = hashing(outputs); - LOG.info("Generated checksum for output data: {}", actualChecksum); - - return actualChecksum.equals(EXPECTED_CHECKSUM); - } catch (IOException e) { - throw new RuntimeException( - String.format("Failed to read from path: %s", outputPath)); - } - } - - private List readLines(String path) throws IOException { - List readData = new ArrayList<>(); - - IOChannelFactory factory = IOChannelUtils.getFactory(path); - - // Match inputPath which may contains glob - Collection files = factory.match(path); - - // Read data from file paths - int i = 0; - for (String file : files) { - try (Reader reader = - Channels.newReader(factory.open(file), StandardCharsets.UTF_8.name())) { - List lines = CharStreams.readLines(reader); - readData.addAll(lines); - LOG.info( - "[{} of {}] Read {} lines from file: {}", i, files.size() - 1, lines.size(), file); - } - i++; - } - return readData; - } - - private String hashing(List strs) { - List hashCodes = new ArrayList<>(); - for (String str : strs) { - hashCodes.add(Hashing.sha1().hashString(str, StandardCharsets.UTF_8)); - } - return Hashing.combineUnordered(hashCodes).toString(); - } - - @Override - public void describeTo(Description description) { - description - .appendText("Expected checksum is (") - .appendText(EXPECTED_CHECKSUM) - .appendText(")"); - } - - @Override - protected void describeMismatchSafely(PipelineResult pResult, Description description) { - description - .appendText("was (") - .appendText(actualChecksum) - .appendText(")"); - } - } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FileChecksumMatcher.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FileChecksumMatcher.java new file mode 100644 index 000000000000..f4bd0626cbbb --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FileChecksumMatcher.java @@ -0,0 +1,137 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.sdk.testing; + +import static com.google.common.base.Preconditions.checkArgument; + +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.util.IOChannelFactory; +import org.apache.beam.sdk.util.IOChannelUtils; + +import com.google.common.base.Strings; +import com.google.common.hash.HashCode; +import com.google.common.hash.Hashing; +import com.google.common.io.CharStreams; + +import org.hamcrest.Description; +import org.hamcrest.TypeSafeMatcher; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.Reader; +import java.nio.channels.Channels; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + +/** + * Matcher to verify file checksum in E2E test. + * + *

For example: + *

{@code [
+ *   assertTrue(job, new FileChecksumMatcher(checksumString, filePath));
+ * ]}
+ */ +public class FileChecksumMatcher extends TypeSafeMatcher + implements SerializableMatcher { + + private static final Logger LOG = LoggerFactory.getLogger(FileChecksumMatcher.class); + + private final String expectedChecksum; + private final String filePath; + private String actualChecksum; + + public FileChecksumMatcher(String checksum, String filePath) { + checkArgument( + !Strings.isNullOrEmpty(checksum), + "Expected valid checksum, but received %s", checksum); + checkArgument( + !Strings.isNullOrEmpty(filePath), + "Expected valid file path, but received %s", filePath); + + this.expectedChecksum = checksum; + this.filePath = filePath; + } + + @Override + public boolean matchesSafely(PipelineResult pipelineResult) { + try { + // Load output data + List outputs = readLines(filePath); + + // Verify outputs. Checksum is computed using SHA-1 algorithm + actualChecksum = hashing(outputs); + LOG.info("Generated checksum for output data: {}", actualChecksum); + + return actualChecksum.equals(expectedChecksum); + } catch (IOException e) { + throw new RuntimeException( + String.format("Failed to read from path: %s", filePath)); + } + } + + private List readLines(String path) throws IOException { + List readData = new ArrayList<>(); + + IOChannelFactory factory = IOChannelUtils.getFactory(path); + + // Match inputPath which may contains glob + Collection files = factory.match(path); + + // Read data from file paths + int i = 0; + for (String file : files) { + try (Reader reader = + Channels.newReader(factory.open(file), StandardCharsets.UTF_8.name())) { + List lines = CharStreams.readLines(reader); + readData.addAll(lines); + LOG.info( + "[{} of {}] Read {} lines from file: {}", i, files.size() - 1, lines.size(), file); + } + i++; + } + return readData; + } + + private String hashing(List strs) { + List hashCodes = new ArrayList<>(); + for (String str : strs) { + hashCodes.add(Hashing.sha1().hashString(str, StandardCharsets.UTF_8)); + } + return Hashing.combineUnordered(hashCodes).toString(); + } + + @Override + public void describeTo(Description description) { + description + .appendText("Expected checksum is (") + .appendText(expectedChecksum) + .appendText(")"); + } + + @Override + public void describeMismatchSafely(PipelineResult pResult, Description description) { + description + .appendText("was (") + .appendText(actualChecksum) + .appendText(")"); + } +} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java new file mode 100644 index 000000000000..00417c746b0a --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.testing; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsString; + +import org.apache.beam.sdk.PipelineResult; + +import com.google.common.io.Files; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.Mock; +import org.mockito.Mockito; + +import java.io.File; +import java.io.IOException; +import java.nio.charset.StandardCharsets; + +/** Tests for {@link FileChecksumMatcher}. */ +@RunWith(JUnit4.class) +public class FileChecksumMatcherTest { + @Rule + public TemporaryFolder tmpFolder = new TemporaryFolder(); + @Rule + public ExpectedException thrown = ExpectedException.none(); + + @Mock + private PipelineResult pResult = Mockito.mock(PipelineResult.class); + + @Test + public void testPreconditionValidChecksumString() throws IOException{ + String tmpPath = tmpFolder.newFile().getPath(); + + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage(containsString("Expected valid checksum, but received")); + new FileChecksumMatcher(null, tmpPath); + new FileChecksumMatcher("", tmpPath); + } + + @Test + public void testPreconditionValidFilePath() throws IOException { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage(containsString("Expected valid file path, but received")); + new FileChecksumMatcher("checksumString", null); + new FileChecksumMatcher("checksumString", ""); + } + + @Test + public void testChecksumVerify() throws IOException{ + File tmpFile = tmpFolder.newFile(); + Files.write("Test for file checksum verifier.", tmpFile, StandardCharsets.UTF_8); + FileChecksumMatcher matcher = + new FileChecksumMatcher("a8772322f5d7b851777f820fc79d050f9d302915", tmpFile.getPath()); + + assertThat(pResult, matcher); + } + } From 0b5da70d296543c00c8c4460107d1c2410c4e55f Mon Sep 17 00:00:00 2001 From: Mark Liu Date: Tue, 2 Aug 2016 17:47:46 -0700 Subject: [PATCH 009/112] Add output checksum to WordCountITOptions --- .../test/java/org/apache/beam/examples/WordCountIT.java | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java b/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java index de6615c95ea5..e438faf4f606 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java +++ b/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java @@ -19,6 +19,7 @@ package org.apache.beam.examples; import org.apache.beam.examples.WordCount.WordCountOptions; +import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.FileChecksumMatcher; import org.apache.beam.sdk.testing.TestPipeline; @@ -39,8 +40,13 @@ public class WordCountIT { /** * Options for the WordCount Integration Test. + * + * Define expected output file checksum to verify WordCount pipeline result with customized input. */ public interface WordCountITOptions extends TestPipelineOptions, WordCountOptions { + @Default.String("c04722202dee29c442b55ead54c6000693e85e77") + String getOutputChecksum(); + void setOutputChecksum(String value); } @Test @@ -54,8 +60,7 @@ public void testE2EWordCount() throws Exception { "output", "results")); options.setOnSuccessMatcher( - new FileChecksumMatcher("c04722202dee29c442b55ead54c6000693e85e77", - options.getOutput() + "*")); + new FileChecksumMatcher(options.getOutputChecksum(), options.getOutput() + "*")); WordCount.main(TestPipeline.convertToArgs(options)); } From a98bbb26c12f96446b314f8229d9218236f0ce06 Mon Sep 17 00:00:00 2001 From: Mark Liu Date: Thu, 11 Aug 2016 11:26:28 -0700 Subject: [PATCH 010/112] More unit test and code style fix --- .../beam/sdk/testing/FileChecksumMatcher.java | 17 +++++----- .../sdk/testing/FileChecksumMatcherTest.java | 34 +++++++++++++++++-- 2 files changed, 39 insertions(+), 12 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FileChecksumMatcher.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FileChecksumMatcher.java index f4bd0626cbbb..303efcb392b8 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FileChecksumMatcher.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FileChecksumMatcher.java @@ -84,13 +84,12 @@ public boolean matchesSafely(PipelineResult pipelineResult) { return actualChecksum.equals(expectedChecksum); } catch (IOException e) { throw new RuntimeException( - String.format("Failed to read from path: %s", filePath)); + String.format("Failed to read from path: %s", filePath)); } } private List readLines(String path) throws IOException { List readData = new ArrayList<>(); - IOChannelFactory factory = IOChannelUtils.getFactory(path); // Match inputPath which may contains glob @@ -100,7 +99,7 @@ private List readLines(String path) throws IOException { int i = 0; for (String file : files) { try (Reader reader = - Channels.newReader(factory.open(file), StandardCharsets.UTF_8.name())) { + Channels.newReader(factory.open(file), StandardCharsets.UTF_8.name())) { List lines = CharStreams.readLines(reader); readData.addAll(lines); LOG.info( @@ -122,16 +121,16 @@ private String hashing(List strs) { @Override public void describeTo(Description description) { description - .appendText("Expected checksum is (") - .appendText(expectedChecksum) - .appendText(")"); + .appendText("Expected checksum is (") + .appendText(expectedChecksum) + .appendText(")"); } @Override public void describeMismatchSafely(PipelineResult pResult, Description description) { description - .appendText("was (") - .appendText(actualChecksum) - .appendText(")"); + .appendText("was (") + .appendText(actualChecksum) + .appendText(")"); } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java index 00417c746b0a..eebb73e33212 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java @@ -49,25 +49,39 @@ public class FileChecksumMatcherTest { private PipelineResult pResult = Mockito.mock(PipelineResult.class); @Test - public void testPreconditionValidChecksumString() throws IOException{ + public void testPreconditionChecksumIsNull() throws IOException { String tmpPath = tmpFolder.newFile().getPath(); thrown.expect(IllegalArgumentException.class); thrown.expectMessage(containsString("Expected valid checksum, but received")); new FileChecksumMatcher(null, tmpPath); + } + + @Test + public void testPreconditionChecksumIsEmpty() throws IOException { + String tmpPath = tmpFolder.newFile().getPath(); + + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage(containsString("Expected valid checksum, but received")); new FileChecksumMatcher("", tmpPath); } @Test - public void testPreconditionValidFilePath() throws IOException { + public void testPreconditionFilePathIsNull() { thrown.expect(IllegalArgumentException.class); thrown.expectMessage(containsString("Expected valid file path, but received")); new FileChecksumMatcher("checksumString", null); + } + + @Test + public void testPreconditionFilePathIsEmpty() { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage(containsString("Expected valid file path, but received")); new FileChecksumMatcher("checksumString", ""); } @Test - public void testChecksumVerify() throws IOException{ + public void testMatcherVerifySingleFile() throws IOException{ File tmpFile = tmpFolder.newFile(); Files.write("Test for file checksum verifier.", tmpFile, StandardCharsets.UTF_8); FileChecksumMatcher matcher = @@ -75,4 +89,18 @@ public void testChecksumVerify() throws IOException{ assertThat(pResult, matcher); } + + @Test + public void testMatcherVerifyMultipleFilesInOneDir() throws IOException { + File tmpFile1 = tmpFolder.newFile(); + File tmpFile2 = tmpFolder.newFile(); + Files.write("To be or not to be, ", tmpFile1, StandardCharsets.UTF_8); + Files.write("it is not a question.", tmpFile2, StandardCharsets.UTF_8); + FileChecksumMatcher matcher = + new FileChecksumMatcher( + "90552392c28396935fe4f123bd0b5c2d0f6260c8", + tmpFolder.getRoot().getPath() + "/*"); + + assertThat(pResult, matcher); + } } From d7566c53d24b76bcdd2e3d61b436edea31bdb752 Mon Sep 17 00:00:00 2001 From: Mark Liu Date: Thu, 11 Aug 2016 11:55:17 -0700 Subject: [PATCH 011/112] Using IOChannelUtils to resolve file path --- .../test/java/org/apache/beam/examples/WordCountIT.java | 3 ++- .../apache/beam/sdk/testing/FileChecksumMatcherTest.java | 7 +++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java b/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java index e438faf4f606..f93dc2b2f779 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java +++ b/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java @@ -41,7 +41,8 @@ public class WordCountIT { /** * Options for the WordCount Integration Test. * - * Define expected output file checksum to verify WordCount pipeline result with customized input. + *

Define expected output file checksum to verify WordCount pipeline result + * with customized input. */ public interface WordCountITOptions extends TestPipelineOptions, WordCountOptions { @Default.String("c04722202dee29c442b55ead54c6000693e85e77") diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java index eebb73e33212..d94ffe2c4761 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java @@ -21,9 +21,8 @@ import static org.hamcrest.Matchers.containsString; import org.apache.beam.sdk.PipelineResult; - +import org.apache.beam.sdk.util.IOChannelUtils; import com.google.common.io.Files; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -99,8 +98,8 @@ public void testMatcherVerifyMultipleFilesInOneDir() throws IOException { FileChecksumMatcher matcher = new FileChecksumMatcher( "90552392c28396935fe4f123bd0b5c2d0f6260c8", - tmpFolder.getRoot().getPath() + "/*"); + IOChannelUtils.resolve(tmpFolder.getRoot().getPath(), "*")); assertThat(pResult, matcher); } - } +} From 705b72ed0fa0644c6130a6dffe741772d1686d83 Mon Sep 17 00:00:00 2001 From: Ian Zhou Date: Fri, 5 Aug 2016 15:31:59 -0700 Subject: [PATCH 012/112] Added unit tests and error handling in removeTemporaryTables --- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 34 +-- .../sdk/io/gcp/bigquery/BigQueryIOTest.java | 193 ++++++++++++------ 2 files changed, 148 insertions(+), 79 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java index 36e09f1dbee0..46c2935942ee 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java @@ -2062,7 +2062,7 @@ public void processElement(ProcessContext c) throws Exception { createDisposition); c.output(toJsonString(ref)); - removeTemporaryFiles(c.getPipelineOptions(), partition); + removeTemporaryFiles(c.getPipelineOptions(), tempFilePrefix, partition); } private void load( @@ -2108,16 +2108,17 @@ private void load( + "retries: %d", jobIdPrefix, Bound.MAX_RETRY_JOBS)); } - private void removeTemporaryFiles(PipelineOptions options, Collection matches) + static void removeTemporaryFiles( + PipelineOptions options, + String tempFilePrefix, + Collection files) throws IOException { - String pattern = tempFilePrefix + "*"; - LOG.debug("Finding temporary files matching {}.", pattern); - IOChannelFactory factory = IOChannelUtils.getFactory(pattern); + IOChannelFactory factory = IOChannelUtils.getFactory(tempFilePrefix); if (factory instanceof GcsIOChannelFactory) { GcsUtil gcsUtil = new GcsUtil.GcsUtilFactory().create(options); - gcsUtil.remove(matches); + gcsUtil.remove(files); } else if (factory instanceof FileIOChannelFactory) { - for (String filename : matches) { + for (String filename : files) { LOG.debug("Removing file {}", filename); boolean exists = Files.deleteIfExists(Paths.get(filename)); if (!exists) { @@ -2175,7 +2176,7 @@ public WriteRename( public void processElement(ProcessContext c) throws Exception { List tempTablesJson = Lists.newArrayList(c.sideInput(tempTablesView)); - // Do not copy if not temp tables are provided + // Do not copy if no temp tables are provided if (tempTablesJson.size() == 0) { return; } @@ -2237,13 +2238,18 @@ private void copy( + "retries: %d", jobIdPrefix, Bound.MAX_RETRY_JOBS)); } - private void removeTemporaryTables(DatasetService tableService, - List tempTables) throws Exception { + static void removeTemporaryTables(DatasetService tableService, + List tempTables) { for (TableReference tableRef : tempTables) { - tableService.deleteTable( - tableRef.getProjectId(), - tableRef.getDatasetId(), - tableRef.getTableId()); + try { + LOG.debug("Deleting table {}", toJsonString(tableRef)); + tableService.deleteTable( + tableRef.getProjectId(), + tableRef.getDatasetId(), + tableRef.getTableId()); + } catch (Exception e) { + LOG.warn("Failed to delete the table {}", toJsonString(tableRef), e); + } } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java index 6d6eb60defe6..fcaa054da456 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java @@ -30,6 +30,8 @@ import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.anyString; import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.when; import org.apache.beam.sdk.Pipeline; @@ -47,6 +49,7 @@ import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Status; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.TransformingSource; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.TableRowWriter; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WritePartition; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteRename; @@ -140,8 +143,6 @@ @RunWith(JUnit4.class) public class BigQueryIOTest implements Serializable { - @Rule public transient TemporaryFolder tmpFolder = new TemporaryFolder(); - // Status.UNKNOWN maps to null private static final Map JOB_STATUS_MAP = ImmutableMap.of( Status.SUCCEEDED, new Job().setStatus(new JobStatus()), @@ -586,12 +587,11 @@ public void processElement(ProcessContext c) throws Exception { @Test @Category(NeedsRunner.class) - public void testCustomSink() throws Exception { + public void testWrite() throws Exception { FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() .withJobService(new FakeJobService() .startJobReturns("done", "done", "done") - .pollJobReturns(Status.FAILED, Status.FAILED, Status.SUCCEEDED)) - .withDatasetService(mockDatasetService); + .pollJobReturns(Status.FAILED, Status.FAILED, Status.SUCCEEDED)); Pipeline p = TestPipeline.create(bqOptions); p.apply(Create.of( @@ -610,17 +610,18 @@ public void testCustomSink() throws Exception { p.run(); logged.verifyInfo("Starting BigQuery load job"); + logged.verifyInfo("BigQuery load job failed"); + logged.verifyInfo("try 0/" + BigQueryIO.Write.Bound.MAX_RETRY_JOBS); + logged.verifyInfo("try 1/" + BigQueryIO.Write.Bound.MAX_RETRY_JOBS); + logged.verifyInfo("try 2/" + BigQueryIO.Write.Bound.MAX_RETRY_JOBS); + logged.verifyNotLogged("try 3/" + BigQueryIO.Write.Bound.MAX_RETRY_JOBS); File tempDir = new File(bqOptions.getTempLocation()); - assertEquals(0, tempDir.listFiles(new FileFilter() { - @Override - public boolean accept(File pathname) { - return pathname.isFile(); - }}).length); + testNumFiles(tempDir, 0); } @Test @Category(NeedsRunner.class) - public void testCustomSinkUnknown() throws Exception { + public void testWriteUnknown() throws Exception { FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() .withJobService(new FakeJobService() .startJobReturns("done", "done") @@ -642,11 +643,7 @@ public void testCustomSinkUnknown() throws Exception { p.run(); File tempDir = new File(bqOptions.getTempLocation()); - assertEquals(0, tempDir.listFiles(new FileFilter() { - @Override - public boolean accept(File pathname) { - return pathname.isFile(); - }}).length); + testNumFiles(tempDir, 0); } @Test @@ -703,7 +700,7 @@ public void testQuerySourcePrimitiveDisplayData() throws IOException, Interrupte @Test - public void testBuildSink() { + public void testBuildWrite() { BigQueryIO.Write.Bound bound = BigQueryIO.Write.to("foo.com:project:somedataset.sometable"); checkWriteObject( bound, "foo.com:project", "somedataset", "sometable", @@ -713,18 +710,18 @@ public void testBuildSink() { @Test @Category(RunnableOnService.class) @Ignore("[BEAM-436] DirectRunner RunnableOnService tempLocation configuration insufficient") - public void testBatchSinkPrimitiveDisplayData() throws IOException, InterruptedException { - testSinkPrimitiveDisplayData(/* streaming: */ false); + public void testBatchWritePrimitiveDisplayData() throws IOException, InterruptedException { + testWritePrimitiveDisplayData(/* streaming: */ false); } @Test @Category(RunnableOnService.class) @Ignore("[BEAM-436] DirectRunner RunnableOnService tempLocation configuration insufficient") - public void testStreamingSinkPrimitiveDisplayData() throws IOException, InterruptedException { - testSinkPrimitiveDisplayData(/* streaming: */ true); + public void testStreamingWritePrimitiveDisplayData() throws IOException, InterruptedException { + testWritePrimitiveDisplayData(/* streaming: */ true); } - private void testSinkPrimitiveDisplayData(boolean streaming) throws IOException, + private void testWritePrimitiveDisplayData(boolean streaming) throws IOException, InterruptedException { PipelineOptions options = TestPipeline.testingPipelineOptions(); options.as(StreamingOptions.class).setStreaming(streaming); @@ -747,7 +744,7 @@ private void testSinkPrimitiveDisplayData(boolean streaming) throws IOException, } @Test - public void testBuildSinkwithoutValidation() { + public void testBuildWriteWithoutValidation() { // This test just checks that using withoutValidation will not trigger object // construction errors. BigQueryIO.Write.Bound bound = @@ -758,7 +755,7 @@ public void testBuildSinkwithoutValidation() { } @Test - public void testBuildSinkDefaultProject() { + public void testBuildWriteDefaultProject() { BigQueryIO.Write.Bound bound = BigQueryIO.Write.to("somedataset.sometable"); checkWriteObject( bound, null, "somedataset", "sometable", @@ -766,7 +763,7 @@ public void testBuildSinkDefaultProject() { } @Test - public void testBuildSinkWithTableReference() { + public void testBuildWriteWithTableReference() { TableReference table = new TableReference() .setProjectId("foo.com:project") .setDatasetId("somedataset") @@ -779,7 +776,7 @@ public void testBuildSinkWithTableReference() { @Test @Category(RunnableOnService.class) - public void testBuildSinkWithoutTable() { + public void testBuildWriteWithoutTable() { Pipeline p = TestPipeline.create(); thrown.expect(IllegalStateException.class); thrown.expectMessage("must set the table reference"); @@ -788,7 +785,7 @@ public void testBuildSinkWithoutTable() { } @Test - public void testBuildSinkWithSchema() { + public void testBuildWriteWithSchema() { TableSchema schema = new TableSchema(); BigQueryIO.Write.Bound bound = BigQueryIO.Write.to("foo.com:project:somedataset.sometable").withSchema(schema); @@ -798,7 +795,7 @@ public void testBuildSinkWithSchema() { } @Test - public void testBuildSinkWithCreateDispositionNever() { + public void testBuildWriteWithCreateDispositionNever() { BigQueryIO.Write.Bound bound = BigQueryIO.Write .to("foo.com:project:somedataset.sometable") .withCreateDisposition(CreateDisposition.CREATE_NEVER); @@ -808,7 +805,7 @@ public void testBuildSinkWithCreateDispositionNever() { } @Test - public void testBuildSinkWithCreateDispositionIfNeeded() { + public void testBuildWriteWithCreateDispositionIfNeeded() { BigQueryIO.Write.Bound bound = BigQueryIO.Write .to("foo.com:project:somedataset.sometable") .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED); @@ -818,7 +815,7 @@ public void testBuildSinkWithCreateDispositionIfNeeded() { } @Test - public void testBuildSinkWithWriteDispositionTruncate() { + public void testBuildWriteWithWriteDispositionTruncate() { BigQueryIO.Write.Bound bound = BigQueryIO.Write .to("foo.com:project:somedataset.sometable") .withWriteDisposition(WriteDisposition.WRITE_TRUNCATE); @@ -828,7 +825,7 @@ public void testBuildSinkWithWriteDispositionTruncate() { } @Test - public void testBuildSinkWithWriteDispositionAppend() { + public void testBuildWriteWithWriteDispositionAppend() { BigQueryIO.Write.Bound bound = BigQueryIO.Write .to("foo.com:project:somedataset.sometable") .withWriteDisposition(WriteDisposition.WRITE_APPEND); @@ -838,7 +835,7 @@ public void testBuildSinkWithWriteDispositionAppend() { } @Test - public void testBuildSinkWithWriteDispositionEmpty() { + public void testBuildWriteWithWriteDispositionEmpty() { BigQueryIO.Write.Bound bound = BigQueryIO.Write .to("foo.com:project:somedataset.sometable") .withWriteDisposition(WriteDisposition.WRITE_EMPTY); @@ -848,7 +845,7 @@ public void testBuildSinkWithWriteDispositionEmpty() { } @Test - public void testBuildSinkDisplayData() { + public void testBuildWriteDisplayData() { String tableSpec = "project:dataset.table"; TableSchema schema = new TableSchema().set("col1", "type1").set("col2", "type2"); @@ -1256,53 +1253,53 @@ void cleanup(PipelineOptions options) throws Exception { @Test public void testWritePartitionEmptyData() throws Exception { - final long numFiles = 0; - final long fileSize = 0; + long numFiles = 0; + long fileSize = 0; // An empty file is created for no input data. One partition is needed. - final long expectedNumPartitions = 1; + long expectedNumPartitions = 1; testWritePartition(numFiles, fileSize, expectedNumPartitions); } @Test public void testWritePartitionSinglePartition() throws Exception { - final long numFiles = BigQueryIO.Write.Bound.MAX_NUM_FILES; - final long fileSize = 1; + long numFiles = BigQueryIO.Write.Bound.MAX_NUM_FILES; + long fileSize = 1; // One partition is needed. - final long expectedNumPartitions = 1; + long expectedNumPartitions = 1; testWritePartition(numFiles, fileSize, expectedNumPartitions); } @Test public void testWritePartitionManyFiles() throws Exception { - final long numFiles = BigQueryIO.Write.Bound.MAX_NUM_FILES * 3; - final long fileSize = 1; + long numFiles = BigQueryIO.Write.Bound.MAX_NUM_FILES * 3; + long fileSize = 1; // One partition is needed for each group of BigQueryWrite.MAX_NUM_FILES files. - final long expectedNumPartitions = 3; + long expectedNumPartitions = 3; testWritePartition(numFiles, fileSize, expectedNumPartitions); } @Test public void testWritePartitionLargeFileSize() throws Exception { - final long numFiles = 10; - final long fileSize = BigQueryIO.Write.Bound.MAX_SIZE_BYTES / 3; + long numFiles = 10; + long fileSize = BigQueryIO.Write.Bound.MAX_SIZE_BYTES / 3; // One partition is needed for each group of three files. - final long expectedNumPartitions = 4; + long expectedNumPartitions = 4; testWritePartition(numFiles, fileSize, expectedNumPartitions); } private void testWritePartition(long numFiles, long fileSize, long expectedNumPartitions) throws Exception { - final List expectedPartitionIds = Lists.newArrayList(); + List expectedPartitionIds = Lists.newArrayList(); for (long i = 1; i <= expectedNumPartitions; ++i) { expectedPartitionIds.add(i); } - final List> files = Lists.newArrayList(); - final List fileNames = Lists.newArrayList(); + List> files = Lists.newArrayList(); + List fileNames = Lists.newArrayList(); for (int i = 0; i < numFiles; ++i) { String fileName = String.format("files%05d", i); fileNames.add(fileName); @@ -1314,7 +1311,7 @@ private void testWritePartition(long numFiles, long fileSize, long expectedNumPa TupleTag>> singlePartitionTag = new TupleTag>>("singlePartitionTag") {}; - final PCollectionView>> filesView = PCollectionViews.iterableView( + PCollectionView>> filesView = PCollectionViews.iterableView( TestPipeline.create(), WindowingStrategy.globalDefault(), KvCoder.of(StringUtf8Coder.of(), VarLongCoder.of())); @@ -1324,7 +1321,7 @@ private void testWritePartition(long numFiles, long fileSize, long expectedNumPa DoFnTester>> tester = DoFnTester.of(writePartition); tester.setSideInput(filesView, GlobalWindow.INSTANCE, files); - tester.processElement(tmpFolder.getRoot().getAbsolutePath()); + tester.processElement(bqOptions.getTempLocation()); List>> partitions; if (expectedNumPartitions > 1) { @@ -1357,18 +1354,17 @@ public void testWriteTables() throws Exception { FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() .withJobService(new FakeJobService() .startJobReturns("done", "done", "done", "done") - .pollJobReturns(Status.FAILED, Status.SUCCEEDED, Status.SUCCEEDED, Status.SUCCEEDED)) - .withDatasetService(mockDatasetService); + .pollJobReturns(Status.FAILED, Status.SUCCEEDED, Status.SUCCEEDED, Status.SUCCEEDED)); - final long numPartitions = 3; - final long numFilesPerPartition = 10; - final String jobIdToken = "jobIdToken"; - final String tempFilePrefix = "tempFilePrefix"; - final String jsonTable = "{}"; - final String jsonSchema = "{}"; - final List expectedTempTables = Lists.newArrayList(); + long numPartitions = 3; + long numFilesPerPartition = 10; + String jobIdToken = "jobIdToken"; + String tempFilePrefix = "tempFilePrefix"; + String jsonTable = "{}"; + String jsonSchema = "{}"; + List expectedTempTables = Lists.newArrayList(); - final List>>> partitions = Lists.newArrayList(); + List>>> partitions = Lists.newArrayList(); for (long i = 0; i < numPartitions; ++i) { List filesPerPartition = Lists.newArrayList(); for (int j = 0; j < numFilesPerPartition; ++j) { @@ -1396,10 +1392,40 @@ public void testWriteTables() throws Exception { List tempTables = tester.takeOutputElements(); logged.verifyInfo("Starting BigQuery load job"); + logged.verifyInfo("BigQuery load job failed"); + logged.verifyInfo("try 0/" + BigQueryIO.Write.Bound.MAX_RETRY_JOBS); + logged.verifyInfo("try 1/" + BigQueryIO.Write.Bound.MAX_RETRY_JOBS); + logged.verifyNotLogged("try 2/" + BigQueryIO.Write.Bound.MAX_RETRY_JOBS); assertEquals(expectedTempTables, tempTables); } + @Test + public void testRemoveTemporaryFiles() throws Exception { + int numFiles = 10; + List fileNames = Lists.newArrayList(); + String tempFilePrefix = bqOptions.getTempLocation() + "/"; + TableRowWriter writer = new TableRowWriter(tempFilePrefix); + for (int i = 0; i < numFiles; ++i) { + String fileName = String.format("files%05d", i); + writer.open(fileName); + fileNames.add(writer.close().getKey()); + } + fileNames.add(tempFilePrefix + String.format("files%05d", numFiles)); + + File tempDir = new File(bqOptions.getTempLocation()); + testNumFiles(tempDir, 10); + + WriteTables.removeTemporaryFiles(bqOptions, tempFilePrefix, fileNames); + + testNumFiles(tempDir, 0); + + for (String fileName : fileNames) { + logged.verifyDebug("Removing file " + fileName); + } + logged.verifyDebug(fileNames.get(numFiles) + " does not exist."); + } + @Test public void testWriteRename() throws Exception { FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() @@ -1408,15 +1434,15 @@ public void testWriteRename() throws Exception { .pollJobReturns(Status.FAILED, Status.SUCCEEDED)) .withDatasetService(mockDatasetService); - final long numTempTables = 3; - final String jobIdToken = "jobIdToken"; - final String jsonTable = "{}"; - final List tempTables = Lists.newArrayList(); + long numTempTables = 3; + String jobIdToken = "jobIdToken"; + String jsonTable = "{}"; + List tempTables = Lists.newArrayList(); for (long i = 0; i < numTempTables; ++i) { tempTables.add(String.format("{\"tableId\":\"%s_%05d\"}", jobIdToken, i)); } - final PCollectionView> tempTablesView = PCollectionViews.iterableView( + PCollectionView> tempTablesView = PCollectionViews.iterableView( TestPipeline.create(), WindowingStrategy.globalDefault(), StringUtf8Coder.of()); @@ -1434,5 +1460,42 @@ public void testWriteRename() throws Exception { tester.processElement(null); logged.verifyInfo("Starting BigQuery copy job"); + logged.verifyInfo("BigQuery copy job failed"); + logged.verifyInfo("try 0/" + BigQueryIO.Write.Bound.MAX_RETRY_JOBS); + logged.verifyInfo("try 1/" + BigQueryIO.Write.Bound.MAX_RETRY_JOBS); + logged.verifyNotLogged("try 2/" + BigQueryIO.Write.Bound.MAX_RETRY_JOBS); + } + + @Test + public void testRemoveTemporaryTables() throws Exception { + String projectId = "someproject"; + String datasetId = "somedataset"; + List tables = Lists.newArrayList("table1", "table2", "table3"); + List tableRefs = Lists.newArrayList( + BigQueryIO.parseTableSpec(String.format("%s:%s.%s", projectId, datasetId, tables.get(0))), + BigQueryIO.parseTableSpec(String.format("%s:%s.%s", projectId, datasetId, tables.get(1))), + BigQueryIO.parseTableSpec(String.format("%s:%s.%s", projectId, datasetId, tables.get(2)))); + + doThrow(new IOException("Unable to delete table")) + .when(mockDatasetService).deleteTable(projectId, datasetId, tables.get(0)); + doNothing().when(mockDatasetService).deleteTable(projectId, datasetId, tables.get(1)); + doNothing().when(mockDatasetService).deleteTable(projectId, datasetId, tables.get(2)); + + WriteRename.removeTemporaryTables(mockDatasetService, tableRefs); + + for (TableReference ref : tableRefs) { + logged.verifyDebug("Deleting table " + toJsonString(ref)); + } + logged.verifyWarn("Failed to delete the table " + toJsonString(tableRefs.get(0))); + logged.verifyNotLogged("Failed to delete the table " + toJsonString(tableRefs.get(1))); + logged.verifyNotLogged("Failed to delete the table " + toJsonString(tableRefs.get(2))); + } + + private static void testNumFiles(File tempDir, int expectedNumFiles) { + assertEquals(expectedNumFiles, tempDir.listFiles(new FileFilter() { + @Override + public boolean accept(File pathname) { + return pathname.isFile(); + }}).length); } } From e21b1c594f4307b2bc5e615d40e1d67f209c527b Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Fri, 12 Aug 2016 17:51:02 +0200 Subject: [PATCH 013/112] [flink] add missing maven config to example pom --- runners/flink/README.md | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/runners/flink/README.md b/runners/flink/README.md index aeb16922a96b..8361112028c9 100644 --- a/runners/flink/README.md +++ b/runners/flink/README.md @@ -171,6 +171,14 @@ The contents of the root `pom.xml` should be slightly changed aftewards (explana beam-runners-flink_2.10 0.2.0-incubating-SNAPSHOT + + + + @@ -196,6 +204,15 @@ The contents of the root `pom.xml` should be slightly changed aftewards (explana + + org.apache.maven.plugins + maven-compiler-plugin + + 1.7 + 1.7 + + + From a83738ba5fc631cc9be8c5294963e2ac2e82429d Mon Sep 17 00:00:00 2001 From: Pei He Date: Mon, 1 Aug 2016 13:41:59 -0700 Subject: [PATCH 014/112] Remove DataflowPipelineJob from examples --- examples/java/pom.xml | 7 +- .../beam/examples/common/ExampleUtils.java | 74 +++++-------------- .../runners/dataflow/DataflowPipelineJob.java | 14 +++- 3 files changed, 31 insertions(+), 64 deletions(-) diff --git a/examples/java/pom.xml b/examples/java/pom.xml index cdf80cb8297f..6efbc54406d9 100644 --- a/examples/java/pom.xml +++ b/examples/java/pom.xml @@ -219,11 +219,6 @@ google-api-client - - com.google.apis - google-api-services-dataflow - - com.google.apis google-api-services-bigquery @@ -286,6 +281,8 @@ org.apache.beam beam-runners-google-cloud-dataflow-java ${project.version} + runtime + true diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/ExampleUtils.java b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleUtils.java index 8b66861107ad..7f03fc0d1c6f 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/common/ExampleUtils.java +++ b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleUtils.java @@ -17,17 +17,10 @@ */ package org.apache.beam.examples.common; -import org.apache.beam.runners.dataflow.BlockingDataflowRunner; -import org.apache.beam.runners.dataflow.DataflowPipelineJob; -import org.apache.beam.runners.dataflow.DataflowRunner; -import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; -import org.apache.beam.runners.dataflow.util.MonitoringUtil; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.options.BigQueryOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PubsubOptions; -import org.apache.beam.sdk.options.StreamingOptions; -import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; import org.apache.beam.sdk.util.Transport; @@ -44,7 +37,6 @@ import com.google.api.services.bigquery.model.Table; import com.google.api.services.bigquery.model.TableReference; import com.google.api.services.bigquery.model.TableSchema; -import com.google.api.services.dataflow.Dataflow; import com.google.api.services.pubsub.Pubsub; import com.google.api.services.pubsub.model.Subscription; import com.google.api.services.pubsub.model.Topic; @@ -71,8 +63,7 @@ public class ExampleUtils { private final PipelineOptions options; private Bigquery bigQueryClient = null; private Pubsub pubsubClient = null; - private Dataflow dataflowClient = null; - private Set jobsToCancel = Sets.newHashSet(); + private Set pipelinesToCancel = Sets.newHashSet(); private List pendingMessages = Lists.newArrayList(); /** @@ -80,7 +71,6 @@ public class ExampleUtils { */ public ExampleUtils(PipelineOptions options) { this.options = options; - setupRunner(); } /** @@ -280,72 +270,47 @@ private void deletePubsubSubscription(String subscription) throws IOException { } } - /** - * Do some runner setup: check that the DirectRunner is not used in conjunction with - * streaming, and if streaming is specified, use the DataflowRunner. - */ - private void setupRunner() { - Class> runner = options.getRunner(); - if (options.as(StreamingOptions.class).isStreaming() - && runner.equals(BlockingDataflowRunner.class)) { - // In order to cancel the pipelines automatically, - // {@literal DataflowRunner} is forced to be used. - options.setRunner(DataflowRunner.class); - } - } - /** * If {@literal DataflowRunner} or {@literal BlockingDataflowRunner} is used, * waits for the pipeline to finish and cancels it (and the injector) before the program exists. */ public void waitToFinish(PipelineResult result) { - if (result instanceof DataflowPipelineJob) { - final DataflowPipelineJob job = (DataflowPipelineJob) result; - jobsToCancel.add(job); - if (!options.as(ExampleOptions.class).getKeepJobsRunning()) { - addShutdownHook(jobsToCancel); - } - try { - job.waitUntilFinish(); - } catch (Exception e) { - throw new RuntimeException("Failed to wait for job to finish: " + job.getJobId()); - } - } else { + pipelinesToCancel.add(result); + if (!options.as(ExampleOptions.class).getKeepJobsRunning()) { + addShutdownHook(pipelinesToCancel); + } + try { + result.waitUntilFinish(); + } catch (UnsupportedOperationException e) { // Do nothing if the given PipelineResult doesn't support waitUntilFinish(), // such as EvaluationResults returned by DirectRunner. tearDown(); printPendingMessages(); + } catch (Exception e) { + throw new RuntimeException("Failed to wait the pipeline until finish: " + result); } } - private void addShutdownHook(final Collection jobs) { - if (dataflowClient == null) { - dataflowClient = options.as(DataflowPipelineOptions.class).getDataflowClient(); - } - + private void addShutdownHook(final Collection pipelineResults) { Runtime.getRuntime().addShutdownHook(new Thread() { @Override public void run() { tearDown(); printPendingMessages(); - for (DataflowPipelineJob job : jobs) { - System.out.println("Canceling example pipeline: " + job.getJobId()); + for (PipelineResult pipelineResult : pipelineResults) { try { - job.cancel(); + pipelineResult.cancel(); } catch (IOException e) { - System.out.println("Failed to cancel the job," - + " please go to the Developers Console to cancel it manually"); - System.out.println( - MonitoringUtil.getJobMonitoringPageURL(job.getProjectId(), job.getJobId())); + System.out.println("Failed to cancel the job."); + System.out.println(e.getMessage()); } } - for (DataflowPipelineJob job : jobs) { + for (PipelineResult pipelineResult : pipelineResults) { boolean cancellationVerified = false; for (int retryAttempts = 6; retryAttempts > 0; retryAttempts--) { - if (job.getState().isTerminal()) { + if (pipelineResult.getState().isTerminal()) { cancellationVerified = true; - System.out.println("Canceled example pipeline: " + job.getJobId()); break; } else { System.out.println( @@ -354,10 +319,7 @@ public void run() { Uninterruptibles.sleepUninterruptibly(10, TimeUnit.SECONDS); } if (!cancellationVerified) { - System.out.println("Failed to verify the cancellation for job: " + job.getJobId()); - System.out.println("Please go to the Developers Console to verify manually:"); - System.out.println( - MonitoringUtil.getJobMonitoringPageURL(job.getProjectId(), job.getJobId())); + System.out.println("Failed to verify the cancellation for job: " + pipelineResult); } } } diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java index e043e23ad162..3d0f1458fbae 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java @@ -271,9 +271,17 @@ public State cancel() throws IOException { content.setProjectId(projectId); content.setId(jobId); content.setRequestedState("JOB_STATE_CANCELLED"); - dataflowOptions.getDataflowClient().projects().jobs() - .update(projectId, jobId, content) - .execute(); + try { + dataflowOptions.getDataflowClient().projects().jobs() + .update(projectId, jobId, content) + .execute(); + } catch (IOException e) { + String errorMsg = String.format( + "Failed to cancel the job, please go to the Developers Console to cancel it manually: %s", + MonitoringUtil.getJobMonitoringPageURL(getProjectId(), getJobId())); + LOG.warn(errorMsg); + throw new IOException(errorMsg, e); + } return State.CANCELLED; } From 54b0514371a1028b271d03cc309e6f66fb909e28 Mon Sep 17 00:00:00 2001 From: mariusz89016 Date: Sun, 14 Aug 2016 00:35:19 +0200 Subject: [PATCH 015/112] [BEAM-432] Corrected BigQueryIO javadoc --- .../java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java index 46c2935942ee..aa168bdab0db 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java @@ -177,7 +177,7 @@ * *

{@code
  * PCollection shakespeare = pipeline.apply(
- *     BigQueryIO.Read.fromQuery("SELECT year, mean_temp FROM samples.weather_stations"));
+ *     BigQueryIO.Read.fromQuery("SELECT year, mean_temp FROM [samples.weather_stations]"));
  * }
* *

When creating a BigQuery input transform, users should provide either a query or a table. From 12b19677280c11b0dca203ef266769b05c90937e Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Fri, 15 Jul 2016 11:27:00 -0700 Subject: [PATCH 016/112] Add TransformEvaluatorFactory#cleanup This cleans up any state stored within the Transform Evaluator Factory. --- .../direct/BoundedReadEvaluatorFactory.java | 4 ++ .../ExecutorServiceParallelExecutor.java | 9 +++- .../direct/FlattenEvaluatorFactory.java | 3 ++ .../GroupAlsoByWindowEvaluatorFactory.java | 6 ++- .../GroupByKeyOnlyEvaluatorFactory.java | 4 +- .../direct/ParDoMultiEvaluatorFactory.java | 5 +++ .../direct/ParDoSingleEvaluatorFactory.java | 5 +++ .../direct/TransformEvaluatorFactory.java | 8 ++++ .../direct/TransformEvaluatorRegistry.java | 41 +++++++++++++++++++ .../direct/UnboundedReadEvaluatorFactory.java | 3 ++ .../runners/direct/ViewEvaluatorFactory.java | 3 ++ .../direct/WindowEvaluatorFactory.java | 3 ++ 12 files changed, 90 insertions(+), 4 deletions(-) diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java index 2f4f86c1ee18..0c4b7fd9bc80 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java @@ -60,6 +60,10 @@ public TransformEvaluator forApplication( return getTransformEvaluator((AppliedPTransform) application, evaluationContext); } + @Override + public void cleanup() { + } + /** * Get a {@link TransformEvaluator} that produces elements for the provided application of * {@link Bounded Read.Bounded}, initializing the queue of evaluators if required. diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java index a0a5ec0e5df3..8c6c6eddb3b3 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java @@ -447,13 +447,18 @@ private void fireTimers() throws Exception { private boolean shouldShutdown() { boolean shouldShutdown = exceptionThrown || evaluationContext.isDone(); if (shouldShutdown) { + LOG.debug("Pipeline has terminated. Shutting down."); + executorService.shutdown(); + try { + registry.cleanup(); + } catch (Exception e) { + visibleUpdates.add(VisibleExecutorUpdate.fromThrowable(e)); + } if (evaluationContext.isDone()) { - LOG.debug("Pipeline is finished. Shutting down. {}"); while (!visibleUpdates.offer(VisibleExecutorUpdate.finished())) { visibleUpdates.poll(); } } - executorService.shutdown(); } return shouldShutdown; } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/FlattenEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/FlattenEvaluatorFactory.java index c84f62035a86..5a0d31dee27d 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/FlattenEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/FlattenEvaluatorFactory.java @@ -43,6 +43,9 @@ public TransformEvaluator forApplication( return evaluator; } + @Override + public void cleanup() throws Exception {} + private TransformEvaluator createInMemoryEvaluator( final AppliedPTransform< PCollectionList, PCollection, FlattenPCollectionList> diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java index e052226c7e5b..d16ffa0bf517 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java @@ -61,11 +61,15 @@ public TransformEvaluator forApplication( return evaluator; } + @Override + public void cleanup() {} + private TransformEvaluator> createEvaluator( AppliedPTransform< PCollection>, PCollection>>, - DirectGroupAlsoByWindow> application, + DirectGroupAlsoByWindow> + application, CommittedBundle> inputBundle, EvaluationContext evaluationContext) { return new GroupAlsoByWindowEvaluator<>( diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java index 0e419c3959e2..dbdbdaf4ea34 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java @@ -18,7 +18,6 @@ package org.apache.beam.runners.direct; import static org.apache.beam.sdk.util.CoderUtils.encodeToByteArray; - import static com.google.common.base.Preconditions.checkState; import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly; @@ -61,6 +60,9 @@ public TransformEvaluator forApplication( return evaluator; } + @Override + public void cleanup() {} + private TransformEvaluator>> createEvaluator( final AppliedPTransform< PCollection>>, diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java index ce770cae37ca..40533c0072af 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java @@ -70,6 +70,11 @@ public TransformEvaluator forApplication( return evaluator; } + @Override + public void cleanup() { + + } + private TransformEvaluator createMultiEvaluator( AppliedPTransform, PCollectionTuple, BoundMulti> application, CommittedBundle inputBundle, diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java index 53af6af7cdf5..201fb46b7453 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java @@ -69,6 +69,11 @@ public TransformEvaluator forApplication( return evaluator; } + @Override + public void cleanup() { + + } + private TransformEvaluator createSingleEvaluator( AppliedPTransform, PCollection, Bound> application, diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorFactory.java index d021b43037e5..3655d26dd553 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorFactory.java @@ -51,4 +51,12 @@ public interface TransformEvaluatorFactory { @Nullable TransformEvaluator forApplication( AppliedPTransform application, @Nullable CommittedBundle inputBundle, EvaluationContext evaluationContext) throws Exception; + + /** + * Cleans up any state maintained by this {@link TransformEvaluatorFactory}. Called after a + * {@link Pipeline} is shut down. No more calls to + * {@link #forApplication(AppliedPTransform, CommittedBundle, EvaluationContext)} will be made + * after a call to {@link #cleanup()}. + */ + void cleanup() throws Exception; } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java index f0afc3be5d7a..b46923775c48 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java @@ -17,6 +17,8 @@ */ package org.apache.beam.runners.direct; +import static com.google.common.base.Preconditions.checkState; + import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupAlsoByWindow; import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; @@ -29,7 +31,13 @@ import com.google.common.collect.ImmutableMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collection; import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; import javax.annotation.Nullable; @@ -38,6 +46,7 @@ * implementations based on the type of {@link PTransform} of the application. */ class TransformEvaluatorRegistry implements TransformEvaluatorFactory { + private static final Logger LOG = LoggerFactory.getLogger(TransformEvaluatorRegistry.class); public static TransformEvaluatorRegistry defaultRegistry() { @SuppressWarnings("rawtypes") ImmutableMap, TransformEvaluatorFactory> primitives = @@ -61,6 +70,8 @@ public static TransformEvaluatorRegistry defaultRegistry() { @SuppressWarnings("rawtypes") private final Map, TransformEvaluatorFactory> factories; + private final AtomicBoolean finished = new AtomicBoolean(false); + private TransformEvaluatorRegistry( @SuppressWarnings("rawtypes") Map, TransformEvaluatorFactory> factories) { @@ -73,7 +84,37 @@ public TransformEvaluator forApplication( @Nullable CommittedBundle inputBundle, EvaluationContext evaluationContext) throws Exception { + checkState( + !finished.get(), "Tried to get an evaluator for a finished TransformEvaluatorRegistry"); TransformEvaluatorFactory factory = factories.get(application.getTransform().getClass()); return factory.forApplication(application, inputBundle, evaluationContext); } + + @Override + public void cleanup() throws Exception { + Collection thrownInCleanup = new ArrayList<>(); + for (TransformEvaluatorFactory factory : factories.values()) { + try { + factory.cleanup(); + } catch (Exception e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + thrownInCleanup.add(e); + } + } + finished.set(true); + if (!thrownInCleanup.isEmpty()) { + LOG.error("Exceptions {} thrown while cleaning up evaluators", thrownInCleanup); + Exception toThrow = null; + for (Exception e : thrownInCleanup) { + if (toThrow == null) { + toThrow = e; + } else { + toThrow.addSuppressed(e); + } + } + throw toThrow; + } + } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java index 0e2745b8bde3..c4d408b232f0 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java @@ -113,6 +113,9 @@ TransformEvaluator getTransformEvaluator( return evaluatorQueue.poll(); } + @Override + public void cleanup() {} + /** * A {@link UnboundedReadEvaluator} produces elements from an underlying {@link UnboundedSource}, * discarding all input elements. Within the call to {@link #finishBundle()}, the evaluator diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java index 362e903a0630..3b0de4b6f8df 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java @@ -59,6 +59,9 @@ public TransformEvaluator forApplication( return evaluator; } + @Override + public void cleanup() throws Exception {} + private TransformEvaluator> createEvaluator( final AppliedPTransform>, PCollectionView, WriteView> application, diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java index 67c2f173f0ed..f2e62cb0445d 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java @@ -66,6 +66,9 @@ private TransformEvaluator createTransformEvaluator( return new WindowIntoEvaluator<>(transform, fn, outputBundle); } + @Override + public void cleanup() {} + private static class WindowIntoEvaluator implements TransformEvaluator { private final AppliedPTransform, PCollection, Window.Bound> transform; From cf0bf3bf9fcab2b01d69ff90d9ba3f602a8a5bd4 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Tue, 19 Jul 2016 11:03:15 -0700 Subject: [PATCH 017/112] Replace CloningThreadLocal with DoFnLifecycleManager This is a more focused interface that interacts with a DoFn before it is available for use and after it has completed and the reference is lost. It is required to properly support setup and teardown, as the fields in a ThreadLocal cannot all be cleaned up without additional tracking. Part of BEAM-452. --- .../runners/direct/CloningThreadLocal.java | 43 ------- .../runners/direct/DoFnLifecycleManager.java | 78 ++++++++++++ ...cleManagerRemovingTransformEvaluator.java} | 41 ++++-- .../direct/ParDoMultiEvaluatorFactory.java | 56 +++++---- .../direct/ParDoSingleEvaluatorFactory.java | 43 ++++--- .../direct/CloningThreadLocalTest.java | 92 -------------- ...anagerRemovingTransformEvaluatorTest.java} | 41 +++--- .../direct/DoFnLifecycleManagerTest.java | 119 ++++++++++++++++++ 8 files changed, 305 insertions(+), 208 deletions(-) delete mode 100644 runners/direct-java/src/main/java/org/apache/beam/runners/direct/CloningThreadLocal.java create mode 100644 runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java rename runners/direct-java/src/main/java/org/apache/beam/runners/direct/{ThreadLocalInvalidatingTransformEvaluator.java => DoFnLifecycleManagerRemovingTransformEvaluator.java} (53%) delete mode 100644 runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningThreadLocalTest.java rename runners/direct-java/src/test/java/org/apache/beam/runners/direct/{ThreadLocalInvalidatingTransformEvaluatorTest.java => DoFnLifecycleManagerRemovingTransformEvaluatorTest.java} (73%) create mode 100644 runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CloningThreadLocal.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CloningThreadLocal.java deleted file mode 100644 index b9dc4ca715ac..000000000000 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CloningThreadLocal.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.beam.runners.direct; - -import org.apache.beam.sdk.util.SerializableUtils; - -import java.io.Serializable; - -/** - * A {@link ThreadLocal} that obtains the initial value by cloning an original value. - */ -class CloningThreadLocal extends ThreadLocal { - public static CloningThreadLocal of(T original) { - return new CloningThreadLocal<>(original); - } - - private final T original; - - private CloningThreadLocal(T original) { - this.original = original; - } - - @Override - public T initialValue() { - return SerializableUtils.clone(original); - } -} diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java new file mode 100644 index 000000000000..27836577b3a7 --- /dev/null +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.runners.direct; + +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.OldDoFn; +import org.apache.beam.sdk.util.SerializableUtils; + +import com.google.common.cache.CacheBuilder; +import com.google.common.cache.CacheLoader; +import com.google.common.cache.LoadingCache; + +/** + * Manages {@link DoFn} setup, teardown, and serialization. + * + *

{@link DoFnLifecycleManager} is similar to a {@link ThreadLocal} storing a {@link DoFn}, but + * calls the {@link DoFn} {@link Setup} the first time the {@link DoFn} is obtained and {@link + * Teardown} whenever the {@link DoFn} is removed, and provides a method for clearing all cached + * {@link DoFn DoFns}. + */ +class DoFnLifecycleManager { + public static DoFnLifecycleManager of(OldDoFn original) { + return new DoFnLifecycleManager(original); + } + + private final LoadingCache> outstanding; + + private DoFnLifecycleManager(OldDoFn original) { + this.outstanding = CacheBuilder.newBuilder().build(new DeserializingCacheLoader(original)); + } + + public OldDoFn get() throws Exception { + Thread currentThread = Thread.currentThread(); + return outstanding.get(currentThread); + } + + public void remove() throws Exception { + Thread currentThread = Thread.currentThread(); + outstanding.invalidate(currentThread); + } + + /** + * Remove all {@link DoFn DoFns} from this {@link DoFnLifecycleManager}. + */ + public void removeAll() throws Exception { + outstanding.invalidateAll(); + } + + private class DeserializingCacheLoader extends CacheLoader> { + private final byte[] original; + + public DeserializingCacheLoader(OldDoFn original) { + this.original = SerializableUtils.serializeToByteArray(original); + } + + @Override + public OldDoFn load(Thread key) throws Exception { + return (OldDoFn) SerializableUtils.deserializeFromByteArray(original, + "DoFn Copy in thread " + key.getName()); + } + } +} diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ThreadLocalInvalidatingTransformEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluator.java similarity index 53% rename from runners/direct-java/src/main/java/org/apache/beam/runners/direct/ThreadLocalInvalidatingTransformEvaluator.java rename to runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluator.java index d8a6bf9e7998..f3d1d4f3d584 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ThreadLocalInvalidatingTransformEvaluator.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluator.java @@ -20,25 +20,28 @@ import org.apache.beam.sdk.util.WindowedValue; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + /** * A {@link TransformEvaluator} which delegates calls to an underlying {@link TransformEvaluator}, - * clearing the value of a {@link ThreadLocal} if any call throws an exception. + * clearing the value of a {@link DoFnLifecycleManager} if any call throws an exception. */ -class ThreadLocalInvalidatingTransformEvaluator - implements TransformEvaluator { +class DoFnLifecycleManagerRemovingTransformEvaluator implements TransformEvaluator { + private static final Logger LOG = + LoggerFactory.getLogger(DoFnLifecycleManagerRemovingTransformEvaluator.class); private final TransformEvaluator underlying; - private final ThreadLocal threadLocal; + private final DoFnLifecycleManager lifecycleManager; public static TransformEvaluator wrapping( - TransformEvaluator underlying, - ThreadLocal threadLocal) { - return new ThreadLocalInvalidatingTransformEvaluator<>(underlying, threadLocal); + TransformEvaluator underlying, DoFnLifecycleManager threadLocal) { + return new DoFnLifecycleManagerRemovingTransformEvaluator<>(underlying, threadLocal); } - private ThreadLocalInvalidatingTransformEvaluator( - TransformEvaluator underlying, ThreadLocal threadLocal) { + private DoFnLifecycleManagerRemovingTransformEvaluator( + TransformEvaluator underlying, DoFnLifecycleManager threadLocal) { this.underlying = underlying; - this.threadLocal = threadLocal; + this.lifecycleManager = threadLocal; } @Override @@ -46,7 +49,14 @@ public void processElement(WindowedValue element) throws Exception { try { underlying.processElement(element); } catch (Exception e) { - threadLocal.remove(); + try { + lifecycleManager.remove(); + } catch (Exception removalException) { + LOG.error( + "Exception encountered while cleaning up after processing an element", + removalException); + e.addSuppressed(removalException); + } throw e; } } @@ -56,7 +66,14 @@ public TransformResult finishBundle() throws Exception { try { return underlying.finishBundle(); } catch (Exception e) { - threadLocal.remove(); + try { + lifecycleManager.remove(); + } catch (Exception removalException) { + LOG.error( + "Exception encountered while cleaning up after finishing a bundle", + removalException); + e.addSuppressed(removalException); + } throw e; } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java index 40533c0072af..f2455e1f069e 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java @@ -31,6 +31,9 @@ import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.util.Map; /** @@ -38,32 +41,26 @@ * {@link BoundMulti} primitive {@link PTransform}. */ class ParDoMultiEvaluatorFactory implements TransformEvaluatorFactory { - private final LoadingCache>, ThreadLocal>> + private static final Logger LOG = LoggerFactory.getLogger(ParDoMultiEvaluatorFactory.class); + private final LoadingCache>, DoFnLifecycleManager> fnClones; public ParDoMultiEvaluatorFactory() { - fnClones = - CacheBuilder.newBuilder() - .build( - new CacheLoader< - AppliedPTransform>, ThreadLocal>>() { - @Override - public ThreadLocal> load( - AppliedPTransform> key) - throws Exception { - @SuppressWarnings({"unchecked", "rawtypes"}) - ThreadLocal threadLocal = - (ThreadLocal) CloningThreadLocal.of(key.getTransform().getFn()); - return threadLocal; - } - }); + fnClones = CacheBuilder.newBuilder() + .build(new CacheLoader>, DoFnLifecycleManager>() { + @Override + public DoFnLifecycleManager load(AppliedPTransform> key) + throws Exception { + return DoFnLifecycleManager.of(key.getTransform().getFn()); + } + }); } @Override public TransformEvaluator forApplication( AppliedPTransform application, CommittedBundle inputBundle, - EvaluationContext evaluationContext) { + EvaluationContext evaluationContext) throws Exception { @SuppressWarnings({"unchecked", "rawtypes"}) TransformEvaluator evaluator = createMultiEvaluator((AppliedPTransform) application, inputBundle, evaluationContext); @@ -71,38 +68,45 @@ public TransformEvaluator forApplication( } @Override - public void cleanup() { - + public void cleanup() throws Exception { + for (DoFnLifecycleManager lifecycleManager : fnClones.asMap().values()) { + lifecycleManager.removeAll(); + } } private TransformEvaluator createMultiEvaluator( AppliedPTransform, PCollectionTuple, BoundMulti> application, CommittedBundle inputBundle, - EvaluationContext evaluationContext) { + EvaluationContext evaluationContext) throws Exception { Map, PCollection> outputs = application.getOutput().getAll(); - @SuppressWarnings({"unchecked", "rawtypes"}) - ThreadLocal> fnLocal = - (ThreadLocal) fnClones.getUnchecked((AppliedPTransform) application); + DoFnLifecycleManager fnLocal = fnClones.getUnchecked((AppliedPTransform) application); String stepName = evaluationContext.getStepName(application); DirectStepContext stepContext = evaluationContext.getExecutionContext(application, inputBundle.getKey()) .getOrCreateStepContext(stepName, stepName); try { + @SuppressWarnings({"unchecked", "rawtypes"}) TransformEvaluator parDoEvaluator = ParDoEvaluator.create( evaluationContext, stepContext, inputBundle, application, - fnLocal.get(), + (OldDoFn) fnLocal.get(), application.getTransform().getSideInputs(), application.getTransform().getMainOutputTag(), application.getTransform().getSideOutputTags().getAll(), outputs); - return ThreadLocalInvalidatingTransformEvaluator.wrapping(parDoEvaluator, fnLocal); + return DoFnLifecycleManagerRemovingTransformEvaluator.wrapping(parDoEvaluator, fnLocal); } catch (Exception e) { - fnLocal.remove(); + try { + fnLocal.remove(); + } catch (Exception removalException) { + LOG.error("Exception encountered while cleaning up in ParDo evaluator construction", + removalException); + e.addSuppressed(removalException); + } throw e; } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java index 201fb46b7453..a0fbd1d71514 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java @@ -31,6 +31,9 @@ import com.google.common.cache.LoadingCache; import com.google.common.collect.ImmutableMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.util.Collections; /** @@ -38,22 +41,18 @@ * {@link Bound ParDo.Bound} primitive {@link PTransform}. */ class ParDoSingleEvaluatorFactory implements TransformEvaluatorFactory { - private final LoadingCache>, ThreadLocal>> - fnClones; + private static final Logger LOG = LoggerFactory.getLogger(ParDoSingleEvaluatorFactory.class); + private final LoadingCache>, DoFnLifecycleManager> fnClones; public ParDoSingleEvaluatorFactory() { fnClones = CacheBuilder.newBuilder() .build( - new CacheLoader< - AppliedPTransform>, ThreadLocal>>() { + new CacheLoader>, DoFnLifecycleManager>() { @Override - public ThreadLocal> load(AppliedPTransform> key) + public DoFnLifecycleManager load(AppliedPTransform> key) throws Exception { - @SuppressWarnings({"unchecked", "rawtypes"}) - ThreadLocal threadLocal = - (ThreadLocal) CloningThreadLocal.of(key.getTransform().getFn()); - return threadLocal; + return DoFnLifecycleManager.of(key.getTransform().getFn()); } }); } @@ -62,7 +61,7 @@ public ParDoSingleEvaluatorFactory() { public TransformEvaluator forApplication( final AppliedPTransform application, CommittedBundle inputBundle, - EvaluationContext evaluationContext) { + EvaluationContext evaluationContext) throws Exception { @SuppressWarnings({"unchecked", "rawtypes"}) TransformEvaluator evaluator = createSingleEvaluator((AppliedPTransform) application, inputBundle, evaluationContext); @@ -70,39 +69,45 @@ public TransformEvaluator forApplication( } @Override - public void cleanup() { - + public void cleanup() throws Exception { + for (DoFnLifecycleManager lifecycleManager : fnClones.asMap().values()) { + lifecycleManager.removeAll(); + } } private TransformEvaluator createSingleEvaluator( AppliedPTransform, PCollection, Bound> application, CommittedBundle inputBundle, - EvaluationContext evaluationContext) { + EvaluationContext evaluationContext) throws Exception { TupleTag mainOutputTag = new TupleTag<>("out"); String stepName = evaluationContext.getStepName(application); DirectStepContext stepContext = evaluationContext.getExecutionContext(application, inputBundle.getKey()) .getOrCreateStepContext(stepName, stepName); - @SuppressWarnings({"unchecked", "rawtypes"}) - ThreadLocal> fnLocal = - (ThreadLocal) fnClones.getUnchecked((AppliedPTransform) application); + DoFnLifecycleManager fnLocal = fnClones.getUnchecked((AppliedPTransform) application); try { + @SuppressWarnings({"unchecked", "rawtypes"}) ParDoEvaluator parDoEvaluator = ParDoEvaluator.create( evaluationContext, stepContext, inputBundle, application, - fnLocal.get(), + (OldDoFn) fnLocal.get(), application.getTransform().getSideInputs(), mainOutputTag, Collections.>emptyList(), ImmutableMap., PCollection>of(mainOutputTag, application.getOutput())); - return ThreadLocalInvalidatingTransformEvaluator.wrapping(parDoEvaluator, fnLocal); + return DoFnLifecycleManagerRemovingTransformEvaluator.wrapping(parDoEvaluator, fnLocal); } catch (Exception e) { - fnLocal.remove(); + try { + fnLocal.remove(); + } catch (Exception removalException) { + LOG.error("Exception encountered constructing ParDo evaluator", removalException); + e.addSuppressed(removalException); + } throw e; } } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningThreadLocalTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningThreadLocalTest.java deleted file mode 100644 index 298db46bec13..000000000000 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningThreadLocalTest.java +++ /dev/null @@ -1,92 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.beam.runners.direct; - -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.nullValue; -import static org.hamcrest.core.IsNot.not; -import static org.hamcrest.core.IsSame.theInstance; -import static org.junit.Assert.assertThat; - -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -import java.io.Serializable; -import java.util.concurrent.Callable; -import java.util.concurrent.Executors; - -/** - * Tests for {@link CloningThreadLocalTest}. - */ -@RunWith(JUnit4.class) -public class CloningThreadLocalTest { - @Test - public void returnsCopiesOfOriginal() throws Exception { - Record original = new Record(); - ThreadLocal loaded = CloningThreadLocal.of(original); - assertThat(loaded.get(), not(nullValue())); - assertThat(loaded.get(), equalTo(original)); - assertThat(loaded.get(), not(theInstance(original))); - } - - @Test - public void returnsDifferentCopiesInDifferentThreads() throws Exception { - Record original = new Record(); - final ThreadLocal loaded = CloningThreadLocal.of(original); - assertThat(loaded.get(), not(nullValue())); - assertThat(loaded.get(), equalTo(original)); - assertThat(loaded.get(), not(theInstance(original))); - - Callable otherThread = - new Callable() { - @Override - public Record call() throws Exception { - return loaded.get(); - } - }; - Record sameThread = loaded.get(); - Record firstOtherThread = Executors.newSingleThreadExecutor().submit(otherThread).get(); - Record secondOtherThread = Executors.newSingleThreadExecutor().submit(otherThread).get(); - - assertThat(sameThread, equalTo(firstOtherThread)); - assertThat(sameThread, equalTo(secondOtherThread)); - assertThat(sameThread, not(theInstance(firstOtherThread))); - assertThat(sameThread, not(theInstance(secondOtherThread))); - assertThat(firstOtherThread, not(theInstance(secondOtherThread))); - } - - private static class Record implements Serializable { - private final double rand = Math.random(); - - @Override - public boolean equals(Object other) { - if (!(other instanceof Record)) { - return false; - } - Record that = (Record) other; - return this.rand == that.rand; - } - - @Override - public int hashCode() { - return 1; - } - } -} diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ThreadLocalInvalidatingTransformEvaluatorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java similarity index 73% rename from runners/direct-java/src/test/java/org/apache/beam/runners/direct/ThreadLocalInvalidatingTransformEvaluatorTest.java rename to runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java index 6e477d37c83c..67f4ff47789f 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ThreadLocalInvalidatingTransformEvaluatorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java @@ -25,8 +25,10 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; +import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.util.WindowedValue; +import org.hamcrest.Matchers; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -36,24 +38,23 @@ import java.util.List; /** - * Tests for {@link ThreadLocalInvalidatingTransformEvaluator}. + * Tests for {@link DoFnLifecycleManagerRemovingTransformEvaluator}. */ @RunWith(JUnit4.class) -public class ThreadLocalInvalidatingTransformEvaluatorTest { - private ThreadLocal threadLocal; +public class DoFnLifecycleManagerRemovingTransformEvaluatorTest { + private DoFnLifecycleManager lifecycleManager; @Before public void setup() { - threadLocal = new ThreadLocal<>(); - threadLocal.set(new Object()); + lifecycleManager = DoFnLifecycleManager.of(new TestFn()); } @Test public void delegatesToUnderlying() throws Exception { RecordingTransformEvaluator underlying = new RecordingTransformEvaluator(); - Object original = threadLocal.get(); + OldDoFn original = lifecycleManager.get(); TransformEvaluator evaluator = - ThreadLocalInvalidatingTransformEvaluator.wrapping(underlying, threadLocal); + DoFnLifecycleManagerRemovingTransformEvaluator.wrapping(underlying, lifecycleManager); WindowedValue first = WindowedValue.valueInGlobalWindow(new Object()); WindowedValue second = WindowedValue.valueInGlobalWindow(new Object()); evaluator.processElement(first); @@ -66,35 +67,36 @@ public void delegatesToUnderlying() throws Exception { } @Test - public void removesOnExceptionInProcessElement() { + public void removesOnExceptionInProcessElement() throws Exception { ThrowingTransformEvaluator underlying = new ThrowingTransformEvaluator(); - Object original = threadLocal.get(); + OldDoFn original = lifecycleManager.get(); assertThat(original, not(nullValue())); TransformEvaluator evaluator = - ThreadLocalInvalidatingTransformEvaluator.wrapping(underlying, threadLocal); + DoFnLifecycleManagerRemovingTransformEvaluator.wrapping(underlying, lifecycleManager); try { evaluator.processElement(WindowedValue.valueInGlobalWindow(new Object())); } catch (Exception e) { - assertThat(threadLocal.get(), nullValue()); + assertThat(lifecycleManager.get(), not(Matchers.>theInstance(original))); return; } fail("Expected ThrowingTransformEvaluator to throw on method call"); } @Test - public void removesOnExceptionInFinishBundle() { + public void removesOnExceptionInFinishBundle() throws Exception { ThrowingTransformEvaluator underlying = new ThrowingTransformEvaluator(); - Object original = threadLocal.get(); - // the ThreadLocal is set when the evaluator starts + OldDoFn original = lifecycleManager.get(); + // the LifecycleManager is set when the evaluator starts assertThat(original, not(nullValue())); TransformEvaluator evaluator = - ThreadLocalInvalidatingTransformEvaluator.wrapping(underlying, threadLocal); + DoFnLifecycleManagerRemovingTransformEvaluator.wrapping(underlying, lifecycleManager); try { evaluator.finishBundle(); } catch (Exception e) { - assertThat(threadLocal.get(), nullValue()); + assertThat(lifecycleManager.get(), + Matchers.not(Matchers.>theInstance(original))); return; } fail("Expected ThrowingTransformEvaluator to throw on method call"); @@ -132,4 +134,11 @@ public TransformResult finishBundle() throws Exception { throw new Exception(); } } + + + private static class TestFn extends OldDoFn { + @Override + public void processElement(ProcessContext c) throws Exception { + } + } } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java new file mode 100644 index 000000000000..f316e19d4879 --- /dev/null +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.runners.direct; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.not; +import static org.hamcrest.Matchers.theInstance; +import static org.junit.Assert.assertThat; + +import org.apache.beam.sdk.transforms.OldDoFn; + +import org.hamcrest.Matchers; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; + +/** + * Tests for {@link DoFnLifecycleManager}. + */ +public class DoFnLifecycleManagerTest { + private TestFn fn = new TestFn(); + private DoFnLifecycleManager mgr = DoFnLifecycleManager.of(fn); + + @Test + public void setupOnGet() throws Exception { + TestFn obtained = (TestFn) mgr.get(); + + assertThat(obtained, not(theInstance(fn))); + } + + @Test + public void getMultipleCallsSingleSetupCall() throws Exception { + TestFn obtained = (TestFn) mgr.get(); + TestFn secondObtained = (TestFn) mgr.get(); + + assertThat(obtained, theInstance(secondObtained)); + } + + @Test + public void getMultipleThreadsDifferentInstances() throws Exception { + CountDownLatch startSignal = new CountDownLatch(1); + ExecutorService executor = Executors.newCachedThreadPool(); + List> futures = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + futures.add(executor.submit(new GetFnCallable(mgr, startSignal))); + } + startSignal.countDown(); + List fns = new ArrayList<>(); + for (Future future : futures) { + fns.add(future.get(1L, TimeUnit.SECONDS)); + } + + for (TestFn fn : fns) { + int sameInstances = 0; + for (TestFn otherFn : fns) { + if (otherFn == fn) { + sameInstances++; + } + } + assertThat(sameInstances, equalTo(1)); + } + } + + @Test + public void teardownOnRemove() throws Exception { + TestFn obtained = (TestFn) mgr.get(); + mgr.remove(); + + assertThat(obtained, not(theInstance(fn))); + + assertThat(mgr.get(), not(Matchers.>theInstance(obtained))); + } + + private static class GetFnCallable implements Callable { + private final DoFnLifecycleManager mgr; + private final CountDownLatch startSignal; + + private GetFnCallable(DoFnLifecycleManager mgr, CountDownLatch startSignal) { + this.mgr = mgr; + this.startSignal = startSignal; + } + + @Override + public TestFn call() throws Exception { + startSignal.await(); + return (TestFn) mgr.get(); + } + } + + + private static class TestFn extends OldDoFn { + @Override + public void processElement(ProcessContext c) throws Exception { + } + } +} From 12abb1b02246b8d36021c7b1a970daf1b64ba4b9 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Thu, 14 Jul 2016 14:51:02 -0700 Subject: [PATCH 018/112] Add DoFn @Setup and @Teardown Methods annotated with these annotations are used to perform expensive setup work and clean up a DoFn after another method throws an exception or the DoFn is discarded. --- .../runners/direct/DoFnLifecycleManager.java | 38 +- ...ycleManagerRemovingTransformEvaluator.java | 39 +- .../runners/direct/DoFnLifecycleManagers.java | 45 ++ .../direct/ParDoMultiEvaluatorFactory.java | 4 +- .../direct/ParDoSingleEvaluatorFactory.java | 4 +- .../direct/DoFnLifecycleManagerTest.java | 49 ++ .../direct/DoFnLifecycleManagersTest.java | 142 ++++++ .../functions/FlinkDoFnFunction.java | 12 +- .../FlinkMultiOutputDoFnFunction.java | 31 +- .../streaming/FlinkAbstractParDoWrapper.java | 2 + .../FlinkGroupAlsoByWindowWrapper.java | 2 + .../spark/translation/DoFnFunction.java | 23 +- .../spark/translation/MultiDoFnFunction.java | 1 + .../translation/SparkProcessContext.java | 17 + .../org/apache/beam/sdk/transforms/DoFn.java | 31 +- .../beam/sdk/transforms/DoFnReflector.java | 70 ++- .../apache/beam/sdk/transforms/OldDoFn.java | 25 ++ .../org/apache/beam/sdk/transforms/ParDo.java | 15 +- .../sdk/transforms/DoFnReflectorTest.java | 65 +++ .../apache/beam/sdk/transforms/ParDoTest.java | 420 +++++++++++++++++- 20 files changed, 970 insertions(+), 65 deletions(-) create mode 100644 runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManagers.java create mode 100644 runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagersTest.java diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java index 27836577b3a7..3f4f2c6a880d 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java @@ -18,6 +18,7 @@ package org.apache.beam.runners.direct; +import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.util.SerializableUtils; @@ -26,6 +27,13 @@ import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Iterator; + /** * Manages {@link DoFn} setup, teardown, and serialization. * @@ -35,6 +43,8 @@ * {@link DoFn DoFns}. */ class DoFnLifecycleManager { + private static final Logger LOG = LoggerFactory.getLogger(DoFnLifecycleManager.class); + public static DoFnLifecycleManager of(OldDoFn original) { return new DoFnLifecycleManager(original); } @@ -52,14 +62,30 @@ private DoFnLifecycleManager(OldDoFn original) { public void remove() throws Exception { Thread currentThread = Thread.currentThread(); - outstanding.invalidate(currentThread); + OldDoFn fn = outstanding.asMap().remove(currentThread); + fn.teardown(); } /** - * Remove all {@link DoFn DoFns} from this {@link DoFnLifecycleManager}. + * Remove all {@link DoFn DoFns} from this {@link DoFnLifecycleManager}. Returns all exceptions + * that were thrown while calling the remove methods. + * + *

If the returned Collection is nonempty, an exception was thrown from at least one + * {@link DoFn#teardown()} method, and the {@link PipelineRunner} should throw an exception. */ - public void removeAll() throws Exception { - outstanding.invalidateAll(); + public Collection removeAll() throws Exception { + Iterator> fns = outstanding.asMap().values().iterator(); + Collection thrown = new ArrayList<>(); + while (fns.hasNext()) { + OldDoFn fn = fns.next(); + fns.remove(); + try { + fn.teardown(); + } catch (Exception e) { + thrown.add(e); + } + } + return thrown; } private class DeserializingCacheLoader extends CacheLoader> { @@ -71,8 +97,10 @@ public DeserializingCacheLoader(OldDoFn original) { @Override public OldDoFn load(Thread key) throws Exception { - return (OldDoFn) SerializableUtils.deserializeFromByteArray(original, + OldDoFn fn = (OldDoFn) SerializableUtils.deserializeFromByteArray(original, "DoFn Copy in thread " + key.getName()); + fn.setup(); + return fn; } } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluator.java index f3d1d4f3d584..523273cdd1ec 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluator.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluator.java @@ -34,14 +34,14 @@ class DoFnLifecycleManagerRemovingTransformEvaluator implements Transfor private final DoFnLifecycleManager lifecycleManager; public static TransformEvaluator wrapping( - TransformEvaluator underlying, DoFnLifecycleManager threadLocal) { - return new DoFnLifecycleManagerRemovingTransformEvaluator<>(underlying, threadLocal); + TransformEvaluator underlying, DoFnLifecycleManager lifecycleManager) { + return new DoFnLifecycleManagerRemovingTransformEvaluator<>(underlying, lifecycleManager); } private DoFnLifecycleManagerRemovingTransformEvaluator( - TransformEvaluator underlying, DoFnLifecycleManager threadLocal) { + TransformEvaluator underlying, DoFnLifecycleManager lifecycleManager) { this.underlying = underlying; - this.lifecycleManager = threadLocal; + this.lifecycleManager = lifecycleManager; } @Override @@ -49,14 +49,7 @@ public void processElement(WindowedValue element) throws Exception { try { underlying.processElement(element); } catch (Exception e) { - try { - lifecycleManager.remove(); - } catch (Exception removalException) { - LOG.error( - "Exception encountered while cleaning up after processing an element", - removalException); - e.addSuppressed(removalException); - } + onException(e, "Exception encountered while cleaning up after processing an element"); throw e; } } @@ -66,15 +59,21 @@ public TransformResult finishBundle() throws Exception { try { return underlying.finishBundle(); } catch (Exception e) { - try { - lifecycleManager.remove(); - } catch (Exception removalException) { - LOG.error( - "Exception encountered while cleaning up after finishing a bundle", - removalException); - e.addSuppressed(removalException); - } + onException(e, "Exception encountered while cleaning up after finishing a bundle"); throw e; } } + + private void onException(Exception e, String msg) { + try { + lifecycleManager.remove(); + } catch (Exception removalException) { + if (removalException instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + LOG.error(msg, removalException); + e.addSuppressed(removalException); + } + } + } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManagers.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManagers.java new file mode 100644 index 000000000000..6a1dd8f24fa1 --- /dev/null +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManagers.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.runners.direct; + +import java.util.ArrayList; +import java.util.Collection; + +/** + * Utility methods for interacting with {@link DoFnLifecycleManager DoFnLifecycleManagers}. + */ +class DoFnLifecycleManagers { + private DoFnLifecycleManagers() { + /* Do not instantiate */ + } + + static void removeAllFromManagers(Iterable managers) throws Exception { + Collection thrown = new ArrayList<>(); + for (DoFnLifecycleManager manager : managers) { + thrown.addAll(manager.removeAll()); + } + if (!thrown.isEmpty()) { + Exception overallException = new Exception("Exceptions thrown while tearing down DoFns"); + for (Exception e : thrown) { + overallException.addSuppressed(e); + } + throw overallException; + } + } +} diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java index f2455e1f069e..2d05e68b6659 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java @@ -69,9 +69,7 @@ public TransformEvaluator forApplication( @Override public void cleanup() throws Exception { - for (DoFnLifecycleManager lifecycleManager : fnClones.asMap().values()) { - lifecycleManager.removeAll(); - } + DoFnLifecycleManagers.removeAllFromManagers(fnClones.asMap().values()); } private TransformEvaluator createMultiEvaluator( diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java index a0fbd1d71514..97cbfa7f7b2f 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java @@ -70,9 +70,7 @@ public TransformEvaluator forApplication( @Override public void cleanup() throws Exception { - for (DoFnLifecycleManager lifecycleManager : fnClones.asMap().values()) { - lifecycleManager.removeAll(); - } + DoFnLifecycleManagers.removeAllFromManagers(fnClones.asMap().values()); } private TransformEvaluator createSingleEvaluator( diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java index f316e19d4879..77b32968f8b4 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java @@ -18,7 +18,9 @@ package org.apache.beam.runners.direct; +import static com.google.common.base.Preconditions.checkState; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.theInstance; import static org.junit.Assert.assertThat; @@ -49,6 +51,8 @@ public void setupOnGet() throws Exception { TestFn obtained = (TestFn) mgr.get(); assertThat(obtained, not(theInstance(fn))); + assertThat(obtained.setupCalled, is(true)); + assertThat(obtained.teardownCalled, is(false)); } @Test @@ -57,6 +61,8 @@ public void getMultipleCallsSingleSetupCall() throws Exception { TestFn secondObtained = (TestFn) mgr.get(); assertThat(obtained, theInstance(secondObtained)); + assertThat(obtained.setupCalled, is(true)); + assertThat(obtained.teardownCalled, is(false)); } @Test @@ -74,6 +80,7 @@ public void getMultipleThreadsDifferentInstances() throws Exception { } for (TestFn fn : fns) { + assertThat(fn.setupCalled, is(true)); int sameInstances = 0; for (TestFn otherFn : fns) { if (otherFn == fn) { @@ -90,10 +97,33 @@ public void teardownOnRemove() throws Exception { mgr.remove(); assertThat(obtained, not(theInstance(fn))); + assertThat(obtained.setupCalled, is(true)); + assertThat(obtained.teardownCalled, is(true)); assertThat(mgr.get(), not(Matchers.>theInstance(obtained))); } + @Test + public void teardownAllOnRemoveAll() throws Exception { + CountDownLatch startSignal = new CountDownLatch(1); + ExecutorService executor = Executors.newCachedThreadPool(); + List> futures = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + futures.add(executor.submit(new GetFnCallable(mgr, startSignal))); + } + startSignal.countDown(); + List fns = new ArrayList<>(); + for (Future future : futures) { + fns.add(future.get(1L, TimeUnit.SECONDS)); + } + mgr.removeAll(); + + for (TestFn fn : fns) { + assertThat(fn.setupCalled, is(true)); + assertThat(fn.teardownCalled, is(true)); + } + } + private static class GetFnCallable implements Callable { private final DoFnLifecycleManager mgr; private final CountDownLatch startSignal; @@ -112,8 +142,27 @@ public TestFn call() throws Exception { private static class TestFn extends OldDoFn { + boolean setupCalled = false; + boolean teardownCalled = false; + + @Override + public void setup() { + checkState(!setupCalled); + checkState(!teardownCalled); + + setupCalled = true; + } + @Override public void processElement(ProcessContext c) throws Exception { } + + @Override + public void teardown() { + checkState(setupCalled); + checkState(!teardownCalled); + + teardownCalled = true; + } } } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagersTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagersTest.java new file mode 100644 index 000000000000..8be3d52763be --- /dev/null +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagersTest.java @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.runners.direct; + +import static org.hamcrest.Matchers.equalTo; + +import org.apache.beam.sdk.transforms.OldDoFn; + +import com.google.common.collect.ImmutableList; + +import org.hamcrest.BaseMatcher; +import org.hamcrest.Description; +import org.hamcrest.Matcher; +import org.hamcrest.Matchers; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.ArrayList; +import java.util.Collection; + +/** + * Tests for {@link DoFnLifecycleManagers}. + */ +@RunWith(JUnit4.class) +public class DoFnLifecycleManagersTest { + @Rule public ExpectedException thrown = ExpectedException.none(); + + @Test + public void removeAllWhenManagersThrowSuppressesAndThrows() throws Exception { + DoFnLifecycleManager first = DoFnLifecycleManager.of(new ThrowsInCleanupFn("foo")); + DoFnLifecycleManager second = DoFnLifecycleManager.of(new ThrowsInCleanupFn("bar")); + DoFnLifecycleManager third = DoFnLifecycleManager.of(new ThrowsInCleanupFn("baz")); + first.get(); + second.get(); + third.get(); + + final Collection> suppressions = new ArrayList<>(); + suppressions.add(new ThrowableMessageMatcher("foo")); + suppressions.add(new ThrowableMessageMatcher("bar")); + suppressions.add(new ThrowableMessageMatcher("baz")); + + thrown.expect( + new BaseMatcher() { + @Override + public void describeTo(Description description) { + description + .appendText("Exception suppressing ") + .appendList("[", ", ", "]", suppressions); + } + + @Override + public boolean matches(Object item) { + if (!(item instanceof Exception)) { + return false; + } + Exception that = (Exception) item; + return Matchers.containsInAnyOrder(suppressions) + .matches(ImmutableList.copyOf(that.getSuppressed())); + } + }); + + DoFnLifecycleManagers.removeAllFromManagers(ImmutableList.of(first, second, third)); + } + + @Test + public void whenManagersSucceedSucceeds() throws Exception { + DoFnLifecycleManager first = DoFnLifecycleManager.of(new EmptyFn()); + DoFnLifecycleManager second = DoFnLifecycleManager.of(new EmptyFn()); + DoFnLifecycleManager third = DoFnLifecycleManager.of(new EmptyFn()); + first.get(); + second.get(); + third.get(); + + DoFnLifecycleManagers.removeAllFromManagers(ImmutableList.of(first, second, third)); + } + + private static class ThrowsInCleanupFn extends OldDoFn { + private final String message; + + private ThrowsInCleanupFn(String message) { + this.message = message; + } + + @Override + public void processElement(ProcessContext c) throws Exception { + } + + @Override + public void teardown() throws Exception { + throw new Exception(message); + } + } + + + private static class ThrowableMessageMatcher extends BaseMatcher { + private final Matcher messageMatcher; + + public ThrowableMessageMatcher(String message) { + this.messageMatcher = equalTo(message); + } + + @Override + public boolean matches(Object item) { + if (!(item instanceof Throwable)) { + return false; + } + Throwable that = (Throwable) item; + return messageMatcher.matches(that.getMessage()); + } + + @Override + public void describeTo(Description description) { + description.appendText("a throwable with a message ").appendDescriptionOf(messageMatcher); + } + } + + + private static class EmptyFn extends OldDoFn { + @Override + public void processElement(ProcessContext c) throws Exception { + } + } +} diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java index a4af1b025a42..fdf1e59e06d3 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java @@ -25,6 +25,7 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.flink.api.common.functions.RichMapPartitionFunction; +import org.apache.flink.configuration.Configuration; import org.apache.flink.util.Collector; import java.util.Map; @@ -86,7 +87,7 @@ public void mapPartition( // side inputs and window access also only works if an element // is in only one window for (WindowedValue value : values) { - for (WindowedValue explodedValue: value.explodeWindows()) { + for (WindowedValue explodedValue : value.explodeWindows()) { context = context.forWindowedValue(value); doFn.processElement(context); } @@ -99,4 +100,13 @@ public void mapPartition( this.doFn.finishBundle(context); } + @Override + public void open(Configuration parameters) throws Exception { + doFn.setup(); + } + + @Override + public void close() throws Exception { + doFn.teardown(); + } } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputDoFnFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputDoFnFunction.java index 6e673fc1df95..5013b90d22d5 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputDoFnFunction.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputDoFnFunction.java @@ -27,6 +27,7 @@ import org.apache.beam.sdk.values.TupleTag; import org.apache.flink.api.common.functions.RichMapPartitionFunction; +import org.apache.flink.configuration.Configuration; import org.apache.flink.util.Collector; import java.util.Map; @@ -75,14 +76,15 @@ public void mapPartition( Iterable> values, Collector> out) throws Exception { - FlinkProcessContext context = new FlinkMultiOutputProcessContext<>( - serializedOptions.getPipelineOptions(), - getRuntimeContext(), - doFn, - windowingStrategy, - out, - outputMap, - sideInputs); + FlinkProcessContext context = + new FlinkMultiOutputProcessContext<>( + serializedOptions.getPipelineOptions(), + getRuntimeContext(), + doFn, + windowingStrategy, + out, + outputMap, + sideInputs); this.doFn.startBundle(context); @@ -97,14 +99,23 @@ public void mapPartition( // side inputs and window access also only works if an element // is in only one window for (WindowedValue value : values) { - for (WindowedValue explodedValue: value.explodeWindows()) { + for (WindowedValue explodedValue : value.explodeWindows()) { context = context.forWindowedValue(value); doFn.processElement(context); } } } - this.doFn.finishBundle(context); } + + @Override + public void open(Configuration parameters) throws Exception { + doFn.setup(); + } + + @Override + public void close() throws Exception { + doFn.teardown(); + } } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java index 74ec66a3155b..a9dd865bf2b3 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java @@ -70,6 +70,7 @@ public FlinkAbstractParDoWrapper(PipelineOptions options, WindowingStrategy(output), this.timerInternals); operator.startBundle(context); } @@ -351,6 +352,7 @@ public void processWatermark(Watermark mark) throws Exception { @Override public void close() throws Exception { operator.finishBundle(context); + operator.teardown(); super.close(); } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java index f4ce51633362..c08d1850d412 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java @@ -24,6 +24,8 @@ import org.apache.beam.sdk.values.TupleTag; import org.apache.spark.api.java.function.FlatMapFunction; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.Iterator; import java.util.LinkedList; @@ -40,6 +42,8 @@ public class DoFnFunction implements FlatMapFunction>, WindowedValue> { private final OldDoFn mFunction; + private static final Logger LOG = LoggerFactory.getLogger(DoFnFunction.class); + private final SparkRuntimeContext mRuntimeContext; private final Map, BroadcastHelper> mSideInputs; @@ -61,8 +65,23 @@ public Iterable> call(Iterator> ite Exception { ProcCtxt ctxt = new ProcCtxt(mFunction, mRuntimeContext, mSideInputs); ctxt.setup(); - mFunction.startBundle(ctxt); - return ctxt.getOutputIterable(iter, mFunction); + try { + mFunction.setup(); + mFunction.startBundle(ctxt); + return ctxt.getOutputIterable(iter, mFunction); + } catch (Exception e) { + try { + // this teardown handles exceptions encountered in setup() and startBundle(). teardown + // after execution or due to exceptions in process element is called in the iterator + // produced by ctxt.getOutputIterable returned from this method. + mFunction.teardown(); + } catch (Exception teardownException) { + LOG.error( + "Suppressing exception while tearing down Function {}", mFunction, teardownException); + e.addSuppressed(teardownException); + } + throw e; + } } private class ProcCtxt extends SparkProcessContext> { diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java index e33578df83d5..abf0e8379a2e 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java @@ -65,6 +65,7 @@ class MultiDoFnFunction public Iterable, WindowedValue>> call(Iterator> iter) throws Exception { ProcCtxt ctxt = new ProcCtxt(mFunction, mRuntimeContext, mSideInputs); + mFunction.setup(); mFunction.startBundle(ctxt); ctxt.setup(); return ctxt.getOutputIterable(iter, mFunction); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java index 2f06a1c8508a..1cdbd92010d8 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java @@ -238,6 +238,7 @@ protected ValueT computeNext() { try { doFn.processElement(SparkProcessContext.this); } catch (Exception e) { + handleProcessingException(e); throw new SparkProcessException(e); } outputIterator = getOutputIterator(); @@ -249,15 +250,31 @@ protected ValueT computeNext() { calledFinish = true; doFn.finishBundle(SparkProcessContext.this); } catch (Exception e) { + handleProcessingException(e); throw new SparkProcessException(e); } outputIterator = getOutputIterator(); continue; // try to consume outputIterator from start of loop } + try { + doFn.teardown(); + } catch (Exception e) { + LOG.error( + "Suppressing teardown exception that occurred after processing entire input", e); + } return endOfData(); } } } + + private void handleProcessingException(Exception e) { + try { + doFn.teardown(); + } catch (Exception e1) { + LOG.error("Exception while cleaning up DoFn", e1); + e.addSuppressed(e1); + } + } } /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java index a06467e9b381..80b67afc8412 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java @@ -342,6 +342,20 @@ public OutputReceiver outputReceiver() { ///////////////////////////////////////////////////////////////////////////// + + /** + * Annotation for the method to use to prepare an instance for processing bundles of elements. The + * method annotated with this must satisfy the following constraints + *

    + *
  • It must have zero arguments. + *
+ */ + @Documented + @Retention(RetentionPolicy.RUNTIME) + @Target(ElementType.METHOD) + public @interface Setup { + } + /** * Annotation for the method to use to prepare an instance for processing a batch of elements. * The method annotated with this must satisfy the following constraints: @@ -371,7 +385,7 @@ public OutputReceiver outputReceiver() { public @interface ProcessElement {} /** - * Annotation for the method to use to prepare an instance for processing a batch of elements. + * Annotation for the method to use to finish processing a batch of elements. * The method annotated with this must satisfy the following constraints: *
    *
  • It must have at least one argument. @@ -383,6 +397,21 @@ public OutputReceiver outputReceiver() { @Target(ElementType.METHOD) public @interface FinishBundle {} + + /** + * Annotation for the method to use to clean up this instance after processing bundles of + * elements. No other method will be called after a call to the annotated method is made. + * The method annotated with this must satisfy the following constraint: + *
      + *
    • It must have zero arguments. + *
    + */ + @Documented + @Retention(RetentionPolicy.RUNTIME) + @Target(ElementType.METHOD) + public @interface Teardown { + } + /** * Returns an {@link Aggregator} with aggregation logic specified by the * {@link CombineFn} argument. The name provided must be unique across diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnReflector.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnReflector.java index 3dfda55caed4..bf040412b5e0 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnReflector.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnReflector.java @@ -17,11 +17,15 @@ */ package org.apache.beam.sdk.transforms; +import static com.google.common.base.Preconditions.checkState; + import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn.ExtraContextFactory; import org.apache.beam.sdk.transforms.DoFn.FinishBundle; import org.apache.beam.sdk.transforms.DoFn.ProcessElement; +import org.apache.beam.sdk.transforms.DoFn.Setup; import org.apache.beam.sdk.transforms.DoFn.StartBundle; +import org.apache.beam.sdk.transforms.DoFn.Teardown; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; @@ -302,6 +306,15 @@ static List verifyBundleMethodArguments(M new TypeParameter() {}); } + @VisibleForTesting + static void verifyLifecycleMethodArguments(Method m) { + if (m == null) { + return; + } + checkState(void.class.equals(m.getReturnType()), "%s must have void return type", format(m)); + checkState(m.getGenericParameterTypes().length == 0, "%s must take zero arguments", format(m)); + } + /** * Verify the method arguments for a given {@link DoFn} method. * @@ -392,6 +405,8 @@ static List verifyMethodArguments( /** Interface for invoking the {@code OldDoFn} processing methods. */ public interface DoFnInvoker { + /** Invoke {@link OldDoFn#setup} on the bound {@code OldDoFn}. */ + void invokeSetup(); /** Invoke {@link OldDoFn#startBundle} on the bound {@code OldDoFn}. */ void invokeStartBundle( DoFn.Context c, @@ -401,6 +416,9 @@ void invokeFinishBundle( DoFn.Context c, ExtraContextFactory extra); + /** Invoke {@link OldDoFn#teardown()} on the bound {@code DoFn}. */ + void invokeTeardown(); + /** Invoke {@link OldDoFn#processElement} on the bound {@code OldDoFn}. */ public void invokeProcessElement( DoFn.ProcessContext c, @@ -412,9 +430,11 @@ public void invokeProcessElement( */ private static class GenericDoFnReflector extends DoFnReflector { + private final Method setup; private final Method startBundle; private final Method processElement; private final Method finishBundle; + private final Method teardown; private final List processElementArgs; private final List startBundleArgs; private final List finishBundleArgs; @@ -424,13 +444,17 @@ private GenericDoFnReflector( @SuppressWarnings("rawtypes") Class fn) { // Locate the annotated methods this.processElement = findAnnotatedMethod(ProcessElement.class, fn, true); + this.setup = findAnnotatedMethod(Setup.class, fn, false); this.startBundle = findAnnotatedMethod(StartBundle.class, fn, false); this.finishBundle = findAnnotatedMethod(FinishBundle.class, fn, false); + this.teardown = findAnnotatedMethod(Teardown.class, fn, false); // Verify that their method arguments satisfy our conditions. this.processElementArgs = verifyProcessMethodArguments(processElement); this.startBundleArgs = verifyBundleMethodArguments(startBundle); this.finishBundleArgs = verifyBundleMethodArguments(finishBundle); + verifyLifecycleMethodArguments(setup); + verifyLifecycleMethodArguments(teardown); this.constructor = createInvokerConstructor(fn); } @@ -552,8 +576,17 @@ public String subclass(Generic superClass) { .intercept(InvokerDelegation.create( startBundle, BeforeDelegation.INVOKE_PREPARE_FOR_PROCESSING, startBundleArgs)) .method(ElementMatchers.named("invokeFinishBundle")) - .intercept(InvokerDelegation.create( - finishBundle, BeforeDelegation.NOOP, finishBundleArgs)); + .intercept(InvokerDelegation.create(finishBundle, + BeforeDelegation.NOOP, + finishBundleArgs)) + .method(ElementMatchers.named("invokeSetup")) + .intercept(InvokerDelegation.create(setup, + BeforeDelegation.NOOP, + Collections.emptyList())) + .method(ElementMatchers.named("invokeTeardown")) + .intercept(InvokerDelegation.create(teardown, + BeforeDelegation.NOOP, + Collections.emptyList())); @SuppressWarnings("unchecked") Class> dynamicClass = (Class>) builder @@ -735,6 +768,11 @@ private SimpleDoFnAdapter(DoFnReflector reflector, DoFn fn) { this.invoker = reflector.bindInvoker(fn); } + @Override + public void setup() throws Exception { + invoker.invokeSetup(); + } + @Override public void startBundle(OldDoFn.Context c) throws Exception { ContextAdapter adapter = new ContextAdapter<>(fn, c); @@ -747,6 +785,11 @@ public void finishBundle(OldDoFn.Context c) throws Exception { invoker.invokeFinishBundle(adapter, adapter); } + @Override + public void teardown() { + invoker.invokeTeardown(); + } + @Override public void processElement(OldDoFn.ProcessContext c) throws Exception { ProcessContextAdapter adapter = new ProcessContextAdapter<>(fn, c); @@ -940,15 +983,20 @@ private StackManipulation invokeTargetMethod(MethodDescription instrumentedMetho new MethodDescription.ForLoadedMethod(target)).resolve(instrumentedMethod); ParameterList params = targetMethod.getParameters(); - // Instructions to setup the parameters for the call - ArrayList parameters = new ArrayList<>(args.size() + 1); - // 1. The first argument in the delegate method must be the context. This corresponds to - // the first argument in the instrumented method, so copy that. - parameters.add(MethodVariableAccess.of( - params.get(0).getType().getSuperClass()).loadOffset(1)); - // 2. For each of the extra arguments push the appropriate value. - for (AdditionalParameter arg : args) { - parameters.add(pushArgument(arg, instrumentedMethod)); + List parameters; + if (!params.isEmpty()) { + // Instructions to setup the parameters for the call + parameters = new ArrayList<>(args.size() + 1); + // 1. The first argument in the delegate method must be the context. This corresponds to + // the first argument in the instrumented method, so copy that. + parameters.add(MethodVariableAccess.of(params.get(0).getType().getSuperClass()) + .loadOffset(1)); + // 2. For each of the extra arguments push the appropriate value. + for (AdditionalParameter arg : args) { + parameters.add(pushArgument(arg, instrumentedMethod)); + } + } else { + parameters = Collections.emptyList(); } return new StackManipulation.Compound( diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java index 443599ac282a..84cd9978c621 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java @@ -338,6 +338,17 @@ public OldDoFn() { */ private boolean aggregatorsAreFinal; + /** + * Prepares this {@link DoFn} instance for processing bundles. + * + *

    {@link #setup()} will be called at most once per {@link DoFn} instance, and before any other + * {@link DoFn} method is called. + * + *

    By default, does nothing. + */ + public void setup() throws Exception { + } + /** * Prepares this {@code OldDoFn} instance for processing a batch of elements. * @@ -372,6 +383,20 @@ public void startBundle(Context c) throws Exception { public void finishBundle(Context c) throws Exception { } + /** + * Cleans up this {@link DoFn}. + * + *

    {@link #teardown()} will be called before the {@link PipelineRunner} discards a {@link DoFn} + * instance, including due to another {@link DoFn} method throwing an {@link Exception}. No other + * {@link DoFn} methods will be called after a call to {@link #teardown()}. + * + *

    By default, does nothing. + */ + public void teardown() throws Exception { + } + + ///////////////////////////////////////////////////////////////////////////// + /** * {@inheritDoc} * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java index ca6d9b2e0031..aa5753197b1d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java @@ -67,11 +67,11 @@ * For each bundle of input elements processing proceeds as follows: * *

      - *
    1. If required, a fresh instance of the argument {@link OldDoFn} is created - * on a worker. This may be through deserialization or other means. A - * {@link PipelineRunner} may reuse {@link OldDoFn} instances for multiple bundles. - * A {@link OldDoFn} that has terminated abnormally (by throwing an {@link Exception} - * will never be reused.
    2. + *
    3. If required, a fresh instance of the argument {@link DoFn} is created + * on a worker, and {@link DoFn#setup()} is called on this instance. This may be through + * deserialization or other means. A {@link PipelineRunner} may reuse {@link DoFn} instances for + * multiple bundles. A {@link DoFn} that has terminated abnormally (by throwing an + * {@link Exception}) will never be reused.
    4. *
    5. The {@link OldDoFn OldDoFn's} {@link OldDoFn#startBundle} method is called to * initialize it. If this method is not overridden, the call may be optimized * away.
    6. @@ -83,6 +83,11 @@ * {@link OldDoFn#finishBundle} * until a new call to {@link OldDoFn#startBundle} has occurred. * If this method is not overridden, this call may be optimized away. + *
    7. If any of {@link DoFn#setup}, {@link DoFn#startBundle}, {@link DoFn#processElement} or + * {@link DoFn#finishBundle} throw an exception, {@link DoFn#teardown} will be called on the + * {@link DoFn} instance.
    8. + *
    9. If a runner will no longer use a {@link DoFn}, {@link DoFn#teardown()} will be called on + * the discarded instance.
    10. *
    * * Each of the calls to any of the {@link OldDoFn OldDoFn's} processing diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnReflectorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnReflectorTest.java index c47e0cf5a9d8..e05e5e2c667a 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnReflectorTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnReflectorTest.java @@ -25,6 +25,8 @@ import org.apache.beam.sdk.transforms.DoFn.ExtraContextFactory; import org.apache.beam.sdk.transforms.DoFn.ProcessContext; import org.apache.beam.sdk.transforms.DoFn.ProcessElement; +import org.apache.beam.sdk.transforms.DoFn.Setup; +import org.apache.beam.sdk.transforms.DoFn.Teardown; import org.apache.beam.sdk.transforms.dofnreflector.DoFnReflectorTestHelper; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.UserCodeException; @@ -53,6 +55,8 @@ public static class Invocations { public boolean wasProcessElementInvoked = false; public boolean wasStartBundleInvoked = false; public boolean wasFinishBundleInvoked = false; + public boolean wasSetupInvoked = false; + public boolean wasTeardownInvoked = false; private final String name; public Invocations(String name) { @@ -144,6 +148,33 @@ private void checkInvokeFinishBundleWorks( } } + private void checkInvokeSetupWorks(DoFnReflector r, Invocations... invocations) throws Exception { + assertTrue("Need at least one invocation to check", invocations.length >= 1); + for (Invocations invocation : invocations) { + assertFalse("Should not yet have called setup on " + invocation.name, + invocation.wasSetupInvoked); + } + r.bindInvoker(fn).invokeSetup(); + for (Invocations invocation : invocations) { + assertTrue("Should have called setup on " + invocation.name, + invocation.wasSetupInvoked); + } + } + + private void checkInvokeTeardownWorks(DoFnReflector r, Invocations... invocations) + throws Exception { + assertTrue("Need at least one invocation to check", invocations.length >= 1); + for (Invocations invocation : invocations) { + assertFalse("Should not yet have called teardown on " + invocation.name, + invocation.wasTeardownInvoked); + } + r.bindInvoker(fn).invokeTeardown(); + for (Invocations invocation : invocations) { + assertTrue("Should have called teardown on " + invocation.name, + invocation.wasTeardownInvoked); + } + } + @Test public void testDoFnWithNoExtraContext() throws Exception { final Invocations invocations = new Invocations("AnonymousClass"); @@ -324,6 +355,40 @@ public void finishBundle(Context c) { checkInvokeFinishBundleWorks(reflector, invocations); } + @Test + public void testDoFnWithSetupTeardown() throws Exception { + final Invocations invocations = new Invocations("AnonymousClass"); + DoFnReflector reflector = underTest(new DoFn() { + @ProcessElement + public void processElement(@SuppressWarnings("unused") ProcessContext c) {} + + @StartBundle + public void startBundle(Context c) { + invocations.wasStartBundleInvoked = true; + assertSame(c, mockContext); + } + + @FinishBundle + public void finishBundle(Context c) { + invocations.wasFinishBundleInvoked = true; + assertSame(c, mockContext); + } + + @Setup + public void before() { + invocations.wasSetupInvoked = true; + } + + @Teardown + public void after() { + invocations.wasTeardownInvoked = true; + } + }); + + checkInvokeSetupWorks(reflector, invocations); + checkInvokeTeardownWorks(reflector, invocations); + } + @Test public void testNoProcessElement() throws Exception { thrown.expect(IllegalStateException.class); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java index 7fe053c29ecc..846012486cad 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java @@ -24,17 +24,18 @@ import static org.apache.beam.sdk.util.SerializableUtils.serializeToByteArray; import static org.apache.beam.sdk.util.StringUtils.byteArrayToJsonString; import static org.apache.beam.sdk.util.StringUtils.jsonStringToByteArray; - import static com.google.common.base.Preconditions.checkNotNull; - import static org.hamcrest.Matchers.allOf; import static org.hamcrest.Matchers.anyOf; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.is; import static org.hamcrest.collection.IsIterableContainingInAnyOrder.containsInAnyOrder; import static org.hamcrest.collection.IsIterableContainingInOrder.contains; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; +import static org.junit.Assert.fail; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.AtomicCoder; @@ -53,6 +54,7 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.common.ElementByteSizeObserver; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TimestampedValue; @@ -60,6 +62,7 @@ import org.apache.beam.sdk.values.TupleTagList; import com.fasterxml.jackson.annotation.JsonCreator; + import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Rule; @@ -77,6 +80,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; /** * Tests for ParDo. @@ -169,8 +173,10 @@ public void processElement(OldDoFn.ProcessContext c) throws Exc } static class TestDoFn extends DoFn { - enum State { UNSTARTED, STARTED, PROCESSING, FINISHED } - State state = State.UNSTARTED; + enum State {NOT_SET_UP, UNSTARTED, STARTED, PROCESSING, FINISHED} + + + State state = State.NOT_SET_UP; final List> sideInputViews = new ArrayList<>(); final List> sideOutputTupleTags = new ArrayList<>(); @@ -184,6 +190,12 @@ public TestDoFn(List> sideInputViews, this.sideOutputTupleTags.addAll(sideOutputTupleTags); } + @Setup + public void prepare() { + assertEquals(State.NOT_SET_UP, state); + state = State.UNSTARTED; + } + @StartBundle public void startBundle(Context c) { assertEquals(State.UNSTARTED, state); @@ -1463,4 +1475,404 @@ public void populateDisplayData(Builder builder) { assertThat(displayData, includesDisplayDataFrom(fn)); assertThat(displayData, hasDisplayItem("fn", fn.getClass())); } + + @Test + @Category(RunnableOnService.class) + public void testFnCallSequence() { + TestPipeline p = TestPipeline.create(); + PCollectionList.of(p.apply("Impolite", Create.of(1, 2, 4))) + .and(p.apply("Polite", Create.of(3, 5, 6, 7))) + .apply(Flatten.pCollections()) + .apply(ParDo.of(new CallSequenceEnforcingOldFn())); + + p.run(); + } + + @Test + @Category(RunnableOnService.class) + public void testFnCallSequenceMulti() { + TestPipeline p = TestPipeline.create(); + PCollectionList.of(p.apply("Impolite", Create.of(1, 2, 4))) + .and(p.apply("Polite", Create.of(3, 5, 6, 7))) + .apply(Flatten.pCollections()) + .apply(ParDo.of(new CallSequenceEnforcingOldFn()) + .withOutputTags(new TupleTag() {}, TupleTagList.empty())); + + p.run(); + } + + private static class CallSequenceEnforcingOldFn extends OldDoFn { + private boolean setupCalled = false; + private int startBundleCalls = 0; + private int finishBundleCalls = 0; + private boolean teardownCalled = false; + + @Override + public void setup() { + assertThat("setup should not be called twice", setupCalled, is(false)); + assertThat("setup should be called before startBundle", startBundleCalls, equalTo(0)); + assertThat("setup should be called before finishBundle", finishBundleCalls, equalTo(0)); + assertThat("setup should be called before teardown", teardownCalled, is(false)); + setupCalled = true; + } + + @Override + public void startBundle(Context c) { + assertThat("setup should have been called", setupCalled, is(true)); + assertThat( + "Even number of startBundle and finishBundle calls in startBundle", + startBundleCalls, + equalTo(finishBundleCalls)); + assertThat("teardown should not have been called", teardownCalled, is(false)); + startBundleCalls++; + } + + @Override + public void processElement(ProcessContext c) throws Exception { + assertThat("startBundle should have been called", startBundleCalls, greaterThan(0)); + assertThat( + "there should be one startBundle call with no call to finishBundle", + startBundleCalls, + equalTo(finishBundleCalls + 1)); + assertThat("teardown should not have been called", teardownCalled, is(false)); + } + + @Override + public void finishBundle(Context c) { + assertThat("startBundle should have been called", startBundleCalls, greaterThan(0)); + assertThat( + "there should be one bundle that has been started but not finished", + startBundleCalls, + equalTo(finishBundleCalls + 1)); + assertThat("teardown should not have been called", teardownCalled, is(false)); + finishBundleCalls++; + } + + @Override + public void teardown() { + assertThat(setupCalled, is(true)); + assertThat(startBundleCalls, anyOf(equalTo(finishBundleCalls))); + assertThat(teardownCalled, is(false)); + teardownCalled = true; + } + } + + @Test + @Category(RunnableOnService.class) + public void testFnWithContextCallSequence() { + TestPipeline p = TestPipeline.create(); + PCollectionList.of(p.apply("Impolite", Create.of(1, 2, 4))) + .and(p.apply("Polite", Create.of(3, 5, 6, 7))) + .apply(Flatten.pCollections()) + .apply(ParDo.of(new CallSequenceEnforcingFn())); + + p.run(); + } + + @Test + @Category(RunnableOnService.class) + public void testFnWithContextCallSequenceMulti() { + TestPipeline p = TestPipeline.create(); + PCollectionList.of(p.apply("Impolite", Create.of(1, 2, 4))) + .and(p.apply("Polite", Create.of(3, 5, 6, 7))) + .apply(Flatten.pCollections()) + .apply(ParDo.of(new CallSequenceEnforcingFn()) + .withOutputTags(new TupleTag() { + }, TupleTagList.empty())); + + p.run(); + } + + private static class CallSequenceEnforcingFn extends DoFn { + private boolean setupCalled = false; + private int startBundleCalls = 0; + private int finishBundleCalls = 0; + private boolean teardownCalled = false; + + @Setup + public void before() { + assertThat("setup should not be called twice", setupCalled, is(false)); + assertThat("setup should be called before startBundle", startBundleCalls, equalTo(0)); + assertThat("setup should be called before finishBundle", finishBundleCalls, equalTo(0)); + assertThat("setup should be called before teardown", teardownCalled, is(false)); + setupCalled = true; + } + + @StartBundle + public void begin(Context c) { + assertThat("setup should have been called", setupCalled, is(true)); + assertThat("Even number of startBundle and finishBundle calls in startBundle", + startBundleCalls, + equalTo(finishBundleCalls)); + assertThat("teardown should not have been called", teardownCalled, is(false)); + startBundleCalls++; + } + + @ProcessElement + public void process(ProcessContext c) throws Exception { + assertThat("startBundle should have been called", startBundleCalls, greaterThan(0)); + assertThat("there should be one startBundle call with no call to finishBundle", + startBundleCalls, + equalTo(finishBundleCalls + 1)); + assertThat("teardown should not have been called", teardownCalled, is(false)); + } + + @FinishBundle + public void end(Context c) { + assertThat("startBundle should have been called", startBundleCalls, greaterThan(0)); + assertThat("there should be one bundle that has been started but not finished", + startBundleCalls, + equalTo(finishBundleCalls + 1)); + assertThat("teardown should not have been called", teardownCalled, is(false)); + finishBundleCalls++; + } + + @Teardown + public void after() { + assertThat(setupCalled, is(true)); + assertThat(startBundleCalls, anyOf(equalTo(finishBundleCalls))); + assertThat(teardownCalled, is(false)); + teardownCalled = true; + } + } + + @Test + @Category(NeedsRunner.class) + public void testTeardownCalledAfterExceptionInSetup() { + TestPipeline p = TestPipeline.create(); + ExceptionThrowingOldFn fn = new ExceptionThrowingOldFn(MethodForException.SETUP); + p + .apply(Create.of(1, 2, 3)) + .apply(ParDo.of(fn)); + try { + p.run(); + fail("Pipeline should have failed with an exception"); + } catch (Exception e) { + assertThat( + "Function should have been torn down after exception", + ExceptionThrowingOldFn.teardownCalled.get(), + is(true)); + } + } + + @Test + @Category(NeedsRunner.class) + public void testTeardownCalledAfterExceptionInStartBundle() { + TestPipeline p = TestPipeline.create(); + ExceptionThrowingOldFn fn = new ExceptionThrowingOldFn(MethodForException.START_BUNDLE); + p + .apply(Create.of(1, 2, 3)) + .apply(ParDo.of(fn)); + try { + p.run(); + fail("Pipeline should have failed with an exception"); + } catch (Exception e) { + assertThat( + "Function should have been torn down after exception", + ExceptionThrowingOldFn.teardownCalled.get(), + is(true)); + } + } + + @Test + @Category(NeedsRunner.class) + public void testTeardownCalledAfterExceptionInProcessElement() { + TestPipeline p = TestPipeline.create(); + ExceptionThrowingOldFn fn = new ExceptionThrowingOldFn(MethodForException.PROCESS_ELEMENT); + p + .apply(Create.of(1, 2, 3)) + .apply(ParDo.of(fn)); + try { + p.run(); + fail("Pipeline should have failed with an exception"); + } catch (Exception e) { + assertThat( + "Function should have been torn down after exception", + ExceptionThrowingOldFn.teardownCalled.get(), + is(true)); + } + } + + @Test + @Category(NeedsRunner.class) + public void testTeardownCalledAfterExceptionInFinishBundle() { + TestPipeline p = TestPipeline.create(); + ExceptionThrowingOldFn fn = new ExceptionThrowingOldFn(MethodForException.FINISH_BUNDLE); + p + .apply(Create.of(1, 2, 3)) + .apply(ParDo.of(fn)); + try { + p.run(); + fail("Pipeline should have failed with an exception"); + } catch (Exception e) { + assertThat( + "Function should have been torn down after exception", + ExceptionThrowingOldFn.teardownCalled.get(), + is(true)); + } + } + + @Test + @Category(NeedsRunner.class) + public void testWithContextTeardownCalledAfterExceptionInSetup() { + TestPipeline p = TestPipeline.create(); + ExceptionThrowingOldFn fn = new ExceptionThrowingOldFn(MethodForException.SETUP); + p.apply(Create.of(1, 2, 3)).apply(ParDo.of(fn)); + try { + p.run(); + fail("Pipeline should have failed with an exception"); + } catch (Exception e) { + assertThat("Function should have been torn down after exception", + ExceptionThrowingOldFn.teardownCalled.get(), + is(true)); + } + } + + @Test + @Category(NeedsRunner.class) + public void testWithContextTeardownCalledAfterExceptionInStartBundle() { + TestPipeline p = TestPipeline.create(); + ExceptionThrowingOldFn fn = new ExceptionThrowingOldFn(MethodForException.START_BUNDLE); + p.apply(Create.of(1, 2, 3)).apply(ParDo.of(fn)); + try { + p.run(); + fail("Pipeline should have failed with an exception"); + } catch (Exception e) { + assertThat("Function should have been torn down after exception", + ExceptionThrowingOldFn.teardownCalled.get(), + is(true)); + } + } + + @Test + @Category(NeedsRunner.class) + public void testWithContextTeardownCalledAfterExceptionInProcessElement() { + TestPipeline p = TestPipeline.create(); + ExceptionThrowingOldFn fn = new ExceptionThrowingOldFn(MethodForException.PROCESS_ELEMENT); + p.apply(Create.of(1, 2, 3)).apply(ParDo.of(fn)); + try { + p.run(); + fail("Pipeline should have failed with an exception"); + } catch (Exception e) { + assertThat("Function should have been torn down after exception", + ExceptionThrowingOldFn.teardownCalled.get(), + is(true)); + } + } + + @Test + @Category(NeedsRunner.class) + public void testWithContextTeardownCalledAfterExceptionInFinishBundle() { + TestPipeline p = TestPipeline.create(); + ExceptionThrowingOldFn fn = new ExceptionThrowingOldFn(MethodForException.FINISH_BUNDLE); + p.apply(Create.of(1, 2, 3)).apply(ParDo.of(fn)); + try { + p.run(); + fail("Pipeline should have failed with an exception"); + } catch (Exception e) { + assertThat("Function should have been torn down after exception", + ExceptionThrowingOldFn.teardownCalled.get(), + is(true)); + } + } + + private static class ExceptionThrowingOldFn extends OldDoFn { + static AtomicBoolean teardownCalled = new AtomicBoolean(false); + + private final MethodForException toThrow; + private boolean thrown; + + private ExceptionThrowingOldFn(MethodForException toThrow) { + this.toThrow = toThrow; + } + + @Override + public void setup() throws Exception { + throwIfNecessary(MethodForException.SETUP); + } + + @Override + public void startBundle(Context c) throws Exception { + throwIfNecessary(MethodForException.START_BUNDLE); + } + + @Override + public void processElement(ProcessContext c) throws Exception { + throwIfNecessary(MethodForException.PROCESS_ELEMENT); + } + + @Override + public void finishBundle(Context c) throws Exception { + throwIfNecessary(MethodForException.FINISH_BUNDLE); + } + + private void throwIfNecessary(MethodForException method) throws Exception { + if (toThrow == method && !thrown) { + thrown = true; + throw new Exception("Hasn't yet thrown"); + } + } + + @Override + public void teardown() { + if (!thrown) { + fail("Excepted to have a processing method throw an exception"); + } + teardownCalled.set(true); + } + } + + + private static class ExceptionThrowingFn extends DoFn { + static AtomicBoolean teardownCalled = new AtomicBoolean(false); + + private final MethodForException toThrow; + private boolean thrown; + + private ExceptionThrowingFn(MethodForException toThrow) { + this.toThrow = toThrow; + } + + @Setup + public void before() throws Exception { + throwIfNecessary(MethodForException.SETUP); + } + + @StartBundle + public void preBundle(Context c) throws Exception { + throwIfNecessary(MethodForException.START_BUNDLE); + } + + @ProcessElement + public void perElement(ProcessContext c) throws Exception { + throwIfNecessary(MethodForException.PROCESS_ELEMENT); + } + + @FinishBundle + public void postBundle(Context c) throws Exception { + throwIfNecessary(MethodForException.FINISH_BUNDLE); + } + + private void throwIfNecessary(MethodForException method) throws Exception { + if (toThrow == method && !thrown) { + thrown = true; + throw new Exception("Hasn't yet thrown"); + } + } + + @Teardown + public void after() { + if (!thrown) { + fail("Excepted to have a processing method throw an exception"); + } + teardownCalled.set(true); + } + } + + private enum MethodForException { + SETUP, + START_BUNDLE, + PROCESS_ELEMENT, + FINISH_BUNDLE + } } From 29cbdceb5b78ce86ad0d90050d7542b0d5b45362 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Thu, 11 Aug 2016 10:45:43 -0700 Subject: [PATCH 019/112] Move ParDo Lifecycle tests to their own file These tests are not yet functional in all runners, and this makes them easier to ignore. --- runners/google-cloud-dataflow-java/pom.xml | 10 + .../sdk/transforms/ParDoLifecycleTest.java | 448 ++++++++++++++++++ .../apache/beam/sdk/transforms/ParDoTest.java | 405 ---------------- 3 files changed, 458 insertions(+), 405 deletions(-) create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoLifecycleTest.java diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml index 86991b788f50..c32e1845616c 100644 --- a/runners/google-cloud-dataflow-java/pom.xml +++ b/runners/google-cloud-dataflow-java/pom.xml @@ -60,6 +60,16 @@ true + + + runnable-on-service-tests + + + org/apache/beam/sdk/transforms/ParDoLifecycleTest.java + + + + com.google.guava guava-testlib ${guava.version} diff --git a/runners/direct-java/pom.xml b/runners/direct-java/pom.xml index 0a2b4b9cb4fa..11481f10f171 100644 --- a/runners/direct-java/pom.xml +++ b/runners/direct-java/pom.xml @@ -192,6 +192,10 @@ the second relocation. --> com.google.common + + + com.google.common.**.testing + org.apache.beam.runners.direct.repackaged.com.google.common @@ -264,6 +268,8 @@ + com.google.guava guava-testlib test diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml index d130281979f8..d5485ef9f157 100644 --- a/runners/google-cloud-dataflow-java/pom.xml +++ b/runners/google-cloud-dataflow-java/pom.xml @@ -196,6 +196,10 @@ the second relocation. --> com.google.common + + + com.google.common.**.testing + org.apache.beam.sdk.repackaged.com.google.common @@ -310,6 +314,8 @@ + com.google.guava guava-testlib test diff --git a/sdks/java/core/pom.xml b/sdks/java/core/pom.xml index aff4f66d78f3..fddccea7874b 100644 --- a/sdks/java/core/pom.xml +++ b/sdks/java/core/pom.xml @@ -193,6 +193,10 @@ exclude 'org.apache.beam.**', and remove the second relocation. --> com.google.common + + + com.google.common.**.testing + org.apache.beam.sdk.repackaged.com.google.common @@ -421,6 +425,8 @@ + com.google.guava guava-testlib test From b7c9a7bf4711a404f206abfe9480312525fe4a46 Mon Sep 17 00:00:00 2001 From: Daniel Halperin Date: Tue, 16 Aug 2016 13:11:16 -0700 Subject: [PATCH 022/112] ByteKeyRangeTracker: synchronize toString It's the only unsynchronized function, and it could potentially produce bad data. --- .../java/org/apache/beam/sdk/io/range/ByteKeyRangeTracker.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRangeTracker.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRangeTracker.java index 7dd9a2cc9b20..24bf6e474b4d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRangeTracker.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRangeTracker.java @@ -176,7 +176,7 @@ public synchronized boolean markDone() { } @Override - public String toString() { + public synchronized String toString() { return toStringHelper(ByteKeyRangeTracker.class) .add("range", range) .add("position", position) From 2a8ceed623e7909e0d67b71263e255e10a1c7693 Mon Sep 17 00:00:00 2001 From: Scott Wegner Date: Wed, 17 Aug 2016 09:28:44 -0700 Subject: [PATCH 023/112] Fix repackaging exclude pattern for guava-testlib --- pom.xml | 2 +- runners/direct-java/pom.xml | 4 ++-- runners/google-cloud-dataflow-java/pom.xml | 4 ++-- sdks/java/core/pom.xml | 4 ++-- 4 files changed, 7 insertions(+), 7 deletions(-) diff --git a/pom.xml b/pom.xml index b5f30c132779..f39f94aca693 100644 --- a/pom.xml +++ b/pom.xml @@ -634,7 +634,7 @@ + excluding com.google.common.**.testing.* --> com.google.guava guava-testlib ${guava.version} diff --git a/runners/direct-java/pom.xml b/runners/direct-java/pom.xml index 11481f10f171..e06883f6e65d 100644 --- a/runners/direct-java/pom.xml +++ b/runners/direct-java/pom.xml @@ -194,7 +194,7 @@ com.google.common - com.google.common.**.testing + com.google.common.**.testing.* org.apache.beam.runners.direct.repackaged.com.google.common @@ -269,7 +269,7 @@ + excluding com.google.common.**.testing.* --> com.google.guava guava-testlib test diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml index d5485ef9f157..00b5a9ba67c2 100644 --- a/runners/google-cloud-dataflow-java/pom.xml +++ b/runners/google-cloud-dataflow-java/pom.xml @@ -198,7 +198,7 @@ com.google.common - com.google.common.**.testing + com.google.common.**.testing.* org.apache.beam.sdk.repackaged.com.google.common @@ -315,7 +315,7 @@ + excluding com.google.common.**.testing.* --> com.google.guava guava-testlib test diff --git a/sdks/java/core/pom.xml b/sdks/java/core/pom.xml index fddccea7874b..78aec852bf84 100644 --- a/sdks/java/core/pom.xml +++ b/sdks/java/core/pom.xml @@ -195,7 +195,7 @@ com.google.common - com.google.common.**.testing + com.google.common.**.testing.* org.apache.beam.sdk.repackaged.com.google.common @@ -426,7 +426,7 @@ + excluding com.google.common.**.testing.* --> com.google.guava guava-testlib test From fbf77f90e0391304a580178f99441256526c4b0e Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Tue, 9 Aug 2016 17:16:00 -0700 Subject: [PATCH 024/112] Rewrites DoFnReflector to go via DoFnSignature DoFnSignature encapsulates type information about a DoFn, in particular which arguments/features its methods actually use. Before this commit, DoFnReflector would parse/verify/generate code in one go; after this commit, these stages are separated: DoFnSignature encapsulates all information needed to generate the code. Additionally, removes the unnecessary genericity in the implementation of DoFnReflector's code generation for the very different methods processElement and start/finishBundle. The code is simpler if decomposed into utility functions, rather than attempting a uniform representation for different methods. --- .../org/apache/beam/sdk/transforms/DoFn.java | 17 +- .../beam/sdk/transforms/DoFnAdapters.java | 281 ++++ .../beam/sdk/transforms/DoFnReflector.java | 1150 ----------------- .../beam/sdk/transforms/DoFnTester.java | 2 +- .../org/apache/beam/sdk/transforms/ParDo.java | 6 +- .../sdk/transforms/reflect/DoFnInvoker.java | 61 + .../sdk/transforms/reflect/DoFnInvokers.java | 506 ++++++++ .../sdk/transforms/reflect/DoFnSignature.java | 113 ++ .../transforms/reflect/DoFnSignatures.java | 321 +++++ .../sdk/transforms/reflect/package-info.java | 23 + .../sdk/transforms/DoFnReflectorTest.java | 822 ------------ .../beam/sdk/transforms/FlattenTest.java | 4 +- .../transforms/reflect/DoFnInvokersTest.java | 498 +++++++ .../DoFnInvokersTestHelper.java} | 8 +- .../reflect/DoFnSignaturesTest.java | 371 ++++++ ...chmark.java => DoFnInvokersBenchmark.java} | 46 +- 16 files changed, 2212 insertions(+), 2017 deletions(-) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnReflector.java create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/package-info.java delete mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnReflectorTest.java create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java rename sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/{dofnreflector/DoFnReflectorTestHelper.java => reflect/DoFnInvokersTestHelper.java} (93%) create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesTest.java rename sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/transforms/{DoFnReflectorBenchmark.java => DoFnInvokersBenchmark.java} (83%) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java index 80b67afc8412..2348783f9f61 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java @@ -26,6 +26,7 @@ import org.apache.beam.sdk.transforms.OldDoFn.DelegatingAggregator; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.display.HasDisplayData; +import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.values.PCollectionView; @@ -247,7 +248,7 @@ public Duration getAllowedTimestampSkew() { ///////////////////////////////////////////////////////////////////////////// - Map> aggregators = new HashMap<>(); + protected Map> aggregators = new HashMap<>(); /** * Protects aggregators from being created after initialization. @@ -283,7 +284,7 @@ protected TypeDescriptor getOutputTypeDescriptor() { /** * Interface for runner implementors to provide implementations of extra context information. * - *

    The methods on this interface are called by {@link DoFnReflector} before invoking an + *

    The methods on this interface are called by {@link DoFnInvoker} before invoking an * annotated {@link StartBundle}, {@link ProcessElement} or {@link FinishBundle} method that * has indicated it needs the given extra context. * @@ -301,23 +302,23 @@ public interface ExtraContextFactory { BoundedWindow window(); /** - * A placeholder for testing purposes. The return type itself is package-private and not - * implemented. + * A placeholder for testing purposes. */ InputProvider inputProvider(); /** - * A placeholder for testing purposes. The return type itself is package-private and not - * implemented. + * A placeholder for testing purposes. */ OutputReceiver outputReceiver(); } - static interface OutputReceiver { + /** A placeholder for testing handling of output types during {@link DoFn} reflection. */ + public interface OutputReceiver { void output(T output); } - static interface InputProvider { + /** A placeholder for testing handling of input types during {@link DoFn} reflection. */ + public interface InputProvider { T get(); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java new file mode 100644 index 000000000000..71a148ffafe9 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java @@ -0,0 +1,281 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.transforms; + +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; +import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; +import org.apache.beam.sdk.transforms.reflect.DoFnSignature; +import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TypeDescriptor; + +import org.joda.time.Duration; +import org.joda.time.Instant; + +import java.io.IOException; + +/** + * Utility class containing adapters for running a {@link DoFn} as an {@link OldDoFn}. + * + * @deprecated This class will go away when we start running {@link DoFn}'s directly (using + * {@link DoFnInvoker}) rather than via {@link OldDoFn}. + */ +@Deprecated +public class DoFnAdapters { + /** Should not be instantiated. */ + private DoFnAdapters() {} + + /** + * If this is an {@link OldDoFn} produced via {@link #toOldDoFn}, returns the class of the + * original {@link DoFn}, otherwise returns {@code fn.getClass()}. + */ + public static Class getDoFnClass(OldDoFn fn) { + if (fn instanceof SimpleDoFnAdapter) { + return ((SimpleDoFnAdapter) fn).fn.getClass(); + } else { + return fn.getClass(); + } + } + + /** Creates an {@link OldDoFn} that delegates to the {@link DoFn}. */ + public static OldDoFn toOldDoFn(DoFn fn) { + DoFnSignature signature = DoFnSignatures.INSTANCE.getOrParseSignature(fn.getClass()); + if (signature.processElement().usesSingleWindow()) { + return new WindowDoFnAdapter<>(fn); + } else { + return new SimpleDoFnAdapter<>(fn); + } + } + + /** + * Wraps a {@link DoFn} that doesn't require access to {@link BoundedWindow} as an {@link + * OldDoFn}. + */ + private static class SimpleDoFnAdapter extends OldDoFn { + private final DoFn fn; + private transient DoFnInvoker invoker; + + SimpleDoFnAdapter(DoFn fn) { + super(fn.aggregators); + this.fn = fn; + this.invoker = DoFnInvokers.INSTANCE.newByteBuddyInvoker(fn); + } + + @Override + public void setup() throws Exception { + this.invoker.invokeSetup(); + } + + @Override + public void startBundle(Context c) throws Exception { + this.fn.prepareForProcessing(); + invoker.invokeStartBundle(new ContextAdapter<>(fn, c)); + } + + @Override + public void finishBundle(Context c) throws Exception { + invoker.invokeFinishBundle(new ContextAdapter<>(fn, c)); + } + + @Override + public void teardown() throws Exception { + this.invoker.invokeTeardown(); + } + + @Override + public void processElement(ProcessContext c) throws Exception { + ProcessContextAdapter adapter = new ProcessContextAdapter<>(fn, c); + invoker.invokeProcessElement(adapter, adapter); + } + + @Override + protected TypeDescriptor getInputTypeDescriptor() { + return fn.getInputTypeDescriptor(); + } + + @Override + protected TypeDescriptor getOutputTypeDescriptor() { + return fn.getOutputTypeDescriptor(); + } + + @Override + public Duration getAllowedTimestampSkew() { + return fn.getAllowedTimestampSkew(); + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder.include(fn); + } + + private void readObject(java.io.ObjectInputStream in) + throws IOException, ClassNotFoundException { + in.defaultReadObject(); + this.invoker = DoFnInvokers.INSTANCE.newByteBuddyInvoker(fn); + } + } + + /** Wraps a {@link DoFn} that requires access to {@link BoundedWindow} as an {@link OldDoFn}. */ + private static class WindowDoFnAdapter extends SimpleDoFnAdapter + implements OldDoFn.RequiresWindowAccess { + + WindowDoFnAdapter(DoFn fn) { + super(fn); + } + } + + /** + * Wraps an {@link OldDoFn.Context} as a {@link DoFn.ExtraContextFactory} inside a {@link + * DoFn.StartBundle} or {@link DoFn.FinishBundle} method, which means the extra context is + * unavailable. + */ + private static class ContextAdapter extends DoFn.Context + implements DoFn.ExtraContextFactory { + + private OldDoFn.Context context; + + private ContextAdapter(DoFn fn, OldDoFn.Context context) { + fn.super(); + this.context = context; + } + + @Override + public PipelineOptions getPipelineOptions() { + return context.getPipelineOptions(); + } + + @Override + public void output(OutputT output) { + context.output(output); + } + + @Override + public void outputWithTimestamp(OutputT output, Instant timestamp) { + context.outputWithTimestamp(output, timestamp); + } + + @Override + public void sideOutput(TupleTag tag, T output) { + context.sideOutput(tag, output); + } + + @Override + public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { + context.sideOutputWithTimestamp(tag, output, timestamp); + } + + @Override + public BoundedWindow window() { + // The DoFn doesn't allow us to ask for these outside ProcessElements, so this + // should be unreachable. + throw new UnsupportedOperationException("Can only get the window in ProcessElements"); + } + + @Override + public DoFn.InputProvider inputProvider() { + throw new UnsupportedOperationException("inputProvider() exists only for testing"); + } + + @Override + public DoFn.OutputReceiver outputReceiver() { + throw new UnsupportedOperationException("outputReceiver() exists only for testing"); + } + } + + /** + * Wraps an {@link OldDoFn.ProcessContext} as a {@link DoFn.ExtraContextFactory} inside a {@link + * DoFn.ProcessElement} method. + */ + private static class ProcessContextAdapter + extends DoFn.ProcessContext + implements DoFn.ExtraContextFactory { + + private OldDoFn.ProcessContext context; + + private ProcessContextAdapter( + DoFn fn, OldDoFn.ProcessContext context) { + fn.super(); + this.context = context; + } + + @Override + public PipelineOptions getPipelineOptions() { + return context.getPipelineOptions(); + } + + @Override + public T sideInput(PCollectionView view) { + return context.sideInput(view); + } + + @Override + public void output(OutputT output) { + context.output(output); + } + + @Override + public void outputWithTimestamp(OutputT output, Instant timestamp) { + context.outputWithTimestamp(output, timestamp); + } + + @Override + public void sideOutput(TupleTag tag, T output) { + context.sideOutput(tag, output); + } + + @Override + public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { + context.sideOutputWithTimestamp(tag, output, timestamp); + } + + @Override + public InputT element() { + return context.element(); + } + + @Override + public Instant timestamp() { + return context.timestamp(); + } + + @Override + public PaneInfo pane() { + return context.pane(); + } + + @Override + public BoundedWindow window() { + return context.window(); + } + + @Override + public DoFn.InputProvider inputProvider() { + throw new UnsupportedOperationException("inputProvider() exists only for testing"); + } + + @Override + public DoFn.OutputReceiver outputReceiver() { + throw new UnsupportedOperationException("outputReceiver() exists only for testing"); + } + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnReflector.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnReflector.java deleted file mode 100644 index bf040412b5e0..000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnReflector.java +++ /dev/null @@ -1,1150 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.transforms; - -import static com.google.common.base.Preconditions.checkState; - -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.transforms.DoFn.ExtraContextFactory; -import org.apache.beam.sdk.transforms.DoFn.FinishBundle; -import org.apache.beam.sdk.transforms.DoFn.ProcessElement; -import org.apache.beam.sdk.transforms.DoFn.Setup; -import org.apache.beam.sdk.transforms.DoFn.StartBundle; -import org.apache.beam.sdk.transforms.DoFn.Teardown; -import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.UserCodeException; -import org.apache.beam.sdk.util.common.ReflectHelpers; -import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.sdk.values.TypeDescriptor; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; -import com.google.common.base.Preconditions; -import com.google.common.base.Predicate; -import com.google.common.collect.FluentIterable; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.reflect.TypeParameter; -import com.google.common.reflect.TypeToken; - -import net.bytebuddy.ByteBuddy; -import net.bytebuddy.NamingStrategy.SuffixingRandom; -import net.bytebuddy.description.field.FieldDescription; -import net.bytebuddy.description.method.MethodDescription; -import net.bytebuddy.description.method.ParameterList; -import net.bytebuddy.description.modifier.FieldManifestation; -import net.bytebuddy.description.modifier.Visibility; -import net.bytebuddy.description.type.TypeDescription; -import net.bytebuddy.description.type.TypeDescription.Generic; -import net.bytebuddy.dynamic.DynamicType; -import net.bytebuddy.dynamic.loading.ClassLoadingStrategy; -import net.bytebuddy.dynamic.scaffold.InstrumentedType; -import net.bytebuddy.dynamic.scaffold.subclass.ConstructorStrategy.Default; -import net.bytebuddy.implementation.Implementation; -import net.bytebuddy.implementation.MethodCall.MethodLocator; -import net.bytebuddy.implementation.StubMethod; -import net.bytebuddy.implementation.bind.MethodDelegationBinder.MethodInvoker; -import net.bytebuddy.implementation.bind.annotation.TargetMethodAnnotationDrivenBinder.TerminationHandler; -import net.bytebuddy.implementation.bytecode.ByteCodeAppender; -import net.bytebuddy.implementation.bytecode.Duplication; -import net.bytebuddy.implementation.bytecode.StackManipulation; -import net.bytebuddy.implementation.bytecode.Throw; -import net.bytebuddy.implementation.bytecode.assign.Assigner; -import net.bytebuddy.implementation.bytecode.member.FieldAccess; -import net.bytebuddy.implementation.bytecode.member.MethodInvocation; -import net.bytebuddy.implementation.bytecode.member.MethodReturn; -import net.bytebuddy.implementation.bytecode.member.MethodVariableAccess; -import net.bytebuddy.jar.asm.Label; -import net.bytebuddy.jar.asm.MethodVisitor; -import net.bytebuddy.jar.asm.Opcodes; -import net.bytebuddy.matcher.ElementMatchers; -import org.joda.time.Duration; -import org.joda.time.Instant; - -import java.io.IOException; -import java.lang.annotation.Annotation; -import java.lang.reflect.Constructor; -import java.lang.reflect.InvocationTargetException; -import java.lang.reflect.Method; -import java.lang.reflect.Modifier; -import java.lang.reflect.ParameterizedType; -import java.lang.reflect.Type; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.LinkedHashMap; -import java.util.LinkedHashSet; -import java.util.List; -import java.util.Map; -import javax.annotation.Nonnull; -import javax.annotation.Nullable; - - -/** - * Utility implementing the necessary reflection for working with {@link DoFn}s. - */ -public abstract class DoFnReflector { - - private static final String FN_DELEGATE_FIELD_NAME = "delegate"; - - private enum Availability { - /** Indicates parameters only available in {@code @ProcessElement} methods. */ - PROCESS_ELEMENT_ONLY, - /** Indicates parameters available in all methods. */ - EVERYWHERE - } - - /** - * Enumeration of the parameters available from the {@link ExtraContextFactory} to use as - * additional parameters for {@link DoFn} methods. - *

    - * We don't rely on looking for properly annotated methods within {@link ExtraContextFactory} - * because erasure would make it impossible to completely fill in the type token for context - * parameters that depend on the input/output type. - */ - private enum AdditionalParameter { - - /** Any {@link BoundedWindow} parameter is populated by the window of the current element. */ - WINDOW_OF_ELEMENT(Availability.PROCESS_ELEMENT_ONLY, BoundedWindow.class, "window") { - @Override - public TypeToken tokenFor(TypeToken in, TypeToken out) { - return TypeToken.of(BoundedWindow.class); - } - }, - - INPUT_PROVIDER(Availability.PROCESS_ELEMENT_ONLY, DoFn.InputProvider.class, "inputProvider") { - @Override - public TypeToken tokenFor(TypeToken in, TypeToken out) { - return new TypeToken>() {}.where( - new TypeParameter() {}, in); - } - - @Override - public boolean isHidden() { - return true; - } - }, - - OUTPUT_RECEIVER( - Availability.PROCESS_ELEMENT_ONLY, DoFn.OutputReceiver.class, "outputReceiver") { - @Override - public TypeToken tokenFor(TypeToken in, TypeToken out) { - return new TypeToken>() {}.where( - new TypeParameter() {}, out); - } - - @Override - public boolean isHidden() { - return true; - } - }; - - /** - * Create a type token representing the given parameter. May use the type token associated - * with the input and output types of the {@link DoFn}, depending on the extra - * context. - */ - abstract TypeToken tokenFor( - TypeToken in, TypeToken out); - - /** - * Indicates whether this enum is for testing only, hence should not appear in error messages, - * etc. Defaults to {@code false}. - */ - boolean isHidden() { - return false; - } - - private final Class rawType; - private final Availability availability; - private final transient MethodDescription method; - - private AdditionalParameter(Availability availability, Class rawType, String method) { - this.availability = availability; - this.rawType = rawType; - try { - this.method = new MethodDescription.ForLoadedMethod( - ExtraContextFactory.class.getMethod(method)); - } catch (NoSuchMethodException | SecurityException e) { - throw new RuntimeException( - "Unable to access method " + method + " on " + ExtraContextFactory.class, e); - } - } - } - - private static final Map, AdditionalParameter> EXTRA_CONTEXTS; - private static final Map, AdditionalParameter> EXTRA_PROCESS_CONTEXTS; - - static { - ImmutableMap.Builder, AdditionalParameter> everywhereBuilder = - ImmutableMap., AdditionalParameter>builder(); - ImmutableMap.Builder, AdditionalParameter> processElementBuilder = - ImmutableMap., AdditionalParameter>builder(); - - for (AdditionalParameter value : AdditionalParameter.values()) { - switch (value.availability) { - case EVERYWHERE: - everywhereBuilder.put(value.rawType, value); - break; - case PROCESS_ELEMENT_ONLY: - processElementBuilder.put(value.rawType, value); - break; - } - } - - EXTRA_CONTEXTS = everywhereBuilder.build(); - EXTRA_PROCESS_CONTEXTS = processElementBuilder - // Process Element contexts include everything available everywhere - .putAll(EXTRA_CONTEXTS) - .build(); - } - - /** - * @return true if the reflected {@link DoFn} uses a Single Window. - */ - public abstract boolean usesSingleWindow(); - - /** Create an {@link DoFnInvoker} bound to the given {@link OldDoFn}. */ - public abstract DoFnInvoker bindInvoker( - DoFn fn); - - private static final Map, DoFnReflector> REFLECTOR_CACHE = - new LinkedHashMap, DoFnReflector>(); - - /** - * @return the {@link DoFnReflector} for the given {@link DoFn}. - */ - public static DoFnReflector of( - @SuppressWarnings("rawtypes") Class fn) { - DoFnReflector reflector = REFLECTOR_CACHE.get(fn); - if (reflector != null) { - return reflector; - } - - reflector = new GenericDoFnReflector(fn); - REFLECTOR_CACHE.put(fn, reflector); - return reflector; - } - - /** - * Create a {@link OldDoFn} that the {@link DoFn}. - */ - public OldDoFn toDoFn(DoFn fn) { - if (usesSingleWindow()) { - return new WindowDoFnAdapter(this, fn); - } else { - return new SimpleDoFnAdapter(this, fn); - } - } - - private static String formatType(TypeToken t) { - return ReflectHelpers.TYPE_SIMPLE_DESCRIPTION.apply(t.getType()); - } - - private static String format(Method m) { - return ReflectHelpers.CLASS_AND_METHOD_FORMATTER.apply(m); - } - - private static Collection describeSupportedTypes( - Map, AdditionalParameter> extraProcessContexts, - final TypeToken in, final TypeToken out) { - return FluentIterable - .from(extraProcessContexts.values()) - .filter(new Predicate() { - @Override - public boolean apply(@Nonnull AdditionalParameter additionalParameter) { - return !additionalParameter.isHidden(); - } - }) - .transform(new Function() { - @Override - @Nonnull - public String apply(@Nonnull AdditionalParameter input) { - return formatType(input.tokenFor(in, out)); - } - }) - .toSortedSet(String.CASE_INSENSITIVE_ORDER); - } - - @VisibleForTesting - static List verifyProcessMethodArguments(Method m) { - return verifyMethodArguments(m, - EXTRA_PROCESS_CONTEXTS, - new TypeToken.ProcessContext>() {}, - new TypeParameter() {}, - new TypeParameter() {}); - } - - @VisibleForTesting - static List verifyBundleMethodArguments(Method m) { - if (m == null) { - return null; - } - return verifyMethodArguments(m, - EXTRA_CONTEXTS, - new TypeToken.Context>() {}, - new TypeParameter() {}, - new TypeParameter() {}); - } - - @VisibleForTesting - static void verifyLifecycleMethodArguments(Method m) { - if (m == null) { - return; - } - checkState(void.class.equals(m.getReturnType()), "%s must have void return type", format(m)); - checkState(m.getGenericParameterTypes().length == 0, "%s must take zero arguments", format(m)); - } - - /** - * Verify the method arguments for a given {@link DoFn} method. - * - *

    The requirements for a method to be valid, are: - *

      - *
    1. The method has at least one argument. - *
    2. The first argument is of type firstContextArg. - *
    3. The remaining arguments have raw types that appear in {@code contexts} - *
    4. Any generics on the extra context arguments match what is expected. Currently, this - * is exercised only by placeholders. For example, {@code InputReceiver must either match - * the {@code InputT} {@code OldDoFn.ProcessContext} or use a wildcard, etc. - *
    - * - * @param m the method to verify - * @param contexts mapping from raw classes to the {@link AdditionalParameter} used - * to create new instances. - * @param firstContextArg the expected type of the first context argument - * @param iParam TypeParameter representing the input type - * @param oParam TypeParameter representing the output type - */ - @VisibleForTesting - static List verifyMethodArguments( - Method m, - Map, AdditionalParameter> contexts, - TypeToken firstContextArg, - TypeParameter iParam, - TypeParameter oParam) { - - if (!void.class.equals(m.getReturnType())) { - throw new IllegalStateException(String.format( - "%s must have a void return type", format(m))); - } - if (m.isVarArgs()) { - throw new IllegalStateException(String.format( - "%s must not have var args", format(m))); - } - - // The first parameter must be present, and must be the specified type - Type[] params = m.getGenericParameterTypes(); - TypeToken contextToken = null; - if (params.length > 0) { - contextToken = TypeToken.of(params[0]); - } - if (contextToken == null - || !contextToken.getRawType().equals(firstContextArg.getRawType())) { - throw new IllegalStateException(String.format( - "%s must take a %s as its first argument", - format(m), firstContextArg.getRawType().getSimpleName())); - } - AdditionalParameter[] contextInfos = new AdditionalParameter[params.length - 1]; - - // Fill in the generics in the allExtraContextArgs interface from the types in the - // Context or ProcessContext OldDoFn. - ParameterizedType pt = (ParameterizedType) contextToken.getType(); - // We actually want the owner, since ProcessContext and Context are owned by DoFn. - pt = (ParameterizedType) pt.getOwnerType(); - @SuppressWarnings("unchecked") - TypeToken iActual = (TypeToken) TypeToken.of(pt.getActualTypeArguments()[0]); - @SuppressWarnings("unchecked") - TypeToken oActual = (TypeToken) TypeToken.of(pt.getActualTypeArguments()[1]); - - // All of the remaining parameters must be a super-interface of allExtraContextArgs - // that is not listed in the EXCLUDED_INTERFACES set. - for (int i = 1; i < params.length; i++) { - TypeToken param = TypeToken.of(params[i]); - - AdditionalParameter info = contexts.get(param.getRawType()); - if (info == null) { - throw new IllegalStateException(String.format( - "%s is not a valid context parameter for method %s. Should be one of %s", - formatType(param), format(m), - describeSupportedTypes(contexts, iActual, oActual))); - } - - // If we get here, the class matches, but maybe the generics don't: - TypeToken expected = info.tokenFor(iActual, oActual); - if (!expected.isSubtypeOf(param)) { - throw new IllegalStateException(String.format( - "Incompatible generics in context parameter %s for method %s. Should be %s", - formatType(param), format(m), formatType(info.tokenFor(iActual, oActual)))); - } - - // Register the (now validated) context info - contextInfos[i - 1] = info; - } - return ImmutableList.copyOf(contextInfos); - } - - /** Interface for invoking the {@code OldDoFn} processing methods. */ - public interface DoFnInvoker { - /** Invoke {@link OldDoFn#setup} on the bound {@code OldDoFn}. */ - void invokeSetup(); - /** Invoke {@link OldDoFn#startBundle} on the bound {@code OldDoFn}. */ - void invokeStartBundle( - DoFn.Context c, - ExtraContextFactory extra); - /** Invoke {@link OldDoFn#finishBundle} on the bound {@code OldDoFn}. */ - void invokeFinishBundle( - DoFn.Context c, - ExtraContextFactory extra); - - /** Invoke {@link OldDoFn#teardown()} on the bound {@code DoFn}. */ - void invokeTeardown(); - - /** Invoke {@link OldDoFn#processElement} on the bound {@code OldDoFn}. */ - public void invokeProcessElement( - DoFn.ProcessContext c, - ExtraContextFactory extra); - } - - /** - * Implementation of {@link DoFnReflector} for the arbitrary {@link DoFn}. - */ - private static class GenericDoFnReflector extends DoFnReflector { - - private final Method setup; - private final Method startBundle; - private final Method processElement; - private final Method finishBundle; - private final Method teardown; - private final List processElementArgs; - private final List startBundleArgs; - private final List finishBundleArgs; - private final Constructor constructor; - - private GenericDoFnReflector( - @SuppressWarnings("rawtypes") Class fn) { - // Locate the annotated methods - this.processElement = findAnnotatedMethod(ProcessElement.class, fn, true); - this.setup = findAnnotatedMethod(Setup.class, fn, false); - this.startBundle = findAnnotatedMethod(StartBundle.class, fn, false); - this.finishBundle = findAnnotatedMethod(FinishBundle.class, fn, false); - this.teardown = findAnnotatedMethod(Teardown.class, fn, false); - - // Verify that their method arguments satisfy our conditions. - this.processElementArgs = verifyProcessMethodArguments(processElement); - this.startBundleArgs = verifyBundleMethodArguments(startBundle); - this.finishBundleArgs = verifyBundleMethodArguments(finishBundle); - verifyLifecycleMethodArguments(setup); - verifyLifecycleMethodArguments(teardown); - - this.constructor = createInvokerConstructor(fn); - } - - private static Collection declaredMethodsWithAnnotation( - Class anno, - Class startClass, Class stopClass) { - Collection matches = new ArrayList<>(); - - Class clazz = startClass; - LinkedHashSet> interfaces = new LinkedHashSet<>(); - - // First, find all declared methods on the startClass and parents (up to stopClass) - while (clazz != null && !clazz.equals(stopClass)) { - for (Method method : clazz.getDeclaredMethods()) { - if (method.isAnnotationPresent(anno)) { - matches.add(method); - } - } - - Collections.addAll(interfaces, clazz.getInterfaces()); - - clazz = clazz.getSuperclass(); - } - - // Now, iterate over all the discovered interfaces - for (Method method : ReflectHelpers.getClosureOfMethodsOnInterfaces(interfaces)) { - if (method.isAnnotationPresent(anno)) { - matches.add(method); - } - } - return matches; - } - - private static Method findAnnotatedMethod( - Class anno, Class fnClazz, boolean required) { - Collection matches = declaredMethodsWithAnnotation( - anno, fnClazz, DoFn.class); - - if (matches.size() == 0) { - if (required) { - throw new IllegalStateException(String.format( - "No method annotated with @%s found in %s", - anno.getSimpleName(), fnClazz.getName())); - } else { - return null; - } - } - - // If we have at least one match, then either it should be the only match - // or it should be an extension of the other matches (which came from parent - // classes). - Method first = matches.iterator().next(); - for (Method other : matches) { - if (!first.getName().equals(other.getName()) - || !Arrays.equals(first.getParameterTypes(), other.getParameterTypes())) { - throw new IllegalStateException(String.format( - "Found multiple methods annotated with @%s. [%s] and [%s]", - anno.getSimpleName(), format(first), format(other))); - } - } - - // We need to be able to call it. We require it is public. - if ((first.getModifiers() & Modifier.PUBLIC) == 0) { - throw new IllegalStateException(format(first) + " must be public"); - } - - // And make sure its not static. - if ((first.getModifiers() & Modifier.STATIC) != 0) { - throw new IllegalStateException(format(first) + " must not be static"); - } - - return first; - } - - @Override - public boolean usesSingleWindow() { - return usesContext(AdditionalParameter.WINDOW_OF_ELEMENT); - } - - private boolean usesContext(AdditionalParameter param) { - return processElementArgs.contains(param) - || (startBundleArgs != null && startBundleArgs.contains(param)) - || (finishBundleArgs != null && finishBundleArgs.contains(param)); - } - - /** - * Use ByteBuddy to generate the code for a {@link DoFnInvoker} that invokes the given - * {@link DoFn}. - * @param clazz - * @return - */ - private Constructor> createInvokerConstructor( - @SuppressWarnings("rawtypes") Class clazz) { - - final TypeDescription clazzDescription = new TypeDescription.ForLoadedType(clazz); - - DynamicType.Builder builder = new ByteBuddy() - // Create subclasses inside the target class, to have access to - // private and package-private bits - .with(new SuffixingRandom("auxiliary") { - @Override - public String subclass(Generic superClass) { - return super.name(clazzDescription); - } - }) - // Create a subclass of DoFnInvoker - .subclass(DoFnInvoker.class, Default.NO_CONSTRUCTORS) - .defineField(FN_DELEGATE_FIELD_NAME, clazz, Visibility.PRIVATE, FieldManifestation.FINAL) - // Define a constructor to populate fields appropriately. - .defineConstructor(Visibility.PUBLIC) - .withParameter(clazz) - .intercept(new InvokerConstructor()) - // Implement the three methods by calling into the appropriate functions on the fn. - .method(ElementMatchers.named("invokeProcessElement")) - .intercept(InvokerDelegation.create( - processElement, BeforeDelegation.NOOP, processElementArgs)) - .method(ElementMatchers.named("invokeStartBundle")) - .intercept(InvokerDelegation.create( - startBundle, BeforeDelegation.INVOKE_PREPARE_FOR_PROCESSING, startBundleArgs)) - .method(ElementMatchers.named("invokeFinishBundle")) - .intercept(InvokerDelegation.create(finishBundle, - BeforeDelegation.NOOP, - finishBundleArgs)) - .method(ElementMatchers.named("invokeSetup")) - .intercept(InvokerDelegation.create(setup, - BeforeDelegation.NOOP, - Collections.emptyList())) - .method(ElementMatchers.named("invokeTeardown")) - .intercept(InvokerDelegation.create(teardown, - BeforeDelegation.NOOP, - Collections.emptyList())); - - @SuppressWarnings("unchecked") - Class> dynamicClass = (Class>) builder - .make() - .load(getClass().getClassLoader(), ClassLoadingStrategy.Default.INJECTION) - .getLoaded(); - try { - return dynamicClass.getConstructor(clazz); - } catch (IllegalArgumentException - | NoSuchMethodException - | SecurityException e) { - throw new RuntimeException(e); - } - } - - @Override - public DoFnInvoker bindInvoker( - DoFn fn) { - try { - @SuppressWarnings("unchecked") - DoFnInvoker invoker = - (DoFnInvoker) constructor.newInstance(fn); - return invoker; - } catch (InstantiationException - | IllegalAccessException - | IllegalArgumentException - | InvocationTargetException - | SecurityException e) { - throw new RuntimeException("Unable to bind invoker for " + fn.getClass(), e); - } - } - } - - private static class ContextAdapter - extends DoFn.Context - implements DoFn.ExtraContextFactory { - - private OldDoFn.Context context; - - private ContextAdapter( - DoFn fn, OldDoFn.Context context) { - fn.super(); - this.context = context; - } - - @Override - public PipelineOptions getPipelineOptions() { - return context.getPipelineOptions(); - } - - @Override - public void output(OutputT output) { - context.output(output); - } - - @Override - public void outputWithTimestamp(OutputT output, Instant timestamp) { - context.outputWithTimestamp(output, timestamp); - } - - @Override - public void sideOutput(TupleTag tag, T output) { - context.sideOutput(tag, output); - } - - @Override - public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { - context.sideOutputWithTimestamp(tag, output, timestamp); - } - - @Override - public BoundedWindow window() { - // The DoFn doesn't allow us to ask for these outside ProcessElements, so this - // should be unreachable. - throw new UnsupportedOperationException("Can only get the window in ProcessElements"); - } - - @Override - public DoFn.InputProvider inputProvider() { - throw new UnsupportedOperationException("inputProvider() exists only for testing"); - } - - @Override - public DoFn.OutputReceiver outputReceiver() { - throw new UnsupportedOperationException("outputReceiver() exists only for testing"); - } - } - - private static class ProcessContextAdapter - extends DoFn.ProcessContext - implements DoFn.ExtraContextFactory { - - private OldDoFn.ProcessContext context; - - private ProcessContextAdapter( - DoFn fn, - OldDoFn.ProcessContext context) { - fn.super(); - this.context = context; - } - - @Override - public PipelineOptions getPipelineOptions() { - return context.getPipelineOptions(); - } - - @Override - public T sideInput(PCollectionView view) { - return context.sideInput(view); - } - - @Override - public void output(OutputT output) { - context.output(output); - } - - @Override - public void outputWithTimestamp(OutputT output, Instant timestamp) { - context.outputWithTimestamp(output, timestamp); - } - - @Override - public void sideOutput(TupleTag tag, T output) { - context.sideOutput(tag, output); - } - - @Override - public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { - context.sideOutputWithTimestamp(tag, output, timestamp); - } - - @Override - public InputT element() { - return context.element(); - } - - @Override - public Instant timestamp() { - return context.timestamp(); - } - - @Override - public PaneInfo pane() { - return context.pane(); - } - - @Override - public BoundedWindow window() { - return context.window(); - } - - @Override - public DoFn.InputProvider inputProvider() { - throw new UnsupportedOperationException("inputProvider() exists only for testing"); - } - - @Override - public DoFn.OutputReceiver outputReceiver() { - throw new UnsupportedOperationException("outputReceiver() exists only for testing"); - } - } - - public static Class getDoFnClass(OldDoFn fn) { - if (fn instanceof SimpleDoFnAdapter) { - return ((SimpleDoFnAdapter) fn).fn.getClass(); - } else { - return fn.getClass(); - } - } - - private static class SimpleDoFnAdapter extends OldDoFn { - - private final DoFn fn; - private transient DoFnInvoker invoker; - - private SimpleDoFnAdapter(DoFnReflector reflector, DoFn fn) { - super(fn.aggregators); - this.fn = fn; - this.invoker = reflector.bindInvoker(fn); - } - - @Override - public void setup() throws Exception { - invoker.invokeSetup(); - } - - @Override - public void startBundle(OldDoFn.Context c) throws Exception { - ContextAdapter adapter = new ContextAdapter<>(fn, c); - invoker.invokeStartBundle(adapter, adapter); - } - - @Override - public void finishBundle(OldDoFn.Context c) throws Exception { - ContextAdapter adapter = new ContextAdapter<>(fn, c); - invoker.invokeFinishBundle(adapter, adapter); - } - - @Override - public void teardown() { - invoker.invokeTeardown(); - } - - @Override - public void processElement(OldDoFn.ProcessContext c) throws Exception { - ProcessContextAdapter adapter = new ProcessContextAdapter<>(fn, c); - invoker.invokeProcessElement(adapter, adapter); - } - - @Override - protected TypeDescriptor getInputTypeDescriptor() { - return fn.getInputTypeDescriptor(); - } - - @Override - protected TypeDescriptor getOutputTypeDescriptor() { - return fn.getOutputTypeDescriptor(); - } - - @Override - public Duration getAllowedTimestampSkew() { - return fn.getAllowedTimestampSkew(); - } - - @Override - public void populateDisplayData(DisplayData.Builder builder) { - builder.include(fn); - } - - private void readObject(java.io.ObjectInputStream in) - throws IOException, ClassNotFoundException { - in.defaultReadObject(); - invoker = DoFnReflector.of(fn.getClass()).bindInvoker(fn); - } - } - - private static class WindowDoFnAdapter - extends SimpleDoFnAdapter implements OldDoFn.RequiresWindowAccess { - - private WindowDoFnAdapter(DoFnReflector reflector, DoFn fn) { - super(reflector, fn); - } - } - - private static enum BeforeDelegation { - NOOP { - @Override - StackManipulation manipulation( - TypeDescription delegateType, MethodDescription instrumentedMethod, boolean finalStep) { - Preconditions.checkArgument(!finalStep, - "Shouldn't use NOOP delegation if there is nothing to do afterwards."); - return StackManipulation.Trivial.INSTANCE; - } - }, - INVOKE_PREPARE_FOR_PROCESSING { - private final Assigner assigner = Assigner.DEFAULT; - - @Override - StackManipulation manipulation( - TypeDescription delegateType, MethodDescription instrumentedMethod, boolean finalStep) { - MethodDescription prepareMethod; - try { - prepareMethod = new MethodLocator.ForExplicitMethod( - new MethodDescription.ForLoadedMethod( - DoFn.class.getDeclaredMethod("prepareForProcessing"))) - .resolve(instrumentedMethod); - } catch (NoSuchMethodException | SecurityException e) { - throw new RuntimeException("Unable to locate prepareForProcessing method", e); - } - - if (finalStep) { - return new StackManipulation.Compound( - // Invoke the prepare method - MethodInvoker.Simple.INSTANCE.invoke(prepareMethod), - // Return from the invokeStartBundle when we're done. - TerminationHandler.Returning.INSTANCE.resolve( - assigner, instrumentedMethod, prepareMethod)); - } else { - return new StackManipulation.Compound( - // Duplicate the delegation target so that it remains after we invoke prepare - Duplication.duplicate(delegateType), - // Invoke the prepare method - MethodInvoker.Simple.INSTANCE.invoke(prepareMethod), - // Drop the return value from prepareForProcessing - TerminationHandler.Dropping.INSTANCE.resolve( - assigner, instrumentedMethod, prepareMethod)); - } - } - }; - - /** - * Stack manipulation to perform prior to the delegate call. - * - *
      - *
    • Precondition: Stack has the delegate target on top of the stack - *
    • Postcondition: If finalStep is true, then we've returned from the method. Otherwise, the - * stack still has the delegate target on top of the stack. - *
    - * - * @param delegateType The type of the delegate target, in case it needs to be duplicated. - * @param instrumentedMethod The method bing instrumented. Necessary for resolving types and - * other information. - * @param finalStep If true, return from the {@code invokeStartBundle} method after invoking - * {@code prepareForProcessing} on the delegate. - */ - abstract StackManipulation manipulation( - TypeDescription delegateType, MethodDescription instrumentedMethod, boolean finalStep); - } - - /** - * A byte-buddy {@link Implementation} that delegates a call that receives - * {@link AdditionalParameter} to the given {@link DoFn} method. - */ - private static final class InvokerDelegation implements Implementation { - @Nullable - private final Method target; - private final BeforeDelegation before; - private final List args; - private final Assigner assigner = Assigner.DEFAULT; - private FieldDescription field; - - /** - * Create the {@link InvokerDelegation} for the specified method. - * - * @param target the method to delegate to - * @param isStartBundle whether or not this is the {@code startBundle} call - * @param args the {@link AdditionalParameter} to be passed to the {@code target} - */ - private InvokerDelegation( - @Nullable Method target, - BeforeDelegation before, - List args) { - this.target = target; - this.before = before; - this.args = args; - } - - /** - * Generate the {@link Implementation} of one of the life-cycle methods of a - * {@link DoFn}. - */ - private static Implementation create( - @Nullable final Method target, BeforeDelegation before, List args) { - if (target == null && before == BeforeDelegation.NOOP) { - // There is no target to call and nothing needs to happen before. Just produce a stub. - return StubMethod.INSTANCE; - } else { - // We need to generate a non-empty method implementation. - return new InvokerDelegation(target, before, args); - } - } - - @Override - public InstrumentedType prepare(InstrumentedType instrumentedType) { - // Remember the field description of the instrumented type. - field = instrumentedType.getDeclaredFields() - .filter(ElementMatchers.named(FN_DELEGATE_FIELD_NAME)).getOnly(); - - // Delegating the method call doesn't require any changes to the instrumented type. - return instrumentedType; - } - - /** - * Stack manipulation to push the {@link DoFn} reference stored in the - * delegate field of the invoker on to the top of the stack. - * - *

    This implementation is derived from the code for - * {@code MethodCall.invoke(m).onInstanceField(clazz, delegateField)} with two key differences. - * First, it doesn't add a synthetic field each time, which is critical to avoid duplicate field - * definitions. Second, it uses the {@link AdditionalParameter} to populate the arguments to the - * method. - */ - private StackManipulation pushDelegateField() { - return new StackManipulation.Compound( - // Push "this" reference to the stack - MethodVariableAccess.REFERENCE.loadOffset(0), - // Access the delegate field of the the invoker - FieldAccess.forField(field).getter()); - } - - private StackManipulation pushArgument( - AdditionalParameter arg, MethodDescription instrumentedMethod) { - MethodDescription transform = arg.method; - - return new StackManipulation.Compound( - // Push the ExtraContextFactory which must have been argument 2 of the instrumented method - MethodVariableAccess.REFERENCE.loadOffset(2), - // Invoke the appropriate method to produce the context argument - MethodInvocation.invoke(transform)); - } - - private StackManipulation invokeTargetMethod(MethodDescription instrumentedMethod) { - MethodDescription targetMethod = new MethodLocator.ForExplicitMethod( - new MethodDescription.ForLoadedMethod(target)).resolve(instrumentedMethod); - ParameterList params = targetMethod.getParameters(); - - List parameters; - if (!params.isEmpty()) { - // Instructions to setup the parameters for the call - parameters = new ArrayList<>(args.size() + 1); - // 1. The first argument in the delegate method must be the context. This corresponds to - // the first argument in the instrumented method, so copy that. - parameters.add(MethodVariableAccess.of(params.get(0).getType().getSuperClass()) - .loadOffset(1)); - // 2. For each of the extra arguments push the appropriate value. - for (AdditionalParameter arg : args) { - parameters.add(pushArgument(arg, instrumentedMethod)); - } - } else { - parameters = Collections.emptyList(); - } - - return new StackManipulation.Compound( - // Push the parameters - new StackManipulation.Compound(parameters), - // Invoke the target method - wrapWithUserCodeException(MethodInvoker.Simple.INSTANCE.invoke(targetMethod)), - // Return from the instrumented method - TerminationHandler.Returning.INSTANCE.resolve( - assigner, instrumentedMethod, targetMethod)); - } - - /** - * Wrap a given stack manipulation in a try catch block. Any exceptions thrown within the - * try are wrapped with a {@link UserCodeException}. - */ - private StackManipulation wrapWithUserCodeException( - final StackManipulation tryBody) { - final MethodDescription createUserCodeException; - try { - createUserCodeException = new MethodDescription.ForLoadedMethod( - UserCodeException.class.getDeclaredMethod("wrap", Throwable.class)); - } catch (NoSuchMethodException | SecurityException e) { - throw new RuntimeException("Unable to find UserCodeException.wrap", e); - } - - return new StackManipulation() { - @Override - public boolean isValid() { - return tryBody.isValid(); - } - - @Override - public Size apply(MethodVisitor mv, Context implementationContext) { - Label tryBlockStart = new Label(); - Label tryBlockEnd = new Label(); - Label catchBlockStart = new Label(); - Label catchBlockEnd = new Label(); - - String throwableName = - new TypeDescription.ForLoadedType(Throwable.class).getInternalName(); - mv.visitTryCatchBlock(tryBlockStart, tryBlockEnd, catchBlockStart, throwableName); - - // The try block attempts to perform the expected operations, then jumps to success - mv.visitLabel(tryBlockStart); - Size trySize = tryBody.apply(mv, implementationContext); - mv.visitJumpInsn(Opcodes.GOTO, catchBlockEnd); - mv.visitLabel(tryBlockEnd); - - // The handler wraps the exception, and then throws. - mv.visitLabel(catchBlockStart); - // Add the exception to the frame - mv.visitFrame(Opcodes.F_SAME1, - // No local variables - 0, new Object[] {}, - // 1 stack element (the throwable) - 1, new Object[] { throwableName }); - - Size catchSize = new StackManipulation.Compound( - MethodInvocation.invoke(createUserCodeException), - Throw.INSTANCE) - .apply(mv, implementationContext); - - mv.visitLabel(catchBlockEnd); - // The frame contents after the try/catch block is the same - // as it was before. - mv.visitFrame(Opcodes.F_SAME, - // No local variables - 0, new Object[] {}, - // No new stack variables - 0, new Object[] {}); - - return new Size( - trySize.getSizeImpact(), - Math.max(trySize.getMaximalSize(), catchSize.getMaximalSize())); - } - }; - } - - @Override - public ByteCodeAppender appender(final Target implementationTarget) { - return new ByteCodeAppender() { - @Override - public Size apply( - MethodVisitor methodVisitor, - Context implementationContext, - MethodDescription instrumentedMethod) { - StackManipulation.Size size = new StackManipulation.Compound( - // Put the target on top of the stack - pushDelegateField(), - // Do any necessary pre-delegation work - before.manipulation(field.getType().asErasure(), instrumentedMethod, target == null), - // Invoke the target method, if there is one. If there wasn't, then isStartBundle was - // true, and we've already emitted the appropriate return instructions. - target != null - ? invokeTargetMethod(instrumentedMethod) - : StackManipulation.Trivial.INSTANCE) - .apply(methodVisitor, implementationContext); - return new Size(size.getMaximalSize(), instrumentedMethod.getStackSize()); - } - }; - } - } - - /** - * A constructor {@link Implementation} for a {@link DoFnInvoker class}. Produces the byte code - * for a constructor that takes a single argument and assigns it to the delegate field. - * {@link AdditionalParameter} to the given {@link DoFn} method. - */ - private static final class InvokerConstructor implements Implementation { - @Override - public InstrumentedType prepare(InstrumentedType instrumentedType) { - return instrumentedType; - } - - @Override - public ByteCodeAppender appender(final Target implementationTarget) { - return new ByteCodeAppender() { - @Override - public Size apply( - MethodVisitor methodVisitor, - Context implementationContext, - MethodDescription instrumentedMethod) { - StackManipulation.Size size = new StackManipulation.Compound( - // Load the this reference - MethodVariableAccess.REFERENCE.loadOffset(0), - // Invoke the super constructor (default constructor of Object) - MethodInvocation - .invoke(new TypeDescription.ForLoadedType(Object.class) - .getDeclaredMethods() - .filter(ElementMatchers.isConstructor() - .and(ElementMatchers.takesArguments(0))) - .getOnly()), - // Load the this reference - MethodVariableAccess.REFERENCE.loadOffset(0), - // Load the delegate argument - MethodVariableAccess.REFERENCE.loadOffset(1), - // Assign the delegate argument to the delegate field - FieldAccess.forField(implementationTarget.getInstrumentedType() - .getDeclaredFields() - .filter(ElementMatchers.named(FN_DELEGATE_FIELD_NAME)) - .getOnly()).putter(), - // Return void. - MethodReturn.VOID - ).apply(methodVisitor, implementationContext); - return new Size(size.getMaximalSize(), instrumentedMethod.getStackSize()); - } - }; - } - } -} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java index dd1baab4a265..4cd410a19597 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java @@ -92,7 +92,7 @@ public static DoFnTester of(OldDoFn DoFnTester of(DoFn fn) { - return new DoFnTester(DoFnReflector.of(fn.getClass()).toDoFn(fn)); + return new DoFnTester(DoFnAdapters.toOldDoFn(fn)); } /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java index aa5753197b1d..af500ba0c9c3 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java @@ -536,7 +536,7 @@ private static Bound of( private static OldDoFn adapt(DoFn fn) { - return DoFnReflector.of(fn.getClass()).toDoFn(fn); + return DoFnAdapters.toOldDoFn(fn); } /** @@ -747,7 +747,7 @@ protected Coder getDefaultOutputCoder(PCollection inp @Override protected String getKindString() { - Class clazz = DoFnReflector.getDoFnClass(fn); + Class clazz = DoFnAdapters.getDoFnClass(fn); if (clazz.isAnonymousClass()) { return "AnonymousParDo"; } else { @@ -968,7 +968,7 @@ public Coder getDefaultOutputCoder( @Override protected String getKindString() { - Class clazz = DoFnReflector.getDoFnClass(fn); + Class clazz = DoFnAdapters.getDoFnClass(fn); if (clazz.isAnonymousClass()) { return "AnonymousParMultiDo"; } else { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java new file mode 100644 index 000000000000..5818a5933f2d --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.transforms.reflect; + +import org.apache.beam.sdk.transforms.DoFn; + +/** + * Interface for invoking the {@code DoFn} processing methods. + * + * Instantiating a {@link DoFnInvoker} associates it with a specific {@link DoFn} instance, + * referred to as the bound {@link DoFn}. + */ +public interface DoFnInvoker { + /** + * Invoke the {@link DoFn.Setup} method on the bound {@link DoFn}. + */ + void invokeSetup(); + + /** + * Invoke the {@link DoFn.StartBundle} method on the bound {@link DoFn}. + * + * @param c The {@link DoFn.Context} to invoke the fn with. + */ + void invokeStartBundle(DoFn.Context c); + + /** + * Invoke the {@link DoFn.FinishBundle} method on the bound {@link DoFn}. + * + * @param c The {@link DoFn.Context} to invoke the fn with. + */ + void invokeFinishBundle(DoFn.Context c); + + /** + * Invoke the {@link DoFn.Teardown} method on the bound {@link DoFn}. + */ + void invokeTeardown(); + + /** + * Invoke the {@link DoFn.ProcessElement} method on the bound {@link DoFn}. + * + * @param c The {@link DoFn.ProcessContext} to invoke the fn with. + * @param extra Factory for producing extra parameter objects (such as window), if necessary. + */ + void invokeProcessElement( + DoFn.ProcessContext c, DoFn.ExtraContextFactory extra); +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java new file mode 100644 index 000000000000..73874d767d3c --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java @@ -0,0 +1,506 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.transforms.reflect; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; + +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.DoFn.FinishBundle; +import org.apache.beam.sdk.transforms.DoFn.ProcessElement; +import org.apache.beam.sdk.transforms.DoFn.Setup; +import org.apache.beam.sdk.transforms.DoFn.StartBundle; +import org.apache.beam.sdk.transforms.DoFn.Teardown; +import org.apache.beam.sdk.util.UserCodeException; + +import net.bytebuddy.ByteBuddy; +import net.bytebuddy.NamingStrategy; +import net.bytebuddy.description.field.FieldDescription; +import net.bytebuddy.description.method.MethodDescription; +import net.bytebuddy.description.modifier.FieldManifestation; +import net.bytebuddy.description.modifier.Visibility; +import net.bytebuddy.description.type.TypeDescription; +import net.bytebuddy.dynamic.DynamicType; +import net.bytebuddy.dynamic.loading.ClassLoadingStrategy; +import net.bytebuddy.dynamic.scaffold.InstrumentedType; +import net.bytebuddy.dynamic.scaffold.subclass.ConstructorStrategy; +import net.bytebuddy.implementation.Implementation; +import net.bytebuddy.implementation.MethodCall; +import net.bytebuddy.implementation.bind.MethodDelegationBinder; +import net.bytebuddy.implementation.bytecode.ByteCodeAppender; +import net.bytebuddy.implementation.bytecode.StackManipulation; +import net.bytebuddy.implementation.bytecode.Throw; +import net.bytebuddy.implementation.bytecode.member.FieldAccess; +import net.bytebuddy.implementation.bytecode.member.MethodInvocation; +import net.bytebuddy.implementation.bytecode.member.MethodReturn; +import net.bytebuddy.implementation.bytecode.member.MethodVariableAccess; +import net.bytebuddy.jar.asm.Label; +import net.bytebuddy.jar.asm.MethodVisitor; +import net.bytebuddy.jar.asm.Opcodes; +import net.bytebuddy.matcher.ElementMatchers; + +import java.lang.reflect.Constructor; +import java.lang.reflect.InvocationTargetException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.EnumMap; +import java.util.LinkedHashMap; +import java.util.Map; + +import javax.annotation.Nullable; + +/** Dynamically generates {@link DoFnInvoker} instances for invoking a {@link DoFn}. */ +public class DoFnInvokers { + public static final DoFnInvokers INSTANCE = new DoFnInvokers(); + + private static final String FN_DELEGATE_FIELD_NAME = "delegate"; + + /** + * A cache of constructors of generated {@link DoFnInvoker} classes, keyed by {@link DoFn} class. + * Needed because generating an invoker class is expensive, and to avoid generating an excessive + * number of classes consuming PermGen memory. + */ + private final Map, Constructor> byteBuddyInvokerConstructorCache = + new LinkedHashMap<>(); + + private DoFnInvokers() {} + + /** @return the {@link DoFnInvoker} for the given {@link DoFn}. */ + public DoFnInvoker newByteBuddyInvoker( + DoFn fn) { + return newByteBuddyInvoker(DoFnSignatures.INSTANCE.getOrParseSignature(fn.getClass()), fn); + } + + /** @return the {@link DoFnInvoker} for the given {@link DoFn}. */ + public DoFnInvoker newByteBuddyInvoker( + DoFnSignature signature, DoFn fn) { + checkArgument( + signature.fnClass().equals(fn.getClass()), + "Signature is for class %s, but fn is of class %s", + signature.fnClass(), + fn.getClass()); + try { + @SuppressWarnings("unchecked") + DoFnInvoker invoker = + (DoFnInvoker) + getOrGenerateByteBuddyInvokerConstructor(signature).newInstance(fn); + return invoker; + } catch (InstantiationException + | IllegalAccessException + | IllegalArgumentException + | InvocationTargetException + | SecurityException e) { + throw new RuntimeException("Unable to bind invoker for " + fn.getClass(), e); + } + } + + /** + * Returns a generated constructor for a {@link DoFnInvoker} for the given {@link DoFn} class and + * caches it. + */ + private synchronized Constructor getOrGenerateByteBuddyInvokerConstructor( + DoFnSignature signature) { + Class fnClass = signature.fnClass(); + Constructor constructor = byteBuddyInvokerConstructorCache.get(fnClass); + if (constructor == null) { + Class> invokerClass = generateInvokerClass(signature); + try { + constructor = invokerClass.getConstructor(fnClass); + } catch (IllegalArgumentException | NoSuchMethodException | SecurityException e) { + throw new RuntimeException(e); + } + byteBuddyInvokerConstructorCache.put(fnClass, constructor); + } + return constructor; + } + + /** Generates a {@link DoFnInvoker} class for the given {@link DoFnSignature}. */ + private static Class> generateInvokerClass(DoFnSignature signature) { + Class fnClass = signature.fnClass(); + + final TypeDescription clazzDescription = new TypeDescription.ForLoadedType(fnClass); + + DynamicType.Builder builder = + new ByteBuddy() + // Create subclasses inside the target class, to have access to + // private and package-private bits + .with( + new NamingStrategy.SuffixingRandom("auxiliary") { + @Override + public String subclass(TypeDescription.Generic superClass) { + return super.name(clazzDescription); + } + }) + // Create a subclass of DoFnInvoker + .subclass(DoFnInvoker.class, ConstructorStrategy.Default.NO_CONSTRUCTORS) + .defineField( + FN_DELEGATE_FIELD_NAME, fnClass, Visibility.PRIVATE, FieldManifestation.FINAL) + .defineConstructor(Visibility.PUBLIC) + .withParameter(fnClass) + .intercept(new InvokerConstructor()) + .method(ElementMatchers.named("invokeProcessElement")) + .intercept(new ProcessElementDelegation(signature.processElement())) + .method(ElementMatchers.named("invokeStartBundle")) + .intercept( + signature.startBundle() == null + ? new NoopMethodImplementation() + : new BundleMethodDelegation(signature.startBundle())) + .method(ElementMatchers.named("invokeFinishBundle")) + .intercept( + signature.finishBundle() == null + ? new NoopMethodImplementation() + : new BundleMethodDelegation(signature.finishBundle())) + .method(ElementMatchers.named("invokeSetup")) + .intercept( + signature.setup() == null + ? new NoopMethodImplementation() + : new LifecycleMethodDelegation(signature.setup())) + .method(ElementMatchers.named("invokeTeardown")) + .intercept( + signature.teardown() == null + ? new NoopMethodImplementation() + : new LifecycleMethodDelegation(signature.teardown())); + + DynamicType.Unloaded unloaded = builder.make(); + + @SuppressWarnings("unchecked") + Class> res = + (Class>) + unloaded + .load(DoFnInvokers.class.getClassLoader(), ClassLoadingStrategy.Default.INJECTION) + .getLoaded(); + return res; + } + + /** Implements an invoker method by doing nothing and immediately returning void. */ + private static class NoopMethodImplementation implements Implementation { + @Override + public InstrumentedType prepare(InstrumentedType instrumentedType) { + return instrumentedType; + } + + @Override + public ByteCodeAppender appender(final Target implementationTarget) { + return new ByteCodeAppender() { + @Override + public Size apply( + MethodVisitor methodVisitor, + Context implementationContext, + MethodDescription instrumentedMethod) { + StackManipulation manipulation = MethodReturn.VOID; + StackManipulation.Size size = manipulation.apply(methodVisitor, implementationContext); + return new Size(size.getMaximalSize(), instrumentedMethod.getStackSize()); + } + }; + } + } + + /** + * Base class for implementing an invoker method by delegating to a method of the target {@link + * DoFn}. + */ + private abstract static class MethodDelegation implements Implementation { + FieldDescription delegateField; + + @Override + public InstrumentedType prepare(InstrumentedType instrumentedType) { + // Remember the field description of the instrumented type. + delegateField = + instrumentedType + .getDeclaredFields() + .filter(ElementMatchers.named(FN_DELEGATE_FIELD_NAME)) + .getOnly(); + + // Delegating the method call doesn't require any changes to the instrumented type. + return instrumentedType; + } + + @Override + public ByteCodeAppender appender(final Target implementationTarget) { + return new ByteCodeAppender() { + @Override + public Size apply( + MethodVisitor methodVisitor, + Context implementationContext, + MethodDescription instrumentedMethod) { + StackManipulation manipulation = + new StackManipulation.Compound( + // Push "this" reference to the stack + MethodVariableAccess.REFERENCE.loadOffset(0), + // Access the delegate field of the the invoker + FieldAccess.forField(delegateField).getter(), + invokeTargetMethod(instrumentedMethod)); + StackManipulation.Size size = manipulation.apply(methodVisitor, implementationContext); + return new Size(size.getMaximalSize(), instrumentedMethod.getStackSize()); + } + }; + } + + /** + * Generates code to invoke the target method. When this is called the delegate field will be on + * top of the stack. This should add any necessary arguments to the stack and then perform the + * method invocation. + */ + protected abstract StackManipulation invokeTargetMethod(MethodDescription instrumentedMethod); + } + + /** + * Implements the invoker's {@link DoFnInvoker#invokeProcessElement} method by delegating to the + * {@link DoFn.ProcessElement} method. + */ + private static final class ProcessElementDelegation extends MethodDelegation { + private static final Map + EXTRA_CONTEXT_FACTORY_METHODS; + + static { + try { + Map methods = + new EnumMap<>(DoFnSignature.ProcessElementMethod.Parameter.class); + methods.put( + DoFnSignature.ProcessElementMethod.Parameter.BOUNDED_WINDOW, + new MethodDescription.ForLoadedMethod( + DoFn.ExtraContextFactory.class.getMethod("window"))); + methods.put( + DoFnSignature.ProcessElementMethod.Parameter.INPUT_PROVIDER, + new MethodDescription.ForLoadedMethod( + DoFn.ExtraContextFactory.class.getMethod("inputProvider"))); + methods.put( + DoFnSignature.ProcessElementMethod.Parameter.OUTPUT_RECEIVER, + new MethodDescription.ForLoadedMethod( + DoFn.ExtraContextFactory.class.getMethod("outputReceiver"))); + EXTRA_CONTEXT_FACTORY_METHODS = Collections.unmodifiableMap(methods); + } catch (Exception e) { + throw new RuntimeException( + "Failed to locate an ExtraContextFactory method that was expected to exist", e); + } + } + + private final DoFnSignature.ProcessElementMethod signature; + + /** Implementation of {@link MethodDelegation} for the {@link ProcessElement} method. */ + private ProcessElementDelegation(DoFnSignature.ProcessElementMethod signature) { + this.signature = signature; + } + + @Override + protected StackManipulation invokeTargetMethod(MethodDescription instrumentedMethod) { + MethodDescription targetMethod = + new MethodCall.MethodLocator.ForExplicitMethod( + new MethodDescription.ForLoadedMethod(signature.targetMethod())) + .resolve(instrumentedMethod); + + // Parameters of the wrapper invoker method: + // DoFn.ProcessContext, ExtraContextFactory. + // Parameters of the wrapped DoFn method: + // DoFn.ProcessContext, [BoundedWindow, InputProvider, OutputReceiver] in any order + ArrayList parameters = new ArrayList<>(); + // Push the ProcessContext argument. + parameters.add(MethodVariableAccess.REFERENCE.loadOffset(1)); + // Push the extra arguments in their actual order. + StackManipulation pushExtraContextFactory = MethodVariableAccess.REFERENCE.loadOffset(2); + for (DoFnSignature.ProcessElementMethod.Parameter param : signature.extraParameters()) { + parameters.add( + new StackManipulation.Compound( + pushExtraContextFactory, + MethodInvocation.invoke(EXTRA_CONTEXT_FACTORY_METHODS.get(param)))); + } + + return new StackManipulation.Compound( + // Push the parameters + new StackManipulation.Compound(parameters), + // Invoke the target method + wrapWithUserCodeException( + MethodDelegationBinder.MethodInvoker.Simple.INSTANCE.invoke(targetMethod)), + // Return from the instrumented method + MethodReturn.VOID); + } + } + + /** + * Implements {@link DoFnInvoker#invokeStartBundle} or {@link DoFnInvoker#invokeFinishBundle} by + * delegating respectively to the {@link StartBundle} and {@link FinishBundle} methods. + */ + private static final class BundleMethodDelegation extends MethodDelegation { + private final DoFnSignature.BundleMethod signature; + + private BundleMethodDelegation(@Nullable DoFnSignature.BundleMethod signature) { + this.signature = signature; + } + + @Override + protected StackManipulation invokeTargetMethod(MethodDescription instrumentedMethod) { + MethodDescription targetMethod = + new MethodCall.MethodLocator.ForExplicitMethod( + new MethodDescription.ForLoadedMethod(checkNotNull(signature).targetMethod())) + .resolve(instrumentedMethod); + return new StackManipulation.Compound( + // Push the parameters + MethodVariableAccess.REFERENCE.loadOffset(1), + // Invoke the target method + wrapWithUserCodeException( + MethodDelegationBinder.MethodInvoker.Simple.INSTANCE.invoke(targetMethod)), + MethodReturn.VOID); + } + } + + /** + * Implements {@link DoFnInvoker#invokeSetup} or {@link DoFnInvoker#invokeTeardown} by delegating + * respectively to the {@link Setup} and {@link Teardown} methods. + */ + private static final class LifecycleMethodDelegation extends MethodDelegation { + private final DoFnSignature.LifecycleMethod signature; + + private LifecycleMethodDelegation(@Nullable DoFnSignature.LifecycleMethod signature) { + this.signature = signature; + } + + @Override + protected StackManipulation invokeTargetMethod(MethodDescription instrumentedMethod) { + MethodDescription targetMethod = + new MethodCall.MethodLocator.ForExplicitMethod( + new MethodDescription.ForLoadedMethod(checkNotNull(signature).targetMethod())) + .resolve(instrumentedMethod); + return new StackManipulation.Compound( + wrapWithUserCodeException( + MethodDelegationBinder.MethodInvoker.Simple.INSTANCE.invoke(targetMethod)), + MethodReturn.VOID); + } + } + + /** + * Wraps a given stack manipulation in a try catch block. Any exceptions thrown within the try are + * wrapped with a {@link UserCodeException}. + */ + private static StackManipulation wrapWithUserCodeException(final StackManipulation tryBody) { + final MethodDescription createUserCodeException; + try { + createUserCodeException = + new MethodDescription.ForLoadedMethod( + UserCodeException.class.getDeclaredMethod("wrap", Throwable.class)); + } catch (NoSuchMethodException | SecurityException e) { + throw new RuntimeException("Unable to find UserCodeException.wrap", e); + } + + return new StackManipulation() { + @Override + public boolean isValid() { + return tryBody.isValid(); + } + + @Override + public Size apply(MethodVisitor mv, Implementation.Context implementationContext) { + Label tryBlockStart = new Label(); + Label tryBlockEnd = new Label(); + Label catchBlockStart = new Label(); + Label catchBlockEnd = new Label(); + + String throwableName = new TypeDescription.ForLoadedType(Throwable.class).getInternalName(); + mv.visitTryCatchBlock(tryBlockStart, tryBlockEnd, catchBlockStart, throwableName); + + // The try block attempts to perform the expected operations, then jumps to success + mv.visitLabel(tryBlockStart); + Size trySize = tryBody.apply(mv, implementationContext); + mv.visitJumpInsn(Opcodes.GOTO, catchBlockEnd); + mv.visitLabel(tryBlockEnd); + + // The handler wraps the exception, and then throws. + mv.visitLabel(catchBlockStart); + // Add the exception to the frame + mv.visitFrame( + Opcodes.F_SAME1, + // No local variables + 0, + new Object[] {}, + // 1 stack element (the throwable) + 1, + new Object[] {throwableName}); + + Size catchSize = + new Compound(MethodInvocation.invoke(createUserCodeException), Throw.INSTANCE) + .apply(mv, implementationContext); + + mv.visitLabel(catchBlockEnd); + // The frame contents after the try/catch block is the same + // as it was before. + mv.visitFrame( + Opcodes.F_SAME, + // No local variables + 0, + new Object[] {}, + // No new stack variables + 0, + new Object[] {}); + + return new Size( + trySize.getSizeImpact(), + Math.max(trySize.getMaximalSize(), catchSize.getMaximalSize())); + } + }; + } + + /** + * A constructor {@link Implementation} for a {@link DoFnInvoker class}. Produces the byte code + * for a constructor that takes a single argument and assigns it to the delegate field. + */ + private static final class InvokerConstructor implements Implementation { + @Override + public InstrumentedType prepare(InstrumentedType instrumentedType) { + return instrumentedType; + } + + @Override + public ByteCodeAppender appender(final Target implementationTarget) { + return new ByteCodeAppender() { + @Override + public Size apply( + MethodVisitor methodVisitor, + Context implementationContext, + MethodDescription instrumentedMethod) { + StackManipulation.Size size = + new StackManipulation.Compound( + // Load the this reference + MethodVariableAccess.REFERENCE.loadOffset(0), + // Invoke the super constructor (default constructor of Object) + MethodInvocation.invoke( + new TypeDescription.ForLoadedType(Object.class) + .getDeclaredMethods() + .filter( + ElementMatchers.isConstructor() + .and(ElementMatchers.takesArguments(0))) + .getOnly()), + // Load the this reference + MethodVariableAccess.REFERENCE.loadOffset(0), + // Load the delegate argument + MethodVariableAccess.REFERENCE.loadOffset(1), + // Assign the delegate argument to the delegate field + FieldAccess.forField( + implementationTarget + .getInstrumentedType() + .getDeclaredFields() + .filter(ElementMatchers.named(FN_DELEGATE_FIELD_NAME)) + .getOnly()) + .putter(), + // Return void. + MethodReturn.VOID) + .apply(methodVisitor, implementationContext); + return new Size(size.getMaximalSize(), instrumentedMethod.getStackSize()); + } + }; + } + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java new file mode 100644 index 000000000000..6730140ac309 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.transforms.reflect; + +import org.apache.beam.sdk.transforms.DoFn; + +import com.google.auto.value.AutoValue; + +import java.lang.reflect.Method; +import java.util.Collections; +import java.util.List; + +import javax.annotation.Nullable; + +/** + * Describes the signature of a {@link DoFn}, in particular, which features it uses, which extra + * context it requires, types of the input and output elements, etc. + * + *

    See A new DoFn. + */ +@AutoValue +public abstract class DoFnSignature { + public abstract Class fnClass(); + + public abstract ProcessElementMethod processElement(); + + @Nullable + public abstract BundleMethod startBundle(); + + @Nullable + public abstract BundleMethod finishBundle(); + + @Nullable + public abstract LifecycleMethod setup(); + + @Nullable + public abstract LifecycleMethod teardown(); + + static DoFnSignature create( + Class fnClass, + ProcessElementMethod processElement, + @Nullable BundleMethod startBundle, + @Nullable BundleMethod finishBundle, + @Nullable LifecycleMethod setup, + @Nullable LifecycleMethod teardown) { + return new AutoValue_DoFnSignature( + fnClass, + processElement, + startBundle, + finishBundle, + setup, + teardown); + } + + /** Describes a {@link DoFn.ProcessElement} method. */ + @AutoValue + public abstract static class ProcessElementMethod { + enum Parameter { + BOUNDED_WINDOW, + INPUT_PROVIDER, + OUTPUT_RECEIVER + } + + public abstract Method targetMethod(); + + public abstract List extraParameters(); + + static ProcessElementMethod create(Method targetMethod, List extraParameters) { + return new AutoValue_DoFnSignature_ProcessElementMethod( + targetMethod, Collections.unmodifiableList(extraParameters)); + } + + /** @return true if the reflected {@link DoFn} uses a Single Window. */ + public boolean usesSingleWindow() { + return extraParameters().contains(Parameter.BOUNDED_WINDOW); + } + } + + /** Describes a {@link DoFn.StartBundle} or {@link DoFn.FinishBundle} method. */ + @AutoValue + public abstract static class BundleMethod { + public abstract Method targetMethod(); + + static BundleMethod create(Method targetMethod) { + return new AutoValue_DoFnSignature_BundleMethod(targetMethod); + } + } + + /** Describes a {@link DoFn.Setup} or {@link DoFn.Teardown} method. */ + @AutoValue + public abstract static class LifecycleMethod { + public abstract Method targetMethod(); + + static LifecycleMethod create(Method targetMethod) { + return new AutoValue_DoFnSignature_LifecycleMethod(targetMethod); + } + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java new file mode 100644 index 000000000000..80b3b4f5b825 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java @@ -0,0 +1,321 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.transforms.reflect; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; + +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.common.ReflectHelpers; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.reflect.TypeParameter; +import com.google.common.reflect.TypeToken; + +import java.lang.annotation.Annotation; +import java.lang.reflect.Method; +import java.lang.reflect.Modifier; +import java.lang.reflect.ParameterizedType; +import java.lang.reflect.Type; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; + +/** + * Parses a {@link DoFn} and computes its {@link DoFnSignature}. See {@link #getOrParseSignature}. + */ +public class DoFnSignatures { + public static final DoFnSignatures INSTANCE = new DoFnSignatures(); + + private DoFnSignatures() {} + + private final Map, DoFnSignature> signatureCache = new LinkedHashMap<>(); + + /** @return the {@link DoFnSignature} for the given {@link DoFn}. */ + public synchronized DoFnSignature getOrParseSignature( + @SuppressWarnings("rawtypes") Class fn) { + DoFnSignature signature = signatureCache.get(fn); + if (signature == null) { + signatureCache.put(fn, signature = parseSignature(fn)); + } + return signature; + } + + /** Analyzes a given {@link DoFn} class and extracts its {@link DoFnSignature}. */ + private static DoFnSignature parseSignature(Class fnClass) { + TypeToken inputT = null; + TypeToken outputT = null; + + // Extract the input and output type. + checkArgument( + DoFn.class.isAssignableFrom(fnClass), + "%s must be subtype of DoFn", + fnClass.getSimpleName()); + TypeToken fnToken = TypeToken.of(fnClass); + for (TypeToken supertype : fnToken.getTypes()) { + if (!supertype.getRawType().equals(DoFn.class)) { + continue; + } + Type[] args = ((ParameterizedType) supertype.getType()).getActualTypeArguments(); + inputT = TypeToken.of(args[0]); + outputT = TypeToken.of(args[1]); + } + checkNotNull(inputT, "Unable to determine input type from %s", fnClass); + + Method processElementMethod = findAnnotatedMethod(DoFn.ProcessElement.class, fnClass, true); + Method startBundleMethod = findAnnotatedMethod(DoFn.StartBundle.class, fnClass, false); + Method finishBundleMethod = findAnnotatedMethod(DoFn.FinishBundle.class, fnClass, false); + Method setupMethod = findAnnotatedMethod(DoFn.Setup.class, fnClass, false); + Method teardownMethod = findAnnotatedMethod(DoFn.Teardown.class, fnClass, false); + + return DoFnSignature.create( + fnClass, + analyzeProcessElementMethod(fnToken, processElementMethod, inputT, outputT), + (startBundleMethod == null) + ? null + : analyzeBundleMethod(fnToken, startBundleMethod, inputT, outputT), + (finishBundleMethod == null) + ? null + : analyzeBundleMethod(fnToken, finishBundleMethod, inputT, outputT), + (setupMethod == null) ? null : analyzeLifecycleMethod(setupMethod), + (teardownMethod == null) ? null : analyzeLifecycleMethod(teardownMethod)); + } + + /** + * Generates a type token for {@code DoFn.ProcessContext} given {@code InputT} + * and {@code OutputT}. + */ + private static + TypeToken.ProcessContext> doFnProcessContextTypeOf( + TypeToken inputT, TypeToken outputT) { + return new TypeToken.ProcessContext>() {}.where( + new TypeParameter() {}, inputT) + .where(new TypeParameter() {}, outputT); + } + + /** + * Generates a type token for {@code DoFn.Context} given {@code InputT} and + * {@code OutputT}. + */ + private static TypeToken.Context> doFnContextTypeOf( + TypeToken inputT, TypeToken outputT) { + return new TypeToken.Context>() {}.where( + new TypeParameter() {}, inputT) + .where(new TypeParameter() {}, outputT); + } + + /** Generates a type token for {@code DoFn.InputProvider} given {@code InputT}. */ + private static TypeToken> inputProviderTypeOf( + TypeToken inputT) { + return new TypeToken>() {}.where( + new TypeParameter() {}, inputT); + } + + /** Generates a type token for {@code DoFn.OutputReceiver} given {@code OutputT}. */ + private static TypeToken> outputReceiverTypeOf( + TypeToken inputT) { + return new TypeToken>() {}.where( + new TypeParameter() {}, inputT); + } + + @VisibleForTesting + static DoFnSignature.ProcessElementMethod analyzeProcessElementMethod( + TypeToken fnClass, Method m, TypeToken inputT, TypeToken outputT) { + checkArgument( + void.class.equals(m.getReturnType()), "%s must have a void return type", format(m)); + checkArgument(!m.isVarArgs(), "%s must not have var args", format(m)); + + TypeToken processContextToken = doFnProcessContextTypeOf(inputT, outputT); + + Type[] params = m.getGenericParameterTypes(); + TypeToken contextToken = null; + if (params.length > 0) { + contextToken = fnClass.resolveType(params[0]); + } + checkArgument( + contextToken != null && contextToken.equals(processContextToken), + "%s must take a %s as its first argument", + format(m), + formatType(processContextToken)); + + List extraParameters = new ArrayList<>(); + TypeToken expectedInputProviderT = inputProviderTypeOf(inputT); + TypeToken expectedOutputReceiverT = outputReceiverTypeOf(outputT); + for (int i = 1; i < params.length; ++i) { + TypeToken param = fnClass.resolveType(params[i]); + Class rawType = param.getRawType(); + if (rawType.equals(BoundedWindow.class)) { + checkArgument( + !extraParameters.contains(DoFnSignature.ProcessElementMethod.Parameter.BOUNDED_WINDOW), + "Multiple BoundedWindow parameters in %s", + format(m)); + extraParameters.add(DoFnSignature.ProcessElementMethod.Parameter.BOUNDED_WINDOW); + } else if (rawType.equals(DoFn.InputProvider.class)) { + checkArgument( + !extraParameters.contains(DoFnSignature.ProcessElementMethod.Parameter.INPUT_PROVIDER), + "Multiple InputProvider parameters in %s", + format(m)); + checkArgument( + param.equals(expectedInputProviderT), + "Wrong type of InputProvider parameter for method %s: %s, should be %s", + format(m), + formatType(param), + formatType(expectedInputProviderT)); + extraParameters.add(DoFnSignature.ProcessElementMethod.Parameter.INPUT_PROVIDER); + } else if (rawType.equals(DoFn.OutputReceiver.class)) { + checkArgument( + !extraParameters.contains(DoFnSignature.ProcessElementMethod.Parameter.OUTPUT_RECEIVER), + "Multiple OutputReceiver parameters in %s", + format(m)); + checkArgument( + param.equals(expectedOutputReceiverT), + "Wrong type of OutputReceiver parameter for method %s: %s, should be %s", + format(m), + formatType(param), + formatType(expectedOutputReceiverT)); + extraParameters.add(DoFnSignature.ProcessElementMethod.Parameter.OUTPUT_RECEIVER); + } else { + List allowedParamTypes = + Arrays.asList(formatType(new TypeToken() {})); + checkArgument( + false, + "%s is not a valid context parameter for method %s. Should be one of %s", + formatType(param), + format(m), + allowedParamTypes); + } + } + + return DoFnSignature.ProcessElementMethod.create(m, extraParameters); + } + + @VisibleForTesting + static DoFnSignature.BundleMethod analyzeBundleMethod( + TypeToken fnToken, Method m, TypeToken inputT, TypeToken outputT) { + checkArgument( + void.class.equals(m.getReturnType()), "%s must have a void return type", format(m)); + checkArgument(!m.isVarArgs(), "%s must not have var args", format(m)); + + TypeToken expectedContextToken = doFnContextTypeOf(inputT, outputT); + + Type[] params = m.getGenericParameterTypes(); + checkArgument( + params.length == 1, + "%s must have a single argument of type %s", + format(m), + formatType(expectedContextToken)); + TypeToken contextToken = fnToken.resolveType(params[0]); + checkArgument( + contextToken.equals(expectedContextToken), + "Wrong type of context argument to %s: %s, must be %s", + format(m), + formatType(contextToken), + formatType(expectedContextToken)); + + return DoFnSignature.BundleMethod.create(m); + } + + private static DoFnSignature.LifecycleMethod analyzeLifecycleMethod(Method m) { + checkArgument( + void.class.equals(m.getReturnType()), "%s must have a void return type", format(m)); + checkArgument( + m.getGenericParameterTypes().length == 0, "%s must take zero arguments", format(m)); + return DoFnSignature.LifecycleMethod.create(m); + } + + private static Collection declaredMethodsWithAnnotation( + Class anno, Class startClass, Class stopClass) { + Collection matches = new ArrayList<>(); + + Class clazz = startClass; + LinkedHashSet> interfaces = new LinkedHashSet<>(); + + // First, find all declared methods on the startClass and parents (up to stopClass) + while (clazz != null && !clazz.equals(stopClass)) { + for (Method method : clazz.getDeclaredMethods()) { + if (method.isAnnotationPresent(anno)) { + matches.add(method); + } + } + + Collections.addAll(interfaces, clazz.getInterfaces()); + + clazz = clazz.getSuperclass(); + } + + // Now, iterate over all the discovered interfaces + for (Method method : ReflectHelpers.getClosureOfMethodsOnInterfaces(interfaces)) { + if (method.isAnnotationPresent(anno)) { + matches.add(method); + } + } + return matches; + } + + private static Method findAnnotatedMethod( + Class anno, Class fnClazz, boolean required) { + Collection matches = declaredMethodsWithAnnotation(anno, fnClazz, DoFn.class); + + if (matches.size() == 0) { + checkArgument( + !required, + "No method annotated with @%s found in %s", + anno.getSimpleName(), + fnClazz.getName()); + return null; + } + + // If we have at least one match, then either it should be the only match + // or it should be an extension of the other matches (which came from parent + // classes). + Method first = matches.iterator().next(); + for (Method other : matches) { + checkArgument( + first.getName().equals(other.getName()) + && Arrays.equals(first.getParameterTypes(), other.getParameterTypes()), + "Found multiple methods annotated with @%s. [%s] and [%s]", + anno.getSimpleName(), + format(first), + format(other)); + } + + // We need to be able to call it. We require it is public. + checkArgument( + (first.getModifiers() & Modifier.PUBLIC) != 0, "%s must be public", format(first)); + + // And make sure its not static. + checkArgument( + (first.getModifiers() & Modifier.STATIC) == 0, "%s must not be static", format(first)); + + return first; + } + + private static String format(Method m) { + return ReflectHelpers.CLASS_AND_METHOD_FORMATTER.apply(m); + } + + private static String formatType(TypeToken t) { + return ReflectHelpers.TYPE_SIMPLE_DESCRIPTION.apply(t.getType()); + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/package-info.java new file mode 100644 index 000000000000..4df5209e6b39 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/package-info.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +/** + * Defines reflection-based utilities for analyzing {@link org.apache.beam.sdk.transforms.DoFn}'s + * and creating {@link org.apache.beam.sdk.transforms.reflect.DoFnSignature}'s and + * {@link org.apache.beam.sdk.transforms.reflect.DoFnInvoker}'s from them. + */ +package org.apache.beam.sdk.transforms.reflect; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnReflectorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnReflectorTest.java deleted file mode 100644 index e05e5e2c667a..000000000000 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnReflectorTest.java +++ /dev/null @@ -1,822 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.transforms; - -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertSame; -import static org.junit.Assert.assertTrue; - -import org.apache.beam.sdk.transforms.DoFn.Context; -import org.apache.beam.sdk.transforms.DoFn.ExtraContextFactory; -import org.apache.beam.sdk.transforms.DoFn.ProcessContext; -import org.apache.beam.sdk.transforms.DoFn.ProcessElement; -import org.apache.beam.sdk.transforms.DoFn.Setup; -import org.apache.beam.sdk.transforms.DoFn.Teardown; -import org.apache.beam.sdk.transforms.dofnreflector.DoFnReflectorTestHelper; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.UserCodeException; - -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; -import org.mockito.Mock; -import org.mockito.MockitoAnnotations; - -import java.lang.reflect.Method; - -/** - * Tests for {@link DoFnReflector}. - */ -@RunWith(JUnit4.class) -public class DoFnReflectorTest { - - /** - * A convenience struct holding flags that indicate whether a particular method was invoked. - */ - public static class Invocations { - public boolean wasProcessElementInvoked = false; - public boolean wasStartBundleInvoked = false; - public boolean wasFinishBundleInvoked = false; - public boolean wasSetupInvoked = false; - public boolean wasTeardownInvoked = false; - private final String name; - - public Invocations(String name) { - this.name = name; - } - } - - private DoFn fn; - - @Rule - public ExpectedException thrown = ExpectedException.none(); - - @Mock - private DoFn.ProcessContext mockContext; - @Mock - private BoundedWindow mockWindow; - @Mock - private DoFn.InputProvider mockInputProvider; - @Mock - private DoFn.OutputReceiver mockOutputReceiver; - - private ExtraContextFactory extraContextFactory; - - @Before - public void setUp() { - MockitoAnnotations.initMocks(this); - this.extraContextFactory = new ExtraContextFactory() { - @Override - public BoundedWindow window() { - return mockWindow; - } - - @Override - public DoFn.InputProvider inputProvider() { - return mockInputProvider; - } - - @Override - public DoFn.OutputReceiver outputReceiver() { - return mockOutputReceiver; - } - }; - } - - private DoFnReflector underTest(DoFn fn) { - this.fn = fn; - return DoFnReflector.of(fn.getClass()); - } - - private void checkInvokeProcessElementWorks( - DoFnReflector r, Invocations... invocations) throws Exception { - assertTrue("Need at least one invocation to check", invocations.length >= 1); - for (Invocations invocation : invocations) { - assertFalse("Should not yet have called processElement on " + invocation.name, - invocation.wasProcessElementInvoked); - } - r.bindInvoker(fn).invokeProcessElement(mockContext, extraContextFactory); - for (Invocations invocation : invocations) { - assertTrue("Should have called processElement on " + invocation.name, - invocation.wasProcessElementInvoked); - } - } - - private void checkInvokeStartBundleWorks( - DoFnReflector r, Invocations... invocations) throws Exception { - assertTrue("Need at least one invocation to check", invocations.length >= 1); - for (Invocations invocation : invocations) { - assertFalse("Should not yet have called startBundle on " + invocation.name, - invocation.wasStartBundleInvoked); - } - r.bindInvoker(fn).invokeStartBundle(mockContext, extraContextFactory); - for (Invocations invocation : invocations) { - assertTrue("Should have called startBundle on " + invocation.name, - invocation.wasStartBundleInvoked); - } - } - - private void checkInvokeFinishBundleWorks( - DoFnReflector r, Invocations... invocations) throws Exception { - assertTrue("Need at least one invocation to check", invocations.length >= 1); - for (Invocations invocation : invocations) { - assertFalse("Should not yet have called finishBundle on " + invocation.name, - invocation.wasFinishBundleInvoked); - } - r.bindInvoker(fn).invokeFinishBundle(mockContext, extraContextFactory); - for (Invocations invocation : invocations) { - assertTrue("Should have called finishBundle on " + invocation.name, - invocation.wasFinishBundleInvoked); - } - } - - private void checkInvokeSetupWorks(DoFnReflector r, Invocations... invocations) throws Exception { - assertTrue("Need at least one invocation to check", invocations.length >= 1); - for (Invocations invocation : invocations) { - assertFalse("Should not yet have called setup on " + invocation.name, - invocation.wasSetupInvoked); - } - r.bindInvoker(fn).invokeSetup(); - for (Invocations invocation : invocations) { - assertTrue("Should have called setup on " + invocation.name, - invocation.wasSetupInvoked); - } - } - - private void checkInvokeTeardownWorks(DoFnReflector r, Invocations... invocations) - throws Exception { - assertTrue("Need at least one invocation to check", invocations.length >= 1); - for (Invocations invocation : invocations) { - assertFalse("Should not yet have called teardown on " + invocation.name, - invocation.wasTeardownInvoked); - } - r.bindInvoker(fn).invokeTeardown(); - for (Invocations invocation : invocations) { - assertTrue("Should have called teardown on " + invocation.name, - invocation.wasTeardownInvoked); - } - } - - @Test - public void testDoFnWithNoExtraContext() throws Exception { - final Invocations invocations = new Invocations("AnonymousClass"); - DoFnReflector reflector = underTest(new DoFn() { - - @ProcessElement - public void processElement(ProcessContext c) - throws Exception { - invocations.wasProcessElementInvoked = true; - assertSame(c, mockContext); - } - }); - - assertFalse(reflector.usesSingleWindow()); - - checkInvokeProcessElementWorks(reflector, invocations); - } - - @Test - public void testDoFnInvokersReused() throws Exception { - // Ensures that we don't create a new Invoker class for every instance of the OldDoFn. - IdentityParent fn1 = new IdentityParent(); - IdentityParent fn2 = new IdentityParent(); - DoFnReflector reflector1 = underTest(fn1); - DoFnReflector reflector2 = underTest(fn2); - assertSame("DoFnReflector instances should be cached and reused for identical types", - reflector1, reflector2); - assertSame("Invoker classes should only be generated once for each type", - reflector1.bindInvoker(fn1).getClass(), - reflector2.bindInvoker(fn2).getClass()); - } - - interface InterfaceWithProcessElement { - @ProcessElement - void processElement(DoFn.ProcessContext c); - } - - interface LayersOfInterfaces extends InterfaceWithProcessElement {} - - private class IdentityUsingInterfaceWithProcessElement - extends DoFn - implements LayersOfInterfaces { - - private Invocations invocations = new Invocations("Named Class"); - - @Override - public void processElement(DoFn.ProcessContext c) { - invocations.wasProcessElementInvoked = true; - assertSame(c, mockContext); - } - } - - @Test - public void testDoFnWithProcessElementInterface() throws Exception { - IdentityUsingInterfaceWithProcessElement fn = new IdentityUsingInterfaceWithProcessElement(); - DoFnReflector reflector = underTest(fn); - assertFalse(reflector.usesSingleWindow()); - checkInvokeProcessElementWorks(reflector, fn.invocations); - } - - private class IdentityParent extends DoFn { - protected Invocations parentInvocations = new Invocations("IdentityParent"); - - @ProcessElement - public void process(ProcessContext c) { - parentInvocations.wasProcessElementInvoked = true; - assertSame(c, mockContext); - } - } - - private class IdentityChildWithoutOverride extends IdentityParent { - } - - private class IdentityChildWithOverride extends IdentityParent { - protected Invocations childInvocations = new Invocations("IdentityChildWithOverride"); - - @Override - public void process(DoFn.ProcessContext c) { - super.process(c); - childInvocations.wasProcessElementInvoked = true; - } - } - - @Test - public void testDoFnWithMethodInSuperclass() throws Exception { - IdentityChildWithoutOverride fn = new IdentityChildWithoutOverride(); - DoFnReflector reflector = underTest(fn); - assertFalse(reflector.usesSingleWindow()); - checkInvokeProcessElementWorks(reflector, fn.parentInvocations); - } - - @Test - public void testDoFnWithMethodInSubclass() throws Exception { - IdentityChildWithOverride fn = new IdentityChildWithOverride(); - DoFnReflector reflector = underTest(fn); - assertFalse(reflector.usesSingleWindow()); - checkInvokeProcessElementWorks(reflector, fn.parentInvocations, fn.childInvocations); - } - - @Test - public void testDoFnWithWindow() throws Exception { - final Invocations invocations = new Invocations("AnonymousClass"); - DoFnReflector reflector = underTest(new DoFn() { - - @ProcessElement - public void processElement(ProcessContext c, BoundedWindow w) - throws Exception { - invocations.wasProcessElementInvoked = true; - assertSame(c, mockContext); - assertSame(w, mockWindow); - } - }); - - assertTrue(reflector.usesSingleWindow()); - - checkInvokeProcessElementWorks(reflector, invocations); - } - - @Test - public void testDoFnWithOutputReceiver() throws Exception { - final Invocations invocations = new Invocations("AnonymousClass"); - DoFnReflector reflector = underTest(new DoFn() { - - @ProcessElement - public void processElement(ProcessContext c, DoFn.OutputReceiver o) - throws Exception { - invocations.wasProcessElementInvoked = true; - assertSame(c, mockContext); - assertSame(o, mockOutputReceiver); - } - }); - - assertFalse(reflector.usesSingleWindow()); - - checkInvokeProcessElementWorks(reflector, invocations); - } - - @Test - public void testDoFnWithInputProvider() throws Exception { - final Invocations invocations = new Invocations("AnonymousClass"); - DoFnReflector reflector = underTest(new DoFn() { - - @ProcessElement - public void processElement(ProcessContext c, DoFn.InputProvider i) - throws Exception { - invocations.wasProcessElementInvoked = true; - assertSame(c, mockContext); - assertSame(i, mockInputProvider); - } - }); - - assertFalse(reflector.usesSingleWindow()); - - checkInvokeProcessElementWorks(reflector, invocations); - } - - @Test - public void testDoFnWithStartBundle() throws Exception { - final Invocations invocations = new Invocations("AnonymousClass"); - DoFnReflector reflector = underTest(new DoFn() { - @ProcessElement - public void processElement(@SuppressWarnings("unused") ProcessContext c) {} - - @StartBundle - public void startBundle(Context c) { - invocations.wasStartBundleInvoked = true; - assertSame(c, mockContext); - } - - @FinishBundle - public void finishBundle(Context c) { - invocations.wasFinishBundleInvoked = true; - assertSame(c, mockContext); - } - }); - - checkInvokeStartBundleWorks(reflector, invocations); - checkInvokeFinishBundleWorks(reflector, invocations); - } - - @Test - public void testDoFnWithSetupTeardown() throws Exception { - final Invocations invocations = new Invocations("AnonymousClass"); - DoFnReflector reflector = underTest(new DoFn() { - @ProcessElement - public void processElement(@SuppressWarnings("unused") ProcessContext c) {} - - @StartBundle - public void startBundle(Context c) { - invocations.wasStartBundleInvoked = true; - assertSame(c, mockContext); - } - - @FinishBundle - public void finishBundle(Context c) { - invocations.wasFinishBundleInvoked = true; - assertSame(c, mockContext); - } - - @Setup - public void before() { - invocations.wasSetupInvoked = true; - } - - @Teardown - public void after() { - invocations.wasTeardownInvoked = true; - } - }); - - checkInvokeSetupWorks(reflector, invocations); - checkInvokeTeardownWorks(reflector, invocations); - } - - @Test - public void testNoProcessElement() throws Exception { - thrown.expect(IllegalStateException.class); - thrown.expectMessage("No method annotated with @ProcessElement found"); - thrown.expectMessage(getClass().getName() + "$"); - underTest(new DoFn() {}); - } - - @Test - public void testMultipleProcessElement() throws Exception { - thrown.expect(IllegalStateException.class); - thrown.expectMessage("Found multiple methods annotated with @ProcessElement"); - thrown.expectMessage("foo()"); - thrown.expectMessage("bar()"); - thrown.expectMessage(getClass().getName() + "$"); - underTest(new DoFn() { - @ProcessElement - public void foo() {} - - @ProcessElement - public void bar() {} - }); - } - - @Test - public void testMultipleStartBundleElement() throws Exception { - thrown.expect(IllegalStateException.class); - thrown.expectMessage("Found multiple methods annotated with @StartBundle"); - thrown.expectMessage("bar()"); - thrown.expectMessage("baz()"); - thrown.expectMessage(getClass().getName() + "$"); - underTest(new DoFn() { - @ProcessElement - public void foo() {} - - @StartBundle - public void bar() {} - - @StartBundle - public void baz() {} - }); - } - - @Test - public void testMultipleFinishBundleElement() throws Exception { - thrown.expect(IllegalStateException.class); - thrown.expectMessage("Found multiple methods annotated with @FinishBundle"); - thrown.expectMessage("bar()"); - thrown.expectMessage("baz()"); - thrown.expectMessage(getClass().getName() + "$"); - underTest(new DoFn() { - @ProcessElement - public void foo() {} - - @FinishBundle - public void bar() {} - - @FinishBundle - public void baz() {} - }); - } - - private static class PrivateDoFnClass extends DoFn { - final Invocations invocations = new Invocations(getClass().getName()); - - @ProcessElement - public void processThis(ProcessContext c) { - invocations.wasProcessElementInvoked = true; - } - } - - @Test - public void testLocalPrivateDoFnClass() throws Exception { - PrivateDoFnClass fn = new PrivateDoFnClass(); - DoFnReflector reflector = underTest(fn); - checkInvokeProcessElementWorks(reflector, fn.invocations); - } - - @Test - public void testStaticPackagePrivateDoFnClass() throws Exception { - Invocations invocations = new Invocations("StaticPackagePrivateDoFn"); - DoFnReflector reflector = - underTest(DoFnReflectorTestHelper.newStaticPackagePrivateDoFn(invocations)); - checkInvokeProcessElementWorks(reflector, invocations); - } - - @Test - public void testInnerPackagePrivateDoFnClass() throws Exception { - Invocations invocations = new Invocations("InnerPackagePrivateDoFn"); - DoFnReflector reflector = - underTest(new DoFnReflectorTestHelper().newInnerPackagePrivateDoFn(invocations)); - checkInvokeProcessElementWorks(reflector, invocations); - } - - @Test - public void testStaticPrivateDoFnClass() throws Exception { - Invocations invocations = new Invocations("StaticPrivateDoFn"); - DoFnReflector reflector = underTest(DoFnReflectorTestHelper.newStaticPrivateDoFn(invocations)); - checkInvokeProcessElementWorks(reflector, invocations); - } - - @Test - public void testInnerPrivateDoFnClass() throws Exception { - Invocations invocations = new Invocations("StaticInnerDoFn"); - DoFnReflector reflector = - underTest(new DoFnReflectorTestHelper().newInnerPrivateDoFn(invocations)); - checkInvokeProcessElementWorks(reflector, invocations); - } - - @Test - public void testAnonymousInnerDoFnInOtherPackage() throws Exception { - Invocations invocations = new Invocations("AnonymousInnerDoFnInOtherPackage"); - DoFnReflector reflector = - underTest(new DoFnReflectorTestHelper().newInnerAnonymousDoFn(invocations)); - checkInvokeProcessElementWorks(reflector, invocations); - } - - @Test - public void testStaticAnonymousDoFnInOtherPackage() throws Exception { - Invocations invocations = new Invocations("AnonymousStaticDoFnInOtherPackage"); - DoFnReflector reflector = - underTest(DoFnReflectorTestHelper.newStaticAnonymousDoFn(invocations)); - checkInvokeProcessElementWorks(reflector, invocations); - } - - @Test - public void testPrivateProcessElement() throws Exception { - thrown.expect(IllegalStateException.class); - thrown.expectMessage("process() must be public"); - thrown.expectMessage(getClass().getName() + "$"); - underTest(new DoFn() { - @ProcessElement - private void process() {} - }); - } - - @Test - public void testPrivateStartBundle() throws Exception { - thrown.expect(IllegalStateException.class); - thrown.expectMessage("startBundle() must be public"); - thrown.expectMessage(getClass().getName() + "$"); - underTest(new DoFn() { - @ProcessElement - public void processElement() {} - - @StartBundle - void startBundle() {} - }); - } - - @Test - public void testPrivateFinishBundle() throws Exception { - thrown.expect(IllegalStateException.class); - thrown.expectMessage("finishBundle() must be public"); - thrown.expectMessage(getClass().getName() + "$"); - underTest(new DoFn() { - @ProcessElement - public void processElement() {} - - @FinishBundle - void finishBundle() {} - }); - } - - @SuppressWarnings({"unused"}) - private void missingProcessContext() {} - - @Test - public void testMissingProcessContext() throws Exception { - thrown.expect(IllegalStateException.class); - thrown.expectMessage(getClass().getName() - + "#missingProcessContext() must take a ProcessContext as its first argument"); - - DoFnReflector.verifyProcessMethodArguments( - getClass().getDeclaredMethod("missingProcessContext")); - } - - @SuppressWarnings({"unused"}) - private void badProcessContext(String s) {} - - @Test - public void testBadProcessContextType() throws Exception { - thrown.expect(IllegalStateException.class); - thrown.expectMessage(getClass().getName() - + "#badProcessContext(String) must take a ProcessContext as its first argument"); - - DoFnReflector.verifyProcessMethodArguments( - getClass().getDeclaredMethod("badProcessContext", String.class)); - } - - @SuppressWarnings({"unused"}) - private void badExtraContext(DoFn.Context c, int n) {} - - @Test - public void testBadExtraContext() throws Exception { - thrown.expect(IllegalStateException.class); - thrown.expectMessage( - "int is not a valid context parameter for method " - + getClass().getName() + "#badExtraContext(Context, int). Should be one of ["); - - DoFnReflector.verifyBundleMethodArguments( - getClass().getDeclaredMethod("badExtraContext", Context.class, int.class)); - } - - @SuppressWarnings({"unused"}) - private void badExtraProcessContext( - DoFn.ProcessContext c, Integer n) {} - - @Test - public void testBadExtraProcessContextType() throws Exception { - thrown.expect(IllegalStateException.class); - thrown.expectMessage( - "Integer is not a valid context parameter for method " - + getClass().getName() + "#badExtraProcessContext(ProcessContext, Integer)" - + ". Should be one of [BoundedWindow]"); - - DoFnReflector.verifyProcessMethodArguments( - getClass().getDeclaredMethod("badExtraProcessContext", - ProcessContext.class, Integer.class)); - } - - @SuppressWarnings("unused") - private int badReturnType() { - return 0; - } - - @Test - public void testBadReturnType() throws Exception { - thrown.expect(IllegalStateException.class); - thrown.expectMessage(getClass().getName() + "#badReturnType() must have a void return type"); - - DoFnReflector.verifyProcessMethodArguments(getClass().getDeclaredMethod("badReturnType")); - } - - @SuppressWarnings("unused") - private void goodGenerics( - DoFn.ProcessContext c, - DoFn.InputProvider input, - DoFn.OutputReceiver output) {} - - @Test - public void testValidGenerics() throws Exception { - Method method = - getClass() - .getDeclaredMethod( - "goodGenerics", - DoFn.ProcessContext.class, - DoFn.InputProvider.class, - DoFn.OutputReceiver.class); - DoFnReflector.verifyProcessMethodArguments(method); - } - - @SuppressWarnings("unused") - private void goodWildcards( - DoFn.ProcessContext c, - DoFn.InputProvider input, - DoFn.OutputReceiver output) {} - - @Test - public void testGoodWildcards() throws Exception { - Method method = - getClass() - .getDeclaredMethod( - "goodWildcards", - DoFn.ProcessContext.class, - DoFn.InputProvider.class, - DoFn.OutputReceiver.class); - DoFnReflector.verifyProcessMethodArguments(method); - } - - @SuppressWarnings("unused") - private void goodBoundedWildcards( - DoFn.ProcessContext c, - DoFn.InputProvider input, - DoFn.OutputReceiver output) {} - - @Test - public void testGoodBoundedWildcards() throws Exception { - Method method = - getClass() - .getDeclaredMethod( - "goodBoundedWildcards", - DoFn.ProcessContext.class, - DoFn.InputProvider.class, - DoFn.OutputReceiver.class); - DoFnReflector.verifyProcessMethodArguments(method); - } - - @SuppressWarnings("unused") - private void goodTypeVariables( - DoFn.ProcessContext c, - DoFn.InputProvider input, - DoFn.OutputReceiver output) {} - - @Test - public void testGoodTypeVariables() throws Exception { - Method method = - getClass() - .getDeclaredMethod( - "goodTypeVariables", - DoFn.ProcessContext.class, - DoFn.InputProvider.class, - DoFn.OutputReceiver.class); - DoFnReflector.verifyProcessMethodArguments(method); - } - - @SuppressWarnings("unused") - private void badGenericTwoArgs( - DoFn.ProcessContext c, - DoFn.InputProvider input, - DoFn.OutputReceiver output) {} - - @Test - public void testBadGenericsTwoArgs() throws Exception { - Method method = - getClass() - .getDeclaredMethod( - "badGenericTwoArgs", - DoFn.ProcessContext.class, - DoFn.InputProvider.class, - DoFn.OutputReceiver.class); - - thrown.expect(IllegalStateException.class); - thrown.expectMessage("Incompatible generics in context parameter " - + "OutputReceiver " - + "for method " + getClass().getName() - + "#badGenericTwoArgs(ProcessContext, InputProvider, OutputReceiver). Should be " - + "OutputReceiver"); - - DoFnReflector.verifyProcessMethodArguments(method); - } - - @SuppressWarnings("unused") - private void badGenericWildCards( - DoFn.ProcessContext c, - DoFn.InputProvider input, - DoFn.OutputReceiver output) {} - - @Test - public void testBadGenericWildCards() throws Exception { - Method method = - getClass() - .getDeclaredMethod( - "badGenericWildCards", - DoFn.ProcessContext.class, - DoFn.InputProvider.class, - DoFn.OutputReceiver.class); - - thrown.expect(IllegalStateException.class); - thrown.expectMessage("Incompatible generics in context parameter " - + "OutputReceiver for method " - + getClass().getName() - + "#badGenericWildCards(ProcessContext, InputProvider, OutputReceiver). Should be " - + "OutputReceiver"); - - DoFnReflector.verifyProcessMethodArguments(method); - } - - @SuppressWarnings("unused") - private void badTypeVariables(DoFn.ProcessContext c, - DoFn.InputProvider input, DoFn.OutputReceiver output) {} - - @Test - public void testBadTypeVariables() throws Exception { - Method method = - getClass() - .getDeclaredMethod( - "badTypeVariables", - DoFn.ProcessContext.class, - DoFn.InputProvider.class, - DoFn.OutputReceiver.class); - - thrown.expect(IllegalStateException.class); - thrown.expectMessage("Incompatible generics in context parameter " - + "OutputReceiver for method " + getClass().getName() - + "#badTypeVariables(ProcessContext, InputProvider, OutputReceiver). Should be " - + "OutputReceiver"); - - DoFnReflector.verifyProcessMethodArguments(method); - } - - @Test - public void testProcessElementException() throws Exception { - DoFn fn = new DoFn() { - @ProcessElement - public void processElement(@SuppressWarnings("unused") ProcessContext c) { - throw new IllegalArgumentException("bogus"); - } - }; - - thrown.expect(UserCodeException.class); - thrown.expectMessage("bogus"); - DoFnReflector.of(fn.getClass()).bindInvoker(fn).invokeProcessElement(null, null); - } - - @Test - public void testStartBundleException() throws Exception { - DoFn fn = new DoFn() { - @StartBundle - public void startBundle(@SuppressWarnings("unused") Context c) { - throw new IllegalArgumentException("bogus"); - } - - @ProcessElement - public void processElement(@SuppressWarnings("unused") ProcessContext c) { - } - }; - - thrown.expect(UserCodeException.class); - thrown.expectMessage("bogus"); - DoFnReflector.of(fn.getClass()).bindInvoker(fn).invokeStartBundle(null, null); - } - - @Test - public void testFinishBundleException() throws Exception { - DoFn fn = new DoFn() { - @FinishBundle - public void finishBundle(@SuppressWarnings("unused") Context c) { - throw new IllegalArgumentException("bogus"); - } - - @ProcessElement - public void processElement(@SuppressWarnings("unused") ProcessContext c) { - } - }; - - thrown.expect(UserCodeException.class); - thrown.expectMessage("bogus"); - DoFnReflector.of(fn.getClass()).bindInvoker(fn).invokeFinishBundle(null, null); - } -} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java index 604536bca80e..3469223c38fe 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java @@ -99,7 +99,7 @@ public void testFlattenPCollectionListThenParDo() { PCollection output = makePCollectionListOfStrings(p, inputs) .apply(Flatten.pCollections()) - .apply(ParDo.of(new IdentityFn(){})); + .apply(ParDo.of(new IdentityFn())); PAssert.that(output).containsInAnyOrder(flattenLists(inputs)); p.run(); @@ -152,7 +152,7 @@ public void testFlattenPCollectionListEmptyThenParDo() { PCollection output = PCollectionList.empty(p) .apply(Flatten.pCollections()).setCoder(StringUtf8Coder.of()) - .apply(ParDo.of(new IdentityFn(){})); + .apply(ParDo.of(new IdentityFn())); PAssert.that(output).empty(); p.run(); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java new file mode 100644 index 000000000000..7e756e245f91 --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java @@ -0,0 +1,498 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.transforms.reflect; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertTrue; + +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.UserCodeException; + +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +/** Tests for {@link DoFnInvokers}. */ +public class DoFnInvokersTest { + /** A convenience struct holding flags that indicate whether a particular method was invoked. */ + public static class Invocations { + public boolean wasProcessElementInvoked = false; + public boolean wasStartBundleInvoked = false; + public boolean wasFinishBundleInvoked = false; + public boolean wasSetupInvoked = false; + public boolean wasTeardownInvoked = false; + private final String name; + + public Invocations(String name) { + this.name = name; + } + } + + @Rule public ExpectedException thrown = ExpectedException.none(); + + @Mock private DoFn.ProcessContext mockContext; + @Mock private BoundedWindow mockWindow; + @Mock private DoFn.InputProvider mockInputProvider; + @Mock private DoFn.OutputReceiver mockOutputReceiver; + + private DoFn.ExtraContextFactory extraContextFactory; + + @Before + public void setUp() { + MockitoAnnotations.initMocks(this); + this.extraContextFactory = + new DoFn.ExtraContextFactory() { + @Override + public BoundedWindow window() { + return mockWindow; + } + + @Override + public DoFn.InputProvider inputProvider() { + return mockInputProvider; + } + + @Override + public DoFn.OutputReceiver outputReceiver() { + return mockOutputReceiver; + } + }; + } + + private void checkInvokeProcessElementWorks(DoFn fn, Invocations... invocations) + throws Exception { + assertTrue("Need at least one invocation to check", invocations.length >= 1); + for (Invocations invocation : invocations) { + assertFalse( + "Should not yet have called processElement on " + invocation.name, + invocation.wasProcessElementInvoked); + } + DoFnInvokers.INSTANCE + .newByteBuddyInvoker(fn) + .invokeProcessElement(mockContext, extraContextFactory); + for (Invocations invocation : invocations) { + assertTrue( + "Should have called processElement on " + invocation.name, + invocation.wasProcessElementInvoked); + } + } + + private void checkInvokeStartBundleWorks(DoFn fn, Invocations... invocations) + throws Exception { + assertTrue("Need at least one invocation to check", invocations.length >= 1); + for (Invocations invocation : invocations) { + assertFalse( + "Should not yet have called startBundle on " + invocation.name, + invocation.wasStartBundleInvoked); + } + DoFnInvokers.INSTANCE.newByteBuddyInvoker(fn).invokeStartBundle(mockContext); + for (Invocations invocation : invocations) { + assertTrue( + "Should have called startBundle on " + invocation.name, invocation.wasStartBundleInvoked); + } + } + + private void checkInvokeFinishBundleWorks(DoFn fn, Invocations... invocations) + throws Exception { + assertTrue("Need at least one invocation to check", invocations.length >= 1); + for (Invocations invocation : invocations) { + assertFalse( + "Should not yet have called finishBundle on " + invocation.name, + invocation.wasFinishBundleInvoked); + } + DoFnInvokers.INSTANCE.newByteBuddyInvoker(fn).invokeFinishBundle(mockContext); + for (Invocations invocation : invocations) { + assertTrue( + "Should have called finishBundle on " + invocation.name, + invocation.wasFinishBundleInvoked); + } + } + + private void checkInvokeSetupWorks(DoFn fn, Invocations... invocations) + throws Exception { + assertTrue("Need at least one invocation to check", invocations.length >= 1); + for (Invocations invocation : invocations) { + assertFalse( + "Should not yet have called setup on " + invocation.name, invocation.wasSetupInvoked); + } + DoFnInvokers.INSTANCE.newByteBuddyInvoker(fn).invokeSetup(); + for (Invocations invocation : invocations) { + assertTrue("Should have called setup on " + invocation.name, invocation.wasSetupInvoked); + } + } + + private void checkInvokeTeardownWorks(DoFn fn, Invocations... invocations) + throws Exception { + assertTrue("Need at least one invocation to check", invocations.length >= 1); + for (Invocations invocation : invocations) { + assertFalse( + "Should not yet have called teardown on " + invocation.name, + invocation.wasTeardownInvoked); + } + DoFnInvokers.INSTANCE.newByteBuddyInvoker(fn).invokeTeardown(); + for (Invocations invocation : invocations) { + assertTrue( + "Should have called teardown on " + invocation.name, invocation.wasTeardownInvoked); + } + } + + @Test + public void testDoFnWithNoExtraContext() throws Exception { + final Invocations invocations = new Invocations("AnonymousClass"); + DoFn fn = + new DoFn() { + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + invocations.wasProcessElementInvoked = true; + assertSame(c, mockContext); + } + }; + + assertFalse( + DoFnSignatures.INSTANCE + .getOrParseSignature(fn.getClass()) + .processElement() + .usesSingleWindow()); + + checkInvokeProcessElementWorks(fn, invocations); + } + + @Test + public void testDoFnInvokersReused() throws Exception { + // Ensures that we don't create a new Invoker class for every instance of the DoFn. + IdentityParent fn1 = new IdentityParent(); + IdentityParent fn2 = new IdentityParent(); + assertSame( + "Invoker classes should only be generated once for each type", + DoFnInvokers.INSTANCE.newByteBuddyInvoker(fn1).getClass(), + DoFnInvokers.INSTANCE.newByteBuddyInvoker(fn2).getClass()); + } + + interface InterfaceWithProcessElement { + @DoFn.ProcessElement + void processElement(DoFn.ProcessContext c); + } + + interface LayersOfInterfaces extends InterfaceWithProcessElement {} + + private class IdentityUsingInterfaceWithProcessElement extends DoFn + implements LayersOfInterfaces { + + private Invocations invocations = new Invocations("Named Class"); + + @Override + public void processElement(DoFn.ProcessContext c) { + invocations.wasProcessElementInvoked = true; + assertSame(c, mockContext); + } + } + + @Test + public void testDoFnWithProcessElementInterface() throws Exception { + IdentityUsingInterfaceWithProcessElement fn = new IdentityUsingInterfaceWithProcessElement(); + assertFalse( + DoFnSignatures.INSTANCE + .getOrParseSignature(fn.getClass()) + .processElement() + .usesSingleWindow()); + checkInvokeProcessElementWorks(fn, fn.invocations); + } + + private class IdentityParent extends DoFn { + protected Invocations parentInvocations = new Invocations("IdentityParent"); + + @ProcessElement + public void process(ProcessContext c) { + parentInvocations.wasProcessElementInvoked = true; + assertSame(c, mockContext); + } + } + + private class IdentityChildWithoutOverride extends IdentityParent {} + + private class IdentityChildWithOverride extends IdentityParent { + protected Invocations childInvocations = new Invocations("IdentityChildWithOverride"); + + @Override + public void process(DoFn.ProcessContext c) { + super.process(c); + childInvocations.wasProcessElementInvoked = true; + } + } + + @Test + public void testDoFnWithMethodInSuperclass() throws Exception { + IdentityChildWithoutOverride fn = new IdentityChildWithoutOverride(); + assertFalse( + DoFnSignatures.INSTANCE + .getOrParseSignature(fn.getClass()) + .processElement() + .usesSingleWindow()); + checkInvokeProcessElementWorks(fn, fn.parentInvocations); + } + + @Test + public void testDoFnWithMethodInSubclass() throws Exception { + IdentityChildWithOverride fn = new IdentityChildWithOverride(); + assertFalse( + DoFnSignatures.INSTANCE + .getOrParseSignature(fn.getClass()) + .processElement() + .usesSingleWindow()); + checkInvokeProcessElementWorks(fn, fn.parentInvocations, fn.childInvocations); + } + + @Test + public void testDoFnWithWindow() throws Exception { + final Invocations invocations = new Invocations("AnonymousClass"); + DoFn fn = + new DoFn() { + @ProcessElement + public void processElement(ProcessContext c, BoundedWindow w) throws Exception { + invocations.wasProcessElementInvoked = true; + assertSame(c, mockContext); + assertSame(w, mockWindow); + } + }; + + assertTrue( + DoFnSignatures.INSTANCE + .getOrParseSignature(fn.getClass()) + .processElement() + .usesSingleWindow()); + + checkInvokeProcessElementWorks(fn, invocations); + } + + @Test + public void testDoFnWithOutputReceiver() throws Exception { + final Invocations invocations = new Invocations("AnonymousClass"); + DoFn fn = + new DoFn() { + @ProcessElement + public void processElement(ProcessContext c, OutputReceiver o) throws Exception { + invocations.wasProcessElementInvoked = true; + assertSame(c, mockContext); + assertSame(o, mockOutputReceiver); + } + }; + + assertFalse( + DoFnSignatures.INSTANCE + .getOrParseSignature(fn.getClass()) + .processElement() + .usesSingleWindow()); + + checkInvokeProcessElementWorks(fn, invocations); + } + + @Test + public void testDoFnWithInputProvider() throws Exception { + final Invocations invocations = new Invocations("AnonymousClass"); + DoFn fn = + new DoFn() { + @ProcessElement + public void processElement(ProcessContext c, InputProvider i) throws Exception { + invocations.wasProcessElementInvoked = true; + assertSame(c, mockContext); + assertSame(i, mockInputProvider); + } + }; + + assertFalse( + DoFnSignatures.INSTANCE + .getOrParseSignature(fn.getClass()) + .processElement() + .usesSingleWindow()); + + checkInvokeProcessElementWorks(fn, invocations); + } + + @Test + public void testDoFnWithStartBundle() throws Exception { + final Invocations invocations = new Invocations("AnonymousClass"); + DoFn fn = + new DoFn() { + @ProcessElement + public void processElement(@SuppressWarnings("unused") ProcessContext c) {} + + @StartBundle + public void startBundle(Context c) { + invocations.wasStartBundleInvoked = true; + assertSame(c, mockContext); + } + + @FinishBundle + public void finishBundle(Context c) { + invocations.wasFinishBundleInvoked = true; + assertSame(c, mockContext); + } + }; + + checkInvokeStartBundleWorks(fn, invocations); + checkInvokeFinishBundleWorks(fn, invocations); + } + + @Test + public void testDoFnWithSetupTeardown() throws Exception { + final Invocations invocations = new Invocations("AnonymousClass"); + DoFn fn = + new DoFn() { + @ProcessElement + public void processElement(@SuppressWarnings("unused") ProcessContext c) {} + + @StartBundle + public void startBundle(Context c) { + invocations.wasStartBundleInvoked = true; + assertSame(c, mockContext); + } + + @FinishBundle + public void finishBundle(Context c) { + invocations.wasFinishBundleInvoked = true; + assertSame(c, mockContext); + } + + @Setup + public void before() { + invocations.wasSetupInvoked = true; + } + + @Teardown + public void after() { + invocations.wasTeardownInvoked = true; + } + }; + + checkInvokeSetupWorks(fn, invocations); + checkInvokeTeardownWorks(fn, invocations); + } + + private static class PrivateDoFnClass extends DoFn { + final Invocations invocations = new Invocations(getClass().getName()); + + @ProcessElement + public void processThis(ProcessContext c) { + invocations.wasProcessElementInvoked = true; + } + } + + @Test + public void testLocalPrivateDoFnClass() throws Exception { + PrivateDoFnClass fn = new PrivateDoFnClass(); + checkInvokeProcessElementWorks(fn, fn.invocations); + } + + @Test + public void testStaticPackagePrivateDoFnClass() throws Exception { + Invocations invocations = new Invocations("StaticPackagePrivateDoFn"); + checkInvokeProcessElementWorks( + DoFnInvokersTestHelper.newStaticPackagePrivateDoFn(invocations), invocations); + } + + @Test + public void testInnerPackagePrivateDoFnClass() throws Exception { + Invocations invocations = new Invocations("InnerPackagePrivateDoFn"); + checkInvokeProcessElementWorks( + new DoFnInvokersTestHelper().newInnerPackagePrivateDoFn(invocations), invocations); + } + + @Test + public void testStaticPrivateDoFnClass() throws Exception { + Invocations invocations = new Invocations("StaticPrivateDoFn"); + checkInvokeProcessElementWorks( + DoFnInvokersTestHelper.newStaticPrivateDoFn(invocations), invocations); + } + + @Test + public void testInnerPrivateDoFnClass() throws Exception { + Invocations invocations = new Invocations("StaticInnerDoFn"); + checkInvokeProcessElementWorks( + new DoFnInvokersTestHelper().newInnerPrivateDoFn(invocations), invocations); + } + + @Test + public void testAnonymousInnerDoFnInOtherPackage() throws Exception { + Invocations invocations = new Invocations("AnonymousInnerDoFnInOtherPackage"); + checkInvokeProcessElementWorks( + new DoFnInvokersTestHelper().newInnerAnonymousDoFn(invocations), invocations); + } + + @Test + public void testStaticAnonymousDoFnInOtherPackage() throws Exception { + Invocations invocations = new Invocations("AnonymousStaticDoFnInOtherPackage"); + checkInvokeProcessElementWorks( + DoFnInvokersTestHelper.newStaticAnonymousDoFn(invocations), invocations); + } + + @Test + public void testProcessElementException() throws Exception { + DoFn fn = + new DoFn() { + @ProcessElement + public void processElement(@SuppressWarnings("unused") ProcessContext c) { + throw new IllegalArgumentException("bogus"); + } + }; + + thrown.expect(UserCodeException.class); + thrown.expectMessage("bogus"); + DoFnInvokers.INSTANCE.newByteBuddyInvoker(fn).invokeProcessElement(null, null); + } + + @Test + public void testStartBundleException() throws Exception { + DoFn fn = + new DoFn() { + @StartBundle + public void startBundle(@SuppressWarnings("unused") Context c) { + throw new IllegalArgumentException("bogus"); + } + + @ProcessElement + public void processElement(@SuppressWarnings("unused") ProcessContext c) {} + }; + + thrown.expect(UserCodeException.class); + thrown.expectMessage("bogus"); + DoFnInvokers.INSTANCE.newByteBuddyInvoker(fn).invokeStartBundle(null); + } + + @Test + public void testFinishBundleException() throws Exception { + DoFn fn = + new DoFn() { + @FinishBundle + public void finishBundle(@SuppressWarnings("unused") Context c) { + throw new IllegalArgumentException("bogus"); + } + + @ProcessElement + public void processElement(@SuppressWarnings("unused") ProcessContext c) {} + }; + + thrown.expect(UserCodeException.class); + thrown.expectMessage("bogus"); + DoFnInvokers.INSTANCE.newByteBuddyInvoker(fn).invokeFinishBundle(null); + } +} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/dofnreflector/DoFnReflectorTestHelper.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTestHelper.java similarity index 93% rename from sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/dofnreflector/DoFnReflectorTestHelper.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTestHelper.java index 90fba12b9599..7bfdddc05226 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/dofnreflector/DoFnReflectorTestHelper.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTestHelper.java @@ -15,16 +15,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.transforms.dofnreflector; +package org.apache.beam.sdk.transforms.reflect; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.DoFnReflectorTest.Invocations; +import org.apache.beam.sdk.transforms.reflect.DoFnInvokersTest.Invocations; /** - * Test helper for DoFnReflectorTest, which needs to test package-private access + * Test helper for {@link DoFnInvokersTest}, which needs to test package-private access * to DoFns in other packages. */ -public class DoFnReflectorTestHelper { +public class DoFnInvokersTestHelper { private static class StaticPrivateDoFn extends DoFn { final Invocations invocations; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesTest.java new file mode 100644 index 000000000000..1a26df22cc34 --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesTest.java @@ -0,0 +1,371 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.transforms.reflect; + +import org.apache.beam.sdk.transforms.DoFn; + +import com.google.common.reflect.TypeToken; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.lang.reflect.Method; +import java.util.List; + +/** Tests for {@link DoFnSignatures}. */ +@RunWith(JUnit4.class) +public class DoFnSignaturesTest { + + @Rule public ExpectedException thrown = ExpectedException.none(); + + private static class FakeDoFn extends DoFn {} + + @SuppressWarnings({"unused"}) + private void missingProcessContext() {} + + @Test + public void testMissingProcessContext() throws Exception { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage( + getClass().getName() + + "#missingProcessContext() must take a ProcessContext<> as its first argument"); + + DoFnSignatures.analyzeProcessElementMethod( + TypeToken.of(FakeDoFn.class), + getClass().getDeclaredMethod("missingProcessContext"), + TypeToken.of(Integer.class), + TypeToken.of(String.class)); + } + + @SuppressWarnings({"unused"}) + private void badProcessContext(String s) {} + + @Test + public void testBadProcessContextType() throws Exception { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage( + getClass().getName() + + "#badProcessContext(String) must take a ProcessContext<> as its first argument"); + + DoFnSignatures.analyzeProcessElementMethod( + TypeToken.of(FakeDoFn.class), + getClass().getDeclaredMethod("badProcessContext", String.class), + TypeToken.of(Integer.class), + TypeToken.of(String.class)); + } + + @SuppressWarnings({"unused"}) + private void badExtraContext(DoFn.Context c, int n) {} + + @Test + public void testBadExtraContext() throws Exception { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage( + getClass().getName() + + "#badExtraContext(Context, int) must have a single argument of type Context"); + + DoFnSignatures.analyzeBundleMethod( + TypeToken.of(FakeDoFn.class), + getClass().getDeclaredMethod("badExtraContext", DoFn.Context.class, int.class), + TypeToken.of(Integer.class), + TypeToken.of(String.class)); + } + + @SuppressWarnings({"unused"}) + private void badExtraProcessContext(DoFn.ProcessContext c, Integer n) {} + + @Test + public void testBadExtraProcessContextType() throws Exception { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage( + "Integer is not a valid context parameter for method " + + getClass().getName() + + "#badExtraProcessContext(ProcessContext, Integer)" + + ". Should be one of [BoundedWindow]"); + + DoFnSignatures.analyzeProcessElementMethod( + TypeToken.of(FakeDoFn.class), + getClass() + .getDeclaredMethod("badExtraProcessContext", DoFn.ProcessContext.class, Integer.class), + TypeToken.of(Integer.class), + TypeToken.of(String.class)); + } + + @SuppressWarnings("unused") + private int badReturnType() { + return 0; + } + + @Test + public void testBadReturnType() throws Exception { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage(getClass().getName() + "#badReturnType() must have a void return type"); + + DoFnSignatures.analyzeProcessElementMethod( + TypeToken.of(FakeDoFn.class), + getClass().getDeclaredMethod("badReturnType"), + TypeToken.of(Integer.class), + TypeToken.of(String.class)); + } + + @SuppressWarnings("unused") + private void goodConcreteTypes( + DoFn.ProcessContext c, + DoFn.InputProvider input, + DoFn.OutputReceiver output) {} + + @Test + public void testGoodConcreteTypes() throws Exception { + Method method = + getClass() + .getDeclaredMethod( + "goodConcreteTypes", + DoFn.ProcessContext.class, + DoFn.InputProvider.class, + DoFn.OutputReceiver.class); + DoFnSignatures.analyzeProcessElementMethod( + TypeToken.of(FakeDoFn.class), + method, + TypeToken.of(Integer.class), + TypeToken.of(String.class)); + } + + private static class GoodTypeVariables extends DoFn { + @ProcessElement + @SuppressWarnings("unused") + public void goodTypeVariables( + DoFn.ProcessContext c, + DoFn.InputProvider input, + DoFn.OutputReceiver output) {} + } + + @Test + public void testGoodTypeVariables() throws Exception { + DoFnSignatures.INSTANCE.getOrParseSignature(GoodTypeVariables.class); + } + + private static class IdentityFn extends DoFn { + @ProcessElement + @SuppressWarnings("unused") + public void processElement(ProcessContext c, InputProvider input, OutputReceiver output) { + c.output(c.element()); + } + } + + private static class IdentityListFn extends IdentityFn> {} + + @Test + public void testIdentityFnApplied() throws Exception { + DoFnSignatures.INSTANCE.getOrParseSignature(new IdentityFn() {}.getClass()); + } + + @SuppressWarnings("unused") + private void badGenericTwoArgs( + DoFn.ProcessContext c, + DoFn.InputProvider input, + DoFn.OutputReceiver output) {} + + @Test + public void testBadGenericsTwoArgs() throws Exception { + Method method = + getClass() + .getDeclaredMethod( + "badGenericTwoArgs", + DoFn.ProcessContext.class, + DoFn.InputProvider.class, + DoFn.OutputReceiver.class); + + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage( + "Wrong type of OutputReceiver parameter " + + "for method " + + getClass().getName() + + "#badGenericTwoArgs(ProcessContext, InputProvider, OutputReceiver): " + + "OutputReceiver, should be " + + "OutputReceiver"); + + DoFnSignatures.analyzeProcessElementMethod( + TypeToken.of(FakeDoFn.class), + method, + TypeToken.of(Integer.class), + TypeToken.of(String.class)); + } + + @SuppressWarnings("unused") + private void badGenericWildCards( + DoFn.ProcessContext c, + DoFn.InputProvider input, + DoFn.OutputReceiver output) {} + + @Test + public void testBadGenericWildCards() throws Exception { + Method method = + getClass() + .getDeclaredMethod( + "badGenericWildCards", + DoFn.ProcessContext.class, + DoFn.InputProvider.class, + DoFn.OutputReceiver.class); + + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage( + "Wrong type of OutputReceiver parameter for method " + + getClass().getName() + + "#badGenericWildCards(ProcessContext, InputProvider, OutputReceiver): " + + "OutputReceiver, should be " + + "OutputReceiver"); + + DoFnSignatures.analyzeProcessElementMethod( + TypeToken.of(FakeDoFn.class), + method, + TypeToken.of(Integer.class), + TypeToken.of(String.class)); + } + + static class BadTypeVariables extends DoFn { + @ProcessElement + @SuppressWarnings("unused") + public void badTypeVariables( + DoFn.ProcessContext c, + DoFn.InputProvider input, + DoFn.OutputReceiver output) {} + } + + @Test + public void testBadTypeVariables() throws Exception { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage( + "Wrong type of OutputReceiver parameter for method " + + BadTypeVariables.class.getName() + + "#badTypeVariables(ProcessContext, InputProvider, OutputReceiver): " + + "OutputReceiver, should be " + + "OutputReceiver"); + + DoFnSignatures.INSTANCE.getOrParseSignature(BadTypeVariables.class); + } + + @Test + public void testNoProcessElement() throws Exception { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("No method annotated with @ProcessElement found"); + thrown.expectMessage(getClass().getName() + "$"); + DoFnSignatures.INSTANCE.getOrParseSignature(new DoFn() {}.getClass()); + } + + @Test + public void testMultipleProcessElement() throws Exception { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("Found multiple methods annotated with @ProcessElement"); + thrown.expectMessage("foo()"); + thrown.expectMessage("bar()"); + thrown.expectMessage(getClass().getName() + "$"); + DoFnSignatures.INSTANCE.getOrParseSignature( + new DoFn() { + @ProcessElement + public void foo() {} + + @ProcessElement + public void bar() {} + }.getClass()); + } + + @Test + public void testMultipleStartBundleElement() throws Exception { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("Found multiple methods annotated with @StartBundle"); + thrown.expectMessage("bar()"); + thrown.expectMessage("baz()"); + thrown.expectMessage(getClass().getName() + "$"); + DoFnSignatures.INSTANCE.getOrParseSignature( + new DoFn() { + @ProcessElement + public void foo() {} + + @StartBundle + public void bar() {} + + @StartBundle + public void baz() {} + }.getClass()); + } + + @Test + public void testMultipleFinishBundleMethods() throws Exception { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("Found multiple methods annotated with @FinishBundle"); + thrown.expectMessage("bar(Context)"); + thrown.expectMessage("baz(Context)"); + thrown.expectMessage(getClass().getName() + "$"); + DoFnSignatures.INSTANCE.getOrParseSignature( + new DoFn() { + @ProcessElement + public void foo(ProcessContext context) {} + + @FinishBundle + public void bar(Context context) {} + + @FinishBundle + public void baz(Context context) {} + }.getClass()); + } + + @Test + public void testPrivateProcessElement() throws Exception { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("process() must be public"); + thrown.expectMessage(getClass().getName() + "$"); + DoFnSignatures.INSTANCE.getOrParseSignature( + new DoFn() { + @ProcessElement + private void process() {} + }.getClass()); + } + + @Test + public void testPrivateStartBundle() throws Exception { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("startBundle() must be public"); + thrown.expectMessage(getClass().getName() + "$"); + DoFnSignatures.INSTANCE.getOrParseSignature( + new DoFn() { + @ProcessElement + public void processElement() {} + + @StartBundle + void startBundle() {} + }.getClass()); + } + + @Test + public void testPrivateFinishBundle() throws Exception { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("finishBundle() must be public"); + thrown.expectMessage(getClass().getName() + "$"); + DoFnSignatures.INSTANCE.getOrParseSignature( + new DoFn() { + @ProcessElement + public void processElement() {} + + @FinishBundle + void finishBundle() {} + }.getClass()); + } +} diff --git a/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/transforms/DoFnReflectorBenchmark.java b/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/transforms/DoFnInvokersBenchmark.java similarity index 83% rename from sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/transforms/DoFnReflectorBenchmark.java rename to sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/transforms/DoFnInvokersBenchmark.java index 91ecd162d875..a574ed83dc42 100644 --- a/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/transforms/DoFnReflectorBenchmark.java +++ b/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/transforms/DoFnInvokersBenchmark.java @@ -22,15 +22,15 @@ import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFn.ExtraContextFactory; -import org.apache.beam.sdk.transforms.DoFnReflector; -import org.apache.beam.sdk.transforms.DoFnReflector.DoFnInvoker; +import org.apache.beam.sdk.transforms.DoFnAdapters; import org.apache.beam.sdk.transforms.OldDoFn; +import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; +import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowingInternals; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; - import org.joda.time.Instant; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.Fork; @@ -40,36 +40,33 @@ import org.openjdk.jmh.annotations.Warmup; /** - * Benchmarks for {@link OldDoFn} and {@link DoFn} invocations, specifically - * for measuring the overhead of {@link DoFnReflector}. + * Benchmarks for {@link OldDoFn} and {@link DoFn} invocations, specifically for measuring the + * overhead of {@link DoFnInvokers}. */ @State(Scope.Benchmark) @Fork(1) @Warmup(iterations = 5) -public class DoFnReflectorBenchmark { +public class DoFnInvokersBenchmark { private static final String ELEMENT = "some string to use for testing"; private OldDoFn oldDoFn = new UpperCaseOldDoFn(); private DoFn doFn = new UpperCaseDoFn(); - private StubOldDoFnProcessContext stubOldDoFnContext = new StubOldDoFnProcessContext(oldDoFn, - ELEMENT); - private StubDoFnProcessContext stubDoFnContext = - new StubDoFnProcessContext(doFn, ELEMENT); + private StubOldDoFnProcessContext stubOldDoFnContext = + new StubOldDoFnProcessContext(oldDoFn, ELEMENT); + private StubDoFnProcessContext stubDoFnContext = new StubDoFnProcessContext(doFn, ELEMENT); private ExtraContextFactory extraContextFactory = new DoFn.FakeExtraContextFactory<>(); - private DoFnReflector doFnReflector; private OldDoFn adaptedDoFnWithContext; private DoFnInvoker invoker; @Setup public void setUp() { - doFnReflector = DoFnReflector.of(doFn.getClass()); - adaptedDoFnWithContext = doFnReflector.toDoFn(doFn); - invoker = doFnReflector.bindInvoker(doFn); + adaptedDoFnWithContext = DoFnAdapters.toOldDoFn(doFn); + invoker = DoFnInvokers.INSTANCE.newByteBuddyInvoker(doFn); } @Benchmark @@ -162,24 +159,21 @@ public void outputWithTimestamp(String output, Instant timestamp) { } @Override - public void sideOutput(TupleTag tag, T output) { - } + public void sideOutput(TupleTag tag, T output) {} @Override - public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { - } + public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) {} @Override - protected Aggregator - createAggregatorInternal(String name, CombineFn combiner) { + protected Aggregator createAggregatorInternal( + String name, CombineFn combiner) { return null; } } - private static class StubDoFnProcessContext - extends DoFn.ProcessContext { + private static class StubDoFnProcessContext extends DoFn.ProcessContext { private final String element; - private String output; + private String output; public StubDoFnProcessContext(DoFn fn, String element) { fn.super(); @@ -222,11 +216,9 @@ public void outputWithTimestamp(String output, Instant timestamp) { } @Override - public void sideOutput(TupleTag tag, T output) { - } + public void sideOutput(TupleTag tag, T output) {} @Override - public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { - } + public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) {} } } From 236945d2504b73de91f7292219e0b15a53e062f5 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Wed, 27 Jul 2016 14:23:15 -0700 Subject: [PATCH 025/112] Replace ParDo with simpler transforms where possible There are a number of places in the Java SDK where we use ParDo.of(DoFn) when MapElements or other higher-level composites are applicable and readable. This change alters a number of those. --- .../apache/beam/sdk/transforms/Combine.java | 28 +++++++++---------- .../org/apache/beam/sdk/transforms/Count.java | 8 +++--- .../beam/sdk/transforms/FlatMapElements.java | 4 +-- .../apache/beam/sdk/transforms/Flatten.java | 12 ++++---- .../org/apache/beam/sdk/transforms/Keys.java | 8 +++--- .../apache/beam/sdk/transforms/KvSwap.java | 9 +++--- .../beam/sdk/transforms/MapElements.java | 16 ++++++++--- .../beam/sdk/transforms/RemoveDuplicates.java | 8 +++--- .../apache/beam/sdk/transforms/Values.java | 8 +++--- .../apache/beam/sdk/transforms/WithKeys.java | 9 +++--- .../beam/sdk/transforms/windowing/Window.java | 11 ++++---- .../org/apache/beam/sdk/PipelineTest.java | 12 ++++---- .../org/apache/beam/sdk/io/WriteTest.java | 4 ++- .../beam/sdk/transforms/MapElementsTest.java | 8 +++--- .../org/apache/beam/sdk/io/kafka/KafkaIO.java | 10 ++++--- 15 files changed, 81 insertions(+), 74 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java index 6ba3f8a8964d..56c0bc4e5c6e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java @@ -2121,14 +2121,14 @@ public void processElement(ProcessContext c) { inputCoder.getValueCoder())) .setWindowingStrategyInternal(preCombineStrategy) .apply("PreCombineHot", Combine.perKey(hotPreCombine)) - .apply("StripNonce", ParDo.of( - new DoFn, AccumT>, - KV>>() { - @ProcessElement - public void processElement(ProcessContext c) { - c.output(KV.of( - c.element().getKey().getKey(), - InputOrAccum.accum(c.element().getValue()))); + .apply("StripNonce", MapElements.via( + new SimpleFunction, AccumT>, + KV>>() { + @Override + public KV> apply(KV, AccumT> elem) { + return KV.of( + elem.getKey().getKey(), + InputOrAccum.accum(elem.getValue())); } })) .setCoder(KvCoder.of(inputCoder.getKeyCoder(), inputOrAccumCoder)) @@ -2137,12 +2137,12 @@ public void processElement(ProcessContext c) { PCollection>> preprocessedCold = split .get(cold) .setCoder(inputCoder) - .apply("PrepareCold", ParDo.of( - new DoFn, KV>>() { - @ProcessElement - public void processElement(ProcessContext c) { - c.output(KV.of(c.element().getKey(), - InputOrAccum.input(c.element().getValue()))); + .apply("PrepareCold", MapElements.via( + new SimpleFunction, KV>>() { + @Override + public KV> apply(KV element) { + return KV.of(element.getKey(), + InputOrAccum.input(element.getValue())); } })) .setCoder(KvCoder.of(inputCoder.getKeyCoder(), inputOrAccumCoder)); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Count.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Count.java index ac59c767504e..195c5d17ed88 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Count.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Count.java @@ -107,10 +107,10 @@ public PerElement() { } public PCollection> apply(PCollection input) { return input - .apply("Init", ParDo.of(new DoFn>() { - @ProcessElement - public void processElement(ProcessContext c) { - c.output(KV.of(c.element(), (Void) null)); + .apply("Init", MapElements.via(new SimpleFunction>() { + @Override + public KV apply(T element) { + return KV.of(element, (Void) null); } })) .apply(Count.perKey()); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/FlatMapElements.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/FlatMapElements.java index 6f9e3d8ac078..2837c40cc3a6 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/FlatMapElements.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/FlatMapElements.java @@ -29,7 +29,7 @@ * {@link PCollection} and merging the results. */ public class FlatMapElements -extends PTransform, PCollection> { +extends PTransform, PCollection> { /** * For a {@code SerializableFunction>} {@code fn}, * returns a {@link PTransform} that applies {@code fn} to every element of the input @@ -130,7 +130,7 @@ private FlatMapElements( } @Override - public PCollection apply(PCollection input) { + public PCollection apply(PCollection input) { return input.apply( "FlatMap", ParDo.of( diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Flatten.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Flatten.java index 7e09d7e4dd3b..f3f4f887078d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Flatten.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Flatten.java @@ -173,13 +173,11 @@ public PCollection apply(PCollection> in) { @SuppressWarnings("unchecked") Coder elemCoder = ((IterableLikeCoder) inCoder).getElemCoder(); - return in.apply("FlattenIterables", ParDo.of( - new DoFn, T>() { - @ProcessElement - public void processElement(ProcessContext c) { - for (T i : c.element()) { - c.output(i); - } + return in.apply("FlattenIterables", FlatMapElements.via( + new SimpleFunction, Iterable>() { + @Override + public Iterable apply(Iterable element) { + return element; } })) .setCoder(elemCoder); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Keys.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Keys.java index 5ac1866a3590..2405adf41e4b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Keys.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Keys.java @@ -58,10 +58,10 @@ private Keys() { } @Override public PCollection apply(PCollection> in) { return - in.apply("Keys", ParDo.of(new DoFn, K>() { - @ProcessElement - public void processElement(ProcessContext c) { - c.output(c.element().getKey()); + in.apply("Keys", MapElements.via(new SimpleFunction, K>() { + @Override + public K apply(KV kv) { + return kv.getKey(); } })); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/KvSwap.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/KvSwap.java index d4386d2a8107..2b81ebfdf1de 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/KvSwap.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/KvSwap.java @@ -62,11 +62,10 @@ private KvSwap() { } @Override public PCollection> apply(PCollection> in) { return - in.apply("KvSwap", ParDo.of(new DoFn, KV>() { - @ProcessElement - public void processElement(ProcessContext c) { - KV e = c.element(); - c.output(KV.of(e.getValue(), e.getKey())); + in.apply("KvSwap", MapElements.via(new SimpleFunction, KV>() { + @Override + public KV apply(KV kv) { + return KV.of(kv.getValue(), kv.getKey()); } })); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/MapElements.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/MapElements.java index 17ad6e74a13f..73e4359831db 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/MapElements.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/MapElements.java @@ -25,7 +25,7 @@ * {@code PTransform}s for mapping a simple function over the elements of a {@link PCollection}. */ public class MapElements -extends PTransform, PCollection> { +extends PTransform, PCollection> { /** * For a {@code SerializableFunction} {@code fn} and output type descriptor, @@ -44,8 +44,16 @@ public class MapElements * descriptor need not be provided. */ public static MissingOutputTypeDescriptor - via(SerializableFunction fn) { - return new MissingOutputTypeDescriptor<>(fn); + via(SerializableFunction fn) { + + // TypeDescriptor interacts poorly with the wildcards needed to correctly express + // covariance and contravariance in Java, so instead we cast it to an invariant + // function here. + @SuppressWarnings("unchecked") // safe covariant cast + SerializableFunction simplerFn = + (SerializableFunction) fn; + + return new MissingOutputTypeDescriptor<>(simplerFn); } /** @@ -103,7 +111,7 @@ private MapElements(SimpleFunction fn, Class fnClass) { } @Override - public PCollection apply(PCollection input) { + public PCollection apply(PCollection input) { return input.apply( "Map", ParDo.of( diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/RemoveDuplicates.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/RemoveDuplicates.java index bba4b5130957..2744b14c8913 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/RemoveDuplicates.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/RemoveDuplicates.java @@ -85,10 +85,10 @@ public static WithRepresentativeValues withRepresentativeValueF @Override public PCollection apply(PCollection in) { return in - .apply("CreateIndex", ParDo.of(new DoFn>() { - @ProcessElement - public void processElement(ProcessContext c) { - c.output(KV.of(c.element(), (Void) null)); + .apply("CreateIndex", MapElements.via(new SimpleFunction>() { + @Override + public KV apply(T element) { + return KV.of(element, (Void) null); } })) .apply(Combine.perKey( diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Values.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Values.java index 34342db53c6b..d21d100764a0 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Values.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Values.java @@ -58,10 +58,10 @@ private Values() { } @Override public PCollection apply(PCollection> in) { return - in.apply("Values", ParDo.of(new DoFn, V>() { - @ProcessElement - public void processElement(ProcessContext c) { - c.output(c.element().getValue()); + in.apply("Values", MapElements.via(new SimpleFunction, V>() { + @Override + public V apply(KV kv) { + return kv.getValue(); } })); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithKeys.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithKeys.java index 2a44963e6fc2..8b061f6bdc57 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithKeys.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithKeys.java @@ -113,11 +113,10 @@ public WithKeys withKeyType(TypeDescriptor keyType) { @Override public PCollection> apply(PCollection in) { PCollection> result = - in.apply("AddKeys", ParDo.of(new DoFn>() { - @ProcessElement - public void processElement(ProcessContext c) { - c.output(KV.of(fn.apply(c.element()), - c.element())); + in.apply("AddKeys", MapElements.via(new SimpleFunction>() { + @Override + public KV apply(V element) { + return KV.of(fn.apply(element), element); } })); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java index c1b0237e7a69..9dd069cf9952 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java @@ -21,10 +21,10 @@ import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.Coder.NonDeterministicException; -import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.GroupByKey; +import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.SimpleFunction; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode; @@ -645,10 +645,9 @@ public PCollection apply(PCollection input) { // We first apply a (trivial) transform to the input PCollection to produce a new // PCollection. This ensures that we don't modify the windowing strategy of the input // which may be used elsewhere. - .apply("Identity", ParDo.of(new DoFn() { - @ProcessElement - public void processElement(ProcessContext c) { - c.output(c.element()); + .apply("Identity", MapElements.via(new SimpleFunction() { + @Override public T apply(T element) { + return element; } })) // Then we modify the windowing strategy. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java index 8b8649994a06..d7b3ac54de9d 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java @@ -36,10 +36,10 @@ import org.apache.beam.sdk.testing.RunnableOnService; import org.apache.beam.sdk.testing.TestPipeline; 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.MapElements; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.SimpleFunction; import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; @@ -146,10 +146,10 @@ public void testMultipleApply() { private static PTransform, PCollection> addSuffix( final String suffix) { - return ParDo.of(new DoFn() { - @ProcessElement - public void processElement(DoFn.ProcessContext c) { - c.output(c.element() + suffix); + return MapElements.via(new SimpleFunction() { + @Override + public String apply(String input) { + return input + suffix; } }); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java index f9bf472eab9d..b9ba53ba6e13 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java @@ -91,7 +91,9 @@ public class WriteTest { // Static counts of the number of records per shard. private static List recordsPerShard = new ArrayList<>(); - private static final MapElements IDENTITY_MAP = + @SuppressWarnings("unchecked") // covariant cast + private static final PTransform, PCollection> IDENTITY_MAP = + (PTransform) MapElements.via(new SimpleFunction() { @Override public String apply(String input) { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MapElementsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MapElementsTest.java index e86a1289f82b..7217bca663fa 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MapElementsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MapElementsTest.java @@ -233,7 +233,7 @@ public Integer apply(Integer input) { } @Test public void testSimpleFunctionDisplayData() { - SimpleFunction simpleFn = new SimpleFunction() { + SimpleFunction simpleFn = new SimpleFunction() { @Override public Integer apply(Integer input) { return input; @@ -255,17 +255,17 @@ public void populateDisplayData(DisplayData.Builder builder) { @Test @Category(RunnableOnService.class) public void testPrimitiveDisplayData() { - SimpleFunction mapFn = new SimpleFunction() { + SimpleFunction mapFn = new SimpleFunction() { @Override public Integer apply(Integer input) { return input; } }; - MapElements map = MapElements.via(mapFn); + MapElements map = MapElements.via(mapFn); DisplayDataEvaluator evaluator = DisplayDataEvaluator.create(); - Set displayData = evaluator.displayDataForPrimitiveTransforms(map); + Set displayData = evaluator.displayDataForPrimitiveTransforms(map); assertThat("MapElements should include the mapFn in its primitive display data", displayData, hasItem(hasDisplayItem("mapFn", mapFn.getClass()))); } diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java index 2383105d2e61..8a0c7880e97d 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java @@ -34,9 +34,11 @@ import org.apache.beam.sdk.io.kafka.KafkaCheckpointMark.PartitionMark; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.transforms.SimpleFunction; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.ExposedByteArrayInputStream; import org.apache.beam.sdk.values.KV; @@ -1314,10 +1316,10 @@ private KafkaValueWrite(TypedWrite kvWriteTransform) { public PDone apply(PCollection input) { return input .apply("Kafka values with default key", - ParDo.of(new DoFn>() { - @ProcessElement - public void processElement(ProcessContext ctx) throws Exception { - ctx.output(KV.of(null, ctx.element())); + MapElements.via(new SimpleFunction>() { + @Override + public KV apply(V element) { + return KV.of(null, element); } })) .setCoder(KvCoder.of(VoidCoder.of(), kvWriteTransform.valueCoder)) From da3081a68c82c6f22ee382dfe0ffe1bd6be5d0e2 Mon Sep 17 00:00:00 2001 From: Pei He Date: Mon, 15 Aug 2016 12:22:11 -0700 Subject: [PATCH 026/112] Set Gcs upload buffer size to 1M in streaming mode in DataflowRunner --- .../beam/runners/dataflow/DataflowRunner.java | 6 ++++ .../runners/dataflow/DataflowRunnerTest.java | 31 +++++++++++++++++++ 2 files changed, 37 insertions(+) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index 62222897184b..6f8180e0d381 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -125,6 +125,7 @@ import com.google.api.services.dataflow.model.Job; import com.google.api.services.dataflow.model.ListJobsResponse; import com.google.api.services.dataflow.model.WorkerPool; +import com.google.cloud.hadoop.util.AbstractGoogleAsyncWriteChannel; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Joiner; @@ -309,6 +310,11 @@ public static DataflowRunner fromOptions(PipelineOptions options) { + "' invalid. Please make sure the value is non-negative."); } + if (dataflowOptions.isStreaming() && dataflowOptions.getGcsUploadBufferSizeBytes() == null) { + dataflowOptions.setGcsUploadBufferSizeBytes( + AbstractGoogleAsyncWriteChannel.UPLOAD_PIPE_BUFFER_SIZE_DEFAULT); + } + return new DataflowRunner(dataflowOptions); } diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java index d7deffdb0323..6f1653b026be 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java @@ -89,6 +89,7 @@ import com.google.api.services.dataflow.model.DataflowPackage; import com.google.api.services.dataflow.model.Job; import com.google.api.services.dataflow.model.ListJobsResponse; +import com.google.cloud.hadoop.util.AbstractGoogleAsyncWriteChannel; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -795,6 +796,36 @@ public void testValidJobName() throws IOException { } } + @Test + public void testGcsUploadBufferSizeDefault() throws IOException { + DataflowPipelineOptions batchOptions = buildPipelineOptions(); + DataflowRunner.fromOptions(batchOptions); + assertNull(batchOptions.getGcsUploadBufferSizeBytes()); + + DataflowPipelineOptions streamingOptions = buildPipelineOptions(); + streamingOptions.setStreaming(true); + DataflowRunner.fromOptions(streamingOptions); + assertEquals( + AbstractGoogleAsyncWriteChannel.UPLOAD_PIPE_BUFFER_SIZE_DEFAULT, + streamingOptions.getGcsUploadBufferSizeBytes().intValue()); + } + + @Test + public void testGcsUploadBufferSize() throws IOException { + int gcsUploadBufferSizeBytes = 12345678; + DataflowPipelineOptions batchOptions = buildPipelineOptions(); + batchOptions.setGcsUploadBufferSizeBytes(gcsUploadBufferSizeBytes); + DataflowRunner.fromOptions(batchOptions); + assertEquals(gcsUploadBufferSizeBytes, batchOptions.getGcsUploadBufferSizeBytes().intValue()); + + DataflowPipelineOptions streamingOptions = buildPipelineOptions(); + streamingOptions.setStreaming(true); + streamingOptions.setGcsUploadBufferSizeBytes(gcsUploadBufferSizeBytes); + DataflowRunner.fromOptions(streamingOptions); + assertEquals( + gcsUploadBufferSizeBytes, streamingOptions.getGcsUploadBufferSizeBytes().intValue()); + } + /** * A fake PTransform for testing. */ From 0e088b7fcb2b35d7fdc5125d4dc66e9fa6ae7ffd Mon Sep 17 00:00:00 2001 From: Pei He Date: Wed, 17 Aug 2016 13:56:37 -0700 Subject: [PATCH 027/112] addressed feedback --- .../beam/runners/dataflow/DataflowRunner.java | 6 ++++-- .../runners/dataflow/DataflowRunnerTest.java | 21 ++++++++++++------- 2 files changed, 18 insertions(+), 9 deletions(-) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index 6f8180e0d381..1a845eae8d14 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -219,6 +219,9 @@ public class DataflowRunner extends PipelineRunner { // The limit of CreateJob request size. private static final int CREATE_JOB_REQUEST_LIMIT_BYTES = 10 * 1024 * 1024; + @VisibleForTesting + static final int GCS_UPLOAD_BUFFER_SIZE_BYTES_DEFAULT = 1 * 1024 * 1024; + private final Set> pcollectionsRequiringIndexedFormat; /** @@ -311,8 +314,7 @@ public static DataflowRunner fromOptions(PipelineOptions options) { } if (dataflowOptions.isStreaming() && dataflowOptions.getGcsUploadBufferSizeBytes() == null) { - dataflowOptions.setGcsUploadBufferSizeBytes( - AbstractGoogleAsyncWriteChannel.UPLOAD_PIPE_BUFFER_SIZE_DEFAULT); + dataflowOptions.setGcsUploadBufferSizeBytes(GCS_UPLOAD_BUFFER_SIZE_BYTES_DEFAULT); } return new DataflowRunner(dataflowOptions); diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java index 6f1653b026be..58b9878713be 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java @@ -797,31 +797,38 @@ public void testValidJobName() throws IOException { } @Test - public void testGcsUploadBufferSizeDefault() throws IOException { + public void testGcsUploadBufferSizeIsUnsetForBatchWhenDefault() throws IOException { DataflowPipelineOptions batchOptions = buildPipelineOptions(); - DataflowRunner.fromOptions(batchOptions); + batchOptions.setRunner(DataflowRunner.class); + Pipeline.create(batchOptions); assertNull(batchOptions.getGcsUploadBufferSizeBytes()); + } + @Test + public void testGcsUploadBufferSizeIsSetForStreamingWhenDefault() throws IOException { DataflowPipelineOptions streamingOptions = buildPipelineOptions(); streamingOptions.setStreaming(true); - DataflowRunner.fromOptions(streamingOptions); + streamingOptions.setRunner(DataflowRunner.class); + Pipeline.create(streamingOptions); assertEquals( - AbstractGoogleAsyncWriteChannel.UPLOAD_PIPE_BUFFER_SIZE_DEFAULT, + DataflowRunner.GCS_UPLOAD_BUFFER_SIZE_BYTES_DEFAULT, streamingOptions.getGcsUploadBufferSizeBytes().intValue()); } @Test - public void testGcsUploadBufferSize() throws IOException { + public void testGcsUploadBufferSizeUnchangedWhenNotDefault() throws IOException { int gcsUploadBufferSizeBytes = 12345678; DataflowPipelineOptions batchOptions = buildPipelineOptions(); batchOptions.setGcsUploadBufferSizeBytes(gcsUploadBufferSizeBytes); - DataflowRunner.fromOptions(batchOptions); + batchOptions.setRunner(DataflowRunner.class); + Pipeline.create(batchOptions); assertEquals(gcsUploadBufferSizeBytes, batchOptions.getGcsUploadBufferSizeBytes().intValue()); DataflowPipelineOptions streamingOptions = buildPipelineOptions(); streamingOptions.setStreaming(true); streamingOptions.setGcsUploadBufferSizeBytes(gcsUploadBufferSizeBytes); - DataflowRunner.fromOptions(streamingOptions); + streamingOptions.setRunner(DataflowRunner.class); + Pipeline.create(streamingOptions); assertEquals( gcsUploadBufferSizeBytes, streamingOptions.getGcsUploadBufferSizeBytes().intValue()); } From d9ff2e42339e04358c66308bd292a5a460547f77 Mon Sep 17 00:00:00 2001 From: Pei He Date: Wed, 17 Aug 2016 14:30:23 -0700 Subject: [PATCH 028/112] fix unused imports --- .../java/org/apache/beam/runners/dataflow/DataflowRunner.java | 1 - .../org/apache/beam/runners/dataflow/DataflowRunnerTest.java | 1 - 2 files changed, 2 deletions(-) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index 1a845eae8d14..c4dd703f1232 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -125,7 +125,6 @@ import com.google.api.services.dataflow.model.Job; import com.google.api.services.dataflow.model.ListJobsResponse; import com.google.api.services.dataflow.model.WorkerPool; -import com.google.cloud.hadoop.util.AbstractGoogleAsyncWriteChannel; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Joiner; diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java index 58b9878713be..92a6bcb4c729 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java @@ -89,7 +89,6 @@ import com.google.api.services.dataflow.model.DataflowPackage; import com.google.api.services.dataflow.model.Job; import com.google.api.services.dataflow.model.ListJobsResponse; -import com.google.cloud.hadoop.util.AbstractGoogleAsyncWriteChannel; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; From a0361ae99e9e39bb5ff9766508501932416129ec Mon Sep 17 00:00:00 2001 From: Vikas Kedigehalli Date: Mon, 15 Aug 2016 15:28:07 -0700 Subject: [PATCH 029/112] DatastoreIO Sink as ParDo --- .../beam/sdk/io/gcp/datastore/V1Beta3.java | 376 +++++++----------- .../sdk/io/gcp/datastore/V1Beta3Test.java | 88 ++-- 2 files changed, 195 insertions(+), 269 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3.java index 052feb34a847..0d2e2cb72cab 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3.java @@ -30,10 +30,6 @@ import static com.google.datastore.v1beta3.client.DatastoreHelper.makeValue; import org.apache.beam.sdk.annotations.Experimental; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.SerializableCoder; -import org.apache.beam.sdk.io.Sink.WriteOperation; -import org.apache.beam.sdk.io.Sink.Writer; import org.apache.beam.sdk.options.GcpOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Create; @@ -167,7 +163,8 @@ public class V1Beta3 { * Datastore has a limit of 500 mutations per batch operation, so we flush * changes to Datastore every 500 entities. */ - private static final int DATASTORE_BATCH_UPDATE_LIMIT = 500; + @VisibleForTesting + static final int DATASTORE_BATCH_UPDATE_LIMIT = 500; /** * Returns an empty {@link V1Beta3.Read} builder. Configure the source {@code projectId}, @@ -634,42 +631,8 @@ public void processElement(ProcessContext context) throws Exception { } } } - - /** - * A wrapper factory class for Datastore singleton classes {@link DatastoreFactory} and - * {@link QuerySplitter} - * - *

    {@link DatastoreFactory} and {@link QuerySplitter} are not java serializable, hence - * wrapping them under this class, which implements {@link Serializable}. - */ - @VisibleForTesting - static class V1Beta3DatastoreFactory implements Serializable { - - /** Builds a Datastore client for the given pipeline options and project. */ - public Datastore getDatastore(PipelineOptions pipelineOptions, String projectId) { - DatastoreOptions.Builder builder = - new DatastoreOptions.Builder() - .projectId(projectId) - .initializer( - new RetryHttpRequestInitializer() - ); - - Credential credential = pipelineOptions.as(GcpOptions.class).getGcpCredential(); - if (credential != null) { - builder.credential(credential); - } - - return DatastoreFactory.get().create(builder.build()); - } - - /** Builds a Datastore {@link QuerySplitter}. */ - public QuerySplitter getQuerySplitter() { - return DatastoreHelper.getQuerySplitter(); - } - } } - /** * Returns an empty {@link V1Beta3.Write} builder. Configure the destination * {@code projectId} using {@link V1Beta3.Write#withProjectId}. @@ -705,8 +668,8 @@ public Write withProjectId(String projectId) { @Override public PDone apply(PCollection input) { - return input.apply( - org.apache.beam.sdk.io.Write.to(new DatastoreSink(projectId))); + input.apply(ParDo.of(new DatastoreWriterFn(projectId))); + return PDone.in(input.getPipeline()); } @Override @@ -733,130 +696,127 @@ public void populateDisplayData(DisplayData.Builder builder) { .addIfNotNull(DisplayData.item("projectId", projectId) .withLabel("Output Project")); } - } - /** - * A {@link org.apache.beam.sdk.io.Sink} that writes data to Datastore. - */ - static class DatastoreSink extends org.apache.beam.sdk.io.Sink { - final String projectId; - - public DatastoreSink(String projectId) { - this.projectId = projectId; - } - - @Override - public void validate(PipelineOptions options) { - checkNotNull(projectId, "projectId"); - } - - @Override - public DatastoreWriteOperation createWriteOperation(PipelineOptions options) { - return new DatastoreWriteOperation(this); - } - - @Override - public void populateDisplayData(DisplayData.Builder builder) { - super.populateDisplayData(builder); - builder - .addIfNotNull(DisplayData.item("projectId", projectId) - .withLabel("Output Project")); - } - } + /** + * A {@link DoFn} that writes {@link Entity} objects to Cloud Datastore. Entities are written in + * batches, where the maximum batch size is {@link V1Beta3#DATASTORE_BATCH_UPDATE_LIMIT}. + * Entities are committed as upsert mutations (either update if the key already exists, or + * insert if it is a new key). If an entity does not have a complete key (i.e., it has no name + * or id), the bundle will fail. + * + *

    See + * Datastore: Entities, Properties, and Keys for information about entity keys and entities. + * + *

    Commits are non-transactional. If a commit fails because of a conflict over an entity + * group, the commit will be retried (up to {@link V1Beta3#DATASTORE_BATCH_UPDATE_LIMIT} + * times). + */ + @VisibleForTesting + static class DatastoreWriterFn extends DoFn { + private static final Logger LOG = LoggerFactory.getLogger(DatastoreWriterFn.class); + private final String projectId; + private transient Datastore datastore; + private final V1Beta3DatastoreFactory datastoreFactory; + // Current batch of entities to be written. + private final List entities = new ArrayList<>(); + /** + * Since a bundle is written in batches, we should retry the commit of a batch in order to + * prevent transient errors from causing the bundle to fail. + */ + private static final int MAX_RETRIES = 5; - /** - * A {@link WriteOperation} that will manage a parallel write to a Datastore sink. - */ - private static class DatastoreWriteOperation - extends WriteOperation { - private static final Logger LOG = LoggerFactory.getLogger(DatastoreWriteOperation.class); + /** + * Initial backoff time for exponential backoff for retry attempts. + */ + private static final int INITIAL_BACKOFF_MILLIS = 5000; - private final DatastoreSink sink; + public DatastoreWriterFn(String projectId) { + this(projectId, new V1Beta3DatastoreFactory()); + } - public DatastoreWriteOperation(DatastoreSink sink) { - this.sink = sink; - } + @VisibleForTesting + DatastoreWriterFn(String projectId, V1Beta3DatastoreFactory datastoreFactory) { + this.projectId = checkNotNull(projectId, "projectId"); + this.datastoreFactory = datastoreFactory; + } - @Override - public Coder getWriterResultCoder() { - return SerializableCoder.of(DatastoreWriteResult.class); - } + @StartBundle + public void startBundle(Context c) { + datastore = datastoreFactory.getDatastore(c.getPipelineOptions(), projectId); + } - @Override - public void initialize(PipelineOptions options) throws Exception {} + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + // Verify that the entity to write has a complete key. + if (!isValidKey(c.element().getKey())) { + throw new IllegalArgumentException( + "Entities to be written to the Datastore must have complete keys"); + } - /** - * Finalizes the write. Logs the number of entities written to the Datastore. - */ - @Override - public void finalize(Iterable writerResults, PipelineOptions options) - throws Exception { - long totalEntities = 0; - for (DatastoreWriteResult result : writerResults) { - totalEntities += result.entitiesWritten; + entities.add(c.element()); + if (entities.size() >= V1Beta3.DATASTORE_BATCH_UPDATE_LIMIT) { + flushBatch(); + } } - LOG.info("Wrote {} elements.", totalEntities); - } - @Override - public DatastoreWriter createWriter(PipelineOptions options) throws Exception { - DatastoreOptions.Builder builder = - new DatastoreOptions.Builder() - .projectId(sink.projectId) - .initializer(new RetryHttpRequestInitializer()); - Credential credential = options.as(GcpOptions.class).getGcpCredential(); - if (credential != null) { - builder.credential(credential); + @FinishBundle + public void finishBundle(Context c) throws Exception { + if (entities.size() > 0) { + flushBatch(); + } } - Datastore datastore = DatastoreFactory.get().create(builder.build()); - return new DatastoreWriter(this, datastore); - } + /** + * Writes a batch of entities to Cloud Datastore. + * + *

    If a commit fails, it will be retried (up to {@link DatastoreWriterFn#MAX_RETRIES} + * times). All entities in the batch will be committed again, even if the commit was partially + * successful. If the retry limit is exceeded, the last exception from the Datastore will be + * thrown. + * + * @throws DatastoreException if the commit fails or IOException or InterruptedException if + * backing off between retries fails. + */ + private void flushBatch() throws DatastoreException, IOException, InterruptedException { + LOG.debug("Writing batch of {} entities", entities.size()); + Sleeper sleeper = Sleeper.DEFAULT; + BackOff backoff = new AttemptBoundedExponentialBackOff(MAX_RETRIES, INITIAL_BACKOFF_MILLIS); + + while (true) { + // Batch upsert entities. + try { + CommitRequest.Builder commitRequest = CommitRequest.newBuilder(); + for (Entity entity: entities) { + commitRequest.addMutations(makeUpsert(entity)); + } + commitRequest.setMode(CommitRequest.Mode.NON_TRANSACTIONAL); + datastore.commit(commitRequest.build()); + // Break if the commit threw no exception. + break; + } catch (DatastoreException exception) { + // Only log the code and message for potentially-transient errors. The entire exception + // will be propagated upon the last retry. + LOG.error("Error writing to the Datastore ({}): {}", exception.getCode(), + exception.getMessage()); + if (!BackOffUtils.next(sleeper, backoff)) { + LOG.error("Aborting after {} retries.", MAX_RETRIES); + throw exception; + } + } + } + LOG.debug("Successfully wrote {} entities", entities.size()); + entities.clear(); + } - @Override - public DatastoreSink getSink() { - return sink; + @Override + public void populateDisplayData(Builder builder) { + super.populateDisplayData(builder); + builder + .addIfNotNull(DisplayData.item("projectId", projectId) + .withLabel("Output Project")); + } } - } - - /** - * {@link Writer} that writes entities to a Datastore Sink. Entities are written in batches, - * where the maximum batch size is {@link V1Beta3#DATASTORE_BATCH_UPDATE_LIMIT}. Entities - * are committed as upsert mutations (either update if the key already exists, or insert if it is - * a new key). If an entity does not have a complete key (i.e., it has no name or id), the bundle - * will fail. - * - *

    See - * Datastore: Entities, Properties, and Keys for information about entity keys and upsert - * mutations. - * - *

    Commits are non-transactional. If a commit fails because of a conflict over an entity - * group, the commit will be retried (up to {@link V1Beta3#DATASTORE_BATCH_UPDATE_LIMIT} - * times). - * - *

    Visible for testing purposes. - */ - @VisibleForTesting - static class DatastoreWriter extends Writer { - private static final Logger LOG = LoggerFactory.getLogger(DatastoreWriter.class); - private final DatastoreWriteOperation writeOp; - private final Datastore datastore; - private long totalWritten = 0; - - // Visible for testing. - final List entities = new ArrayList<>(); - - /** - * Since a bundle is written in batches, we should retry the commit of a batch in order to - * prevent transient errors from causing the bundle to fail. - */ - private static final int MAX_RETRIES = 5; - - /** - * Initial backoff time for exponential backoff for retry attempts. - */ - private static final int INITIAL_BACKOFF_MILLIS = 5000; /** * Returns true if a Datastore key is complete. A key is complete if its last element @@ -870,100 +830,38 @@ static boolean isValidKey(Key key) { PathElement lastElement = elementList.get(elementList.size() - 1); return (lastElement.getId() != 0 || !lastElement.getName().isEmpty()); } + } - DatastoreWriter(DatastoreWriteOperation writeOp, Datastore datastore) { - this.writeOp = writeOp; - this.datastore = datastore; - } - - @Override - public void open(String uId) throws Exception {} - - /** - * Writes an entity to the Datastore. Writes are batched, up to {@link - * V1Beta3#DATASTORE_BATCH_UPDATE_LIMIT}. If an entity does not have a complete key, an - * {@link IllegalArgumentException} will be thrown. - */ - @Override - public void write(Entity value) throws Exception { - // Verify that the entity to write has a complete key. - if (!isValidKey(value.getKey())) { - throw new IllegalArgumentException( - "Entities to be written to the Datastore must have complete keys"); - } - - entities.add(value); + /** + * A wrapper factory class for Datastore singleton classes {@link DatastoreFactory} and + * {@link QuerySplitter} + * + *

    {@link DatastoreFactory} and {@link QuerySplitter} are not java serializable, hence + * wrapping them under this class, which implements {@link Serializable}. + */ + @VisibleForTesting + static class V1Beta3DatastoreFactory implements Serializable { - if (entities.size() >= V1Beta3.DATASTORE_BATCH_UPDATE_LIMIT) { - flushBatch(); - } - } + /** Builds a Datastore client for the given pipeline options and project. */ + public Datastore getDatastore(PipelineOptions pipelineOptions, String projectId) { + DatastoreOptions.Builder builder = + new DatastoreOptions.Builder() + .projectId(projectId) + .initializer( + new RetryHttpRequestInitializer() + ); - /** - * Flushes any pending batch writes and returns a DatastoreWriteResult. - */ - @Override - public DatastoreWriteResult close() throws Exception { - if (entities.size() > 0) { - flushBatch(); + Credential credential = pipelineOptions.as(GcpOptions.class).getGcpCredential(); + if (credential != null) { + builder.credential(credential); } - return new DatastoreWriteResult(totalWritten); - } - @Override - public DatastoreWriteOperation getWriteOperation() { - return writeOp; + return DatastoreFactory.get().create(builder.build()); } - /** - * Writes a batch of entities to the Datastore. - * - *

    If a commit fails, it will be retried (up to {@link DatastoreWriter#MAX_RETRIES} - * times). All entities in the batch will be committed again, even if the commit was partially - * successful. If the retry limit is exceeded, the last exception from the Datastore will be - * thrown. - * - * @throws DatastoreException if the commit fails or IOException or InterruptedException if - * backing off between retries fails. - */ - private void flushBatch() throws DatastoreException, IOException, InterruptedException { - LOG.debug("Writing batch of {} entities", entities.size()); - Sleeper sleeper = Sleeper.DEFAULT; - BackOff backoff = new AttemptBoundedExponentialBackOff(MAX_RETRIES, INITIAL_BACKOFF_MILLIS); - - while (true) { - // Batch upsert entities. - try { - CommitRequest.Builder commitRequest = CommitRequest.newBuilder(); - for (Entity entity: entities) { - commitRequest.addMutations(makeUpsert(entity)); - } - commitRequest.setMode(CommitRequest.Mode.NON_TRANSACTIONAL); - datastore.commit(commitRequest.build()); - // Break if the commit threw no exception. - break; - } catch (DatastoreException exception) { - // Only log the code and message for potentially-transient errors. The entire exception - // will be propagated upon the last retry. - LOG.error("Error writing to the Datastore ({}): {}", exception.getCode(), - exception.getMessage()); - if (!BackOffUtils.next(sleeper, backoff)) { - LOG.error("Aborting after {} retries.", MAX_RETRIES); - throw exception; - } - } - } - totalWritten += entities.size(); - LOG.debug("Successfully wrote {} entities", entities.size()); - entities.clear(); - } - } - - private static class DatastoreWriteResult implements Serializable { - final long entitiesWritten; - - public DatastoreWriteResult(long recordsWritten) { - this.entitiesWritten = recordsWritten; + /** Builds a Datastore {@link QuerySplitter}. */ + public QuerySplitter getQuerySplitter() { + return DatastoreHelper.getQuerySplitter(); } } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3Test.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3Test.java index 9947c6087a46..8fa34dae119f 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3Test.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3Test.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.io.gcp.datastore; +import static org.apache.beam.sdk.io.gcp.datastore.V1Beta3.DATASTORE_BATCH_UPDATE_LIMIT; import static org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Read.DEFAULT_BUNDLE_SIZE_BYTES; import static org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Read.QUERY_BATCH_LIMIT; import static org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Read.getEstimatedSizeBytes; @@ -27,8 +28,8 @@ import static com.google.datastore.v1beta3.client.DatastoreHelper.makeFilter; import static com.google.datastore.v1beta3.client.DatastoreHelper.makeKey; import static com.google.datastore.v1beta3.client.DatastoreHelper.makeOrder; +import static com.google.datastore.v1beta3.client.DatastoreHelper.makeUpsert; import static com.google.datastore.v1beta3.client.DatastoreHelper.makeValue; -import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.lessThanOrEqualTo; @@ -44,11 +45,12 @@ import static org.mockito.Mockito.verifyZeroInteractions; import static org.mockito.Mockito.when; -import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.DatastoreWriter; import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Read.ReadFn; import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Read.SplitQueryFn; -import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Read.V1Beta3DatastoreFactory; import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Read.V1Beta3Options; +import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.V1Beta3DatastoreFactory; +import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Write; +import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Write.DatastoreWriterFn; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.testing.RunnableOnService; import org.apache.beam.sdk.transforms.DoFnTester; @@ -61,7 +63,7 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.POutput; -import com.google.common.collect.Lists; +import com.google.datastore.v1beta3.CommitRequest; import com.google.datastore.v1beta3.Entity; import com.google.datastore.v1beta3.EntityResult; import com.google.datastore.v1beta3.Key; @@ -87,7 +89,6 @@ import org.mockito.stubbing.Answer; import java.util.ArrayList; -import java.util.Collections; import java.util.HashSet; import java.util.LinkedList; import java.util.List; @@ -285,33 +286,33 @@ public void testHasNameOrId() { Key key; // Complete with name, no ancestor key = makeKey("bird", "finch").build(); - assertTrue(DatastoreWriter.isValidKey(key)); + assertTrue(Write.isValidKey(key)); // Complete with id, no ancestor key = makeKey("bird", 123).build(); - assertTrue(DatastoreWriter.isValidKey(key)); + assertTrue(Write.isValidKey(key)); // Incomplete, no ancestor key = makeKey("bird").build(); - assertFalse(DatastoreWriter.isValidKey(key)); + assertFalse(Write.isValidKey(key)); // Complete with name and ancestor key = makeKey("bird", "owl").build(); key = makeKey(key, "bird", "horned").build(); - assertTrue(DatastoreWriter.isValidKey(key)); + assertTrue(Write.isValidKey(key)); // Complete with id and ancestor key = makeKey("bird", "owl").build(); key = makeKey(key, "bird", 123).build(); - assertTrue(DatastoreWriter.isValidKey(key)); + assertTrue(Write.isValidKey(key)); // Incomplete with ancestor key = makeKey("bird", "owl").build(); key = makeKey(key, "bird").build(); - assertFalse(DatastoreWriter.isValidKey(key)); + assertFalse(Write.isValidKey(key)); key = makeKey().build(); - assertFalse(DatastoreWriter.isValidKey(key)); + assertFalse(Write.isValidKey(key)); } /** @@ -321,35 +322,62 @@ public void testHasNameOrId() { public void testAddEntitiesWithIncompleteKeys() throws Exception { Key key = makeKey("bird").build(); Entity entity = Entity.newBuilder().setKey(key).build(); - DatastoreWriter writer = new DatastoreWriter(null, mockDatastore); + DatastoreWriterFn writer = new DatastoreWriterFn(PROJECT_ID, mockDatastoreFactory); + DoFnTester doFnTester = DoFnTester.of(writer); + doFnTester.setCloningBehavior(CloningBehavior.DO_NOT_CLONE); thrown.expect(IllegalArgumentException.class); thrown.expectMessage("Entities to be written to the Datastore must have complete keys"); - writer.write(entity); + doFnTester.processBundle(entity); + } + + /** Tests {@link DatastoreWriterFn} with entities less than one batch. */ + @Test + public void testDatatoreWriterFnWithOneBatch() throws Exception { + datastoreWriterFnTest(100); + } + + /** Tests {@link DatastoreWriterFn} with entities of more than one batches, but not a multiple. */ + @Test + public void testDatatoreWriterFnWithMultipleBatches() throws Exception { + datastoreWriterFnTest(DATASTORE_BATCH_UPDATE_LIMIT * 3 + 100); } /** - * Test that entities are added to the batch to update. + * Tests {@link DatastoreWriterFn} with entities of several batches, using an exact multiple of + * write batch size. */ @Test - public void testAddingEntities() throws Exception { - List expected = Lists.newArrayList( - Entity.newBuilder().setKey(makeKey("bird", "jay").build()).build(), - Entity.newBuilder().setKey(makeKey("bird", "condor").build()).build(), - Entity.newBuilder().setKey(makeKey("bird", "robin").build()).build()); - - List allEntities = Lists.newArrayList(expected); - Collections.shuffle(allEntities); - - DatastoreWriter writer = new DatastoreWriter(null, mockDatastore); - writer.open("test_id"); - for (Entity entity : allEntities) { - writer.write(entity); + public void testDatatoreWriterFnWithBatchesExactMultiple() throws Exception { + datastoreWriterFnTest(DATASTORE_BATCH_UPDATE_LIMIT * 2); + } + + // A helper method to test DatastoreWriterFn for various batch sizes. + private void datastoreWriterFnTest(int numEntities) throws Exception { + // Create the requested number of mutations. + List entities = new ArrayList<>(numEntities); + for (int i = 0; i < numEntities; ++i) { + entities.add(Entity.newBuilder().setKey(makeKey("key" + i, i + 1)).build()); } - assertEquals(expected.size(), writer.entities.size()); - assertThat(writer.entities, containsInAnyOrder(expected.toArray())); + DatastoreWriterFn datastoreWriter = new DatastoreWriterFn(PROJECT_ID, mockDatastoreFactory); + DoFnTester doFnTester = DoFnTester.of(datastoreWriter); + doFnTester.setCloningBehavior(CloningBehavior.DO_NOT_CLONE); + doFnTester.processBundle(entities); + + int start = 0; + while (start < numEntities) { + int end = Math.min(numEntities, start + DATASTORE_BATCH_UPDATE_LIMIT); + CommitRequest.Builder commitRequest = CommitRequest.newBuilder(); + commitRequest.setMode(CommitRequest.Mode.NON_TRANSACTIONAL); + for (Entity entity: entities.subList(start, end)) { + commitRequest.addMutations(makeUpsert(entity)); + } + // Verify all the batch requests were made with the expected entities. + verify(mockDatastore, times(1)).commit(commitRequest.build()); + start = end; + } } /** From 214776e5f06da91b808c51bd3ae69a3811c30cef Mon Sep 17 00:00:00 2001 From: Frank Yellin Date: Tue, 16 Aug 2016 14:40:41 -0700 Subject: [PATCH 030/112] Change name of result returned by BigQueryIO.Read --- .../java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java index aa168bdab0db..ce0446788156 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java @@ -165,7 +165,7 @@ *

    To read from a BigQuery table, apply a {@link BigQueryIO.Read} transformation. * This produces a {@link PCollection} of {@link TableRow TableRows} as output: *

    {@code
    - * PCollection shakespeare = pipeline.apply(
    + * PCollection weatherData = pipeline.apply(
      *     BigQueryIO.Read.from("clouddataflow-readonly:samples.weather_stations"));
      * }
    * @@ -176,7 +176,7 @@ * input transform. * *
    {@code
    - * PCollection shakespeare = pipeline.apply(
    + * PCollection meanTemperatureData = pipeline.apply(
      *     BigQueryIO.Read.fromQuery("SELECT year, mean_temp FROM [samples.weather_stations]"));
      * }
    * From 2c8a6546af2adb1f7694f29a092338898f851d16 Mon Sep 17 00:00:00 2001 From: Pei He Date: Mon, 15 Aug 2016 17:23:20 -0700 Subject: [PATCH 031/112] Fix NPE in BigQueryIO.TransformingReader --- .../beam/sdk/testing/SourceTestUtils.java | 132 ++++++++++++++++++ .../beam/sdk/testing/SourceTestUtilsTest.java | 66 +++++++++ .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 12 +- .../sdk/io/gcp/bigquery/BigQueryIOTest.java | 31 ++++ 4 files changed, 235 insertions(+), 6 deletions(-) create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SourceTestUtilsTest.java diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SourceTestUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SourceTestUtils.java index e0b88900dbc5..9ce9c5e518da 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SourceTestUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SourceTestUtils.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.testing; +import static com.google.common.base.Preconditions.checkNotNull; + import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertEquals; @@ -27,10 +29,15 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.io.BoundedSource.BoundedReader; import org.apache.beam.sdk.io.Source; import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.values.KV; +import com.google.common.collect.ImmutableList; + +import org.joda.time.Instant; import org.junit.Assert; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -38,6 +45,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.NoSuchElementException; import java.util.Objects; import java.util.concurrent.Callable; import java.util.concurrent.CountDownLatch; @@ -45,6 +53,8 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; +import javax.annotation.Nullable; + /** * Helper functions and test harnesses for checking correctness of {@link Source} * implementations. @@ -673,4 +683,126 @@ public KV, BoundedSource> call() throws Exception { numItemsToReadBeforeSplitting, fraction, options); return (res.numResidualItems > 0); } + + /** + * Returns an equivalent unsplittable {@code BoundedSource}. + * + *

    It forwards most methods to the given {@code boundedSource}, except: + *

      + *
    1. {@link BoundedSource#splitIntoBundles} rejects initial splitting + * by returning itself in a list. + *
    2. {@link BoundedReader#splitAtFraction} rejects dynamic splitting by returning null. + *
    + */ + public static BoundedSource toUnsplittableSource(BoundedSource boundedSource) { + return new UnsplittableSource<>(boundedSource); + } + + private static class UnsplittableSource extends BoundedSource { + + private final BoundedSource boundedSource; + + private UnsplittableSource(BoundedSource boundedSource) { + this.boundedSource = checkNotNull(boundedSource, "boundedSource"); + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + this.boundedSource.populateDisplayData(builder); + } + + @Override + public List> splitIntoBundles( + long desiredBundleSizeBytes, PipelineOptions options) throws Exception { + return ImmutableList.of(this); + } + + @Override + public long getEstimatedSizeBytes(PipelineOptions options) throws Exception { + return boundedSource.getEstimatedSizeBytes(options); + } + + @Override + public boolean producesSortedKeys(PipelineOptions options) throws Exception { + return boundedSource.producesSortedKeys(options); + } + + @Override + public BoundedReader createReader(PipelineOptions options) throws IOException { + return new UnsplittableReader<>(boundedSource, boundedSource.createReader(options)); + } + + @Override + public void validate() { + boundedSource.validate(); + } + + @Override + public Coder getDefaultOutputCoder() { + return boundedSource.getDefaultOutputCoder(); + } + + private static class UnsplittableReader extends BoundedReader { + + private final BoundedSource boundedSource; + private final BoundedReader boundedReader; + + private UnsplittableReader(BoundedSource boundedSource, BoundedReader boundedReader) { + this.boundedSource = checkNotNull(boundedSource, "boundedSource"); + this.boundedReader = checkNotNull(boundedReader, "boundedReader"); + } + + @Override + public BoundedSource getCurrentSource() { + return boundedSource; + } + + @Override + public boolean start() throws IOException { + return boundedReader.start(); + } + + @Override + public boolean advance() throws IOException { + return boundedReader.advance(); + } + + @Override + public T getCurrent() throws NoSuchElementException { + return boundedReader.getCurrent(); + } + + @Override + public void close() throws IOException { + boundedReader.close(); + } + + @Override + @Nullable + public BoundedSource splitAtFraction(double fraction) { + return null; + } + + @Override + @Nullable + public Double getFractionConsumed() { + return boundedReader.getFractionConsumed(); + } + + @Override + public long getSplitPointsConsumed() { + return boundedReader.getSplitPointsConsumed(); + } + + @Override + public long getSplitPointsRemaining() { + return boundedReader.getSplitPointsRemaining(); + } + + @Override + public Instant getCurrentTimestamp() throws NoSuchElementException { + return boundedReader.getCurrentTimestamp(); + } + } + } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SourceTestUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SourceTestUtilsTest.java new file mode 100644 index 000000000000..f2b332bb7dd3 --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SourceTestUtilsTest.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.testing; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.io.BoundedSource.BoundedReader; +import org.apache.beam.sdk.io.CountingSource; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; + +import com.google.common.collect.Sets; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.List; +import java.util.Set; + +/** + * Tests for {@link SourceTestUtils}. + */ +@RunWith(JUnit4.class) +public class SourceTestUtilsTest { + + @Test + public void testToUnsplittableSource() throws Exception { + PipelineOptions options = PipelineOptionsFactory.create(); + BoundedSource baseSource = CountingSource.upTo(100); + BoundedSource unsplittableSource = SourceTestUtils.toUnsplittableSource(baseSource); + List splits = unsplittableSource.splitIntoBundles(1, options); + assertEquals(splits.size(), 1); + assertEquals(splits.get(0), unsplittableSource); + + BoundedReader unsplittableReader = unsplittableSource.createReader(options); + assertEquals(0, unsplittableReader.getFractionConsumed(), 1e-15); + + Set expected = Sets.newHashSet(SourceTestUtils.readFromSource(baseSource, options)); + Set actual = Sets.newHashSet(); + actual.addAll(SourceTestUtils.readNItemsFromUnstartedReader(unsplittableReader, 40)); + assertNull(unsplittableReader.splitAtFraction(0.5)); + actual.addAll(SourceTestUtils.readRemainingFromReader(unsplittableReader, true /* started */)); + assertEquals(1, unsplittableReader.getFractionConsumed(), 1e-15); + + assertEquals(100, actual.size()); + assertEquals(Sets.newHashSet(expected), Sets.newHashSet(actual)); + } +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java index ce0446788156..e61dcca993a0 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java @@ -1123,9 +1123,9 @@ static class TransformingSource extends BoundedSource { BoundedSource boundedSource, SerializableFunction function, Coder outputCoder) { - this.boundedSource = boundedSource; - this.function = function; - this.outputCoder = outputCoder; + this.boundedSource = checkNotNull(boundedSource, "boundedSource"); + this.function = checkNotNull(function, "function"); + this.outputCoder = checkNotNull(outputCoder, "outputCoder"); } @Override @@ -1170,7 +1170,7 @@ private class TransformingReader extends BoundedReader { private final BoundedReader boundedReader; private TransformingReader(BoundedReader boundedReader) { - this.boundedReader = boundedReader; + this.boundedReader = checkNotNull(boundedReader, "boundedReader"); } @Override @@ -1201,8 +1201,8 @@ public void close() throws IOException { @Override public synchronized BoundedSource splitAtFraction(double fraction) { - return new TransformingSource<>( - boundedReader.splitAtFraction(fraction), function, outputCoder); + BoundedSource split = boundedReader.splitAtFraction(fraction); + return split == null ? null : new TransformingSource<>(split, function, outputCoder); } @Override diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java index fcaa054da456..ca60696960e8 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java @@ -1215,6 +1215,37 @@ public String apply(Long input) { stringSource, stringSource.splitIntoBundles(100, options), options); } + @Test + public void testTransformingSourceUnsplittable() throws Exception { + int numElements = 10000; + @SuppressWarnings("deprecation") + BoundedSource longSource = + SourceTestUtils.toUnsplittableSource(CountingSource.upTo(numElements)); + SerializableFunction toStringFn = + new SerializableFunction() { + @Override + public String apply(Long input) { + return input.toString(); + } + }; + BoundedSource stringSource = + new TransformingSource<>(longSource, toStringFn, StringUtf8Coder.of()); + + List expected = Lists.newArrayList(); + for (int i = 0; i < numElements; i++) { + expected.add(String.valueOf(i)); + } + + PipelineOptions options = PipelineOptionsFactory.create(); + Assert.assertThat( + SourceTestUtils.readFromSource(stringSource, options), CoreMatchers.is(expected)); + SourceTestUtils.assertSplitAtFractionBehavior( + stringSource, 100, 0.3, ExpectedSplitOutcome.MUST_BE_CONSISTENT_IF_SUCCEEDS, options); + + SourceTestUtils.assertSourcesEqualReferenceSource( + stringSource, stringSource.splitIntoBundles(100, options), options); + } + @Test @Category(RunnableOnService.class) public void testPassThroughThenCleanup() throws Exception { From f15fab8ccdb3b40004583e8f7e4e32a0b8ba5121 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Thu, 11 Aug 2016 15:46:10 -0700 Subject: [PATCH 032/112] Add inEarlyPanesInGlobalWindow as a PAssert Extractor This is for use in asserting the contents of speculative panes in the global window. --- .../org/apache/beam/sdk/testing/PAssert.java | 18 ++++++++++++++++++ .../beam/sdk/testing/PaneExtractors.java | 18 ++++++++++++++++++ 2 files changed, 36 insertions(+) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java index e07ee3dad4f4..3f1a741a4dff 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java @@ -49,6 +49,7 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.Never; +import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing; import org.apache.beam.sdk.transforms.windowing.Trigger; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior; @@ -175,6 +176,13 @@ public interface IterableAssert { */ IterableAssert inCombinedNonLatePanes(BoundedWindow window); + /** + * Creates a new {@link IterableAssert} like this one, but with the assertion restricted to only + * run on panes in the {@link GlobalWindow} that were emitted before the {@link GlobalWindow} + * closed. These panes have {@link Timing#EARLY}. + */ + IterableAssert inEarlyGlobalWindowPanes(); + /** * Asserts that the iterable in question contains the provided elements. * @@ -381,6 +389,11 @@ public PCollectionContentsAssert inCombinedNonLatePanes(BoundedWindow window) return withPane(window, PaneExtractors.nonLatePanes()); } + @Override + public IterableAssert inEarlyGlobalWindowPanes() { + return withPane(GlobalWindow.INSTANCE, PaneExtractors.earlyPanes()); + } + private PCollectionContentsAssert withPane( BoundedWindow window, SimpleFunction>, Iterable> paneExtractor) { @@ -557,6 +570,11 @@ public PCollectionSingletonIterableAssert inCombinedNonLatePanes(BoundedWindo return withPanes(window, PaneExtractors.>nonLatePanes()); } + @Override + public IterableAssert inEarlyGlobalWindowPanes() { + return withPanes(GlobalWindow.INSTANCE, PaneExtractors.>earlyPanes()); + } + private PCollectionSingletonIterableAssert withPanes( BoundedWindow window, SimpleFunction>>, Iterable>> paneExtractor) { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PaneExtractors.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PaneExtractors.java index f699bfc83cce..899612b3ef2a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PaneExtractors.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PaneExtractors.java @@ -59,6 +59,10 @@ static SimpleFunction>, Iterable> nonLatePanes( return new ExtractNonLatePanes<>(); } + static SimpleFunction>, Iterable> earlyPanes() { + return new ExtractEarlyPanes<>(); + } + static SimpleFunction>, Iterable> allPanes() { return new ExtractAllPanes<>(); } @@ -137,4 +141,18 @@ public Iterable apply(Iterable> input) { return outputs; } } + + private static class ExtractEarlyPanes + extends SimpleFunction>, Iterable> { + @Override + public Iterable apply(Iterable> input) { + List outputs = new ArrayList<>(); + for (WindowedValue value : input) { + if (value.getPane().getTiming() == PaneInfo.Timing.EARLY) { + outputs.add(value.getValue()); + } + } + return outputs; + } + } } From c72d4fcd4ca68c00c7edc6094976228a7e999953 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Mon, 15 Aug 2016 19:43:28 -0700 Subject: [PATCH 033/112] Add TestStream to the Testing package This is a source suitable for use with tests that have interesting triggering behavior. It is an Unbounded source that emits elements in bundles, and advances the watermark and processing time appropriately. --- runners/direct-java/pom.xml | 3 + .../apache/beam/sdk/testing/TestStream.java | 326 ++++++++++++++++++ .../beam/sdk/testing/TestStreamTest.java | 169 +++++++++ 3 files changed, 498 insertions(+) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestStream.java create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java diff --git a/runners/direct-java/pom.xml b/runners/direct-java/pom.xml index e06883f6e65d..8b0f91d858a8 100644 --- a/runners/direct-java/pom.xml +++ b/runners/direct-java/pom.xml @@ -85,6 +85,9 @@ org.apache.beam:beam-sdks-java-core org.apache.beam:beam-runners-java-core + + org/apache/beam/sdk/testing/TestStreamTest.java + [ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestStream.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestStream.java new file mode 100644 index 000000000000..6d11f72bb979 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestStream.java @@ -0,0 +1,326 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.sdk.testing; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; + +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.DurationCoder; +import org.apache.beam.sdk.coders.InstantCoder; +import org.apache.beam.sdk.coders.IterableCoder; +import org.apache.beam.sdk.coders.StandardCoder; +import org.apache.beam.sdk.runners.PipelineRunner; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.PropertyNames; +import org.apache.beam.sdk.util.VarInt; +import org.apache.beam.sdk.util.WindowingStrategy; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollection.IsBounded; +import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.sdk.values.TimestampedValue.TimestampedValueCoder; + +import com.google.auto.value.AutoValue; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.joda.time.ReadableDuration; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Collections; +import java.util.List; + +/** + * A testing input that generates an unbounded {@link PCollection} of elements, advancing the + * watermark and processing time as elements are emitted. After all of the specified elements are + * emitted, ceases to produce output. + * + *

    Each call to a {@link TestStream.Builder} method will only be reflected in the state of the + * {@link Pipeline} after each method before it has completed and no more progress can be made by + * the {@link Pipeline}. A {@link PipelineRunner} must ensure that no more progress can be made in + * the {@link Pipeline} before advancing the state of the {@link TestStream}. + */ +public final class TestStream extends PTransform> { + private final List> events; + private final Coder coder; + + /** + * Create a new {@link TestStream.Builder} with no elements and watermark equal to {@link + * BoundedWindow#TIMESTAMP_MIN_VALUE}. + */ + public static Builder create(Coder coder) { + return new Builder<>(coder); + } + + private TestStream(Coder coder, List> events) { + this.coder = coder; + this.events = checkNotNull(events); + } + + public Coder> getEventCoder() { + return EventCoder.of(coder); + } + + /** + * An incomplete {@link TestStream}. Elements added to this builder will be produced in sequence + * when the pipeline created by the {@link TestStream} is run. + */ + public static class Builder { + private final Coder coder; + private final ImmutableList.Builder> events; + private Instant currentWatermark; + + private Builder(Coder coder) { + this.coder = coder; + events = ImmutableList.builder(); + + currentWatermark = BoundedWindow.TIMESTAMP_MIN_VALUE; + } + + /** + * Adds the specified elements to the source with timestamp equal to the current watermark. + * + * @return this {@link TestStream.Builder} + */ + @SafeVarargs + public final Builder addElements(T element, T... elements) { + TimestampedValue firstElement = TimestampedValue.of(element, currentWatermark); + @SuppressWarnings("unchecked") + TimestampedValue[] remainingElements = new TimestampedValue[elements.length]; + for (int i = 0; i < elements.length; i++) { + remainingElements[i] = TimestampedValue.of(elements[i], currentWatermark); + } + return addElements(firstElement, remainingElements); + } + + /** + * Adds the specified elements to the source with the provided timestamps. + * + * @return this {@link TestStream.Builder} + */ + @SafeVarargs + public final Builder addElements( + TimestampedValue element, TimestampedValue... elements) { + events.add(ElementEvent.add(element, elements)); + return this; + } + + /** + * Advance the watermark of this source to the specified instant. + * + *

    The watermark must advance monotonically and to at most {@link + * BoundedWindow#TIMESTAMP_MAX_VALUE}. + * + * @return this {@link TestStream.Builder} + */ + public Builder advanceWatermarkTo(Instant newWatermark) { + checkArgument( + newWatermark.isAfter(currentWatermark), "The watermark must monotonically advance"); + checkArgument( + newWatermark.isBefore(BoundedWindow.TIMESTAMP_MAX_VALUE), + "The Watermark cannot progress beyond the maximum. Got: %s. Maximum: %s", + newWatermark, + BoundedWindow.TIMESTAMP_MAX_VALUE); + events.add(WatermarkEvent.advanceTo(newWatermark)); + currentWatermark = newWatermark; + return this; + } + + /** + * Advance the processing time by the specified amount. + * + * @return this {@link TestStream.Builder} + */ + public Builder advanceProcessingTime(Duration amount) { + checkArgument( + amount.getMillis() > 0, + "Must advance the processing time by a positive amount. Got: ", + amount); + events.add(ProcessingTimeEvent.advanceBy(amount)); + return this; + } + + /** + * Advance the watermark to infinity, completing this {@link TestStream}. Future calls to the + * same builder will not affect the returned {@link TestStream}. + */ + public TestStream advanceWatermarkToInfinity() { + events.add(WatermarkEvent.advanceTo(BoundedWindow.TIMESTAMP_MAX_VALUE)); + return new TestStream<>(coder, events.build()); + } + } + + /** + * An event in a {@link TestStream}. A marker interface for all events that happen while + * evaluating a {@link TestStream}. + */ + public interface Event { + EventType getType(); + } + + /** + * The types of {@link Event} that are supported by {@link TestStream}. + */ + public enum EventType { + ELEMENT, + WATERMARK, + PROCESSING_TIME + } + + /** A {@link Event} that produces elements. */ + @AutoValue + public abstract static class ElementEvent implements Event { + public abstract Iterable> getElements(); + + @SafeVarargs + static Event add(TimestampedValue element, TimestampedValue... elements) { + return add(ImmutableList.>builder().add(element).add(elements).build()); + } + + static Event add(Iterable> elements) { + return new AutoValue_TestStream_ElementEvent<>(EventType.ELEMENT, elements); + } + } + + /** A {@link Event} that advances the watermark. */ + @AutoValue + public abstract static class WatermarkEvent implements Event { + public abstract Instant getWatermark(); + + static Event advanceTo(Instant newWatermark) { + return new AutoValue_TestStream_WatermarkEvent<>(EventType.WATERMARK, newWatermark); + } + } + + /** A {@link Event} that advances the processing time clock. */ + @AutoValue + public abstract static class ProcessingTimeEvent implements Event { + public abstract Duration getProcessingTimeAdvance(); + + static Event advanceBy(Duration amount) { + return new AutoValue_TestStream_ProcessingTimeEvent<>(EventType.PROCESSING_TIME, amount); + } + } + + @Override + public PCollection apply(PBegin input) { + return PCollection.createPrimitiveOutputInternal( + input.getPipeline(), WindowingStrategy.globalDefault(), IsBounded.UNBOUNDED) + .setCoder(coder); + } + + public List> getStreamEvents() { + return events; + } + + /** + * A {@link Coder} that encodes and decodes {@link TestStream.Event Events}. + * + * @param the type of elements in {@link ElementEvent ElementEvents} encoded and decoded by + * this {@link EventCoder} + */ + @VisibleForTesting + static final class EventCoder extends StandardCoder> { + private static final Coder DURATION_CODER = DurationCoder.of(); + private static final Coder INSTANT_CODER = InstantCoder.of(); + private final Coder valueCoder; + private final Coder>> elementCoder; + + public static EventCoder of(Coder valueCoder) { + return new EventCoder<>(valueCoder); + } + + @JsonCreator + public static EventCoder of( + @JsonProperty(PropertyNames.COMPONENT_ENCODINGS) List> components) { + checkArgument( + components.size() == 1, + "Was expecting exactly one component coder, got %s", + components.size()); + return new EventCoder<>((Coder) components.get(0)); + } + + private EventCoder(Coder valueCoder) { + this.valueCoder = valueCoder; + this.elementCoder = IterableCoder.of(TimestampedValueCoder.of(valueCoder)); + } + + @Override + public void encode( + Event value, OutputStream outStream, Context context) + throws IOException { + VarInt.encode(value.getType().ordinal(), outStream); + switch (value.getType()) { + case ELEMENT: + Iterable> elems = ((ElementEvent) value).getElements(); + elementCoder.encode(elems, outStream, context); + break; + case WATERMARK: + Instant ts = ((WatermarkEvent) value).getWatermark(); + INSTANT_CODER.encode(ts, outStream, context); + break; + case PROCESSING_TIME: + Duration processingAdvance = ((ProcessingTimeEvent) value).getProcessingTimeAdvance(); + DURATION_CODER.encode(processingAdvance, outStream, context); + break; + default: + throw new AssertionError("Unreachable"); + } + } + + @Override + public Event decode( + InputStream inStream, Context context) throws IOException { + switch (EventType.values()[VarInt.decodeInt(inStream)]) { + case ELEMENT: + Iterable> elements = elementCoder.decode(inStream, context); + return ElementEvent.add(elements); + case WATERMARK: + return WatermarkEvent.advanceTo(INSTANT_CODER.decode(inStream, context)); + case PROCESSING_TIME: + return ProcessingTimeEvent.advanceBy( + DURATION_CODER.decode(inStream, context).toDuration()); + default: + throw new AssertionError("Unreachable"); + } + } + + @Override + public List> getCoderArguments() { + return Collections.singletonList(valueCoder); + } + + @Override + public void verifyDeterministic() throws NonDeterministicException { + elementCoder.verifyDeterministic(); + DURATION_CODER.verifyDeterministic(); + INSTANT_CODER.verifyDeterministic(); + } + } +} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java new file mode 100644 index 000000000000..09bccfa17126 --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java @@ -0,0 +1,169 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.sdk.testing; + +import static org.hamcrest.Matchers.allOf; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.lessThanOrEqualTo; +import static org.junit.Assert.assertThat; + +import org.apache.beam.sdk.coders.VarIntCoder; +import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.transforms.Count; +import org.apache.beam.sdk.transforms.Flatten; +import org.apache.beam.sdk.transforms.GroupByKey; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.transforms.Sum; +import org.apache.beam.sdk.transforms.Values; +import org.apache.beam.sdk.transforms.WithKeys; +import org.apache.beam.sdk.transforms.windowing.AfterPane; +import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime; +import org.apache.beam.sdk.transforms.windowing.AfterWatermark; +import org.apache.beam.sdk.transforms.windowing.FixedWindows; +import org.apache.beam.sdk.transforms.windowing.IntervalWindow; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TimestampedValue; + +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.Serializable; + +/** + * Tests for {@link TestStream}. + */ +@RunWith(JUnit4.class) +public class TestStreamTest implements Serializable { + @Test + @Category(NeedsRunner.class) + public void testLateDataAccumulating() { + Instant instant = new Instant(0); + TestStream source = TestStream.create(VarIntCoder.of()) + .addElements(TimestampedValue.of(1, instant), + TimestampedValue.of(2, instant), + TimestampedValue.of(3, instant)) + .advanceWatermarkTo(instant.plus(Duration.standardMinutes(6))) + // These elements are late but within the allowed lateness + .addElements(TimestampedValue.of(4, instant), TimestampedValue.of(5, instant)) + .advanceWatermarkTo(instant.plus(Duration.standardMinutes(20))) + // These elements are droppably late + .addElements(TimestampedValue.of(-1, instant), + TimestampedValue.of(-2, instant), + TimestampedValue.of(-3, instant)) + .advanceWatermarkToInfinity(); + + TestPipeline p = TestPipeline.create(); + PCollection windowed = p + .apply(source) + .apply(Window.into(FixedWindows.of(Duration.standardMinutes(5))).triggering( + AfterWatermark.pastEndOfWindow() + .withEarlyFirings(AfterProcessingTime.pastFirstElementInPane() + .plusDelayOf(Duration.standardMinutes(2))) + .withLateFirings(AfterPane.elementCountAtLeast(1))) + .accumulatingFiredPanes() + .withAllowedLateness(Duration.standardMinutes(5), ClosingBehavior.FIRE_ALWAYS)); + PCollection triggered = windowed.apply(WithKeys.of(1)) + .apply(GroupByKey.create()) + .apply(Values.>create()) + .apply(Flatten.iterables()); + PCollection count = windowed.apply(Count.globally().withoutDefaults()); + PCollection sum = windowed.apply(Sum.integersGlobally().withoutDefaults()); + + IntervalWindow window = new IntervalWindow(instant, instant.plus(Duration.standardMinutes(5L))); + PAssert.that(triggered) + .inFinalPane(window) + .containsInAnyOrder(1, 2, 3, 4, 5); + PAssert.that(triggered) + .inOnTimePane(window) + .containsInAnyOrder(1, 2, 3); + PAssert.that(count) + .inWindow(window) + .satisfies(new SerializableFunction, Void>() { + @Override + public Void apply(Iterable input) { + for (Long count : input) { + assertThat(count, allOf(greaterThanOrEqualTo(3L), lessThanOrEqualTo(5L))); + } + return null; + } + }); + PAssert.that(sum) + .inWindow(window) + .satisfies(new SerializableFunction, Void>() { + @Override + public Void apply(Iterable input) { + for (Integer sum : input) { + assertThat(sum, allOf(greaterThanOrEqualTo(6), lessThanOrEqualTo(15))); + } + return null; + } + }); + + p.run(); + } + + @Test + @Category(NeedsRunner.class) + public void testProcessingTimeTrigger() { + TestStream source = TestStream.create(VarLongCoder.of()) + .addElements(TimestampedValue.of(1L, new Instant(1000L)), + TimestampedValue.of(2L, new Instant(2000L))) + .advanceProcessingTime(Duration.standardMinutes(12)) + .addElements(TimestampedValue.of(3L, new Instant(3000L))) + .advanceProcessingTime(Duration.standardMinutes(6)) + .advanceWatermarkToInfinity(); + + TestPipeline p = TestPipeline.create(); + PCollection sum = p.apply(source) + .apply(Window.triggering(AfterWatermark.pastEndOfWindow() + .withEarlyFirings(AfterProcessingTime.pastFirstElementInPane() + .plusDelayOf(Duration.standardMinutes(5)))).accumulatingFiredPanes() + .withAllowedLateness(Duration.ZERO)) + .apply(Sum.longsGlobally()); + + PAssert.that(sum).inEarlyGlobalWindowPanes().containsInAnyOrder(3L, 6L); + + p.run(); + } + + @Test + public void testEncodeDecode() throws Exception { + TestStream.Event elems = + TestStream.ElementEvent.add( + TimestampedValue.of(1, new Instant()), + TimestampedValue.of(-10, new Instant()), + TimestampedValue.of(Integer.MAX_VALUE, new Instant())); + TestStream.Event wm = TestStream.WatermarkEvent.advanceTo(new Instant(100)); + TestStream.Event procTime = + TestStream.ProcessingTimeEvent.advanceBy(Duration.millis(90548)); + + TestStream.EventCoder coder = TestStream.EventCoder.of(VarIntCoder.of()); + + CoderProperties.coderSerializable(coder); + CoderProperties.coderDecodeEncodeEqual(coder, elems); + CoderProperties.coderDecodeEncodeEqual(coder, wm); + CoderProperties.coderDecodeEncodeEqual(coder, procTime); + } +} From a5ef9a9689147105854f12d3ea054b3887a94e24 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Mon, 15 Aug 2016 19:45:58 -0700 Subject: [PATCH 034/112] Implement TestStream in the DirectRunner --- runners/direct-java/pom.xml | 3 - .../beam/runners/direct/DirectOptions.java | 45 +--- .../beam/runners/direct/DirectRunner.java | 47 +++- .../runners/direct/EvaluationContext.java | 10 +- ...FixedThreadPoolExecutorServiceFactory.java | 45 ---- .../beam/runners/direct/NanosOffsetClock.java | 13 -- .../direct/TestStreamEvaluatorFactory.java | 204 ++++++++++++++++++ .../direct/TransformEvaluatorRegistry.java | 11 + .../direct/WriteWithShardingFactory.java | 2 +- .../runners/direct/EvaluationContextTest.java | 1 + .../apache/beam/sdk/testing/TestStream.java | 114 +++++++--- .../beam/sdk/testing/TestStreamTest.java | 159 ++++++++++++++ 12 files changed, 508 insertions(+), 146 deletions(-) delete mode 100644 runners/direct-java/src/main/java/org/apache/beam/runners/direct/FixedThreadPoolExecutorServiceFactory.java create mode 100644 runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java diff --git a/runners/direct-java/pom.xml b/runners/direct-java/pom.xml index 8b0f91d858a8..e06883f6e65d 100644 --- a/runners/direct-java/pom.xml +++ b/runners/direct-java/pom.xml @@ -85,9 +85,6 @@ org.apache.beam:beam-sdks-java-core org.apache.beam:beam-runners-java-core - - org/apache/beam/sdk/testing/TestStreamTest.java - [ diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectOptions.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectOptions.java index 3901c04cebac..798fda4c4e9d 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectOptions.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectOptions.java @@ -17,59 +17,16 @@ */ package org.apache.beam.runners.direct; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.options.ApplicationNameOptions; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.Description; -import org.apache.beam.sdk.options.Hidden; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.options.Validation.Required; -import org.apache.beam.sdk.transforms.PTransform; - -import com.fasterxml.jackson.annotation.JsonIgnore; - -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; /** * Options that can be used to configure the {@link org.apache.beam.runners.direct.DirectRunner}. */ public interface DirectOptions extends PipelineOptions, ApplicationNameOptions { - /** - * Gets the {@link ExecutorServiceFactory} to use to create instances of {@link ExecutorService} - * to execute {@link PTransform PTransforms}. - * - *

    Note that {@link ExecutorService ExecutorServices} returned by the factory must ensure that - * it cannot enter a state in which it will not schedule additional pending work unless currently - * scheduled work completes, as this may cause the {@link Pipeline} to cease processing. - * - *

    Defaults to a {@link FixedThreadPoolExecutorServiceFactory}, which produces instances of - * {@link Executors#newCachedThreadPool()}. - */ - @JsonIgnore - @Required - @Hidden - @Default.InstanceFactory(FixedThreadPoolExecutorServiceFactory.class) - ExecutorServiceFactory getExecutorServiceFactory(); - - void setExecutorServiceFactory(ExecutorServiceFactory executorService); - - /** - * Gets the {@link Clock} used by this pipeline. The clock is used in place of accessing the - * system time when time values are required by the evaluator. - */ - @Default.InstanceFactory(NanosOffsetClock.Factory.class) - @JsonIgnore - @Required - @Hidden - @Description( - "The processing time source used by the pipeline. When the current time is " - + "needed by the evaluator, the result of clock#now() is used.") - Clock getClock(); - - void setClock(Clock clock); - - @Default.Boolean(false) + @Default.Boolean(true) @Description( "If the pipeline should shut down producers which have reached the maximum " + "representable watermark. If this is set to true, a pipeline in which all PTransforms " 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 f2b781ec5e35..68184dec7d85 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 @@ -19,6 +19,7 @@ import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly; import org.apache.beam.runners.direct.DirectRunner.DirectPipelineResult; +import org.apache.beam.runners.direct.TestStreamEvaluatorFactory.DirectTestStreamFactory; import org.apache.beam.runners.direct.ViewEvaluatorFactory.ViewOverrideFactory; import org.apache.beam.sdk.AggregatorRetrievalException; import org.apache.beam.sdk.AggregatorValues; @@ -29,6 +30,7 @@ import org.apache.beam.sdk.io.Write; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.runners.PipelineRunner; +import org.apache.beam.sdk.testing.TestStream; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.GroupByKey; @@ -46,6 +48,7 @@ import org.apache.beam.sdk.values.PValue; import com.google.common.base.MoreObjects; +import com.google.common.base.Supplier; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; @@ -58,6 +61,7 @@ import java.util.HashMap; import java.util.Map; import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; /** * An In-Memory implementation of the Dataflow Programming Model. Supports Unbounded @@ -76,8 +80,9 @@ public class DirectRunner private static Map, PTransformOverrideFactory> defaultTransformOverrides = ImmutableMap., PTransformOverrideFactory>builder() - .put(GroupByKey.class, new DirectGroupByKeyOverrideFactory()) .put(CreatePCollectionView.class, new ViewOverrideFactory()) + .put(GroupByKey.class, new DirectGroupByKeyOverrideFactory()) + .put(TestStream.class, new DirectTestStreamFactory()) .put(Write.Bound.class, new WriteWithShardingFactory()) .build(); @@ -175,6 +180,8 @@ public static interface PCollectionViewWriter { //////////////////////////////////////////////////////////////////////////////////////////////// private final DirectOptions options; + private Supplier executorServiceSupplier = new FixedThreadPoolSupplier(); + private Supplier clockSupplier = new NanosOffsetClockSupplier(); public static DirectRunner fromOptions(PipelineOptions options) { return new DirectRunner(options.as(DirectOptions.class)); @@ -191,6 +198,14 @@ public DirectOptions getPipelineOptions() { return options; } + Supplier getClockSupplier() { + return clockSupplier; + } + + void setClockSupplier(Supplier supplier) { + this.clockSupplier = supplier; + } + @Override public OutputT apply( PTransform transform, InputT input) { @@ -223,6 +238,7 @@ public DirectPipelineResult run(Pipeline pipeline) { EvaluationContext context = EvaluationContext.create( getPipelineOptions(), + clockSupplier.get(), createBundleFactory(getPipelineOptions()), consumerTrackingVisitor.getRootTransforms(), consumerTrackingVisitor.getValueToConsumers(), @@ -230,14 +246,15 @@ public DirectPipelineResult run(Pipeline pipeline) { consumerTrackingVisitor.getViews()); // independent executor service for each run - ExecutorService executorService = - context.getPipelineOptions().getExecutorServiceFactory().create(); + ExecutorService executorService = executorServiceSupplier.get(); + + TransformEvaluatorRegistry registry = TransformEvaluatorRegistry.defaultRegistry(); PipelineExecutor executor = ExecutorServiceParallelExecutor.create( executorService, consumerTrackingVisitor.getValueToConsumers(), keyedPValueVisitor.getKeyedPValues(), - TransformEvaluatorRegistry.defaultRegistry(), + registry, defaultModelEnforcements(options), context); executor.start(consumerTrackingVisitor.getRootTransforms()); @@ -392,4 +409,26 @@ public State waitUntilFinish(Duration duration) throws IOException { "DirectPipelineResult does not support waitUntilFinish."); } } + + /** + * A {@link Supplier} that creates a {@link ExecutorService} based on + * {@link Executors#newFixedThreadPool(int)}. + */ + private static class FixedThreadPoolSupplier implements Supplier { + @Override + public ExecutorService get() { + return Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors()); + } + } + + + /** + * A {@link Supplier} that creates a {@link NanosOffsetClock}. + */ + private static class NanosOffsetClockSupplier implements Supplier { + @Override + public Clock get() { + return NanosOffsetClock.create(); + } + } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java index 94f28e2acb95..b9f159a2d598 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java @@ -102,24 +102,26 @@ class EvaluationContext { public static EvaluationContext create( DirectOptions options, + Clock clock, BundleFactory bundleFactory, Collection> rootTransforms, Map>> valueToConsumers, Map, String> stepNames, Collection> views) { return new EvaluationContext( - options, bundleFactory, rootTransforms, valueToConsumers, stepNames, views); + options, clock, bundleFactory, rootTransforms, valueToConsumers, stepNames, views); } private EvaluationContext( DirectOptions options, + Clock clock, BundleFactory bundleFactory, Collection> rootTransforms, Map>> valueToConsumers, Map, String> stepNames, Collection> views) { this.options = checkNotNull(options); - this.clock = options.getClock(); + this.clock = clock; this.bundleFactory = checkNotNull(bundleFactory); checkNotNull(rootTransforms); checkNotNull(valueToConsumers); @@ -433,4 +435,8 @@ public boolean isDone() { public Instant now() { return clock.now(); } + + Clock getClock() { + return clock; + } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/FixedThreadPoolExecutorServiceFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/FixedThreadPoolExecutorServiceFactory.java deleted file mode 100644 index 74c42926b115..000000000000 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/FixedThreadPoolExecutorServiceFactory.java +++ /dev/null @@ -1,45 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.direct; - -import org.apache.beam.sdk.options.DefaultValueFactory; -import org.apache.beam.sdk.options.PipelineOptions; - -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; - -/** - * A {@link ExecutorServiceFactory} that produces fixed thread pools via - * {@link Executors#newFixedThreadPool(int)}, with the number of threads equal to the available - * processors as provided by {@link Runtime#availableProcessors()}. - */ -class FixedThreadPoolExecutorServiceFactory - implements DefaultValueFactory, ExecutorServiceFactory { - private static final FixedThreadPoolExecutorServiceFactory INSTANCE = - new FixedThreadPoolExecutorServiceFactory(); - - @Override - public ExecutorServiceFactory create(PipelineOptions options) { - return INSTANCE; - } - - @Override - public ExecutorService create() { - return Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors()); - } -} diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/NanosOffsetClock.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/NanosOffsetClock.java index ffdee9d14af2..77fa1967c434 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/NanosOffsetClock.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/NanosOffsetClock.java @@ -17,9 +17,6 @@ */ package org.apache.beam.runners.direct; -import org.apache.beam.sdk.options.DefaultValueFactory; -import org.apache.beam.sdk.options.PipelineOptions; - import org.joda.time.Instant; import java.util.concurrent.TimeUnit; @@ -46,14 +43,4 @@ public Instant now() { baseMillis + (TimeUnit.MILLISECONDS.convert( System.nanoTime() - nanosAtBaseMillis, TimeUnit.NANOSECONDS))); } - - /** - * Creates instances of {@link NanosOffsetClock}. - */ - public static class Factory implements DefaultValueFactory { - @Override - public Clock create(PipelineOptions options) { - return new NanosOffsetClock(); - } - } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java new file mode 100644 index 000000000000..90a83b0b840f --- /dev/null +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java @@ -0,0 +1,204 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.runners.direct; + +import static com.google.common.base.Preconditions.checkState; + +import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; +import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.runners.PipelineRunner; +import org.apache.beam.sdk.testing.TestStream; +import org.apache.beam.sdk.testing.TestStream.ElementEvent; +import org.apache.beam.sdk.testing.TestStream.Event; +import org.apache.beam.sdk.testing.TestStream.EventType; +import org.apache.beam.sdk.testing.TestStream.ProcessingTimeEvent; +import org.apache.beam.sdk.testing.TestStream.WatermarkEvent; +import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.WindowingStrategy; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollection.IsBounded; +import org.apache.beam.sdk.values.PInput; +import org.apache.beam.sdk.values.POutput; +import org.apache.beam.sdk.values.TimestampedValue; + +import com.google.common.base.Supplier; + +import org.joda.time.Duration; +import org.joda.time.Instant; + +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; + +import javax.annotation.Nullable; + +/** + * The {@link TransformEvaluatorFactory} for the {@link TestStream} primitive. + */ +class TestStreamEvaluatorFactory implements TransformEvaluatorFactory { + private final AtomicBoolean inUse = new AtomicBoolean(false); + private final AtomicReference> evaluator = new AtomicReference<>(); + + @Nullable + @Override + public TransformEvaluator forApplication( + AppliedPTransform application, + @Nullable CommittedBundle inputBundle, + EvaluationContext evaluationContext) throws Exception { + return createEvaluator((AppliedPTransform) application, evaluationContext); + } + + @Override + public void cleanup() throws Exception {} + + private TransformEvaluator createEvaluator( + AppliedPTransform, TestStream> application, + EvaluationContext evaluationContext) { + if (evaluator.get() == null) { + Evaluator createdEvaluator = new Evaluator<>(application, evaluationContext, inUse); + evaluator.compareAndSet(null, createdEvaluator); + } + if (inUse.compareAndSet(false, true)) { + return evaluator.get(); + } else { + return null; + } + } + + private static class Evaluator implements TransformEvaluator { + private final AppliedPTransform, TestStream> application; + private final EvaluationContext context; + private final AtomicBoolean inUse; + private final List> events; + private int index; + private Instant currentWatermark; + + private Evaluator( + AppliedPTransform, TestStream> application, + EvaluationContext context, + AtomicBoolean inUse) { + this.application = application; + this.context = context; + this.inUse = inUse; + this.events = application.getTransform().getEvents(); + index = 0; + currentWatermark = BoundedWindow.TIMESTAMP_MIN_VALUE; + } + + @Override + public void processElement(WindowedValue element) throws Exception { + } + + @Override + public TransformResult finishBundle() throws Exception { + if (index >= events.size()) { + return StepTransformResult.withHold(application, BoundedWindow.TIMESTAMP_MAX_VALUE).build(); + } + Event event = events.get(index); + if (event.getType().equals(EventType.WATERMARK)) { + currentWatermark = ((WatermarkEvent) event).getWatermark(); + } + StepTransformResult.Builder result = + StepTransformResult.withHold(application, currentWatermark); + if (event.getType().equals(EventType.ELEMENT)) { + UncommittedBundle bundle = context.createRootBundle(application.getOutput()); + for (TimestampedValue elem : ((ElementEvent) event).getElements()) { + bundle.add(WindowedValue.timestampedValueInGlobalWindow(elem.getValue(), + elem.getTimestamp())); + } + result.addOutput(bundle); + } + if (event.getType().equals(EventType.PROCESSING_TIME)) { + ((TestClock) context.getClock()) + .advance(((ProcessingTimeEvent) event).getProcessingTimeAdvance()); + } + index++; + checkState(inUse.compareAndSet(true, false), + "The InUse flag of a %s was changed while the source evaluator was executing. " + + "%s cannot be split or evaluated in parallel.", + TestStream.class.getSimpleName(), + TestStream.class.getSimpleName()); + return result.build(); + } + } + + private static class TestClock implements Clock { + private final AtomicReference currentTime = + new AtomicReference<>(BoundedWindow.TIMESTAMP_MIN_VALUE); + + public void advance(Duration amount) { + Instant now = currentTime.get(); + currentTime.compareAndSet(now, now.plus(amount)); + } + + @Override + public Instant now() { + return currentTime.get(); + } + } + + private static class TestClockSupplier implements Supplier { + @Override + public Clock get() { + return new TestClock(); + } + } + + static class DirectTestStreamFactory implements PTransformOverrideFactory { + @Override + public PTransform override( + PTransform transform) { + if (transform instanceof TestStream) { + return (PTransform) + new DirectTestStream((TestStream) transform); + } + return transform; + } + + private static class DirectTestStream extends PTransform> { + private final TestStream original; + + private DirectTestStream(TestStream transform) { + this.original = transform; + } + + @Override + public PCollection apply(PBegin input) { + setup(input.getPipeline()); + return PCollection.createPrimitiveOutputInternal( + input.getPipeline(), WindowingStrategy.globalDefault(), IsBounded.UNBOUNDED) + .setCoder(original.getValueCoder()); + } + + private void setup(Pipeline p) { + PipelineRunner runner = p.getRunner(); + checkState(runner instanceof DirectRunner, + "%s can only be used when running with the %s", + getClass().getSimpleName(), + DirectRunner.class.getSimpleName()); + ((DirectRunner) runner).setClockSupplier(new TestClockSupplier()); + } + } + } +} diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java index b46923775c48..c35e8b14c84a 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java @@ -23,6 +23,7 @@ import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.sdk.io.Read; +import org.apache.beam.sdk.testing.TestStream; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Flatten.FlattenPCollectionList; import org.apache.beam.sdk.transforms.PTransform; @@ -61,6 +62,7 @@ public static TransformEvaluatorRegistry defaultRegistry() { // Runner-specific primitives used in expansion of GroupByKey .put(DirectGroupByKeyOnly.class, new GroupByKeyOnlyEvaluatorFactory()) .put(DirectGroupAlsoByWindow.class, new GroupAlsoByWindowEvaluatorFactory()) + .put(TestStream.class, new TestStreamEvaluatorFactory()) .build(); return new TransformEvaluatorRegistry(primitives); } @@ -117,4 +119,13 @@ public void cleanup() throws Exception { throw toThrow; } } + + /** + * A factory to create Transform Evaluator Registries. + */ + public static class Factory { + public TransformEvaluatorRegistry create() { + return TransformEvaluatorRegistry.defaultRegistry(); + } + } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java index c2157b8634ce..1ab3403f45bc 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java @@ -66,7 +66,7 @@ public PTransform extends PTransform, PDone> { + private static class DynamicallyReshardedWrite extends PTransform, PDone> { private final transient Write.Bound original; private DynamicallyReshardedWrite(Bound original) { diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java index d4b577349166..7ac0caadc55b 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java @@ -117,6 +117,7 @@ public void setup() { context = EvaluationContext.create( runner.getPipelineOptions(), + NanosOffsetClock.create(), ImmutableListBundleFactory.create(), rootTransforms, valueToConsumers, diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestStream.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestStream.java index 6d11f72bb979..e2eda3222855 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestStream.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestStream.java @@ -32,10 +32,8 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.PropertyNames; import org.apache.beam.sdk.util.VarInt; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TimestampedValue.TimestampedValueCoder; @@ -83,30 +81,30 @@ private TestStream(Coder coder, List> events) { this.events = checkNotNull(events); } - public Coder> getEventCoder() { - return EventCoder.of(coder); - } - /** * An incomplete {@link TestStream}. Elements added to this builder will be produced in sequence * when the pipeline created by the {@link TestStream} is run. */ public static class Builder { private final Coder coder; - private final ImmutableList.Builder> events; - private Instant currentWatermark; + private final ImmutableList> events; + private final Instant currentWatermark; private Builder(Coder coder) { - this.coder = coder; - events = ImmutableList.builder(); + this(coder, ImmutableList.>of(), BoundedWindow.TIMESTAMP_MIN_VALUE); + } - currentWatermark = BoundedWindow.TIMESTAMP_MIN_VALUE; + private Builder(Coder coder, ImmutableList> events, Instant currentWatermark) { + this.coder = coder; + this.events = events; + this.currentWatermark = currentWatermark; } /** * Adds the specified elements to the source with timestamp equal to the current watermark. * - * @return this {@link TestStream.Builder} + * @return A {@link TestStream.Builder} like this one that will add the provided elements + * after all earlier events have completed. */ @SafeVarargs public final Builder addElements(T element, T... elements) { @@ -122,22 +120,40 @@ public final Builder addElements(T element, T... elements) { /** * Adds the specified elements to the source with the provided timestamps. * - * @return this {@link TestStream.Builder} + * @return A {@link TestStream.Builder} like this one that will add the provided elements + * after all earlier events have completed. */ @SafeVarargs public final Builder addElements( TimestampedValue element, TimestampedValue... elements) { - events.add(ElementEvent.add(element, elements)); - return this; + checkArgument( + element.getTimestamp().isBefore(BoundedWindow.TIMESTAMP_MAX_VALUE), + "Elements must have timestamps before %s. Got: %s", + BoundedWindow.TIMESTAMP_MAX_VALUE, + element.getTimestamp()); + for (TimestampedValue multiElement : elements) { + checkArgument( + multiElement.getTimestamp().isBefore(BoundedWindow.TIMESTAMP_MAX_VALUE), + "Elements must have timestamps before %s. Got: %s", + BoundedWindow.TIMESTAMP_MAX_VALUE, + multiElement.getTimestamp()); + } + ImmutableList> newEvents = + ImmutableList.>builder() + .addAll(events) + .add(ElementEvent.add(element, elements)) + .build(); + return new Builder(coder, newEvents, currentWatermark); } /** * Advance the watermark of this source to the specified instant. * - *

    The watermark must advance monotonically and to at most {@link - * BoundedWindow#TIMESTAMP_MAX_VALUE}. + *

    The watermark must advance monotonically and cannot advance to {@link + * BoundedWindow#TIMESTAMP_MAX_VALUE} or beyond. * - * @return this {@link TestStream.Builder} + * @return A {@link TestStream.Builder} like this one that will advance the watermark to the + * specified point after all earlier events have completed. */ public Builder advanceWatermarkTo(Instant newWatermark) { checkArgument( @@ -147,23 +163,30 @@ public Builder advanceWatermarkTo(Instant newWatermark) { "The Watermark cannot progress beyond the maximum. Got: %s. Maximum: %s", newWatermark, BoundedWindow.TIMESTAMP_MAX_VALUE); - events.add(WatermarkEvent.advanceTo(newWatermark)); - currentWatermark = newWatermark; - return this; + ImmutableList> newEvents = ImmutableList.>builder() + .addAll(events) + .add(WatermarkEvent.advanceTo(newWatermark)) + .build(); + return new Builder(coder, newEvents, newWatermark); } /** * Advance the processing time by the specified amount. * - * @return this {@link TestStream.Builder} + * @return A {@link TestStream.Builder} like this one that will advance the processing time by + * the specified amount after all earlier events have completed. */ public Builder advanceProcessingTime(Duration amount) { checkArgument( amount.getMillis() > 0, "Must advance the processing time by a positive amount. Got: ", amount); - events.add(ProcessingTimeEvent.advanceBy(amount)); - return this; + ImmutableList> newEvents = + ImmutableList.>builder() + .addAll(events) + .add(ProcessingTimeEvent.advanceBy(amount)) + .build(); + return new Builder(coder, newEvents, currentWatermark); } /** @@ -171,8 +194,12 @@ public Builder advanceProcessingTime(Duration amount) { * same builder will not affect the returned {@link TestStream}. */ public TestStream advanceWatermarkToInfinity() { - events.add(WatermarkEvent.advanceTo(BoundedWindow.TIMESTAMP_MAX_VALUE)); - return new TestStream<>(coder, events.build()); + ImmutableList> newEvents = + ImmutableList.>builder() + .addAll(events) + .add(WatermarkEvent.advanceTo(BoundedWindow.TIMESTAMP_MAX_VALUE)) + .build(); + return new TestStream<>(coder, newEvents); } } @@ -230,12 +257,30 @@ static Event advanceBy(Duration amount) { @Override public PCollection apply(PBegin input) { - return PCollection.createPrimitiveOutputInternal( - input.getPipeline(), WindowingStrategy.globalDefault(), IsBounded.UNBOUNDED) - .setCoder(coder); + throw new IllegalStateException( + String.format( + "Pipeline Runner %s does not provide a required override for %s", + input.getPipeline().getRunner().getClass().getSimpleName(), + getClass().getSimpleName())); + } + + public Coder getValueCoder() { + return coder; + } + + /** + * Returns a coder suitable for encoding {@link TestStream.Event}. + */ + public Coder> getEventCoder() { + return EventCoder.of(coder); } - public List> getStreamEvents() { + /** + * Returns the sequence of {@link Event Events} in this {@link TestStream}. + * + *

    For use by {@link PipelineRunner} authors. + */ + public List> getEvents() { return events; } @@ -243,7 +288,7 @@ public List> getStreamEvents() { * A {@link Coder} that encodes and decodes {@link TestStream.Event Events}. * * @param the type of elements in {@link ElementEvent ElementEvents} encoded and decoded by - * this {@link EventCoder} + * this {@link EventCoder} */ @VisibleForTesting static final class EventCoder extends StandardCoder> { @@ -290,14 +335,15 @@ public void encode( DURATION_CODER.encode(processingAdvance, outStream, context); break; default: - throw new AssertionError("Unreachable"); + throw new AssertionError("Unreachable: Unsupported Event Type " + value.getType()); } } @Override public Event decode( InputStream inStream, Context context) throws IOException { - switch (EventType.values()[VarInt.decodeInt(inStream)]) { + EventType eventType = EventType.values()[VarInt.decodeInt(inStream)]; + switch (eventType) { case ELEMENT: Iterable> elements = elementCoder.decode(inStream, context); return ElementEvent.add(elements); @@ -307,7 +353,7 @@ public Event decode( return ProcessingTimeEvent.advanceBy( DURATION_CODER.decode(inStream, context).toDuration()); default: - throw new AssertionError("Unreachable"); + throw new AssertionError("Unreachable: Unsupported Event Type " + eventType); } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java index 09bccfa17126..df37d7f967a5 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java @@ -23,8 +23,13 @@ import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.junit.Assert.assertThat; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.testing.TestStream.Builder; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.transforms.GroupByKey; @@ -35,8 +40,12 @@ import org.apache.beam.sdk.transforms.windowing.AfterPane; import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime; import org.apache.beam.sdk.transforms.windowing.AfterWatermark; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.DefaultTrigger; import org.apache.beam.sdk.transforms.windowing.FixedWindows; +import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; +import org.apache.beam.sdk.transforms.windowing.Never; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior; import org.apache.beam.sdk.values.PCollection; @@ -44,8 +53,10 @@ import org.joda.time.Duration; import org.joda.time.Instant; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -56,6 +67,8 @@ */ @RunWith(JUnit4.class) public class TestStreamTest implements Serializable { + @Rule public transient ExpectedException thrown = ExpectedException.none(); + @Test @Category(NeedsRunner.class) public void testLateDataAccumulating() { @@ -148,6 +161,152 @@ public void testProcessingTimeTrigger() { p.run(); } + @Test + @Category(NeedsRunner.class) + public void testDiscardingMode() { + TestStream stream = + TestStream.create(StringUtf8Coder.of()) + .advanceWatermarkTo(new Instant(0)) + .addElements( + TimestampedValue.of("firstPane", new Instant(100)), + TimestampedValue.of("alsoFirstPane", new Instant(200))) + .addElements(TimestampedValue.of("onTimePane", new Instant(500))) + .advanceWatermarkTo(new Instant(1001L)) + .addElements( + TimestampedValue.of("finalLatePane", new Instant(750)), + TimestampedValue.of("alsoFinalLatePane", new Instant(250))) + .advanceWatermarkToInfinity(); + + TestPipeline p = TestPipeline.create(); + FixedWindows windowFn = FixedWindows.of(Duration.millis(1000L)); + Duration allowedLateness = Duration.millis(5000L); + PCollection values = + p.apply(stream) + .apply( + Window.into(windowFn) + .triggering( + AfterWatermark.pastEndOfWindow() + .withEarlyFirings(AfterPane.elementCountAtLeast(2)) + .withLateFirings(Never.ever())) + .discardingFiredPanes() + .withAllowedLateness(allowedLateness)) + .apply(WithKeys.of(1)) + .apply(GroupByKey.create()) + .apply(Values.>create()) + .apply(Flatten.iterables()); + + IntervalWindow window = windowFn.assignWindow(new Instant(100)); + PAssert.that(values) + .inWindow(window) + .containsInAnyOrder( + "firstPane", "alsoFirstPane", "onTimePane", "finalLatePane", "alsoFinalLatePane"); + PAssert.that(values) + .inCombinedNonLatePanes(window) + .containsInAnyOrder("firstPane", "alsoFirstPane", "onTimePane"); + PAssert.that(values).inOnTimePane(window).containsInAnyOrder("onTimePane"); + PAssert.that(values) + .inFinalPane(window) + .containsInAnyOrder("finalLatePane", "alsoFinalLatePane"); + + p.run(); + } + + @Test + @Category(NeedsRunner.class) + public void testFirstElementLate() { + Instant lateElementTimestamp = new Instant(-1_000_000); + TestStream stream = + TestStream.create(StringUtf8Coder.of()) + .advanceWatermarkTo(new Instant(0)) + .addElements(TimestampedValue.of("late", lateElementTimestamp)) + .addElements(TimestampedValue.of("onTime", new Instant(100))) + .advanceWatermarkToInfinity(); + + TestPipeline p = TestPipeline.create(); + FixedWindows windowFn = FixedWindows.of(Duration.millis(1000L)); + Duration allowedLateness = Duration.millis(5000L); + PCollection values = p.apply(stream) + .apply(Window.into(windowFn).triggering(DefaultTrigger.of()) + .discardingFiredPanes() + .withAllowedLateness(allowedLateness)) + .apply(WithKeys.of(1)) + .apply(GroupByKey.create()) + .apply(Values.>create()) + .apply(Flatten.iterables()); + + PAssert.that(values).inWindow(windowFn.assignWindow(lateElementTimestamp)).empty(); + PAssert.that(values) + .inWindow(windowFn.assignWindow(new Instant(100))) + .containsInAnyOrder("onTime"); + + p.run(); + } + + @Test + @Category(NeedsRunner.class) + public void testElementsAtAlmostPositiveInfinity() { + Instant endOfGlobalWindow = GlobalWindow.INSTANCE.maxTimestamp(); + TestStream stream = TestStream.create(StringUtf8Coder.of()) + .addElements(TimestampedValue.of("foo", endOfGlobalWindow), + TimestampedValue.of("bar", endOfGlobalWindow)) + .advanceWatermarkToInfinity(); + + TestPipeline p = TestPipeline.create(); + FixedWindows windows = FixedWindows.of(Duration.standardHours(6)); + PCollection windowedValues = p.apply(stream) + .apply(Window.into(windows)) + .apply(WithKeys.of(1)) + .apply(GroupByKey.create()) + .apply(Values.>create()) + .apply(Flatten.iterables()); + + PAssert.that(windowedValues) + .inWindow(windows.assignWindow(GlobalWindow.INSTANCE.maxTimestamp())) + .containsInAnyOrder("foo", "bar"); + p.run(); + } + + @Test + public void testElementAtPositiveInfinityThrows() { + Builder stream = + TestStream.create(VarIntCoder.of()) + .addElements(TimestampedValue.of(-1, BoundedWindow.TIMESTAMP_MAX_VALUE.minus(1L))); + thrown.expect(IllegalArgumentException.class); + stream.addElements(TimestampedValue.of(1, BoundedWindow.TIMESTAMP_MAX_VALUE)); + } + + @Test + public void testAdvanceWatermarkNonMonotonicThrows() { + Builder stream = + TestStream.create(VarIntCoder.of()) + .advanceWatermarkTo(new Instant(0L)); + thrown.expect(IllegalArgumentException.class); + stream.advanceWatermarkTo(new Instant(-1L)); + } + + @Test + public void testAdvanceWatermarkEqualToPositiveInfinityThrows() { + Builder stream = + TestStream.create(VarIntCoder.of()) + .advanceWatermarkTo(BoundedWindow.TIMESTAMP_MAX_VALUE.minus(1L)); + thrown.expect(IllegalArgumentException.class); + stream.advanceWatermarkTo(BoundedWindow.TIMESTAMP_MAX_VALUE); + } + + @Test + public void testUnsupportedRunnerThrows() { + PipelineOptions opts = PipelineOptionsFactory.create(); + opts.setRunner(CrashingRunner.class); + + Pipeline p = Pipeline.create(opts); + + thrown.expect(IllegalStateException.class); + thrown.expectMessage("does not provide a required override"); + thrown.expectMessage(TestStream.class.getSimpleName()); + thrown.expectMessage(CrashingRunner.class.getSimpleName()); + p.apply(TestStream.create(VarIntCoder.of()).advanceWatermarkToInfinity()); + } + @Test public void testEncodeDecode() throws Exception { TestStream.Event elems = From ac7e6f202812cd54ba72dd332c5bf7c2ace4c989 Mon Sep 17 00:00:00 2001 From: sammcveety Date: Thu, 18 Aug 2016 18:02:06 -0400 Subject: [PATCH 035/112] Incorporate private IP option for Dataflow runner --- pom.xml | 2 +- .../dataflow/DataflowPipelineTranslator.java | 7 +++++++ .../options/DataflowPipelineWorkerPoolOptions.java | 14 ++++++++++++++ 3 files changed, 22 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index f39f94aca693..58adbe7a86cc 100644 --- a/pom.xml +++ b/pom.xml @@ -105,7 +105,7 @@ v2-rev295-1.22.0 0.0.2 v2-rev8-1.22.0 - v1b3-rev30-1.22.0 + v1b3-rev36-1.22.0 0.5.160222 1.0.0-beta.2 1.0.0-beta diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java index 0ca92c849f43..b8d94452872c 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java @@ -425,6 +425,13 @@ public Job translate(List packages) { workerPool.setMachineType(options.getWorkerMachineType()); } + if (options.getUsePublicIps() != null) { + if (options.getUsePublicIps()) { + workerPool.setIpConfiguration("WORKER_IP_PUBLIC"); + } else { + workerPool.setIpConfiguration("WORKER_IP_PRIVATE"); + } + } workerPool.setPackages(packages); workerPool.setNumWorkers(options.getNumWorkers()); diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java index e3a1a0f5bdef..9b7a9ceb7f74 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java @@ -29,6 +29,8 @@ import java.util.List; +import javax.annotation.Nullable; + /** * Options that are used to configure the Dataflow pipeline worker pool. */ @@ -260,4 +262,16 @@ public String getTeardownPolicyName() { + "https://cloud.google.com/compute/docs/reference/latest/diskTypes") String getWorkerDiskType(); void setWorkerDiskType(String value); + + /** + * Specifies whether worker pools should be started with public IP addresses. + * + *

    WARNING: This feature is experimental. You must be whitelisted to use it. + */ + @Description("Specifies whether worker pools should be started with public IP addresses. WARNING:" + + "This feature is experimental. You must be whitelisted to use it.") + @Experimental + @JsonIgnore + @Nullable Boolean getUsePublicIps(); + void setUsePublicIps(@Nullable Boolean value); } From 953306584073044c41bcfdc4ea5e14870ddea5e4 Mon Sep 17 00:00:00 2001 From: Vikas Kedigehalli Date: Wed, 17 Aug 2016 18:19:52 -0700 Subject: [PATCH 036/112] Datastore Sink support for writing Mutations This generalizes Write to Write and Delete cleanly. --- .../beam/sdk/io/gcp/datastore/V1Beta3.java | 436 ++++++++++++------ .../sdk/io/gcp/datastore/V1Beta3Test.java | 237 ++++++++-- 2 files changed, 508 insertions(+), 165 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3.java index 0d2e2cb72cab..8503b668e25a 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3.java @@ -24,6 +24,7 @@ import static com.google.datastore.v1beta3.PropertyFilter.Operator.EQUAL; import static com.google.datastore.v1beta3.PropertyOrder.Direction.DESCENDING; import static com.google.datastore.v1beta3.QueryResultBatch.MoreResultsType.NOT_FINISHED; +import static com.google.datastore.v1beta3.client.DatastoreHelper.makeDelete; import static com.google.datastore.v1beta3.client.DatastoreHelper.makeFilter; import static com.google.datastore.v1beta3.client.DatastoreHelper.makeOrder; import static com.google.datastore.v1beta3.client.DatastoreHelper.makeUpsert; @@ -36,8 +37,10 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.transforms.GroupByKey; +import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.SimpleFunction; import org.apache.beam.sdk.transforms.Values; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.display.DisplayData.Builder; @@ -60,6 +63,7 @@ import com.google.datastore.v1beta3.EntityResult; import com.google.datastore.v1beta3.Key; import com.google.datastore.v1beta3.Key.PathElement; +import com.google.datastore.v1beta3.Mutation; import com.google.datastore.v1beta3.PartitionId; import com.google.datastore.v1beta3.Query; import com.google.datastore.v1beta3.QueryResultBatch; @@ -84,7 +88,7 @@ import javax.annotation.Nullable; /** - *

    {@link V1Beta3} provides an API to Read and Write {@link PCollection PCollections} of + *

    {@link V1Beta3} provides an API to Read, Write and Delete {@link PCollection PCollections} of * Google Cloud Datastore version v1beta3 * {@link Entity} objects. * @@ -129,7 +133,25 @@ * p.run(); * } * - *

    {@link Entity Entities} in the {@code PCollection} to be written must have complete + *

    To delete a {@link PCollection} of {@link Entity Entities} from Datastore, use + * {@link V1Beta3#deleteEntity()}, specifying the Cloud Datastore project to write to: + * + *

     {@code
    + * PCollection entities = ...;
    + * entities.apply(DatastoreIO.v1beta3().deleteEntity().withProjectId(projectId));
    + * p.run();
    + * } 
    + * + *

    To delete entities associated with a {@link PCollection} of {@link Key Keys} from Datastore, + * use {@link V1Beta3#deleteKey}, specifying the Cloud Datastore project to write to: + * + *

     {@code
    + * PCollection entities = ...;
    + * entities.apply(DatastoreIO.v1beta3().deleteKey().withProjectId(projectId));
    + * p.run();
    + * } 
    + * + *

    {@link Entity Entities} in the {@code PCollection} to be written or deleted must have complete * {@link Key Keys}. Complete {@code Keys} specify the {@code name} and {@code id} of the * {@code Entity}, where incomplete {@code Keys} do not. A {@code namespace} other than * {@code projectId} default may be used by specifying it in the {@code Entity} {@code Keys}. @@ -139,9 +161,9 @@ * keyBuilder.getPartitionIdBuilder().setNamespace(namespace); * } * - *

    {@code Entities} will be committed as upsert (update or insert) mutations. Please read - * Entities, Properties, and - * Keys for more information about {@code Entity} keys. + *

    {@code Entities} will be committed as upsert (update or insert) or delete mutations. Please + * read Entities, Properties, + * and Keys for more information about {@code Entity} keys. * *

    Permissions

    * Permission requirements depend on the {@code PipelineRunner} that is used to execute the @@ -641,21 +663,34 @@ public Write write() { return new Write(null); } + /** + * Returns an empty {@link DeleteEntity} builder. Configure the destination + * {@code projectId} using {@link DeleteEntity#withProjectId}. + */ + public DeleteEntity deleteEntity() { + return new DeleteEntity(null); + } + + /** + * Returns an empty {@link DeleteKey} builder. Configure the destination + * {@code projectId} using {@link DeleteKey#withProjectId}. + */ + public DeleteKey deleteKey() { + return new DeleteKey(null); + } + /** * A {@link PTransform} that writes {@link Entity} objects to Cloud Datastore. * * @see DatastoreIO */ - public static class Write extends PTransform, PDone> { - @Nullable - private final String projectId; - + public static class Write extends Mutate { /** * Note that {@code projectId} is only {@code @Nullable} as a matter of build order, but if * it is {@code null} at instantiation time, an error will be thrown. */ - public Write(@Nullable String projectId) { - this.projectId = projectId; + Write(@Nullable String projectId) { + super(projectId, new UpsertFn()); } /** @@ -665,27 +700,99 @@ public Write withProjectId(String projectId) { checkNotNull(projectId, "projectId"); return new Write(projectId); } + } - @Override - public PDone apply(PCollection input) { - input.apply(ParDo.of(new DatastoreWriterFn(projectId))); - return PDone.in(input.getPipeline()); + /** + * A {@link PTransform} that deletes {@link Entity Entities} from Cloud Datastore. + * + * @see DatastoreIO + */ + public static class DeleteEntity extends Mutate { + /** + * Note that {@code projectId} is only {@code @Nullable} as a matter of build order, but if + * it is {@code null} at instantiation time, an error will be thrown. + */ + DeleteEntity(@Nullable String projectId) { + super(projectId, new DeleteEntityFn()); } - @Override - public void validate(PCollection input) { + /** + * Returns a new {@link DeleteEntity} that deletes entities from the Cloud Datastore for the + * specified project. + */ + public DeleteEntity withProjectId(String projectId) { + checkNotNull(projectId, "projectId"); + return new DeleteEntity(projectId); + } + } + + /** + * A {@link PTransform} that deletes {@link Entity Entities} associated with the given + * {@link Key Keys} from Cloud Datastore. + * + * @see DatastoreIO + */ + public static class DeleteKey extends Mutate { + /** + * Note that {@code projectId} is only {@code @Nullable} as a matter of build order, but if + * it is {@code null} at instantiation time, an error will be thrown. + */ + DeleteKey(@Nullable String projectId) { + super(projectId, new DeleteKeyFn()); + } + + /** + * Returns a new {@link DeleteKey} that deletes entities from the Cloud Datastore for the + * specified project. + */ + public DeleteKey withProjectId(String projectId) { checkNotNull(projectId, "projectId"); + return new DeleteKey(projectId); } + } + /** + * A {@link PTransform} that writes mutations to Cloud Datastore. + * + *

    It requires a {@link DoFn} that tranforms an object of type {@code T} to a {@link Mutation}. + * {@code T} is usually either an {@link Entity} or a {@link Key} + * Note: Only idempotent Cloud Datastore mutation operations (upsert and delete) should + * be used by the {@code DoFn} provided, as the commits are retried when failures occur. + */ + private abstract static class Mutate extends PTransform, PDone> { @Nullable - public String getProjectId() { - return projectId; + private final String projectId; + /** A function that transforms each {@code T} into a mutation. */ + private final SimpleFunction mutationFn; + + /** + * Note that {@code projectId} is only {@code @Nullable} as a matter of build order, but if + * it is {@code null} at instantiation time, an error will be thrown. + */ + Mutate(@Nullable String projectId, SimpleFunction mutationFn) { + this.projectId = projectId; + this.mutationFn = checkNotNull(mutationFn); + } + + @Override + public PDone apply(PCollection input) { + input.apply("Convert to Mutation", MapElements.via(mutationFn)) + .apply("Write Mutation to Datastore", ParDo.of(new DatastoreWriterFn(projectId))); + + return PDone.in(input.getPipeline()); + } + + @Override + public void validate(PCollection input) { + checkNotNull(projectId, "projectId"); + checkNotNull(mutationFn, "mutationFn"); } @Override public String toString() { return MoreObjects.toStringHelper(getClass()) .add("projectId", projectId) + .add("mutationFn", mutationFn.getClass().getName()) .toString(); } @@ -694,141 +801,200 @@ public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder .addIfNotNull(DisplayData.item("projectId", projectId) - .withLabel("Output Project")); + .withLabel("Output Project")) + .include(mutationFn); } + public String getProjectId() { + return projectId; + } + } + + /** + * {@link DoFn} that writes {@link Mutation}s to Cloud Datastore. Mutations are written in + * batches, where the maximum batch size is {@link V1Beta3#DATASTORE_BATCH_UPDATE_LIMIT}. + * + *

    See + * Datastore: Entities, Properties, and Keys for information about entity keys and mutations. + * + *

    Commits are non-transactional. If a commit fails because of a conflict over an entity + * group, the commit will be retried (up to {@link V1Beta3#DATASTORE_BATCH_UPDATE_LIMIT} + * times). This means that the mutation operation should be idempotent. Thus, the writer should + * only be used for {code upsert} and {@code delete} mutation operations, as these are the only + * two Cloud Datastore mutations that are idempotent. + */ + @VisibleForTesting + static class DatastoreWriterFn extends DoFn { + private static final Logger LOG = LoggerFactory.getLogger(DatastoreWriterFn.class); + private final String projectId; + private transient Datastore datastore; + private final V1Beta3DatastoreFactory datastoreFactory; + // Current batch of mutations to be written. + private final List mutations = new ArrayList<>(); + /** + * Since a bundle is written in batches, we should retry the commit of a batch in order to + * prevent transient errors from causing the bundle to fail. + */ + private static final int MAX_RETRIES = 5; + /** - * A {@link DoFn} that writes {@link Entity} objects to Cloud Datastore. Entities are written in - * batches, where the maximum batch size is {@link V1Beta3#DATASTORE_BATCH_UPDATE_LIMIT}. - * Entities are committed as upsert mutations (either update if the key already exists, or - * insert if it is a new key). If an entity does not have a complete key (i.e., it has no name - * or id), the bundle will fail. - * - *

    See - * Datastore: Entities, Properties, and Keys for information about entity keys and entities. - * - *

    Commits are non-transactional. If a commit fails because of a conflict over an entity - * group, the commit will be retried (up to {@link V1Beta3#DATASTORE_BATCH_UPDATE_LIMIT} - * times). + * Initial backoff time for exponential backoff for retry attempts. */ - @VisibleForTesting - static class DatastoreWriterFn extends DoFn { - private static final Logger LOG = LoggerFactory.getLogger(DatastoreWriterFn.class); - private final String projectId; - private transient Datastore datastore; - private final V1Beta3DatastoreFactory datastoreFactory; - // Current batch of entities to be written. - private final List entities = new ArrayList<>(); - /** - * Since a bundle is written in batches, we should retry the commit of a batch in order to - * prevent transient errors from causing the bundle to fail. - */ - private static final int MAX_RETRIES = 5; + private static final int INITIAL_BACKOFF_MILLIS = 5000; - /** - * Initial backoff time for exponential backoff for retry attempts. - */ - private static final int INITIAL_BACKOFF_MILLIS = 5000; + DatastoreWriterFn(String projectId) { + this(projectId, new V1Beta3DatastoreFactory()); + } - public DatastoreWriterFn(String projectId) { - this(projectId, new V1Beta3DatastoreFactory()); - } + @VisibleForTesting + DatastoreWriterFn(String projectId, V1Beta3DatastoreFactory datastoreFactory) { + this.projectId = checkNotNull(projectId, "projectId"); + this.datastoreFactory = datastoreFactory; + } - @VisibleForTesting - DatastoreWriterFn(String projectId, V1Beta3DatastoreFactory datastoreFactory) { - this.projectId = checkNotNull(projectId, "projectId"); - this.datastoreFactory = datastoreFactory; - } + @StartBundle + public void startBundle(Context c) { + datastore = datastoreFactory.getDatastore(c.getPipelineOptions(), projectId); + } - @StartBundle - public void startBundle(Context c) { - datastore = datastoreFactory.getDatastore(c.getPipelineOptions(), projectId); + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + mutations.add(c.element()); + if (mutations.size() >= V1Beta3.DATASTORE_BATCH_UPDATE_LIMIT) { + flushBatch(); } + } - @ProcessElement - public void processElement(ProcessContext c) throws Exception { - // Verify that the entity to write has a complete key. - if (!isValidKey(c.element().getKey())) { - throw new IllegalArgumentException( - "Entities to be written to the Datastore must have complete keys"); - } - - entities.add(c.element()); - if (entities.size() >= V1Beta3.DATASTORE_BATCH_UPDATE_LIMIT) { - flushBatch(); - } + @FinishBundle + public void finishBundle(Context c) throws Exception { + if (mutations.size() > 0) { + flushBatch(); } + } - @FinishBundle - public void finishBundle(Context c) throws Exception { - if (entities.size() > 0) { - flushBatch(); - } - } + /** + * Writes a batch of mutations to Cloud Datastore. + * + *

    If a commit fails, it will be retried (up to {@link DatastoreWriterFn#MAX_RETRIES} + * times). All mutations in the batch will be committed again, even if the commit was partially + * successful. If the retry limit is exceeded, the last exception from the Datastore will be + * thrown. + * + * @throws DatastoreException if the commit fails or IOException or InterruptedException if + * backing off between retries fails. + */ + private void flushBatch() throws DatastoreException, IOException, InterruptedException { + LOG.debug("Writing batch of {} mutations", mutations.size()); + Sleeper sleeper = Sleeper.DEFAULT; + BackOff backoff = new AttemptBoundedExponentialBackOff(MAX_RETRIES, INITIAL_BACKOFF_MILLIS); - /** - * Writes a batch of entities to Cloud Datastore. - * - *

    If a commit fails, it will be retried (up to {@link DatastoreWriterFn#MAX_RETRIES} - * times). All entities in the batch will be committed again, even if the commit was partially - * successful. If the retry limit is exceeded, the last exception from the Datastore will be - * thrown. - * - * @throws DatastoreException if the commit fails or IOException or InterruptedException if - * backing off between retries fails. - */ - private void flushBatch() throws DatastoreException, IOException, InterruptedException { - LOG.debug("Writing batch of {} entities", entities.size()); - Sleeper sleeper = Sleeper.DEFAULT; - BackOff backoff = new AttemptBoundedExponentialBackOff(MAX_RETRIES, INITIAL_BACKOFF_MILLIS); - - while (true) { - // Batch upsert entities. - try { - CommitRequest.Builder commitRequest = CommitRequest.newBuilder(); - for (Entity entity: entities) { - commitRequest.addMutations(makeUpsert(entity)); - } - commitRequest.setMode(CommitRequest.Mode.NON_TRANSACTIONAL); - datastore.commit(commitRequest.build()); - // Break if the commit threw no exception. - break; - } catch (DatastoreException exception) { - // Only log the code and message for potentially-transient errors. The entire exception - // will be propagated upon the last retry. - LOG.error("Error writing to the Datastore ({}): {}", exception.getCode(), - exception.getMessage()); - if (!BackOffUtils.next(sleeper, backoff)) { - LOG.error("Aborting after {} retries.", MAX_RETRIES); - throw exception; - } + while (true) { + // Batch upsert entities. + try { + CommitRequest.Builder commitRequest = CommitRequest.newBuilder(); + commitRequest.addAllMutations(mutations); + commitRequest.setMode(CommitRequest.Mode.NON_TRANSACTIONAL); + datastore.commit(commitRequest.build()); + // Break if the commit threw no exception. + break; + } catch (DatastoreException exception) { + // Only log the code and message for potentially-transient errors. The entire exception + // will be propagated upon the last retry. + LOG.error("Error writing to the Datastore ({}): {}", exception.getCode(), + exception.getMessage()); + if (!BackOffUtils.next(sleeper, backoff)) { + LOG.error("Aborting after {} retries.", MAX_RETRIES); + throw exception; } } - LOG.debug("Successfully wrote {} entities", entities.size()); - entities.clear(); } + LOG.debug("Successfully wrote {} mutations", mutations.size()); + mutations.clear(); + } - @Override - public void populateDisplayData(Builder builder) { - super.populateDisplayData(builder); - builder - .addIfNotNull(DisplayData.item("projectId", projectId) - .withLabel("Output Project")); - } + @Override + public void populateDisplayData(Builder builder) { + super.populateDisplayData(builder); + builder + .addIfNotNull(DisplayData.item("projectId", projectId) + .withLabel("Output Project")); } + } - /** - * Returns true if a Datastore key is complete. A key is complete if its last element - * has either an id or a name. - */ - static boolean isValidKey(Key key) { - List elementList = key.getPathList(); - if (elementList.isEmpty()) { - return false; - } - PathElement lastElement = elementList.get(elementList.size() - 1); - return (lastElement.getId() != 0 || !lastElement.getName().isEmpty()); + /** + * Returns true if a Datastore key is complete. A key is complete if its last element + * has either an id or a name. + */ + static boolean isValidKey(Key key) { + List elementList = key.getPathList(); + if (elementList.isEmpty()) { + return false; + } + PathElement lastElement = elementList.get(elementList.size() - 1); + return (lastElement.getId() != 0 || !lastElement.getName().isEmpty()); + } + + /** + * A function that constructs an upsert {@link Mutation} from an {@link Entity}. + */ + @VisibleForTesting + static class UpsertFn extends SimpleFunction { + @Override + public Mutation apply(Entity entity) { + // Verify that the entity to write has a complete key. + checkArgument(isValidKey(entity.getKey()), + "Entities to be written to the Datastore must have complete keys:\n%s", entity); + + return makeUpsert(entity).build(); + } + + @Override + public void populateDisplayData(Builder builder) { + builder.add(DisplayData.item("upsertFn", this.getClass()) + .withLabel("Create Upsert Mutation")); + } + } + + /** + * A function that constructs a delete {@link Mutation} from an {@link Entity}. + */ + @VisibleForTesting + static class DeleteEntityFn extends SimpleFunction { + @Override + public Mutation apply(Entity entity) { + // Verify that the entity to delete has a complete key. + checkArgument(isValidKey(entity.getKey()), + "Entities to be deleted from the Datastore must have complete keys:\n%s", entity); + + return makeDelete(entity.getKey()).build(); + } + + @Override + public void populateDisplayData(Builder builder) { + builder.add(DisplayData.item("deleteEntityFn", this.getClass()) + .withLabel("Create Delete Mutation")); + } + } + + /** + * A function that constructs a delete {@link Mutation} from a {@link Key}. + */ + @VisibleForTesting + static class DeleteKeyFn extends SimpleFunction { + @Override + public Mutation apply(Key key) { + // Verify that the entity to delete has a complete key. + checkArgument(isValidKey(key), + "Keys to be deleted from the Datastore must be complete:\n%s", key); + + return makeDelete(key).build(); + } + + @Override + public void populateDisplayData(Builder builder) { + builder.add(DisplayData.item("deleteKeyFn", this.getClass()) + .withLabel("Create Delete Mutation")); } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3Test.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3Test.java index 8fa34dae119f..b0c6c185f812 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3Test.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3Test.java @@ -22,9 +22,11 @@ import static org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Read.QUERY_BATCH_LIMIT; import static org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Read.getEstimatedSizeBytes; import static org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Read.makeRequest; +import static org.apache.beam.sdk.io.gcp.datastore.V1Beta3.isValidKey; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static com.google.datastore.v1beta3.PropertyFilter.Operator.EQUAL; import static com.google.datastore.v1beta3.PropertyOrder.Direction.DESCENDING; +import static com.google.datastore.v1beta3.client.DatastoreHelper.makeDelete; import static com.google.datastore.v1beta3.client.DatastoreHelper.makeFilter; import static com.google.datastore.v1beta3.client.DatastoreHelper.makeKey; import static com.google.datastore.v1beta3.client.DatastoreHelper.makeOrder; @@ -45,12 +47,17 @@ import static org.mockito.Mockito.verifyZeroInteractions; import static org.mockito.Mockito.when; +import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.DatastoreWriterFn; +import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.DeleteEntity; +import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.DeleteEntityFn; +import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.DeleteKey; +import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.DeleteKeyFn; import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Read.ReadFn; import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Read.SplitQueryFn; import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Read.V1Beta3Options; +import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.UpsertFn; import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.V1Beta3DatastoreFactory; import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Write; -import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Write.DatastoreWriterFn; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.testing.RunnableOnService; import org.apache.beam.sdk.transforms.DoFnTester; @@ -67,6 +74,7 @@ import com.google.datastore.v1beta3.Entity; import com.google.datastore.v1beta3.EntityResult; import com.google.datastore.v1beta3.Key; +import com.google.datastore.v1beta3.Mutation; import com.google.datastore.v1beta3.PartitionId; import com.google.datastore.v1beta3.Query; import com.google.datastore.v1beta3.QueryResultBatch; @@ -233,7 +241,7 @@ public void testWriteDoesNotAllowNullProject() throws Exception { @Test public void testWriteValidationFailsWithNoProject() throws Exception { - V1Beta3.Write write = DatastoreIO.v1beta3().write(); + Write write = DatastoreIO.v1beta3().write(); thrown.expect(NullPointerException.class); thrown.expectMessage("projectId"); @@ -242,24 +250,89 @@ public void testWriteValidationFailsWithNoProject() throws Exception { } @Test - public void testSinkValidationSucceedsWithProject() throws Exception { - V1Beta3.Write write = DatastoreIO.v1beta3().write().withProjectId(PROJECT_ID); + public void testWriteValidationSucceedsWithProject() throws Exception { + Write write = DatastoreIO.v1beta3().write().withProjectId(PROJECT_ID); write.validate(null); } @Test public void testWriteDisplayData() { - V1Beta3.Write write = DatastoreIO.v1beta3().write() - .withProjectId(PROJECT_ID); + Write write = DatastoreIO.v1beta3().write().withProjectId(PROJECT_ID); DisplayData displayData = DisplayData.from(write); assertThat(displayData, hasDisplayItem("projectId", PROJECT_ID)); } + @Test + public void testDeleteEntityDoesNotAllowNullProject() throws Exception { + thrown.expect(NullPointerException.class); + thrown.expectMessage("projectId"); + + DatastoreIO.v1beta3().deleteEntity().withProjectId(null); + } + + @Test + public void testDeleteEntityValidationFailsWithNoProject() throws Exception { + DeleteEntity deleteEntity = DatastoreIO.v1beta3().deleteEntity(); + + thrown.expect(NullPointerException.class); + thrown.expectMessage("projectId"); + + deleteEntity.validate(null); + } + + @Test + public void testDeleteEntityValidationSucceedsWithProject() throws Exception { + DeleteEntity deleteEntity = DatastoreIO.v1beta3().deleteEntity().withProjectId(PROJECT_ID); + deleteEntity.validate(null); + } + + @Test + public void testDeleteEntityDisplayData() { + DeleteEntity deleteEntity = DatastoreIO.v1beta3().deleteEntity().withProjectId(PROJECT_ID); + + DisplayData displayData = DisplayData.from(deleteEntity); + + assertThat(displayData, hasDisplayItem("projectId", PROJECT_ID)); + } + + @Test + public void testDeleteKeyDoesNotAllowNullProject() throws Exception { + thrown.expect(NullPointerException.class); + thrown.expectMessage("projectId"); + + DatastoreIO.v1beta3().deleteKey().withProjectId(null); + } + + @Test + public void testDeleteKeyValidationFailsWithNoProject() throws Exception { + DeleteKey deleteKey = DatastoreIO.v1beta3().deleteKey(); + + thrown.expect(NullPointerException.class); + thrown.expectMessage("projectId"); + + deleteKey.validate(null); + } + + @Test + public void testDeleteKeyValidationSucceedsWithProject() throws Exception { + DeleteKey deleteKey = DatastoreIO.v1beta3().deleteKey().withProjectId(PROJECT_ID); + deleteKey.validate(null); + } + + @Test + public void testDeleteKeyDisplayData() { + DeleteKey deleteKey = DatastoreIO.v1beta3().deleteKey().withProjectId(PROJECT_ID); + + DisplayData displayData = DisplayData.from(deleteKey); + + assertThat(displayData, hasDisplayItem("projectId", PROJECT_ID)); + } + @Test @Category(RunnableOnService.class) - public void testSinkPrimitiveDisplayData() { + public void testWritePrimitiveDisplayData() { DisplayDataEvaluator evaluator = DisplayDataEvaluator.create(); PTransform, ?> write = DatastoreIO.v1beta3().write().withProjectId("myProject"); @@ -267,6 +340,39 @@ public void testSinkPrimitiveDisplayData() { Set displayData = evaluator.displayDataForPrimitiveTransforms(write); assertThat("DatastoreIO write should include the project in its primitive display data", displayData, hasItem(hasDisplayItem("projectId"))); + assertThat("DatastoreIO write should include the upsertFn in its primitive display data", + displayData, hasItem(hasDisplayItem("upsertFn"))); + + } + + @Test + @Category(RunnableOnService.class) + public void testDeleteEntityPrimitiveDisplayData() { + DisplayDataEvaluator evaluator = DisplayDataEvaluator.create(); + PTransform, ?> write = + DatastoreIO.v1beta3().deleteEntity().withProjectId("myProject"); + + Set displayData = evaluator.displayDataForPrimitiveTransforms(write); + assertThat("DatastoreIO write should include the project in its primitive display data", + displayData, hasItem(hasDisplayItem("projectId"))); + assertThat("DatastoreIO write should include the deleteEntityFn in its primitive display data", + displayData, hasItem(hasDisplayItem("deleteEntityFn"))); + + } + + @Test + @Category(RunnableOnService.class) + public void testDeleteKeyPrimitiveDisplayData() { + DisplayDataEvaluator evaluator = DisplayDataEvaluator.create(); + PTransform, ?> write = + DatastoreIO.v1beta3().deleteKey().withProjectId("myProject"); + + Set displayData = evaluator.displayDataForPrimitiveTransforms(write); + assertThat("DatastoreIO write should include the project in its primitive display data", + displayData, hasItem(hasDisplayItem("projectId"))); + assertThat("DatastoreIO write should include the deleteKeyFn in its primitive display data", + displayData, hasItem(hasDisplayItem("deleteKeyFn"))); + } /** @@ -286,33 +392,33 @@ public void testHasNameOrId() { Key key; // Complete with name, no ancestor key = makeKey("bird", "finch").build(); - assertTrue(Write.isValidKey(key)); + assertTrue(isValidKey(key)); // Complete with id, no ancestor key = makeKey("bird", 123).build(); - assertTrue(Write.isValidKey(key)); + assertTrue(isValidKey(key)); // Incomplete, no ancestor key = makeKey("bird").build(); - assertFalse(Write.isValidKey(key)); + assertFalse(isValidKey(key)); // Complete with name and ancestor key = makeKey("bird", "owl").build(); key = makeKey(key, "bird", "horned").build(); - assertTrue(Write.isValidKey(key)); + assertTrue(isValidKey(key)); // Complete with id and ancestor key = makeKey("bird", "owl").build(); key = makeKey(key, "bird", 123).build(); - assertTrue(Write.isValidKey(key)); + assertTrue(isValidKey(key)); // Incomplete with ancestor key = makeKey("bird", "owl").build(); key = makeKey(key, "bird").build(); - assertFalse(Write.isValidKey(key)); + assertFalse(isValidKey(key)); key = makeKey().build(); - assertFalse(Write.isValidKey(key)); + assertFalse(isValidKey(key)); } /** @@ -322,14 +428,86 @@ public void testHasNameOrId() { public void testAddEntitiesWithIncompleteKeys() throws Exception { Key key = makeKey("bird").build(); Entity entity = Entity.newBuilder().setKey(key).build(); - DatastoreWriterFn writer = new DatastoreWriterFn(PROJECT_ID, mockDatastoreFactory); - DoFnTester doFnTester = DoFnTester.of(writer); - doFnTester.setCloningBehavior(CloningBehavior.DO_NOT_CLONE); + UpsertFn upsertFn = new UpsertFn(); thrown.expect(IllegalArgumentException.class); thrown.expectMessage("Entities to be written to the Datastore must have complete keys"); - doFnTester.processBundle(entity); + upsertFn.apply(entity); + } + + @Test + /** + * Test that entities with valid keys are transformed to upsert mutations. + */ + public void testAddEntities() throws Exception { + Key key = makeKey("bird", "finch").build(); + Entity entity = Entity.newBuilder().setKey(key).build(); + UpsertFn upsertFn = new UpsertFn(); + + Mutation exceptedMutation = makeUpsert(entity).build(); + assertEquals(upsertFn.apply(entity), exceptedMutation); + } + + /** + * Test that entities with incomplete keys cannot be deleted. + */ + @Test + public void testDeleteEntitiesWithIncompleteKeys() throws Exception { + Key key = makeKey("bird").build(); + Entity entity = Entity.newBuilder().setKey(key).build(); + DeleteEntityFn deleteEntityFn = new DeleteEntityFn(); + + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("Entities to be deleted from the Datastore must have complete keys"); + + deleteEntityFn.apply(entity); + } + + /** + * Test that entities with valid keys are transformed to delete mutations. + */ + @Test + public void testDeleteEntities() throws Exception { + Key key = makeKey("bird", "finch").build(); + Entity entity = Entity.newBuilder().setKey(key).build(); + DeleteEntityFn deleteEntityFn = new DeleteEntityFn(); + + Mutation exceptedMutation = makeDelete(entity.getKey()).build(); + assertEquals(deleteEntityFn.apply(entity), exceptedMutation); + } + + /** + * Test that incomplete keys cannot be deleted. + */ + @Test + public void testDeleteIncompleteKeys() throws Exception { + Key key = makeKey("bird").build(); + DeleteKeyFn deleteKeyFn = new DeleteKeyFn(); + + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("Keys to be deleted from the Datastore must be complete"); + + deleteKeyFn.apply(key); + } + + /** + * Test that valid keys are transformed to delete mutations. + */ + @Test + public void testDeleteKeys() throws Exception { + Key key = makeKey("bird", "finch").build(); + DeleteKeyFn deleteKeyFn = new DeleteKeyFn(); + + Mutation exceptedMutation = makeDelete(key).build(); + assertEquals(deleteKeyFn.apply(key), exceptedMutation); + } + + @Test + public void testDatastoreWriteFnDisplayData() { + DatastoreWriterFn datastoreWriter = new DatastoreWriterFn(PROJECT_ID); + DisplayData displayData = DisplayData.from(datastoreWriter); + assertThat(displayData, hasDisplayItem("projectId", PROJECT_ID)); } /** Tests {@link DatastoreWriterFn} with entities less than one batch. */ @@ -354,27 +532,26 @@ public void testDatatoreWriterFnWithBatchesExactMultiple() throws Exception { } // A helper method to test DatastoreWriterFn for various batch sizes. - private void datastoreWriterFnTest(int numEntities) throws Exception { + private void datastoreWriterFnTest(int numMutations) throws Exception { // Create the requested number of mutations. - List entities = new ArrayList<>(numEntities); - for (int i = 0; i < numEntities; ++i) { - entities.add(Entity.newBuilder().setKey(makeKey("key" + i, i + 1)).build()); + List mutations = new ArrayList<>(numMutations); + for (int i = 0; i < numMutations; ++i) { + mutations.add( + makeUpsert(Entity.newBuilder().setKey(makeKey("key" + i, i + 1)).build()).build()); } DatastoreWriterFn datastoreWriter = new DatastoreWriterFn(PROJECT_ID, mockDatastoreFactory); - DoFnTester doFnTester = DoFnTester.of(datastoreWriter); + DoFnTester doFnTester = DoFnTester.of(datastoreWriter); doFnTester.setCloningBehavior(CloningBehavior.DO_NOT_CLONE); - doFnTester.processBundle(entities); + doFnTester.processBundle(mutations); int start = 0; - while (start < numEntities) { - int end = Math.min(numEntities, start + DATASTORE_BATCH_UPDATE_LIMIT); + while (start < numMutations) { + int end = Math.min(numMutations, start + DATASTORE_BATCH_UPDATE_LIMIT); CommitRequest.Builder commitRequest = CommitRequest.newBuilder(); commitRequest.setMode(CommitRequest.Mode.NON_TRANSACTIONAL); - for (Entity entity: entities.subList(start, end)) { - commitRequest.addMutations(makeUpsert(entity)); - } - // Verify all the batch requests were made with the expected entities. + commitRequest.addAllMutations(mutations.subList(start, end)); + // Verify all the batch requests were made with the expected mutations. verify(mockDatastore, times(1)).commit(commitRequest.build()); start = end; } From c29afb119be034b6b93083d9e8ec5542f13b4373 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Fri, 19 Aug 2016 10:24:00 -0700 Subject: [PATCH 037/112] Mark JAXBContext as Volatile in JAXBCoder This corrects the double-checked locking and lazy intialization --- .../build-tools/src/main/resources/beam/findbugs-filter.xml | 6 ------ .../src/main/java/org/apache/beam/sdk/coders/JAXBCoder.java | 2 +- 2 files changed, 1 insertion(+), 7 deletions(-) diff --git a/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml b/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml index d135cbb671d9..cd50408fd395 100644 --- a/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml +++ b/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml @@ -37,12 +37,6 @@ - - - - - - diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/JAXBCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/JAXBCoder.java index 2284aafe9add..689f993a1cde 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/JAXBCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/JAXBCoder.java @@ -46,7 +46,7 @@ public class JAXBCoder extends AtomicCoder { private final Class jaxbClass; - private transient JAXBContext jaxbContext; + private transient volatile JAXBContext jaxbContext; public Class getJAXBClass() { return jaxbClass; From cb30ba310929934fdf4cfdb324fc70f799108742 Mon Sep 17 00:00:00 2001 From: Luke Cwik Date: Thu, 18 Aug 2016 16:35:33 -0700 Subject: [PATCH 038/112] Modify example dependencies to only add runners as optional dependencies. Also support excluding runners by disabling the include-runners profile. Tested ITs BQ and WordCount pass on Dataflow WordCount passes on Flink and Spark --- examples/java/pom.xml | 129 ++++++++++++------ .../org/apache/beam/examples/WordCount.java | 3 +- examples/java8/pom.xml | 40 +++++- 3 files changed, 125 insertions(+), 47 deletions(-) diff --git a/examples/java/pom.xml b/examples/java/pom.xml index 6efbc54406d9..21d7a3a197cd 100644 --- a/examples/java/pom.xml +++ b/examples/java/pom.xml @@ -38,6 +38,78 @@ 1.6.2 + + + + include-runners + + true + + + + org.apache.beam + beam-runners-direct-java + ${project.version} + runtime + true + + + + org.apache.beam + beam-runners-flink_2.10 + ${project.version} + runtime + true + + + + org.apache.beam + beam-runners-google-cloud-dataflow-java + ${project.version} + runtime + true + + + + org.apache.beam + beam-runners-spark + ${project.version} + runtime + true + + + + org.apache.spark + spark-core_2.10 + ${spark.version} + runtime + true + + + + org.apache.spark + spark-streaming_2.10 + ${spark.version} + runtime + true + + + + + @@ -269,53 +341,11 @@ slf4j-api - - org.apache.beam - beam-runners-direct-java - ${project.version} - runtime - true - - - - org.apache.beam - beam-runners-google-cloud-dataflow-java - ${project.version} - runtime - true - - - - org.apache.beam - beam-runners-flink_2.10 - ${project.version} - runtime - - - - org.apache.beam - beam-runners-spark - ${project.version} - - - - org.apache.spark - spark-core_2.10 - ${spark.version} - runtime - - - - org.apache.spark - spark-streaming_2.10 - ${spark.version} - runtime - - org.slf4j slf4j-jdk14 runtime + true + + + + org.apache.beam + beam-runners-direct-java + ${project.version} + test + + org.mockito mockito-all diff --git a/examples/java/src/main/java/org/apache/beam/examples/WordCount.java b/examples/java/src/main/java/org/apache/beam/examples/WordCount.java index 72ad35f04e54..42d30bb5fa94 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/WordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/WordCount.java @@ -17,7 +17,6 @@ */ package org.apache.beam.examples; -import org.apache.beam.runners.spark.SparkRunner; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.options.Default; @@ -211,7 +210,7 @@ public String create(PipelineOptions options) { public static class InputFactory implements DefaultValueFactory { @Override public String create(PipelineOptions options) { - if (options.getRunner().isAssignableFrom(SparkRunner.class)) { + if (options.getRunner().getName().contains("SparkRunner")) { return Resources.getResource("LICENSE").getPath(); } else { return "gs://apache-beam-samples/apache/LICENSE"; diff --git a/examples/java8/pom.xml b/examples/java8/pom.xml index 609dcfac09fe..f3f2c3c83f17 100644 --- a/examples/java8/pom.xml +++ b/examples/java8/pom.xml @@ -35,8 +35,11 @@ jar - + + 1.6.2 + + include-runners - true + + true + org.apache.beam @@ -71,6 +76,14 @@ true + + org.apache.beam + beam-runners-google-cloud-dataflow-java + ${project.version} + runtime + true + + org.apache.beam beam-runners-spark @@ -78,6 +91,22 @@ runtime true + + + org.apache.spark + spark-core_2.10 + ${spark.version} + runtime + true + + + + org.apache.spark + spark-streaming_2.10 + ${spark.version} + runtime + true + @@ -164,6 +193,13 @@ slf4j-api + + org.slf4j + slf4j-jdk14 + runtime + true + + org.apache.avro avro From 54e4cb123187992b64b1580869ae5857f0ef613b Mon Sep 17 00:00:00 2001 From: Vikas Kedigehalli Date: Fri, 19 Aug 2016 16:18:37 -0700 Subject: [PATCH 039/112] DatastoreIO v1beta3 to v1 --- examples/java/pom.xml | 4 +- .../beam/examples/complete/AutoComplete.java | 12 +- .../examples/cookbook/DatastoreWordCount.java | 24 +-- pom.xml | 8 +- runners/google-cloud-dataflow-java/pom.xml | 2 +- sdks/java/io/google-cloud-platform/pom.xml | 4 +- .../sdk/io/gcp/datastore/DatastoreIO.java | 8 +- .../{V1Beta3.java => DatastoreV1.java} | 175 +++++++++--------- ...{V1Beta3Test.java => DatastoreV1Test.java} | 162 ++++++++-------- .../{V1Beta3ReadIT.java => V1ReadIT.java} | 42 ++--- ...ta3TestOptions.java => V1TestOptions.java} | 4 +- .../{V1Beta3TestUtil.java => V1TestUtil.java} | 74 ++++---- .../{V1Beta3WriteIT.java => V1WriteIT.java} | 22 +-- 13 files changed, 271 insertions(+), 270 deletions(-) rename sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/{V1Beta3.java => DatastoreV1.java} (85%) rename sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/{V1Beta3Test.java => DatastoreV1Test.java} (82%) rename sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/{V1Beta3ReadIT.java => V1ReadIT.java} (69%) rename sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/{V1Beta3TestOptions.java => V1TestOptions.java} (92%) rename sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/{V1Beta3TestUtil.java => V1TestUtil.java} (82%) rename sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/{V1Beta3WriteIT.java => V1WriteIT.java} (76%) diff --git a/examples/java/pom.xml b/examples/java/pom.xml index 21d7a3a197cd..096bc4e32fef 100644 --- a/examples/java/pom.xml +++ b/examples/java/pom.xml @@ -323,12 +323,12 @@ com.google.cloud.datastore - datastore-v1beta3-proto-client + datastore-v1-proto-client com.google.cloud.datastore - datastore-v1beta3-protos + datastore-v1-protos diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java b/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java index baae100bcf38..120c64fae54a 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java @@ -18,8 +18,8 @@ package org.apache.beam.examples.complete; import static com.google.common.base.Preconditions.checkArgument; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeKey; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeValue; +import static com.google.datastore.v1.client.DatastoreHelper.makeKey; +import static com.google.datastore.v1.client.DatastoreHelper.makeValue; import org.apache.beam.examples.common.ExampleBigQueryTableOptions; import org.apache.beam.examples.common.ExampleOptions; @@ -59,9 +59,9 @@ import com.google.api.services.bigquery.model.TableRow; import com.google.api.services.bigquery.model.TableSchema; import com.google.common.base.MoreObjects; -import com.google.datastore.v1beta3.Entity; -import com.google.datastore.v1beta3.Key; -import com.google.datastore.v1beta3.Value; +import com.google.datastore.v1.Entity; +import com.google.datastore.v1.Key; +import com.google.datastore.v1.Value; import org.joda.time.Duration; @@ -488,7 +488,7 @@ public static void main(String[] args) throws IOException { toWrite .apply("FormatForDatastore", ParDo.of(new FormatForDatastore(options.getKind(), options.getDatastoreAncestorKey()))) - .apply(DatastoreIO.v1beta3().write().withProjectId(MoreObjects.firstNonNull( + .apply(DatastoreIO.v1().write().withProjectId(MoreObjects.firstNonNull( options.getOutputProject(), options.getProject()))); } if (options.getOutputToBigQuery()) { diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java index 21220b8fac8f..215e2ffc7898 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java @@ -17,16 +17,16 @@ */ package org.apache.beam.examples.cookbook; -import static com.google.datastore.v1beta3.client.DatastoreHelper.getString; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeFilter; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeKey; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeValue; +import static com.google.datastore.v1.client.DatastoreHelper.getString; +import static com.google.datastore.v1.client.DatastoreHelper.makeFilter; +import static com.google.datastore.v1.client.DatastoreHelper.makeKey; +import static com.google.datastore.v1.client.DatastoreHelper.makeValue; import org.apache.beam.examples.WordCount; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.io.gcp.datastore.DatastoreIO; -import org.apache.beam.sdk.io.gcp.datastore.V1Beta3; +import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.PipelineOptions; @@ -36,11 +36,11 @@ import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.ParDo; -import com.google.datastore.v1beta3.Entity; -import com.google.datastore.v1beta3.Key; -import com.google.datastore.v1beta3.PropertyFilter; -import com.google.datastore.v1beta3.Query; -import com.google.datastore.v1beta3.Value; +import com.google.datastore.v1.Entity; +import com.google.datastore.v1.Key; +import com.google.datastore.v1.PropertyFilter; +import com.google.datastore.v1.Query; +import com.google.datastore.v1.Value; import java.util.Map; import java.util.UUID; @@ -194,7 +194,7 @@ public static void writeDataToDatastore(Options options) { Pipeline p = Pipeline.create(options); p.apply("ReadLines", TextIO.Read.from(options.getInput())) .apply(ParDo.of(new CreateEntityFn(options.getNamespace(), options.getKind()))) - .apply(DatastoreIO.v1beta3().write().withProjectId(options.getProject())); + .apply(DatastoreIO.v1().write().withProjectId(options.getProject())); p.run(); } @@ -225,7 +225,7 @@ public static void readDataFromDatastore(Options options) { Query query = makeAncestorKindQuery(options); // For Datastore sources, the read namespace can be set on the entire query. - V1Beta3.Read read = DatastoreIO.v1beta3().read() + DatastoreV1.Read read = DatastoreIO.v1().read() .withProjectId(options.getProject()) .withQuery(query) .withNamespace(options.getNamespace()); diff --git a/pom.xml b/pom.xml index 58adbe7a86cc..f9e04795417d 100644 --- a/pom.xml +++ b/pom.xml @@ -107,8 +107,8 @@ v2-rev8-1.22.0 v1b3-rev36-1.22.0 0.5.160222 - 1.0.0-beta.2 - 1.0.0-beta + 1.1.0 + 1.0.1 1.0-rc2 1.1 1.22.0 @@ -450,13 +450,13 @@ com.google.cloud.datastore - datastore-v1beta3-proto-client + datastore-v1-proto-client ${datastore.client.version} com.google.cloud.datastore - datastore-v1beta3-protos + datastore-v1-protos ${datastore.proto.version} diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml index 00b5a9ba67c2..0044823068c0 100644 --- a/runners/google-cloud-dataflow-java/pom.xml +++ b/runners/google-cloud-dataflow-java/pom.xml @@ -409,7 +409,7 @@ com.google.cloud.datastore - datastore-v1beta3-protos + datastore-v1-protos test diff --git a/sdks/java/io/google-cloud-platform/pom.xml b/sdks/java/io/google-cloud-platform/pom.xml index 1596a66b82be..8075335bd712 100644 --- a/sdks/java/io/google-cloud-platform/pom.xml +++ b/sdks/java/io/google-cloud-platform/pom.xml @@ -115,12 +115,12 @@ com.google.cloud.datastore - datastore-v1beta3-proto-client + datastore-v1-proto-client com.google.cloud.datastore - datastore-v1beta3-protos + datastore-v1-protos diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreIO.java index bde0aba8a103..5abf0153ac10 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreIO.java @@ -24,7 +24,7 @@ * Google Cloud Datastore over different * versions of the Datastore Client libraries. * - *

    To use the v1beta3 version see {@link V1Beta3}. + *

    To use the v1 version see {@link DatastoreV1}. */ @Experimental(Experimental.Kind.SOURCE_SINK) public class DatastoreIO { @@ -32,10 +32,10 @@ public class DatastoreIO { private DatastoreIO() {} /** - * Returns a {@link V1Beta3} that provides an API for accessing Datastore through v1beta3 version + * Returns a {@link DatastoreV1} that provides an API for accessing Datastore through v1 version * of Datastore Client library. */ - public static V1Beta3 v1beta3() { - return new V1Beta3(); + public static DatastoreV1 v1() { + return new DatastoreV1(); } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java similarity index 85% rename from sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3.java rename to sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java index 8503b668e25a..852595aa79ab 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java @@ -21,14 +21,14 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Verify.verify; -import static com.google.datastore.v1beta3.PropertyFilter.Operator.EQUAL; -import static com.google.datastore.v1beta3.PropertyOrder.Direction.DESCENDING; -import static com.google.datastore.v1beta3.QueryResultBatch.MoreResultsType.NOT_FINISHED; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeDelete; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeFilter; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeOrder; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeUpsert; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeValue; +import static com.google.datastore.v1.PropertyFilter.Operator.EQUAL; +import static com.google.datastore.v1.PropertyOrder.Direction.DESCENDING; +import static com.google.datastore.v1.QueryResultBatch.MoreResultsType.NOT_FINISHED; +import static com.google.datastore.v1.client.DatastoreHelper.makeDelete; +import static com.google.datastore.v1.client.DatastoreHelper.makeFilter; +import static com.google.datastore.v1.client.DatastoreHelper.makeOrder; +import static com.google.datastore.v1.client.DatastoreHelper.makeUpsert; +import static com.google.datastore.v1.client.DatastoreHelper.makeValue; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.options.GcpOptions; @@ -58,23 +58,23 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.MoreObjects; import com.google.common.collect.ImmutableList; -import com.google.datastore.v1beta3.CommitRequest; -import com.google.datastore.v1beta3.Entity; -import com.google.datastore.v1beta3.EntityResult; -import com.google.datastore.v1beta3.Key; -import com.google.datastore.v1beta3.Key.PathElement; -import com.google.datastore.v1beta3.Mutation; -import com.google.datastore.v1beta3.PartitionId; -import com.google.datastore.v1beta3.Query; -import com.google.datastore.v1beta3.QueryResultBatch; -import com.google.datastore.v1beta3.RunQueryRequest; -import com.google.datastore.v1beta3.RunQueryResponse; -import com.google.datastore.v1beta3.client.Datastore; -import com.google.datastore.v1beta3.client.DatastoreException; -import com.google.datastore.v1beta3.client.DatastoreFactory; -import com.google.datastore.v1beta3.client.DatastoreHelper; -import com.google.datastore.v1beta3.client.DatastoreOptions; -import com.google.datastore.v1beta3.client.QuerySplitter; +import com.google.datastore.v1.CommitRequest; +import com.google.datastore.v1.Entity; +import com.google.datastore.v1.EntityResult; +import com.google.datastore.v1.Key; +import com.google.datastore.v1.Key.PathElement; +import com.google.datastore.v1.Mutation; +import com.google.datastore.v1.PartitionId; +import com.google.datastore.v1.Query; +import com.google.datastore.v1.QueryResultBatch; +import com.google.datastore.v1.RunQueryRequest; +import com.google.datastore.v1.RunQueryResponse; +import com.google.datastore.v1.client.Datastore; +import com.google.datastore.v1.client.DatastoreException; +import com.google.datastore.v1.client.DatastoreFactory; +import com.google.datastore.v1.client.DatastoreHelper; +import com.google.datastore.v1.client.DatastoreOptions; +import com.google.datastore.v1.client.QuerySplitter; import com.google.protobuf.Int32Value; import org.slf4j.Logger; @@ -88,21 +88,21 @@ import javax.annotation.Nullable; /** - *

    {@link V1Beta3} provides an API to Read, Write and Delete {@link PCollection PCollections} of - * Google Cloud Datastore version v1beta3 + *

    {@link DatastoreV1} provides an API to Read, Write and Delete {@link PCollection PCollections} + * of Google Cloud Datastore version v1 * {@link Entity} objects. * - *

    This API currently requires an authentication workaround. To use {@link V1Beta3}, users + *

    This API currently requires an authentication workaround. To use {@link DatastoreV1}, users * must use the {@code gcloud} command line tool to get credentials for Datastore: *

      * $ gcloud auth login
      * 
    * - *

    To read a {@link PCollection} from a query to Datastore, use {@link V1Beta3#read} and - * its methods {@link V1Beta3.Read#withProjectId} and {@link V1Beta3.Read#withQuery} to + *

    To read a {@link PCollection} from a query to Datastore, use {@link DatastoreV1#read} and + * its methods {@link DatastoreV1.Read#withProjectId} and {@link DatastoreV1.Read#withQuery} to * specify the project to query and the query to read from. You can optionally provide a namespace - * to query within using {@link V1Beta3.Read#withNamespace}. You could also optionally specify - * how many splits you want for the query using {@link V1Beta3.Read#withNumQuerySplits}. + * to query within using {@link DatastoreV1.Read#withNamespace}. You could also optionally specify + * how many splits you want for the query using {@link DatastoreV1.Read#withNumQuerySplits}. * *

    For example: * @@ -114,40 +114,40 @@ * * Pipeline p = Pipeline.create(options); * PCollection entities = p.apply( - * DatastoreIO.v1beta3().read() + * DatastoreIO.v1().read() * .withProjectId(projectId) * .withQuery(query)); * } * *

    Note: Normally, a Cloud Dataflow job will read from Cloud Datastore in parallel across * many workers. However, when the {@link Query} is configured with a limit using - * {@link com.google.datastore.v1beta3.Query.Builder#setLimit(Int32Value)}, then + * {@link com.google.datastore.v1.Query.Builder#setLimit(Int32Value)}, then * all returned results will be read by a single Dataflow worker in order to ensure correct data. * - *

    To write a {@link PCollection} to a Datastore, use {@link V1Beta3#write}, + *

    To write a {@link PCollection} to a Datastore, use {@link DatastoreV1#write}, * specifying the Cloud Datastore project to write to: * *

     {@code
      * PCollection entities = ...;
    - * entities.apply(DatastoreIO.v1beta3().write().withProjectId(projectId));
    + * entities.apply(DatastoreIO.v1().write().withProjectId(projectId));
      * p.run();
      * } 
    * *

    To delete a {@link PCollection} of {@link Entity Entities} from Datastore, use - * {@link V1Beta3#deleteEntity()}, specifying the Cloud Datastore project to write to: + * {@link DatastoreV1#deleteEntity()}, specifying the Cloud Datastore project to write to: * *

     {@code
      * PCollection entities = ...;
    - * entities.apply(DatastoreIO.v1beta3().deleteEntity().withProjectId(projectId));
    + * entities.apply(DatastoreIO.v1().deleteEntity().withProjectId(projectId));
      * p.run();
      * } 
    * *

    To delete entities associated with a {@link PCollection} of {@link Key Keys} from Datastore, - * use {@link V1Beta3#deleteKey}, specifying the Cloud Datastore project to write to: + * use {@link DatastoreV1#deleteKey}, specifying the Cloud Datastore project to write to: * *

     {@code
      * PCollection entities = ...;
    - * entities.apply(DatastoreIO.v1beta3().deleteKey().withProjectId(projectId));
    + * entities.apply(DatastoreIO.v1().deleteKey().withProjectId(projectId));
      * p.run();
      * } 
    * @@ -176,10 +176,10 @@ * @see org.apache.beam.sdk.runners.PipelineRunner */ @Experimental(Experimental.Kind.SOURCE_SINK) -public class V1Beta3 { +public class DatastoreV1 { // A package-private constructor to prevent direct instantiation from outside of this package - V1Beta3() {} + DatastoreV1() {} /** * Datastore has a limit of 500 mutations per batch operation, so we flush @@ -189,13 +189,13 @@ public class V1Beta3 { static final int DATASTORE_BATCH_UPDATE_LIMIT = 500; /** - * Returns an empty {@link V1Beta3.Read} builder. Configure the source {@code projectId}, + * Returns an empty {@link DatastoreV1.Read} builder. Configure the source {@code projectId}, * {@code query}, and optionally {@code namespace} and {@code numQuerySplits} using - * {@link V1Beta3.Read#withProjectId}, {@link V1Beta3.Read#withQuery}, - * {@link V1Beta3.Read#withNamespace}, {@link V1Beta3.Read#withNumQuerySplits}. + * {@link DatastoreV1.Read#withProjectId}, {@link DatastoreV1.Read#withQuery}, + * {@link DatastoreV1.Read#withNamespace}, {@link DatastoreV1.Read#withNumQuerySplits}. */ - public V1Beta3.Read read() { - return new V1Beta3.Read(null, null, null, 0); + public DatastoreV1.Read read() { + return new DatastoreV1.Read(null, null, null, 0); } /** @@ -329,37 +329,38 @@ private Read(@Nullable String projectId, @Nullable Query query, @Nullable String } /** - * Returns a new {@link V1Beta3.Read} that reads from the Datastore for the specified project. + * Returns a new {@link DatastoreV1.Read} that reads from the Datastore for the specified + * project. */ - public V1Beta3.Read withProjectId(String projectId) { + public DatastoreV1.Read withProjectId(String projectId) { checkNotNull(projectId, "projectId"); - return new V1Beta3.Read(projectId, query, namespace, numQuerySplits); + return new DatastoreV1.Read(projectId, query, namespace, numQuerySplits); } /** - * Returns a new {@link V1Beta3.Read} that reads the results of the specified query. + * Returns a new {@link DatastoreV1.Read} that reads the results of the specified query. * *

    Note: Normally, {@code DatastoreIO} will read from Cloud Datastore in parallel * across many workers. However, when the {@link Query} is configured with a limit using * {@link Query.Builder#setLimit}, then all results will be read by a single worker in order * to ensure correct results. */ - public V1Beta3.Read withQuery(Query query) { + public DatastoreV1.Read withQuery(Query query) { checkNotNull(query, "query"); checkArgument(!query.hasLimit() || query.getLimit().getValue() > 0, "Invalid query limit %s: must be positive", query.getLimit().getValue()); - return new V1Beta3.Read(projectId, query, namespace, numQuerySplits); + return new DatastoreV1.Read(projectId, query, namespace, numQuerySplits); } /** - * Returns a new {@link V1Beta3.Read} that reads from the given namespace. + * Returns a new {@link DatastoreV1.Read} that reads from the given namespace. */ - public V1Beta3.Read withNamespace(String namespace) { - return new V1Beta3.Read(projectId, query, namespace, numQuerySplits); + public DatastoreV1.Read withNamespace(String namespace) { + return new DatastoreV1.Read(projectId, query, namespace, numQuerySplits); } /** - * Returns a new {@link V1Beta3.Read} that reads by splitting the given {@code query} into + * Returns a new {@link DatastoreV1.Read} that reads by splitting the given {@code query} into * {@code numQuerySplits}. * *

    The semantics for the query splitting is defined below: @@ -374,8 +375,8 @@ public V1Beta3.Read withNamespace(String namespace) { * splits. In such cases we just use whatever the Datastore returns. * */ - public V1Beta3.Read withNumQuerySplits(int numQuerySplits) { - return new V1Beta3.Read(projectId, query, namespace, + public DatastoreV1.Read withNumQuerySplits(int numQuerySplits) { + return new DatastoreV1.Read(projectId, query, namespace, Math.min(Math.max(numQuerySplits, 0), NUM_QUERY_SPLITS_MAX)); } @@ -400,7 +401,7 @@ public String getNamespace() { */ @Override public PCollection apply(PBegin input) { - V1Beta3Options v1Beta3Options = V1Beta3Options.from(getProjectId(), getQuery(), + V1Options v1Options = V1Options.from(getProjectId(), getQuery(), getNamespace()); /* @@ -422,7 +423,7 @@ public PCollection apply(PBegin input) { */ PCollection> queries = input .apply(Create.of(query)) - .apply(ParDo.of(new SplitQueryFn(v1Beta3Options, numQuerySplits))); + .apply(ParDo.of(new SplitQueryFn(v1Options, numQuerySplits))); PCollection shardedQueries = queries .apply(GroupByKey.create()) @@ -430,7 +431,7 @@ public PCollection apply(PBegin input) { .apply(Flatten.iterables()); PCollection entities = shardedQueries - .apply(ParDo.of(new ReadFn(v1Beta3Options))); + .apply(ParDo.of(new ReadFn(v1Options))); return entities; } @@ -463,23 +464,23 @@ public String toString() { } /** - * A class for v1beta3 Datastore related options. + * A class for v1 Datastore related options. */ @VisibleForTesting - static class V1Beta3Options implements Serializable { + static class V1Options implements Serializable { private final Query query; private final String projectId; @Nullable private final String namespace; - private V1Beta3Options(String projectId, Query query, @Nullable String namespace) { + private V1Options(String projectId, Query query, @Nullable String namespace) { this.projectId = checkNotNull(projectId, "projectId"); this.query = checkNotNull(query, "query"); this.namespace = namespace; } - public static V1Beta3Options from(String projectId, Query query, @Nullable String namespace) { - return new V1Beta3Options(projectId, query, namespace); + public static V1Options from(String projectId, Query query, @Nullable String namespace) { + return new V1Options(projectId, query, namespace); } public Query getQuery() { @@ -502,23 +503,23 @@ public String getNamespace() { */ @VisibleForTesting static class SplitQueryFn extends DoFn> { - private final V1Beta3Options options; + private final V1Options options; // number of splits to make for a given query private final int numSplits; - private final V1Beta3DatastoreFactory datastoreFactory; + private final V1DatastoreFactory datastoreFactory; // Datastore client private transient Datastore datastore; // Query splitter private transient QuerySplitter querySplitter; - public SplitQueryFn(V1Beta3Options options, int numSplits) { - this(options, numSplits, new V1Beta3DatastoreFactory()); + public SplitQueryFn(V1Options options, int numSplits) { + this(options, numSplits, new V1DatastoreFactory()); } @VisibleForTesting - SplitQueryFn(V1Beta3Options options, int numSplits, - V1Beta3DatastoreFactory datastoreFactory) { + SplitQueryFn(V1Options options, int numSplits, + V1DatastoreFactory datastoreFactory) { this.options = options; this.numSplits = numSplits; this.datastoreFactory = datastoreFactory; @@ -582,17 +583,17 @@ public void populateDisplayData(Builder builder) { */ @VisibleForTesting static class ReadFn extends DoFn { - private final V1Beta3Options options; - private final V1Beta3DatastoreFactory datastoreFactory; + private final V1Options options; + private final V1DatastoreFactory datastoreFactory; // Datastore client private transient Datastore datastore; - public ReadFn(V1Beta3Options options) { - this(options, new V1Beta3DatastoreFactory()); + public ReadFn(V1Options options) { + this(options, new V1DatastoreFactory()); } @VisibleForTesting - ReadFn(V1Beta3Options options, V1Beta3DatastoreFactory datastoreFactory) { + ReadFn(V1Options options, V1DatastoreFactory datastoreFactory) { this.options = options; this.datastoreFactory = datastoreFactory; } @@ -656,8 +657,8 @@ public void processElement(ProcessContext context) throws Exception { } /** - * Returns an empty {@link V1Beta3.Write} builder. Configure the destination - * {@code projectId} using {@link V1Beta3.Write#withProjectId}. + * Returns an empty {@link DatastoreV1.Write} builder. Configure the destination + * {@code projectId} using {@link DatastoreV1.Write#withProjectId}. */ public Write write() { return new Write(null); @@ -812,14 +813,14 @@ public String getProjectId() { /** * {@link DoFn} that writes {@link Mutation}s to Cloud Datastore. Mutations are written in - * batches, where the maximum batch size is {@link V1Beta3#DATASTORE_BATCH_UPDATE_LIMIT}. + * batches, where the maximum batch size is {@link DatastoreV1#DATASTORE_BATCH_UPDATE_LIMIT}. * *

    See * Datastore: Entities, Properties, and Keys for information about entity keys and mutations. * *

    Commits are non-transactional. If a commit fails because of a conflict over an entity - * group, the commit will be retried (up to {@link V1Beta3#DATASTORE_BATCH_UPDATE_LIMIT} + * group, the commit will be retried (up to {@link DatastoreV1#DATASTORE_BATCH_UPDATE_LIMIT} * times). This means that the mutation operation should be idempotent. Thus, the writer should * only be used for {code upsert} and {@code delete} mutation operations, as these are the only * two Cloud Datastore mutations that are idempotent. @@ -829,7 +830,7 @@ static class DatastoreWriterFn extends DoFn { private static final Logger LOG = LoggerFactory.getLogger(DatastoreWriterFn.class); private final String projectId; private transient Datastore datastore; - private final V1Beta3DatastoreFactory datastoreFactory; + private final V1DatastoreFactory datastoreFactory; // Current batch of mutations to be written. private final List mutations = new ArrayList<>(); /** @@ -844,11 +845,11 @@ static class DatastoreWriterFn extends DoFn { private static final int INITIAL_BACKOFF_MILLIS = 5000; DatastoreWriterFn(String projectId) { - this(projectId, new V1Beta3DatastoreFactory()); + this(projectId, new V1DatastoreFactory()); } @VisibleForTesting - DatastoreWriterFn(String projectId, V1Beta3DatastoreFactory datastoreFactory) { + DatastoreWriterFn(String projectId, V1DatastoreFactory datastoreFactory) { this.projectId = checkNotNull(projectId, "projectId"); this.datastoreFactory = datastoreFactory; } @@ -861,7 +862,7 @@ public void startBundle(Context c) { @ProcessElement public void processElement(ProcessContext c) throws Exception { mutations.add(c.element()); - if (mutations.size() >= V1Beta3.DATASTORE_BATCH_UPDATE_LIMIT) { + if (mutations.size() >= DatastoreV1.DATASTORE_BATCH_UPDATE_LIMIT) { flushBatch(); } } @@ -1006,7 +1007,7 @@ public void populateDisplayData(Builder builder) { * wrapping them under this class, which implements {@link Serializable}. */ @VisibleForTesting - static class V1Beta3DatastoreFactory implements Serializable { + static class V1DatastoreFactory implements Serializable { /** Builds a Datastore client for the given pipeline options and project. */ public Datastore getDatastore(PipelineOptions pipelineOptions, String projectId) { diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3Test.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java similarity index 82% rename from sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3Test.java rename to sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java index b0c6c185f812..31b5da43aa1e 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3Test.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java @@ -17,21 +17,21 @@ */ package org.apache.beam.sdk.io.gcp.datastore; -import static org.apache.beam.sdk.io.gcp.datastore.V1Beta3.DATASTORE_BATCH_UPDATE_LIMIT; -import static org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Read.DEFAULT_BUNDLE_SIZE_BYTES; -import static org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Read.QUERY_BATCH_LIMIT; -import static org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Read.getEstimatedSizeBytes; -import static org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Read.makeRequest; -import static org.apache.beam.sdk.io.gcp.datastore.V1Beta3.isValidKey; +import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.DATASTORE_BATCH_UPDATE_LIMIT; +import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.DEFAULT_BUNDLE_SIZE_BYTES; +import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.QUERY_BATCH_LIMIT; +import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.getEstimatedSizeBytes; +import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.makeRequest; +import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.isValidKey; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; -import static com.google.datastore.v1beta3.PropertyFilter.Operator.EQUAL; -import static com.google.datastore.v1beta3.PropertyOrder.Direction.DESCENDING; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeDelete; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeFilter; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeKey; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeOrder; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeUpsert; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeValue; +import static com.google.datastore.v1.PropertyFilter.Operator.EQUAL; +import static com.google.datastore.v1.PropertyOrder.Direction.DESCENDING; +import static com.google.datastore.v1.client.DatastoreHelper.makeDelete; +import static com.google.datastore.v1.client.DatastoreHelper.makeFilter; +import static com.google.datastore.v1.client.DatastoreHelper.makeKey; +import static com.google.datastore.v1.client.DatastoreHelper.makeOrder; +import static com.google.datastore.v1.client.DatastoreHelper.makeUpsert; +import static com.google.datastore.v1.client.DatastoreHelper.makeValue; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.lessThanOrEqualTo; @@ -47,17 +47,17 @@ import static org.mockito.Mockito.verifyZeroInteractions; import static org.mockito.Mockito.when; -import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.DatastoreWriterFn; -import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.DeleteEntity; -import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.DeleteEntityFn; -import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.DeleteKey; -import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.DeleteKeyFn; -import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Read.ReadFn; -import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Read.SplitQueryFn; -import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Read.V1Beta3Options; -import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.UpsertFn; -import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.V1Beta3DatastoreFactory; -import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Write; +import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.DatastoreWriterFn; +import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.DeleteEntity; +import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.DeleteEntityFn; +import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.DeleteKey; +import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.DeleteKeyFn; +import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.ReadFn; +import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.SplitQueryFn; +import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.V1Options; +import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.UpsertFn; +import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.V1DatastoreFactory; +import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Write; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.testing.RunnableOnService; import org.apache.beam.sdk.transforms.DoFnTester; @@ -70,18 +70,18 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.POutput; -import com.google.datastore.v1beta3.CommitRequest; -import com.google.datastore.v1beta3.Entity; -import com.google.datastore.v1beta3.EntityResult; -import com.google.datastore.v1beta3.Key; -import com.google.datastore.v1beta3.Mutation; -import com.google.datastore.v1beta3.PartitionId; -import com.google.datastore.v1beta3.Query; -import com.google.datastore.v1beta3.QueryResultBatch; -import com.google.datastore.v1beta3.RunQueryRequest; -import com.google.datastore.v1beta3.RunQueryResponse; -import com.google.datastore.v1beta3.client.Datastore; -import com.google.datastore.v1beta3.client.QuerySplitter; +import com.google.datastore.v1.CommitRequest; +import com.google.datastore.v1.Entity; +import com.google.datastore.v1.EntityResult; +import com.google.datastore.v1.Key; +import com.google.datastore.v1.Mutation; +import com.google.datastore.v1.PartitionId; +import com.google.datastore.v1.Query; +import com.google.datastore.v1.QueryResultBatch; +import com.google.datastore.v1.RunQueryRequest; +import com.google.datastore.v1.RunQueryResponse; +import com.google.datastore.v1.client.Datastore; +import com.google.datastore.v1.client.QuerySplitter; import com.google.protobuf.Int32Value; import org.junit.Before; @@ -103,29 +103,29 @@ import java.util.Set; /** - * Tests for {@link V1Beta3}. + * Tests for {@link DatastoreV1}. */ @RunWith(JUnit4.class) -public class V1Beta3Test { +public class DatastoreV1Test { private static final String PROJECT_ID = "testProject"; private static final String NAMESPACE = "testNamespace"; private static final String KIND = "testKind"; private static final Query QUERY; - private static final V1Beta3Options v1Beta3Options; + private static final V1Options V_1_OPTIONS; static { Query.Builder q = Query.newBuilder(); q.addKindBuilder().setName(KIND); QUERY = q.build(); - v1Beta3Options = V1Beta3Options.from(PROJECT_ID, QUERY, NAMESPACE); + V_1_OPTIONS = V1Options.from(PROJECT_ID, QUERY, NAMESPACE); } - private V1Beta3.Read initialRead; + private DatastoreV1.Read initialRead; @Mock Datastore mockDatastore; @Mock QuerySplitter mockQuerySplitter; @Mock - V1Beta3DatastoreFactory mockDatastoreFactory; + V1DatastoreFactory mockDatastoreFactory; @Rule public final ExpectedException thrown = ExpectedException.none(); @@ -134,7 +134,7 @@ public class V1Beta3Test { public void setUp() { MockitoAnnotations.initMocks(this); - initialRead = DatastoreIO.v1beta3().read() + initialRead = DatastoreIO.v1().read() .withProjectId(PROJECT_ID).withQuery(QUERY).withNamespace(NAMESPACE); when(mockDatastoreFactory.getDatastore(any(PipelineOptions.class), any(String.class))) @@ -145,7 +145,7 @@ public void setUp() { @Test public void testBuildRead() throws Exception { - V1Beta3.Read read = DatastoreIO.v1beta3().read() + DatastoreV1.Read read = DatastoreIO.v1().read() .withProjectId(PROJECT_ID).withQuery(QUERY).withNamespace(NAMESPACE); assertEquals(QUERY, read.getQuery()); assertEquals(PROJECT_ID, read.getProjectId()); @@ -157,7 +157,7 @@ public void testBuildRead() throws Exception { */ @Test public void testBuildReadAlt() throws Exception { - V1Beta3.Read read = DatastoreIO.v1beta3().read() + DatastoreV1.Read read = DatastoreIO.v1().read() .withProjectId(PROJECT_ID).withNamespace(NAMESPACE).withQuery(QUERY); assertEquals(QUERY, read.getQuery()); assertEquals(PROJECT_ID, read.getProjectId()); @@ -166,7 +166,7 @@ public void testBuildReadAlt() throws Exception { @Test public void testReadValidationFailsProject() throws Exception { - V1Beta3.Read read = DatastoreIO.v1beta3().read().withQuery(QUERY); + DatastoreV1.Read read = DatastoreIO.v1().read().withQuery(QUERY); thrown.expect(NullPointerException.class); thrown.expectMessage("project"); read.validate(null); @@ -174,7 +174,7 @@ public void testReadValidationFailsProject() throws Exception { @Test public void testReadValidationFailsQuery() throws Exception { - V1Beta3.Read read = DatastoreIO.v1beta3().read().withProjectId(PROJECT_ID); + DatastoreV1.Read read = DatastoreIO.v1().read().withProjectId(PROJECT_ID); thrown.expect(NullPointerException.class); thrown.expectMessage("query"); read.validate(null); @@ -186,7 +186,7 @@ public void testReadValidationFailsQueryLimitZero() throws Exception { thrown.expect(IllegalArgumentException.class); thrown.expectMessage("Invalid query limit 0: must be positive"); - DatastoreIO.v1beta3().read().withQuery(invalidLimit); + DatastoreIO.v1().read().withQuery(invalidLimit); } @Test @@ -195,22 +195,22 @@ public void testReadValidationFailsQueryLimitNegative() throws Exception { thrown.expect(IllegalArgumentException.class); thrown.expectMessage("Invalid query limit -5: must be positive"); - DatastoreIO.v1beta3().read().withQuery(invalidLimit); + DatastoreIO.v1().read().withQuery(invalidLimit); } @Test public void testReadValidationSucceedsNamespace() throws Exception { - V1Beta3.Read read = DatastoreIO.v1beta3().read().withProjectId(PROJECT_ID).withQuery(QUERY); + DatastoreV1.Read read = DatastoreIO.v1().read().withProjectId(PROJECT_ID).withQuery(QUERY); /* Should succeed, as a null namespace is fine. */ read.validate(null); } @Test public void testReadDisplayData() { - V1Beta3.Read read = DatastoreIO.v1beta3().read() - .withProjectId(PROJECT_ID) - .withQuery(QUERY) - .withNamespace(NAMESPACE); + DatastoreV1.Read read = DatastoreIO.v1().read() + .withProjectId(PROJECT_ID) + .withQuery(QUERY) + .withNamespace(NAMESPACE); DisplayData displayData = DisplayData.from(read); @@ -223,7 +223,7 @@ public void testReadDisplayData() { @Category(RunnableOnService.class) public void testSourcePrimitiveDisplayData() { DisplayDataEvaluator evaluator = DisplayDataEvaluator.create(); - PTransform read = DatastoreIO.v1beta3().read().withProjectId( + PTransform read = DatastoreIO.v1().read().withProjectId( "myProject").withQuery(Query.newBuilder().build()); Set displayData = evaluator.displayDataForPrimitiveSourceTransforms(read); @@ -236,12 +236,12 @@ public void testWriteDoesNotAllowNullProject() throws Exception { thrown.expect(NullPointerException.class); thrown.expectMessage("projectId"); - DatastoreIO.v1beta3().write().withProjectId(null); + DatastoreIO.v1().write().withProjectId(null); } @Test public void testWriteValidationFailsWithNoProject() throws Exception { - Write write = DatastoreIO.v1beta3().write(); + Write write = DatastoreIO.v1().write(); thrown.expect(NullPointerException.class); thrown.expectMessage("projectId"); @@ -251,13 +251,13 @@ public void testWriteValidationFailsWithNoProject() throws Exception { @Test public void testWriteValidationSucceedsWithProject() throws Exception { - Write write = DatastoreIO.v1beta3().write().withProjectId(PROJECT_ID); + Write write = DatastoreIO.v1().write().withProjectId(PROJECT_ID); write.validate(null); } @Test public void testWriteDisplayData() { - Write write = DatastoreIO.v1beta3().write().withProjectId(PROJECT_ID); + Write write = DatastoreIO.v1().write().withProjectId(PROJECT_ID); DisplayData displayData = DisplayData.from(write); @@ -269,12 +269,12 @@ public void testDeleteEntityDoesNotAllowNullProject() throws Exception { thrown.expect(NullPointerException.class); thrown.expectMessage("projectId"); - DatastoreIO.v1beta3().deleteEntity().withProjectId(null); + DatastoreIO.v1().deleteEntity().withProjectId(null); } @Test public void testDeleteEntityValidationFailsWithNoProject() throws Exception { - DeleteEntity deleteEntity = DatastoreIO.v1beta3().deleteEntity(); + DeleteEntity deleteEntity = DatastoreIO.v1().deleteEntity(); thrown.expect(NullPointerException.class); thrown.expectMessage("projectId"); @@ -284,13 +284,13 @@ public void testDeleteEntityValidationFailsWithNoProject() throws Exception { @Test public void testDeleteEntityValidationSucceedsWithProject() throws Exception { - DeleteEntity deleteEntity = DatastoreIO.v1beta3().deleteEntity().withProjectId(PROJECT_ID); + DeleteEntity deleteEntity = DatastoreIO.v1().deleteEntity().withProjectId(PROJECT_ID); deleteEntity.validate(null); } @Test public void testDeleteEntityDisplayData() { - DeleteEntity deleteEntity = DatastoreIO.v1beta3().deleteEntity().withProjectId(PROJECT_ID); + DeleteEntity deleteEntity = DatastoreIO.v1().deleteEntity().withProjectId(PROJECT_ID); DisplayData displayData = DisplayData.from(deleteEntity); @@ -302,12 +302,12 @@ public void testDeleteKeyDoesNotAllowNullProject() throws Exception { thrown.expect(NullPointerException.class); thrown.expectMessage("projectId"); - DatastoreIO.v1beta3().deleteKey().withProjectId(null); + DatastoreIO.v1().deleteKey().withProjectId(null); } @Test public void testDeleteKeyValidationFailsWithNoProject() throws Exception { - DeleteKey deleteKey = DatastoreIO.v1beta3().deleteKey(); + DeleteKey deleteKey = DatastoreIO.v1().deleteKey(); thrown.expect(NullPointerException.class); thrown.expectMessage("projectId"); @@ -317,13 +317,13 @@ public void testDeleteKeyValidationFailsWithNoProject() throws Exception { @Test public void testDeleteKeyValidationSucceedsWithProject() throws Exception { - DeleteKey deleteKey = DatastoreIO.v1beta3().deleteKey().withProjectId(PROJECT_ID); + DeleteKey deleteKey = DatastoreIO.v1().deleteKey().withProjectId(PROJECT_ID); deleteKey.validate(null); } @Test public void testDeleteKeyDisplayData() { - DeleteKey deleteKey = DatastoreIO.v1beta3().deleteKey().withProjectId(PROJECT_ID); + DeleteKey deleteKey = DatastoreIO.v1().deleteKey().withProjectId(PROJECT_ID); DisplayData displayData = DisplayData.from(deleteKey); @@ -335,7 +335,7 @@ public void testDeleteKeyDisplayData() { public void testWritePrimitiveDisplayData() { DisplayDataEvaluator evaluator = DisplayDataEvaluator.create(); PTransform, ?> write = - DatastoreIO.v1beta3().write().withProjectId("myProject"); + DatastoreIO.v1().write().withProjectId("myProject"); Set displayData = evaluator.displayDataForPrimitiveTransforms(write); assertThat("DatastoreIO write should include the project in its primitive display data", @@ -350,7 +350,7 @@ public void testWritePrimitiveDisplayData() { public void testDeleteEntityPrimitiveDisplayData() { DisplayDataEvaluator evaluator = DisplayDataEvaluator.create(); PTransform, ?> write = - DatastoreIO.v1beta3().deleteEntity().withProjectId("myProject"); + DatastoreIO.v1().deleteEntity().withProjectId("myProject"); Set displayData = evaluator.displayDataForPrimitiveTransforms(write); assertThat("DatastoreIO write should include the project in its primitive display data", @@ -365,7 +365,7 @@ public void testDeleteEntityPrimitiveDisplayData() { public void testDeleteKeyPrimitiveDisplayData() { DisplayDataEvaluator evaluator = DisplayDataEvaluator.create(); PTransform, ?> write = - DatastoreIO.v1beta3().deleteKey().withProjectId("myProject"); + DatastoreIO.v1().deleteKey().withProjectId("myProject"); Set displayData = evaluator.displayDataForPrimitiveTransforms(write); assertThat("DatastoreIO write should include the project in its primitive display data", @@ -380,7 +380,7 @@ public void testDeleteKeyPrimitiveDisplayData() { */ @Test public void testBuildWrite() throws Exception { - V1Beta3.Write write = DatastoreIO.v1beta3().write().withProjectId(PROJECT_ID); + DatastoreV1.Write write = DatastoreIO.v1().write().withProjectId(PROJECT_ID); assertEquals(PROJECT_ID, write.getProjectId()); } @@ -550,7 +550,7 @@ private void datastoreWriterFnTest(int numMutations) throws Exception { int end = Math.min(numMutations, start + DATASTORE_BATCH_UPDATE_LIMIT); CommitRequest.Builder commitRequest = CommitRequest.newBuilder(); commitRequest.setMode(CommitRequest.Mode.NON_TRANSACTIONAL); - commitRequest.addAllMutations(mutations.subList(start, end)); + commitRequest.addAllMutations(mutations.subList(start, end)); // Verify all the batch requests were made with the expected mutations. verify(mockDatastore, times(1)).commit(commitRequest.build()); start = end; @@ -558,7 +558,7 @@ private void datastoreWriterFnTest(int numMutations) throws Exception { } /** - * Tests {@link V1Beta3.Read#getEstimatedSizeBytes} to fetch and return estimated size for a + * Tests {@link DatastoreV1.Read#getEstimatedSizeBytes} to fetch and return estimated size for a * query. */ @Test @@ -585,7 +585,7 @@ public void testSplitQueryFnWithNumSplits() throws Exception { eq(QUERY), any(PartitionId.class), eq(numSplits), any(Datastore.class))) .thenReturn(splitQuery(QUERY, numSplits)); - SplitQueryFn splitQueryFn = new SplitQueryFn(v1Beta3Options, numSplits, mockDatastoreFactory); + SplitQueryFn splitQueryFn = new SplitQueryFn(V_1_OPTIONS, numSplits, mockDatastoreFactory); DoFnTester> doFnTester = DoFnTester.of(splitQueryFn); /** * Although Datastore client is marked transient in {@link SplitQueryFn}, when injected through @@ -623,7 +623,7 @@ public void testSplitQueryFnWithoutNumSplits() throws Exception { eq(QUERY), any(PartitionId.class), eq(expectedNumSplits), any(Datastore.class))) .thenReturn(splitQuery(QUERY, expectedNumSplits)); - SplitQueryFn splitQueryFn = new SplitQueryFn(v1Beta3Options, numSplits, mockDatastoreFactory); + SplitQueryFn splitQueryFn = new SplitQueryFn(V_1_OPTIONS, numSplits, mockDatastoreFactory); DoFnTester> doFnTester = DoFnTester.of(splitQueryFn); doFnTester.setCloningBehavior(CloningBehavior.DO_NOT_CLONE); List> queries = doFnTester.processBundle(QUERY); @@ -636,7 +636,7 @@ public void testSplitQueryFnWithoutNumSplits() throws Exception { } /** - * Tests {@link V1Beta3.Read.SplitQueryFn} when the query has a user specified limit. + * Tests {@link DatastoreV1.Read.SplitQueryFn} when the query has a user specified limit. */ @Test public void testSplitQueryFnWithQueryLimit() throws Exception { @@ -644,7 +644,7 @@ public void testSplitQueryFnWithQueryLimit() throws Exception { .setLimit(Int32Value.newBuilder().setValue(1)) .build(); - SplitQueryFn splitQueryFn = new SplitQueryFn(v1Beta3Options, 10, mockDatastoreFactory); + SplitQueryFn splitQueryFn = new SplitQueryFn(V_1_OPTIONS, 10, mockDatastoreFactory); DoFnTester> doFnTester = DoFnTester.of(splitQueryFn); doFnTester.setCloningBehavior(CloningBehavior.DO_NOT_CLONE); List> queries = doFnTester.processBundle(queryWithLimit); @@ -687,10 +687,10 @@ private void verifyUniqueKeys(List> queries) { /** * A helper function that creates mock {@link Entity} results in response to a query. Always * indicates that more results are available, unless the batch is limited to fewer than - * {@link V1Beta3.Read#QUERY_BATCH_LIMIT} results. + * {@link DatastoreV1.Read#QUERY_BATCH_LIMIT} results. */ private static RunQueryResponse mockResponseForQuery(Query q) { - // Every query V1Beta3 sends should have a limit. + // Every query DatastoreV1 sends should have a limit. assertTrue(q.hasLimit()); // The limit should be in the range [1, QUERY_BATCH_LIMIT] @@ -736,7 +736,7 @@ public RunQueryResponse answer(InvocationOnMock invocationOnMock) throws Throwab } }); - ReadFn readFn = new ReadFn(v1Beta3Options, mockDatastoreFactory); + ReadFn readFn = new ReadFn(V_1_OPTIONS, mockDatastoreFactory); DoFnTester doFnTester = DoFnTester.of(readFn); /** * Although Datastore client is marked transient in {@link ReadFn}, when injected through diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3ReadIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1ReadIT.java similarity index 69% rename from sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3ReadIT.java rename to sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1ReadIT.java index ddb6d81ae995..8fedc774f661 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3ReadIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1ReadIT.java @@ -18,24 +18,24 @@ package org.apache.beam.sdk.io.gcp.datastore; -import static org.apache.beam.sdk.io.gcp.datastore.V1Beta3TestUtil.deleteAllEntities; -import static org.apache.beam.sdk.io.gcp.datastore.V1Beta3TestUtil.getDatastore; -import static org.apache.beam.sdk.io.gcp.datastore.V1Beta3TestUtil.makeAncestorKey; -import static org.apache.beam.sdk.io.gcp.datastore.V1Beta3TestUtil.makeEntity; +import static org.apache.beam.sdk.io.gcp.datastore.V1TestUtil.deleteAllEntities; +import static org.apache.beam.sdk.io.gcp.datastore.V1TestUtil.getDatastore; +import static org.apache.beam.sdk.io.gcp.datastore.V1TestUtil.makeAncestorKey; +import static org.apache.beam.sdk.io.gcp.datastore.V1TestUtil.makeEntity; import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.io.gcp.datastore.V1Beta3TestUtil.UpsertMutationBuilder; -import org.apache.beam.sdk.io.gcp.datastore.V1Beta3TestUtil.V1Beta3TestWriter; +import org.apache.beam.sdk.io.gcp.datastore.V1TestUtil.UpsertMutationBuilder; +import org.apache.beam.sdk.io.gcp.datastore.V1TestUtil.V1TestWriter; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.values.PCollection; -import com.google.datastore.v1beta3.Entity; -import com.google.datastore.v1beta3.Key; -import com.google.datastore.v1beta3.Query; -import com.google.datastore.v1beta3.client.Datastore; +import com.google.datastore.v1.Entity; +import com.google.datastore.v1.Key; +import com.google.datastore.v1.Query; +import com.google.datastore.v1.client.Datastore; import org.junit.After; import org.junit.Before; @@ -46,38 +46,38 @@ import java.util.UUID; /** - * End-to-end tests for Datastore V1Beta3.Read. + * End-to-end tests for Datastore DatastoreV1.Read. */ @RunWith(JUnit4.class) -public class V1Beta3ReadIT { - private V1Beta3TestOptions options; +public class V1ReadIT { + private V1TestOptions options; private String ancestor; private final long numEntities = 1000; @Before public void setup() { - PipelineOptionsFactory.register(V1Beta3TestOptions.class); - options = TestPipeline.testingPipelineOptions().as(V1Beta3TestOptions.class); + PipelineOptionsFactory.register(V1TestOptions.class); + options = TestPipeline.testingPipelineOptions().as(V1TestOptions.class); ancestor = UUID.randomUUID().toString(); } /** - * An end-to-end test for {@link V1Beta3.Read}. + * An end-to-end test for {@link DatastoreV1.Read}. * * Write some test entities to datastore and then run a dataflow pipeline that * reads and counts the total number of entities. Verify that the count matches the * number of entities written. */ @Test - public void testE2EV1Beta3Read() throws Exception { + public void testE2EV1Read() throws Exception { // Create entities and write them to datastore writeEntitiesToDatastore(options, ancestor, numEntities); // Read from datastore - Query query = V1Beta3TestUtil.makeAncestorKindQuery( + Query query = V1TestUtil.makeAncestorKindQuery( options.getKind(), options.getNamespace(), ancestor); - V1Beta3.Read read = DatastoreIO.v1beta3().read() + DatastoreV1.Read read = DatastoreIO.v1().read() .withProjectId(options.getProject()) .withQuery(query) .withNamespace(options.getNamespace()); @@ -93,11 +93,11 @@ public void testE2EV1Beta3Read() throws Exception { } // Creates entities and write them to datastore - private static void writeEntitiesToDatastore(V1Beta3TestOptions options, String ancestor, + private static void writeEntitiesToDatastore(V1TestOptions options, String ancestor, long numEntities) throws Exception { Datastore datastore = getDatastore(options, options.getProject()); // Write test entities to datastore - V1Beta3TestWriter writer = new V1Beta3TestWriter(datastore, new UpsertMutationBuilder()); + V1TestWriter writer = new V1TestWriter(datastore, new UpsertMutationBuilder()); Key ancestorKey = makeAncestorKey(options.getNamespace(), options.getKind(), ancestor); for (long i = 0; i < numEntities; i++) { diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3TestOptions.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestOptions.java similarity index 92% rename from sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3TestOptions.java rename to sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestOptions.java index 099ebe05c9b7..360855fee0f2 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3TestOptions.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestOptions.java @@ -25,9 +25,9 @@ import javax.annotation.Nullable; /** - * V1Beta3 Datastore related pipeline options. + * DatastoreV1 Datastore related pipeline options. */ -public interface V1Beta3TestOptions extends TestPipelineOptions { +public interface V1TestOptions extends TestPipelineOptions { @Description("Project ID to read from datastore") @Default.String("apache-beam-testing") String getProject(); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3TestUtil.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java similarity index 82% rename from sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3TestUtil.java rename to sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java index 7eaf23e80568..1e323ec06a28 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3TestUtil.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java @@ -18,12 +18,12 @@ package org.apache.beam.sdk.io.gcp.datastore; -import static com.google.datastore.v1beta3.QueryResultBatch.MoreResultsType.NOT_FINISHED; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeDelete; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeFilter; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeKey; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeUpsert; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeValue; +import static com.google.datastore.v1.QueryResultBatch.MoreResultsType.NOT_FINISHED; +import static com.google.datastore.v1.client.DatastoreHelper.makeDelete; +import static com.google.datastore.v1.client.DatastoreHelper.makeFilter; +import static com.google.datastore.v1.client.DatastoreHelper.makeKey; +import static com.google.datastore.v1.client.DatastoreHelper.makeUpsert; +import static com.google.datastore.v1.client.DatastoreHelper.makeValue; import org.apache.beam.sdk.options.GcpOptions; import org.apache.beam.sdk.options.PipelineOptions; @@ -35,21 +35,21 @@ import com.google.api.client.util.BackOff; import com.google.api.client.util.BackOffUtils; import com.google.api.client.util.Sleeper; -import com.google.datastore.v1beta3.CommitRequest; -import com.google.datastore.v1beta3.Entity; -import com.google.datastore.v1beta3.EntityResult; -import com.google.datastore.v1beta3.Key; -import com.google.datastore.v1beta3.Key.PathElement; -import com.google.datastore.v1beta3.Mutation; -import com.google.datastore.v1beta3.PropertyFilter; -import com.google.datastore.v1beta3.Query; -import com.google.datastore.v1beta3.QueryResultBatch; -import com.google.datastore.v1beta3.RunQueryRequest; -import com.google.datastore.v1beta3.RunQueryResponse; -import com.google.datastore.v1beta3.client.Datastore; -import com.google.datastore.v1beta3.client.DatastoreException; -import com.google.datastore.v1beta3.client.DatastoreFactory; -import com.google.datastore.v1beta3.client.DatastoreOptions; +import com.google.datastore.v1.CommitRequest; +import com.google.datastore.v1.Entity; +import com.google.datastore.v1.EntityResult; +import com.google.datastore.v1.Key; +import com.google.datastore.v1.Key.PathElement; +import com.google.datastore.v1.Mutation; +import com.google.datastore.v1.PropertyFilter; +import com.google.datastore.v1.Query; +import com.google.datastore.v1.QueryResultBatch; +import com.google.datastore.v1.RunQueryRequest; +import com.google.datastore.v1.RunQueryResponse; +import com.google.datastore.v1.client.Datastore; +import com.google.datastore.v1.client.DatastoreException; +import com.google.datastore.v1.client.DatastoreFactory; +import com.google.datastore.v1.client.DatastoreOptions; import com.google.protobuf.Int32Value; import org.slf4j.Logger; @@ -62,8 +62,8 @@ import java.util.UUID; import javax.annotation.Nullable; -class V1Beta3TestUtil { - private static final Logger LOG = LoggerFactory.getLogger(V1Beta3TestUtil.class); +class V1TestUtil { + private static final Logger LOG = LoggerFactory.getLogger(V1TestUtil.class); /** * A helper function to create the ancestor key for all created and queried entities. @@ -161,13 +161,13 @@ private static RunQueryRequest makeRequest(Query query, @Nullable String namespa /** * Delete all entities with the given ancestor. */ - static void deleteAllEntities(V1Beta3TestOptions options, String ancestor) throws Exception { + static void deleteAllEntities(V1TestOptions options, String ancestor) throws Exception { Datastore datastore = getDatastore(options, options.getProject()); - Query query = V1Beta3TestUtil.makeAncestorKindQuery( + Query query = V1TestUtil.makeAncestorKindQuery( options.getKind(), options.getNamespace(), ancestor); - V1Beta3TestReader reader = new V1Beta3TestReader(datastore, query, options.getNamespace()); - V1Beta3TestWriter writer = new V1Beta3TestWriter(datastore, new DeleteMutationBuilder()); + V1TestReader reader = new V1TestReader(datastore, query, options.getNamespace()); + V1TestWriter writer = new V1TestWriter(datastore, new DeleteMutationBuilder()); long numEntities = 0; while (reader.advance()) { @@ -183,13 +183,13 @@ static void deleteAllEntities(V1Beta3TestOptions options, String ancestor) throw /** * Returns the total number of entities for the given datastore. */ - static long countEntities(V1Beta3TestOptions options, String ancestor) throws Exception { + static long countEntities(V1TestOptions options, String ancestor) throws Exception { // Read from datastore. - Datastore datastore = V1Beta3TestUtil.getDatastore(options, options.getProject()); - Query query = V1Beta3TestUtil.makeAncestorKindQuery( + Datastore datastore = V1TestUtil.getDatastore(options, options.getProject()); + Query query = V1TestUtil.makeAncestorKindQuery( options.getKind(), options.getNamespace(), ancestor); - V1Beta3TestReader reader = new V1Beta3TestReader(datastore, query, options.getNamespace()); + V1TestReader reader = new V1TestReader(datastore, query, options.getNamespace()); long numEntitiesRead = 0; while (reader.advance()) { @@ -228,8 +228,8 @@ public Mutation.Builder apply(Entity entity) { /** * A helper class to write entities to datastore. */ - static class V1Beta3TestWriter { - private static final Logger LOG = LoggerFactory.getLogger(V1Beta3TestWriter.class); + static class V1TestWriter { + private static final Logger LOG = LoggerFactory.getLogger(V1TestWriter.class); // Limits the number of entities updated per batch private static final int DATASTORE_BATCH_UPDATE_LIMIT = 500; // Number of times to retry on update failure @@ -252,7 +252,7 @@ static boolean isValidKey(Key key) { private final MutationBuilder mutationBuilder; private final List entities = new ArrayList<>(); - V1Beta3TestWriter(Datastore datastore, MutationBuilder mutationBuilder) { + V1TestWriter(Datastore datastore, MutationBuilder mutationBuilder) { this.datastore = datastore; this.mutationBuilder = mutationBuilder; } @@ -312,7 +312,7 @@ private void flushBatch() throws DatastoreException, IOException, InterruptedExc /** * A helper class to read entities from datastore. */ - static class V1Beta3TestReader { + static class V1TestReader { private static final int QUERY_BATCH_LIMIT = 500; private final Datastore datastore; private final Query query; @@ -324,7 +324,7 @@ static class V1Beta3TestReader { private QueryResultBatch currentBatch; private Entity currentEntity; - V1Beta3TestReader(Datastore datastore, Query query, @Nullable String namespace) { + V1TestReader(Datastore datastore, Query query, @Nullable String namespace) { this.datastore = datastore; this.query = query; this.namespace = namespace; @@ -368,7 +368,7 @@ private Iterator getIteratorAndMoveCursor() throws DatastoreExcept int numFetch = currentBatch.getEntityResultsCount(); // All indications from the API are that there are/may be more results. moreResults = ((numFetch == QUERY_BATCH_LIMIT) - || (currentBatch.getMoreResults() == NOT_FINISHED)); + || (currentBatch.getMoreResults() == NOT_FINISHED)); // May receive a batch of 0 results if the number of records is a multiple // of the request limit. diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3WriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1WriteIT.java similarity index 76% rename from sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3WriteIT.java rename to sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1WriteIT.java index 782065fab101..b97c05c8c6c4 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3WriteIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1WriteIT.java @@ -18,13 +18,13 @@ package org.apache.beam.sdk.io.gcp.datastore; -import static org.apache.beam.sdk.io.gcp.datastore.V1Beta3TestUtil.countEntities; -import static org.apache.beam.sdk.io.gcp.datastore.V1Beta3TestUtil.deleteAllEntities; +import static org.apache.beam.sdk.io.gcp.datastore.V1TestUtil.countEntities; +import static org.apache.beam.sdk.io.gcp.datastore.V1TestUtil.deleteAllEntities; import static org.junit.Assert.assertEquals; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.CountingInput; -import org.apache.beam.sdk.io.gcp.datastore.V1Beta3TestUtil.CreateEntityFn; +import org.apache.beam.sdk.io.gcp.datastore.V1TestUtil.CreateEntityFn; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.ParDo; @@ -38,37 +38,37 @@ import java.util.UUID; /** - * End-to-end tests for Datastore V1Beta3.Write. + * End-to-end tests for Datastore DatastoreV1.Write. */ @RunWith(JUnit4.class) -public class V1Beta3WriteIT { - private V1Beta3TestOptions options; +public class V1WriteIT { + private V1TestOptions options; private String ancestor; private final long numEntities = 1000; @Before public void setup() { - PipelineOptionsFactory.register(V1Beta3TestOptions.class); - options = TestPipeline.testingPipelineOptions().as(V1Beta3TestOptions.class); + PipelineOptionsFactory.register(V1TestOptions.class); + options = TestPipeline.testingPipelineOptions().as(V1TestOptions.class); ancestor = UUID.randomUUID().toString(); } /** - * An end-to-end test for {@link V1Beta3.Write}. + * An end-to-end test for {@link DatastoreV1.Write}. * * Write some test entities to datastore through a dataflow pipeline. * Read and count all the entities. Verify that the count matches the * number of entities written. */ @Test - public void testE2EV1Beta3Write() throws Exception { + public void testE2EV1Write() throws Exception { Pipeline p = Pipeline.create(options); // Write to datastore p.apply(CountingInput.upTo(numEntities)) .apply(ParDo.of(new CreateEntityFn( options.getKind(), options.getNamespace(), ancestor))) - .apply(DatastoreIO.v1beta3().write().withProjectId(options.getProject())); + .apply(DatastoreIO.v1().write().withProjectId(options.getProject())); p.run(); From b562072b868bff14d7c33a64d05056c604710884 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Mon, 22 Aug 2016 09:23:57 -0700 Subject: [PATCH 040/112] Remove unused constant in ExecutorServiceParallelExecutor --- .../beam/runners/direct/ExecutorServiceParallelExecutor.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java index 8c6c6eddb3b3..35b6239de81c 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java @@ -338,8 +338,6 @@ public boolean isDone() { } private class MonitorRunnable implements Runnable { - // arbitrary termination condition to ensure progress in the presence of pushback - private final long maxTimeProcessingUpdatesNanos = TimeUnit.MILLISECONDS.toNanos(5L); private final String runnableName = String.format("%s$%s-monitor", evaluationContext.getPipelineOptions().getAppName(), ExecutorServiceParallelExecutor.class.getSimpleName()); From a65be9f92c26ffbd00a72a644d619e636ba04b6e Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Mon, 22 Aug 2016 10:05:35 -0700 Subject: [PATCH 041/112] Remove extra timer firings in WatermarkManager These timers should not be fired - the windows should be expired via the GC timer, and any elements should be emitted if neccessary. --- .../beam/runners/direct/WatermarkManager.java | 15 ++++----------- 1 file changed, 4 insertions(+), 11 deletions(-) diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java index c8dfa8c914c4..a44fa501c434 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java @@ -1139,17 +1139,10 @@ private Map, FiredTimers> extractFiredTimers() { inputWatermark.extractFiredEventTimeTimers(); Map, List> processingTimers; Map, List> synchronizedTimers; - if (inputWatermark.get().equals(BoundedWindow.TIMESTAMP_MAX_VALUE)) { - processingTimers = synchronizedProcessingInputWatermark.extractFiredDomainTimers( - TimeDomain.PROCESSING_TIME, BoundedWindow.TIMESTAMP_MAX_VALUE); - synchronizedTimers = synchronizedProcessingInputWatermark.extractFiredDomainTimers( - TimeDomain.PROCESSING_TIME, BoundedWindow.TIMESTAMP_MAX_VALUE); - } else { - processingTimers = synchronizedProcessingInputWatermark.extractFiredDomainTimers( - TimeDomain.PROCESSING_TIME, clock.now()); - synchronizedTimers = synchronizedProcessingInputWatermark.extractFiredDomainTimers( - TimeDomain.SYNCHRONIZED_PROCESSING_TIME, getSynchronizedProcessingInputTime()); - } + processingTimers = synchronizedProcessingInputWatermark.extractFiredDomainTimers( + TimeDomain.PROCESSING_TIME, clock.now()); + synchronizedTimers = synchronizedProcessingInputWatermark.extractFiredDomainTimers( + TimeDomain.SYNCHRONIZED_PROCESSING_TIME, getSynchronizedProcessingInputTime()); Map, Map>> groupedTimers = new HashMap<>(); groupFiredTimers(groupedTimers, eventTimeTimers, processingTimers, synchronizedTimers); From f08f21cdf4c067745a10b31a6481ed470f97dadc Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Sun, 14 Aug 2016 23:08:21 -0700 Subject: [PATCH 042/112] FileBasedSink: improve parallelism in GCS copy/remove --- .../org/apache/beam/sdk/util/GcsUtil.java | 241 ++++++++---------- .../org/apache/beam/sdk/util/GcsUtilTest.java | 69 +++++ 2 files changed, 178 insertions(+), 132 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java index 4e9ee6e62ec8..06685e577bf9 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java @@ -30,11 +30,9 @@ import com.google.api.client.googleapis.json.GoogleJsonError; import com.google.api.client.googleapis.json.GoogleJsonResponseException; import com.google.api.client.http.HttpHeaders; -import com.google.api.client.http.HttpRequestInitializer; import com.google.api.client.util.BackOff; import com.google.api.client.util.Sleeper; import com.google.api.services.storage.Storage; -import com.google.api.services.storage.StorageRequest; import com.google.api.services.storage.model.Objects; import com.google.api.services.storage.model.StorageObject; import com.google.cloud.hadoop.gcsio.GoogleCloudStorageReadChannel; @@ -47,6 +45,11 @@ import com.google.cloud.hadoop.util.RetryDeterminer; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -59,12 +62,16 @@ import java.util.Collections; import java.util.LinkedList; import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; import java.util.regex.Matcher; import java.util.regex.Pattern; import javax.annotation.Nullable; -import javax.annotation.concurrent.NotThreadSafe; /** * Provides operations on GCS. @@ -110,7 +117,11 @@ public GcsUtil create(PipelineOptions options) { /** * Maximum number of requests permitted in a GCS batch request. */ - private static final int MAX_REQUESTS_PER_BATCH = 1000; + private static final int MAX_REQUESTS_PER_BATCH = 100; + /** + * Maximum number of concurrent batches of requests executing on GCS. + */ + private static final int MAX_CONCURRENT_BATCHES = 256; ///////////////////////////////////////////////////////////////////////////// @@ -125,7 +136,6 @@ public GcsUtil create(PipelineOptions options) { // Exposed for testing. final ExecutorService executorService; - private final BatchHelper batchHelper; /** * Returns true if the given GCS pattern is supported otherwise fails with an * exception. @@ -145,8 +155,6 @@ private GcsUtil( this.storageClient = storageClient; this.uploadBufferSizeBytes = uploadBufferSizeBytes; this.executorService = executorService; - this.batchHelper = new BatchHelper( - storageClient.getRequestFactory().getInitializer(), storageClient, MAX_REQUESTS_PER_BATCH); } // Use this only for testing purposes. @@ -372,154 +380,123 @@ public boolean shouldRetry(IOException e) { } } + private static void executeBatches(List batches) throws IOException { + ListeningExecutorService executor = MoreExecutors.listeningDecorator( + MoreExecutors.getExitingExecutorService( + new ThreadPoolExecutor(MAX_CONCURRENT_BATCHES, MAX_CONCURRENT_BATCHES, + 0L, TimeUnit.MILLISECONDS, + new LinkedBlockingQueue()))); + + List> futures = new LinkedList<>(); + for (final BatchRequest batch : batches) { + futures.add(executor.submit(new Callable() { + public Void call() throws IOException { + batch.execute(); + return null; + } + })); + } + + try { + Futures.allAsList(futures).get(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IOException("Interrupted while executing batch GCS request", e); + } catch (ExecutionException e) { + throw new IOException("Error executing batch GCS request", e); + } finally { + executor.shutdown(); + } + } + public void copy(List srcFilenames, List destFilenames) throws IOException { + executeBatches(makeCopyBatches(srcFilenames, destFilenames)); + } + + List makeCopyBatches(List srcFilenames, List destFilenames) + throws IOException { checkArgument( srcFilenames.size() == destFilenames.size(), "Number of source files %s must equal number of destination files %s", srcFilenames.size(), destFilenames.size()); + + List batches = new LinkedList<>(); + BatchRequest batch = storageClient.batch(); for (int i = 0; i < srcFilenames.size(); i++) { final GcsPath sourcePath = GcsPath.fromUri(srcFilenames.get(i)); final GcsPath destPath = GcsPath.fromUri(destFilenames.get(i)); - LOG.debug("Copying {} to {}", sourcePath, destPath); - Storage.Objects.Copy copyObject = storageClient.objects().copy(sourcePath.getBucket(), - sourcePath.getObject(), destPath.getBucket(), destPath.getObject(), null); - batchHelper.queue(copyObject, new JsonBatchCallback() { - @Override - public void onSuccess(StorageObject obj, HttpHeaders responseHeaders) { - LOG.debug("Successfully copied {} to {}", sourcePath, destPath); - } - - @Override - public void onFailure(GoogleJsonError e, HttpHeaders responseHeaders) throws IOException { - // Do nothing on item not found. - if (!errorExtractor.itemNotFound(e)) { - throw new IOException(e.toString()); - } - LOG.debug("{} does not exist.", sourcePath); - } - }); + enqueueCopy(sourcePath, destPath, batch); + if (batch.size() >= MAX_REQUESTS_PER_BATCH) { + batches.add(batch); + batch = storageClient.batch(); + } } - batchHelper.flush(); - } - - public void remove(Collection filenames) throws IOException { - for (String filename : filenames) { - final GcsPath path = GcsPath.fromUri(filename); - LOG.debug("Removing: " + path); - Storage.Objects.Delete deleteObject = - storageClient.objects().delete(path.getBucket(), path.getObject()); - batchHelper.queue(deleteObject, new JsonBatchCallback() { - @Override - public void onSuccess(Void obj, HttpHeaders responseHeaders) throws IOException { - LOG.debug("Successfully removed {}", path); - } - - @Override - public void onFailure(GoogleJsonError e, HttpHeaders responseHeaders) throws IOException { - // Do nothing on item not found. - if (!errorExtractor.itemNotFound(e)) { - throw new IOException(e.toString()); - } - LOG.debug("{} does not exist.", path); - } - }); + if (batch.size() > 0) { + batches.add(batch); } - batchHelper.flush(); + return batches; } - /** - * BatchHelper abstracts out the logic for the maximum requests per batch for GCS. - * - *

    Copy of - * https://github.com/GoogleCloudPlatform/bigdata-interop/blob/master/gcs/src/main/java/com/google/cloud/hadoop/gcsio/BatchHelper.java - * - *

    Copied to prevent Dataflow from depending on the Hadoop-related dependencies that are not - * used in Dataflow. Hadoop-related dependencies will be removed from the Google Cloud Storage - * Connector (https://cloud.google.com/hadoop/google-cloud-storage-connector) so that this project - * and others may use the connector without introducing unnecessary dependencies. - * - *

    This class is not thread-safe; create a new BatchHelper instance per single-threaded logical - * grouping of requests. - */ - @NotThreadSafe - private static class BatchHelper { - /** - * Callback that causes a single StorageRequest to be added to the BatchRequest. - */ - protected static interface QueueRequestCallback { - void enqueue() throws IOException; + List makeRemoveBatches(Collection filenames) throws IOException { + List batches = new LinkedList<>(); + for (List filesToDelete : + Lists.partition(Lists.newArrayList(filenames), MAX_REQUESTS_PER_BATCH)) { + BatchRequest batch = storageClient.batch(); + for (String file : filesToDelete) { + enqueueDelete(GcsPath.fromUri(file), batch); + } + batches.add(batch); } + return batches; + } - private final List pendingBatchEntries; - private final BatchRequest batch; - - // Number of requests that can be queued into a single actual HTTP request - // before a sub-batch is sent. - private final long maxRequestsPerBatch; - - // Flag that indicates whether there is an in-progress flush. - private boolean flushing = false; + public void remove(Collection filenames) throws IOException { + executeBatches(makeRemoveBatches(filenames)); + } - /** - * Primary constructor, generally accessed only via the inner Factory class. - */ - public BatchHelper( - HttpRequestInitializer requestInitializer, Storage gcs, long maxRequestsPerBatch) { - this.pendingBatchEntries = new LinkedList<>(); - this.batch = gcs.batch(requestInitializer); - this.maxRequestsPerBatch = maxRequestsPerBatch; - } + private void enqueueCopy(final GcsPath from, final GcsPath to, BatchRequest batch) + throws IOException { + Storage.Objects.Copy copyRequest = storageClient.objects() + .copy(from.getBucket(), from.getObject(), to.getBucket(), to.getObject(), null); + copyRequest.queue(batch, new JsonBatchCallback() { + @Override + public void onSuccess(StorageObject obj, HttpHeaders responseHeaders) { + LOG.debug("Successfully copied {} to {}", from, to); + } - /** - * Adds an additional request to the batch, and possibly flushes the current contents of the - * batch if {@code maxRequestsPerBatch} has been reached. - */ - public void queue(final StorageRequest req, final JsonBatchCallback callback) - throws IOException { - QueueRequestCallback queueCallback = new QueueRequestCallback() { - @Override - public void enqueue() throws IOException { - req.queue(batch, callback); + @Override + public void onFailure(GoogleJsonError e, HttpHeaders responseHeaders) throws IOException { + if (errorExtractor.itemNotFound(e)) { + // Do nothing on item not found. + LOG.debug("{} does not exist, assuming this is a retry after deletion.", from); + return; } - }; - pendingBatchEntries.add(queueCallback); - - flushIfPossibleAndRequired(); - } - - // Flush our buffer if we have more pending entries than maxRequestsPerBatch - private void flushIfPossibleAndRequired() throws IOException { - if (pendingBatchEntries.size() > maxRequestsPerBatch) { - flushIfPossible(); + throw new IOException( + String.format("Error trying to copy %s to %s: %s", from, to, e)); } - } + }); + } - // Flush our buffer if we are not already in a flush operation and we have data to flush. - private void flushIfPossible() throws IOException { - if (!flushing && pendingBatchEntries.size() > 0) { - flushing = true; - try { - while (batch.size() < maxRequestsPerBatch && pendingBatchEntries.size() > 0) { - QueueRequestCallback head = pendingBatchEntries.remove(0); - head.enqueue(); - } + private void enqueueDelete(final GcsPath file, BatchRequest batch) throws IOException { + Storage.Objects.Delete deleteRequest = storageClient.objects() + .delete(file.getBucket(), file.getObject()); + deleteRequest.queue(batch, new JsonBatchCallback() { + @Override + public void onSuccess(Void obj, HttpHeaders responseHeaders) { + LOG.debug("Successfully deleted {}", file); + } - batch.execute(); - } finally { - flushing = false; + @Override + public void onFailure(GoogleJsonError e, HttpHeaders responseHeaders) throws IOException { + if (errorExtractor.itemNotFound(e)) { + // Do nothing on item not found. + LOG.debug("{} does not exist.", file); + return; } + throw new IOException(String.format("Error trying to delete %s: %s", file, e)); } - } - - - /** - * Sends any currently remaining requests in the batch; should be called at the end of any - * series of batched requests to ensure everything has been sent. - */ - public void flush() throws IOException { - flushIfPossible(); - } + }); } /** diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java index 49c7bc48ec4d..997340a22fa2 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java @@ -18,6 +18,8 @@ package org.apache.beam.sdk.util; import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; @@ -32,6 +34,7 @@ import org.apache.beam.sdk.testing.FastNanoClockAndSleeper; import org.apache.beam.sdk.util.gcsfs.GcsPath; +import com.google.api.client.googleapis.batch.BatchRequest; import com.google.api.client.googleapis.json.GoogleJsonError.ErrorInfo; import com.google.api.client.googleapis.json.GoogleJsonResponseException; import com.google.api.client.http.HttpRequest; @@ -490,4 +493,70 @@ public LowLevelHttpRequest buildRequest(String method, String url) throws IOExce HttpResponse response = request.execute(); return GoogleJsonResponseException.from(jsonFactory, response); } + + private static List makeStrings(String s, int n) { + ImmutableList.Builder ret = ImmutableList.builder(); + for (int i = 0; i < n; ++i) { + ret.add(String.format("gs://bucket/%s%d", s, i)); + } + return ret.build(); + } + + private static int sumBatchSizes(List batches) { + int ret = 0; + for (BatchRequest b : batches) { + ret += b.size(); + assertThat(b.size(), greaterThan(0)); + } + return ret; + } + + @Test + public void testMakeCopyBatches() throws IOException { + GcsUtil gcsUtil = gcsOptionsWithTestCredential().getGcsUtil(); + + // Small number of files fits in 1 batch + List batches = gcsUtil.makeCopyBatches(makeStrings("s", 3), makeStrings("d", 3)); + assertThat(batches.size(), equalTo(1)); + assertThat(sumBatchSizes(batches), equalTo(3)); + + // 1 batch of files fits in 1 batch + batches = gcsUtil.makeCopyBatches(makeStrings("s", 100), makeStrings("d", 100)); + assertThat(batches.size(), equalTo(1)); + assertThat(sumBatchSizes(batches), equalTo(100)); + + // A little more than 5 batches of files fits in 6 batches + batches = gcsUtil.makeCopyBatches(makeStrings("s", 501), makeStrings("d", 501)); + assertThat(batches.size(), equalTo(6)); + assertThat(sumBatchSizes(batches), equalTo(501)); + } + + @Test + public void testInvalidCopyBatches() throws IOException { + GcsUtil gcsUtil = gcsOptionsWithTestCredential().getGcsUtil(); + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("Number of source files 3"); + + gcsUtil.makeCopyBatches(makeStrings("s", 3), makeStrings("d", 1)); + } + + @Test + public void testMakeRemoveBatches() throws IOException { + GcsUtil gcsUtil = gcsOptionsWithTestCredential().getGcsUtil(); + + // Small number of files fits in 1 batch + List batches = gcsUtil.makeRemoveBatches(makeStrings("s", 3)); + assertThat(batches.size(), equalTo(1)); + assertThat(sumBatchSizes(batches), equalTo(3)); + + // 1 batch of files fits in 1 batch + batches = gcsUtil.makeRemoveBatches(makeStrings("s", 100)); + assertThat(batches.size(), equalTo(1)); + assertThat(sumBatchSizes(batches), equalTo(100)); + + // A little more than 5 batches of files fits in 6 batches + batches = gcsUtil.makeRemoveBatches(makeStrings("s", 501)); + assertThat(batches.size(), equalTo(6)); + assertThat(sumBatchSizes(batches), equalTo(501)); + } } From b1123417a6b55fba1560fa68096d0b5551efa960 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Mon, 22 Aug 2016 17:25:32 +0200 Subject: [PATCH 043/112] [BEAM-574] Remove log when new records have not been read yet (KafkaIO) --- .../src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java | 1 - 1 file changed, 1 deletion(-) diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java index 8a0c7880e97d..6cc5d6a597d6 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java @@ -1076,7 +1076,6 @@ private void updateLatestOffsets() { @Override public Instant getWatermark() { if (curRecord == null) { - LOG.warn("{}: getWatermark() : no records have been read yet.", name); return initialWatermark; } From c64e9dca8cc8ab9424ac29894ed6bf9358dd9fdf Mon Sep 17 00:00:00 2001 From: Daniel Halperin Date: Tue, 23 Aug 2016 17:39:19 -0700 Subject: [PATCH 044/112] Write: Remove invalid import --- sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java index fea65ef23398..a474528e04e6 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java @@ -45,7 +45,7 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PDone; -import com.google.api.client.util.Lists; +import com.google.common.collect.Lists; import org.slf4j.Logger; import org.slf4j.LoggerFactory; From 3b221732a298111e0612caa61b7acf69d13dd2d2 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Tue, 23 Aug 2016 19:32:34 -0700 Subject: [PATCH 045/112] JUnit tests: add @RunWith annotation --- .../apache/beam/runners/direct/AggregatorContainerTest.java | 5 +++++ .../apache/beam/runners/direct/DoFnLifecycleManagerTest.java | 3 +++ .../beam/runners/direct/WriteWithShardingFactoryTest.java | 3 +++ .../java/org/apache/beam/sdk/testing/WindowSupplierTest.java | 3 +++ .../apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java | 3 +++ .../test/java/org/apache/beam/sdk/util/PubsubClientTest.java | 3 +++ .../java/org/apache/beam/sdk/util/PubsubGrpcClientTest.java | 4 ++++ .../java/org/apache/beam/sdk/util/PubsubJsonClientTest.java | 3 +++ .../java/org/apache/beam/sdk/util/PubsubTestClientTest.java | 3 +++ 9 files changed, 30 insertions(+) diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/AggregatorContainerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/AggregatorContainerTest.java index 035a1b03be07..b131b4c524c2 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/AggregatorContainerTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/AggregatorContainerTest.java @@ -24,12 +24,16 @@ import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.Sum.SumIntegerFn; import org.apache.beam.sdk.util.ExecutionContext.StepContext; + import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; import org.mockito.Mock; import org.mockito.MockitoAnnotations; + import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; @@ -37,6 +41,7 @@ /** * Tests for {@link AggregatorContainer}. */ +@RunWith(JUnit4.class) public class AggregatorContainerTest { @Rule diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java index 77b32968f8b4..9da46f265b7d 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java @@ -29,6 +29,8 @@ import org.hamcrest.Matchers; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; import java.util.ArrayList; import java.util.List; @@ -42,6 +44,7 @@ /** * Tests for {@link DoFnLifecycleManager}. */ +@RunWith(JUnit4.class) public class DoFnLifecycleManagerTest { private TestFn fn = new TestFn(); private DoFnLifecycleManager mgr = DoFnLifecycleManager.of(fn); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java index a53bc64ecd7f..20b2776890e9 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java @@ -53,6 +53,8 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; import java.io.File; import java.io.FileReader; @@ -68,6 +70,7 @@ /** * Tests for {@link WriteWithShardingFactory}. */ +@RunWith(JUnit4.class) public class WriteWithShardingFactoryTest { public static final int INPUT_SIZE = 10000; @Rule public TemporaryFolder tmp = new TemporaryFolder(); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/WindowSupplierTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/WindowSupplierTest.java index 178c67ca7956..1a2fd1d56041 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/WindowSupplierTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/WindowSupplierTest.java @@ -32,6 +32,8 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; import java.io.IOException; import java.io.InputStream; @@ -41,6 +43,7 @@ /** * Tests for {@link WindowSupplier}. */ +@RunWith(JUnit4.class) public class WindowSupplierTest { private final IntervalWindow window = new IntervalWindow(new Instant(0L), new Instant(100L)); private final IntervalWindow otherWindow = diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java index 7e756e245f91..28f0143d69c1 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java @@ -29,10 +29,13 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; import org.mockito.Mock; import org.mockito.MockitoAnnotations; /** Tests for {@link DoFnInvokers}. */ +@RunWith(JUnit4.class) public class DoFnInvokersTest { /** A convenience struct holding flags that indicate whether a particular method was invoked. */ public static class Invocations { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubClientTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubClientTest.java index 22508572e5a0..07031240d16c 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubClientTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubClientTest.java @@ -30,12 +30,15 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; import java.util.Map; /** * Tests for helper classes and methods in PubsubClient. */ +@RunWith(JUnit4.class) public class PubsubClientTest { @Rule public ExpectedException thrown = ExpectedException.none(); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubGrpcClientTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubGrpcClientTest.java index 71ee27c86aae..4d0ec29ca282 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubGrpcClientTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubGrpcClientTest.java @@ -40,9 +40,12 @@ import com.google.pubsub.v1.SubscriberGrpc; import io.grpc.ManagedChannel; + import org.junit.After; import org.junit.Before; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; import org.mockito.Mockito; import java.io.IOException; @@ -51,6 +54,7 @@ /** * Tests for PubsubGrpcClient. */ +@RunWith(JUnit4.class) public class PubsubGrpcClientTest { private ManagedChannel mockChannel; private GoogleCredentials mockCredentials; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubJsonClientTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubJsonClientTest.java index dfdc46ecea68..21689358197d 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubJsonClientTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubJsonClientTest.java @@ -38,6 +38,8 @@ import org.junit.After; import org.junit.Before; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; import org.mockito.Mockito; import java.io.IOException; @@ -46,6 +48,7 @@ /** * Tests for PubsubJsonClient. */ +@RunWith(JUnit4.class) public class PubsubJsonClientTest { private Pubsub mockPubsub; private PubsubClient client; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubTestClientTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubTestClientTest.java index d788f1070cec..d86f5e7ca89d 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubTestClientTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubTestClientTest.java @@ -32,6 +32,8 @@ import com.google.common.collect.Sets; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; import java.io.IOException; import java.util.List; @@ -40,6 +42,7 @@ /** * Tests for PubsubTestClient. */ +@RunWith(JUnit4.class) public class PubsubTestClientTest { private static final TopicPath TOPIC = PubsubClient.topicPathFromName("testProject", "testTopic"); private static final SubscriptionPath SUBSCRIPTION = From ddbfcdb6d36815e00c266be11d727135d75913f0 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Tue, 23 Aug 2016 14:44:53 -0700 Subject: [PATCH 046/112] Remove ParDoTest Suppression in Google Cloud Dataflow This reenables the lifecycle tests now that they are properly supported. Update the container image. --- runners/google-cloud-dataflow-java/pom.xml | 11 ----------- .../apache/beam/runners/dataflow/DataflowRunner.java | 4 ++-- 2 files changed, 2 insertions(+), 13 deletions(-) diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml index 0044823068c0..bf66f388c944 100644 --- a/runners/google-cloud-dataflow-java/pom.xml +++ b/runners/google-cloud-dataflow-java/pom.xml @@ -60,17 +60,6 @@ true - - - runnable-on-service-tests - - - org/apache/beam/sdk/transforms/ParDoLifecycleTest.java - org/apache/beam/sdk/transforms/ParDoTest.java - - - - - - + From c623a271a3f58a834a803cf9d3b5e5c0920caea7 Mon Sep 17 00:00:00 2001 From: bchambers Date: Wed, 24 Aug 2016 13:01:03 -0700 Subject: [PATCH 060/112] Optimize imports --- .../beam/examples/DebuggingWordCount.java | 8 +- .../beam/examples/WindowedWordCount.java | 17 ++- .../org/apache/beam/examples/WordCount.java | 8 +- .../common/ExampleBigQueryTableOptions.java | 3 +- .../beam/examples/common/ExampleOptions.java | 7 +- .../beam/examples/common/ExampleUtils.java | 14 +-- .../examples/common/PubsubFileInjector.java | 14 +-- .../beam/examples/complete/AutoComplete.java | 33 +++--- .../complete/StreamingWordExtract.java | 12 +- .../apache/beam/examples/complete/TfIdf.java | 14 +-- .../complete/TopWikipediaSessions.java | 7 +- .../examples/complete/TrafficMaxLaneFlow.java | 19 ++- .../beam/examples/complete/TrafficRoutes.java | 29 ++--- .../examples/cookbook/BigQueryTornadoes.java | 12 +- .../cookbook/CombinePerKeyExamples.java | 12 +- .../examples/cookbook/DatastoreWordCount.java | 18 ++- .../examples/cookbook/FilterExamples.java | 14 +-- .../beam/examples/cookbook/JoinExamples.java | 3 +- .../examples/cookbook/MaxPerKeyExamples.java | 12 +- .../examples/cookbook/TriggerExample.java | 17 ++- .../beam/examples/DebuggingWordCountTest.java | 6 +- .../org/apache/beam/examples/WordCountIT.java | 4 +- .../apache/beam/examples/WordCountTest.java | 6 +- .../examples/complete/AutoCompleteTest.java | 12 +- .../beam/examples/complete/TfIdfTest.java | 6 +- .../complete/TopWikipediaSessionsTest.java | 7 +- .../cookbook/BigQueryTornadoesIT.java | 1 - .../cookbook/BigQueryTornadoesTest.java | 7 +- .../cookbook/CombinePerKeyExamplesTest.java | 7 +- .../examples/cookbook/DeDupExampleTest.java | 6 +- .../examples/cookbook/FilterExamplesTest.java | 9 +- .../examples/cookbook/JoinExamplesTest.java | 9 +- .../cookbook/MaxPerKeyExamplesTest.java | 9 +- .../examples/cookbook/TriggerExampleTest.java | 17 ++- .../beam/examples/MinimalWordCountJava8.java | 3 +- .../examples/complete/game/GameStats.java | 8 +- .../complete/game/HourlyTeamScore.java | 8 +- .../examples/complete/game/LeaderBoard.java | 8 +- .../examples/complete/game/UserScore.java | 8 +- .../complete/game/injector/Injector.java | 8 +- .../complete/game/injector/InjectorUtils.java | 1 - .../injector/RetryHttpInitializerWrapper.java | 1 - .../complete/game/utils/WriteToBigQuery.java | 18 ++- .../game/utils/WriteWindowedToBigQuery.java | 6 +- .../examples/MinimalWordCountJava8Test.java | 21 ++-- .../examples/complete/game/GameStatsTest.java | 8 +- .../complete/game/HourlyTeamScoreTest.java | 8 +- .../examples/complete/game/UserScoreTest.java | 8 +- .../beam/runners/core/SideInputHandler.java | 17 ++- .../core/UnboundedReadFromBoundedSource.java | 37 +++--- .../beam/sdk/util/AssignWindowsDoFn.java | 7 +- .../beam/sdk/util/BatchTimerInternals.java | 8 +- .../apache/beam/sdk/util/DoFnRunnerBase.java | 22 ++-- .../org/apache/beam/sdk/util/DoFnRunners.java | 3 +- ...GroupAlsoByWindowsViaOutputBufferDoFn.java | 7 +- .../sdk/util/GroupByKeyViaGroupByKeyOnly.java | 9 +- .../sdk/util/LateDataDroppingDoFnRunner.java | 10 +- .../apache/beam/sdk/util/PaneInfoTracker.java | 6 +- .../sdk/util/PushbackSideInputDoFnRunner.java | 6 +- .../org/apache/beam/sdk/util/ReduceFn.java | 4 +- .../beam/sdk/util/ReduceFnContextFactory.java | 12 +- .../apache/beam/sdk/util/ReduceFnRunner.java | 26 ++-- .../beam/sdk/util/SimpleDoFnRunner.java | 2 +- .../apache/beam/sdk/util/SystemReduceFn.java | 3 +- .../apache/beam/sdk/util/TriggerRunner.java | 16 +-- .../apache/beam/sdk/util/WatermarkHold.java | 12 +- .../runners/core/SideInputHandlerTest.java | 4 +- .../UnboundedReadFromBoundedSourceTest.java | 27 ++--- .../sdk/util/BatchTimerInternalsTest.java | 1 - .../util/GroupAlsoByWindowsProperties.java | 19 ++- ...pAlsoByWindowsViaOutputBufferDoFnTest.java | 1 - .../util/LateDataDroppingDoFnRunnerTest.java | 9 +- .../util/PushbackSideInputDoFnRunnerTest.java | 9 +- .../beam/sdk/util/ReduceFnRunnerTest.java | 13 +- .../apache/beam/sdk/util/ReduceFnTester.java | 45 ++++--- .../beam/sdk/util/SimpleDoFnRunnerTest.java | 6 +- .../runners/direct/AggregatorContainer.java | 8 +- .../direct/BoundedReadEvaluatorFactory.java | 14 +-- .../beam/runners/direct/CommittedResult.java | 9 +- .../ConsumerTrackingPipelineVisitor.java | 13 +- .../beam/runners/direct/DirectRegistrar.java | 5 +- .../beam/runners/direct/DirectRunner.java | 25 ++-- .../runners/direct/DirectTimerInternals.java | 4 +- .../runners/direct/DoFnLifecycleManager.java | 15 +-- ...ycleManagerRemovingTransformEvaluator.java | 1 - .../runners/direct/EvaluationContext.java | 26 ++-- .../ExecutorServiceParallelExecutor.java | 34 +++--- .../GroupAlsoByWindowEvaluatorFactory.java | 6 +- .../GroupByKeyOnlyEvaluatorFactory.java | 13 +- .../ImmutabilityCheckingBundleFactory.java | 6 +- .../ImmutabilityEnforcementFactory.java | 5 +- .../direct/ImmutableListBundleFactory.java | 6 +- .../direct/KeyedPValueTrackingVisitor.java | 5 +- .../beam/runners/direct/NanosOffsetClock.java | 3 +- .../beam/runners/direct/ParDoEvaluator.java | 14 +-- .../direct/ParDoMultiEvaluatorFactory.java | 11 +- .../direct/ParDoSingleEvaluatorFactory.java | 13 +- .../beam/runners/direct/PipelineExecutor.java | 3 +- .../runners/direct/SideInputContainer.java | 19 ++- .../beam/runners/direct/StepAndKey.java | 4 +- .../runners/direct/StepTransformResult.java | 16 +-- .../direct/TestStreamEvaluatorFactory.java | 14 +-- .../direct/TransformEvaluatorFactory.java | 3 +- .../direct/TransformEvaluatorRegistry.java | 16 +-- .../runners/direct/TransformExecutor.java | 8 +- .../direct/TransformExecutorServices.java | 1 - .../beam/runners/direct/TransformResult.java | 7 +- .../direct/UnboundedReadDeduplicator.java | 11 +- .../direct/UnboundedReadEvaluatorFactory.java | 16 +-- .../UncommittedBundleOutputManager.java | 3 +- .../runners/direct/ViewEvaluatorFactory.java | 5 +- .../direct/WatermarkCallbackExecutor.java | 11 +- .../beam/runners/direct/WatermarkManager.java | 28 ++--- .../direct/WindowEvaluatorFactory.java | 10 +- .../direct/WriteWithShardingFactory.java | 7 +- .../direct/AggregatorContainerTest.java | 8 +- .../BoundedReadEvaluatorFactoryTest.java | 13 +- .../runners/direct/CommittedResultTest.java | 13 +- .../ConsumerTrackingPipelineVisitorTest.java | 6 +- .../runners/direct/DirectRegistrarTest.java | 9 +- .../beam/runners/direct/DirectRunnerTest.java | 14 +-- .../direct/DirectTimerInternalsTest.java | 1 - ...ManagerRemovingTransformEvaluatorTest.java | 6 +- .../direct/DoFnLifecycleManagerTest.java | 12 +- .../direct/DoFnLifecycleManagersTest.java | 9 +- .../EncodabilityEnforcementFactoryTest.java | 10 +- .../runners/direct/EvaluationContextTest.java | 17 ++- .../direct/FlattenEvaluatorFactoryTest.java | 1 - .../direct/ForwardingPTransformTest.java | 1 - .../GroupByKeyEvaluatorFactoryTest.java | 8 +- .../GroupByKeyOnlyEvaluatorFactoryTest.java | 8 +- ...ImmutabilityCheckingBundleFactoryTest.java | 1 - .../ImmutabilityEnforcementFactoryTest.java | 6 +- .../ImmutableListBundleFactoryTest.java | 11 +- .../KeyedPValueTrackingVisitorTest.java | 9 +- .../runners/direct/ParDoEvaluatorTest.java | 14 +-- .../ParDoMultiEvaluatorFactoryTest.java | 2 +- .../ParDoSingleEvaluatorFactoryTest.java | 2 +- .../direct/SideInputContainerTest.java | 17 ++- .../direct/StepTransformResultTest.java | 1 - .../runners/direct/StructuralKeyTest.java | 1 - .../direct/TransformExecutorServicesTest.java | 4 +- .../runners/direct/TransformExecutorTest.java | 23 ++-- .../direct/UnboundedReadDeduplicatorTest.java | 12 +- .../UnboundedReadEvaluatorFactoryTest.java | 26 ++-- .../direct/ViewEvaluatorFactoryTest.java | 4 +- .../direct/WatermarkCallbackExecutorTest.java | 8 +- .../runners/direct/WatermarkManagerTest.java | 21 ++-- .../direct/WindowEvaluatorFactoryTest.java | 14 +-- .../direct/WriteWithShardingFactoryTest.java | 27 ++--- .../beam/runners/flink/examples/TFIDF.java | 14 +-- .../runners/flink/examples/WordCount.java | 1 - .../examples/streaming/AutoComplete.java | 6 +- .../examples/streaming/JoinExamples.java | 1 - .../examples/streaming/KafkaIOExamples.java | 12 +- .../KafkaWindowedWordCountExample.java | 4 +- .../examples/streaming/WindowedWordCount.java | 4 +- .../flink/DefaultParallelismFactory.java | 1 - .../FlinkPipelineExecutionEnvironment.java | 4 +- .../runners/flink/FlinkPipelineOptions.java | 6 +- .../beam/runners/flink/FlinkRunner.java | 23 ++-- .../runners/flink/FlinkRunnerRegistrar.java | 5 +- .../beam/runners/flink/FlinkRunnerResult.java | 10 +- .../beam/runners/flink/TestFlinkRunner.java | 2 - .../FlinkBatchPipelineTranslator.java | 1 - .../FlinkBatchTransformTranslators.java | 19 ++- .../FlinkBatchTranslationContext.java | 6 +- .../FlinkStreamingPipelineTranslator.java | 1 - .../FlinkStreamingTransformTranslators.java | 25 ++-- .../FlinkStreamingTranslationContext.java | 6 +- .../functions/FlinkAssignContext.java | 6 +- .../functions/FlinkAssignWindows.java | 4 +- .../functions/FlinkDoFnFunction.java | 4 +- .../FlinkMergingNonShuffleReduceFunction.java | 17 ++- .../FlinkMergingPartialReduceFunction.java | 17 ++- .../functions/FlinkMergingReduceFunction.java | 21 ++-- .../FlinkMultiOutputDoFnFunction.java | 4 +- .../FlinkMultiOutputProcessContext.java | 6 +- .../FlinkMultiOutputPruningFunction.java | 1 - .../FlinkNoElementAssignContext.java | 1 - .../functions/FlinkPartialReduceFunction.java | 17 ++- .../functions/FlinkProcessContext.java | 15 +-- .../functions/FlinkReduceFunction.java | 21 ++-- .../functions/SideInputInitializer.java | 10 +- .../types/CoderTypeInformation.java | 1 - .../types/CoderTypeSerializer.java | 8 +- .../types/EncodedValueComparator.java | 6 +- .../types/EncodedValueSerializer.java | 4 +- .../types/EncodedValueTypeInformation.java | 4 - .../flink/translation/types/FlinkCoder.java | 11 +- .../translation/types/KvKeySelector.java | 1 - .../utils/SerializedPipelineOptions.java | 4 +- .../wrappers/DataInputViewWrapper.java | 3 +- .../wrappers/DataOutputViewWrapper.java | 3 +- .../SerializableFnAggregatorWrapper.java | 9 +- .../wrappers/SourceInputFormat.java | 6 +- .../wrappers/SourceInputSplit.java | 1 - .../wrappers/streaming/DoFnOperator.java | 22 ++-- .../streaming/FlinkStateInternals.java | 13 +- .../streaming/SingletonKeyedWorkItem.java | 3 +- .../SingletonKeyedWorkItemCoder.java | 17 ++- .../streaming/WindowDoFnOperator.java | 34 +++--- .../streaming/WorkItemKeySelector.java | 5 +- .../streaming/io/BoundedSourceWrapper.java | 9 +- .../streaming/io/UnboundedFlinkSink.java | 14 +-- .../streaming/io/UnboundedFlinkSource.java | 7 +- .../streaming/io/UnboundedSocketSource.java | 17 ++- .../streaming/io/UnboundedSourceWrapper.java | 15 +-- .../flink/EncodedValueComparatorTest.java | 1 - .../flink/FlinkRunnerRegistrarTest.java | 1 - .../runners/flink/PipelineOptionsTest.java | 6 +- .../beam/runners/flink/ReadSourceITCase.java | 9 +- .../flink/ReadSourceStreamingITCase.java | 4 +- .../beam/runners/flink/WriteSinkITCase.java | 13 +- .../flink/streaming/DoFnOperatorTest.java | 11 +- .../streaming/FlinkStateInternalsTest.java | 6 +- .../flink/streaming/GroupByNullKeyTest.java | 11 +- .../flink/streaming/TestCountingSource.java | 12 +- .../streaming/TopWikipediaSessionsITCase.java | 11 +- .../streaming/UnboundedSourceWrapperTest.java | 8 +- .../dataflow/BlockingDataflowRunner.java | 7 +- .../dataflow/DataflowJobException.java | 1 - .../runners/dataflow/DataflowPipelineJob.java | 30 ++--- .../dataflow/DataflowPipelineRegistrar.java | 5 +- .../dataflow/DataflowPipelineTranslator.java | 48 ++++---- .../beam/runners/dataflow/DataflowRunner.java | 107 ++++++++--------- .../runners/dataflow/DataflowRunnerHooks.java | 3 +- .../dataflow/internal/CustomSources.java | 18 ++- .../DataflowAggregatorTransforms.java | 10 +- .../DataflowMetricUpdateExtractor.java | 10 +- ...ataflowUnboundedReadFromBoundedSource.java | 37 +++--- .../runners/dataflow/internal/IsmFormat.java | 35 +++--- .../dataflow/internal/ReadTranslator.java | 8 +- .../options/CloudDebuggerOptions.java | 6 +- .../options/DataflowPipelineDebugOptions.java | 11 +- .../options/DataflowPipelineOptions.java | 7 +- .../DataflowPipelineWorkerPoolOptions.java | 9 +- .../options/DataflowProfilingOptions.java | 3 +- .../options/DataflowWorkerLoggingOptions.java | 8 +- .../dataflow/testing/TestDataflowRunner.java | 27 ++--- .../dataflow/util/DataflowTransport.java | 6 +- .../beam/runners/dataflow/util/DoFnInfo.java | 3 +- .../beam/runners/dataflow/util/GcsStager.java | 6 +- .../runners/dataflow/util/MonitoringUtil.java | 14 +-- .../runners/dataflow/util/PackageUtil.java | 18 ++- .../dataflow/util/RandomAccessData.java | 16 +-- .../beam/runners/dataflow/util/Stager.java | 1 - .../beam/runners/dataflow/util/TimeUtil.java | 8 +- .../dataflow/BlockingDataflowRunnerTest.java | 1 - .../dataflow/DataflowPipelineJobTest.java | 33 +++--- .../DataflowPipelineRegistrarTest.java | 9 +- .../DataflowPipelineTranslatorTest.java | 38 +++--- .../runners/dataflow/DataflowRunnerTest.java | 48 ++++---- .../dataflow/RecordingPipelineVisitor.java | 5 +- .../DataflowPipelineDebugOptionsTest.java | 1 - .../options/DataflowPipelineOptionsTest.java | 1 - .../options/DataflowProfilingOptionsTest.java | 4 +- .../DataflowWorkerLoggingOptionsTest.java | 8 +- .../testing/TestDataflowRunnerTest.java | 35 +++--- .../transforms/DataflowGroupByKeyTest.java | 6 +- .../dataflow/transforms/DataflowViewTest.java | 1 - .../dataflow/util/MonitoringUtilTest.java | 17 ++- .../dataflow/util/PackageUtilTest.java | 41 +++---- .../dataflow/util/RandomAccessDataTest.java | 13 +- .../runners/dataflow/util/TimeUtilTest.java | 1 - .../beam/runners/spark/SparkRunner.java | 1 - .../runners/spark/SparkRunnerRegistrar.java | 5 +- .../spark/aggregators/NamedAggregators.java | 10 +- .../runners/spark/coders/CoderHelpers.java | 12 +- .../spark/coders/NullWritableCoder.java | 6 +- .../runners/spark/coders/WritableCoder.java | 14 +-- .../apache/beam/runners/spark/io/KafkaIO.java | 8 +- .../runners/spark/io/hadoop/HadoopIO.java | 6 +- .../spark/io/hadoop/ShardNameBuilder.java | 3 +- .../io/hadoop/ShardNameTemplateHelper.java | 3 +- .../hadoop/TemplatedAvroKeyOutputFormat.java | 5 +- .../TemplatedSequenceFileOutputFormat.java | 3 +- .../io/hadoop/TemplatedTextOutputFormat.java | 3 +- .../spark/translation/DoFnFunction.java | 10 +- .../spark/translation/EvaluationContext.java | 19 ++- .../spark/translation/MultiDoFnFunction.java | 16 +-- .../translation/SparkProcessContext.java | 15 +-- .../translation/SparkRuntimeContext.java | 19 ++- .../spark/translation/TransformEvaluator.java | 3 +- .../translation/TransformTranslator.java | 30 ++--- .../streaming/StreamingEvaluationContext.java | 14 +-- .../StreamingTransformTranslator.java | 26 ++-- .../runners/spark/util/BroadcastHelper.java | 8 +- .../beam/runners/spark/util/ByteArray.java | 1 - .../apache/beam/runners/spark/DeDupTest.java | 11 +- .../beam/runners/spark/EmptyInputTest.java | 9 +- .../runners/spark/SimpleWordCountTest.java | 15 +-- .../spark/SparkRunnerRegistrarTest.java | 9 +- .../apache/beam/runners/spark/TfIdfTest.java | 6 +- .../spark/coders/WritableCoderTest.java | 1 - .../runners/spark/io/AvroPipelineTest.java | 25 ++-- .../beam/runners/spark/io/NumShardsTest.java | 21 ++-- .../hadoop/HadoopFileFormatPipelineTest.java | 6 +- .../spark/io/hadoop/ShardNameBuilderTest.java | 1 - .../translation/CombineGloballyTest.java | 9 +- .../spark/translation/CombinePerKeyTest.java | 11 +- .../spark/translation/DoFnOutputTest.java | 4 +- .../translation/MultiOutputWordCountTest.java | 9 +- .../spark/translation/SerializationTest.java | 23 ++-- .../spark/translation/SideEffectsTest.java | 6 +- .../translation/SparkPipelineOptionsTest.java | 1 - .../translation/TransformTranslatorTest.java | 21 ++-- .../translation/WindowedWordCountTest.java | 9 +- .../streaming/FlattenStreamingTest.java | 8 +- .../streaming/KafkaStreamingTest.java | 16 +-- .../SimpleStreamingWordCountTest.java | 10 +- .../streaming/utils/EmbeddedKafkaCluster.java | 12 +- .../streaming/utils/PAssertStreaming.java | 4 +- .../beam/sdk/AggregatorPipelineExtractor.java | 12 +- .../org/apache/beam/sdk/AggregatorValues.java | 5 +- .../java/org/apache/beam/sdk/Pipeline.java | 21 ++-- .../org/apache/beam/sdk/PipelineResult.java | 4 +- .../org/apache/beam/sdk/coders/AvroCoder.java | 35 +++--- .../beam/sdk/coders/BigDecimalCoder.java | 1 - .../sdk/coders/BigEndianIntegerCoder.java | 1 - .../beam/sdk/coders/BigEndianLongCoder.java | 1 - .../beam/sdk/coders/BigIntegerCoder.java | 1 - .../beam/sdk/coders/ByteArrayCoder.java | 11 +- .../org/apache/beam/sdk/coders/ByteCoder.java | 1 - .../beam/sdk/coders/ByteStringCoder.java | 7 +- .../org/apache/beam/sdk/coders/Coder.java | 11 +- .../beam/sdk/coders/CoderFactories.java | 3 +- .../beam/sdk/coders/CoderProviders.java | 6 +- .../apache/beam/sdk/coders/CoderRegistry.java | 24 ++-- .../beam/sdk/coders/CollectionCoder.java | 4 +- .../apache/beam/sdk/coders/CustomCoder.java | 16 +-- .../apache/beam/sdk/coders/DefaultCoder.java | 3 +- .../apache/beam/sdk/coders/DelegateCoder.java | 1 - .../apache/beam/sdk/coders/DoubleCoder.java | 1 - .../apache/beam/sdk/coders/DurationCoder.java | 8 +- .../apache/beam/sdk/coders/InstantCoder.java | 12 +- .../apache/beam/sdk/coders/IterableCoder.java | 9 +- .../beam/sdk/coders/IterableLikeCoder.java | 9 +- .../org/apache/beam/sdk/coders/JAXBCoder.java | 12 +- .../org/apache/beam/sdk/coders/KvCoder.java | 13 +- .../org/apache/beam/sdk/coders/ListCoder.java | 4 +- .../org/apache/beam/sdk/coders/MapCoder.java | 9 +- .../apache/beam/sdk/coders/NullableCoder.java | 12 +- .../beam/sdk/coders/SerializableCoder.java | 6 +- .../org/apache/beam/sdk/coders/SetCoder.java | 4 +- .../apache/beam/sdk/coders/StandardCoder.java | 11 +- .../beam/sdk/coders/StringDelegateCoder.java | 5 +- .../beam/sdk/coders/StringUtf8Coder.java | 11 +- .../beam/sdk/coders/TableRowJsonCoder.java | 4 +- .../beam/sdk/coders/TextualIntegerCoder.java | 1 - .../apache/beam/sdk/coders/VarIntCoder.java | 4 +- .../apache/beam/sdk/coders/VarLongCoder.java | 4 +- .../org/apache/beam/sdk/coders/VoidCoder.java | 1 - .../beam/sdk/coders/protobuf/ProtoCoder.java | 28 ++--- .../sdk/coders/protobuf/ProtobufUtil.java | 4 +- .../java/org/apache/beam/sdk/io/AvroIO.java | 24 ++-- .../org/apache/beam/sdk/io/AvroSource.java | 45 ++++--- .../apache/beam/sdk/io/BlockBasedSource.java | 6 +- .../io/BoundedReadFromUnboundedSource.java | 17 ++- .../org/apache/beam/sdk/io/BoundedSource.java | 11 +- .../apache/beam/sdk/io/CompressedSource.java | 16 +-- .../org/apache/beam/sdk/io/CountingInput.java | 4 +- .../apache/beam/sdk/io/CountingSource.java | 11 +- .../org/apache/beam/sdk/io/FileBasedSink.java | 29 ++--- .../apache/beam/sdk/io/FileBasedSource.java | 17 ++- .../apache/beam/sdk/io/OffsetBasedSource.java | 10 +- .../java/org/apache/beam/sdk/io/PubsubIO.java | 20 ++-- .../beam/sdk/io/PubsubUnboundedSink.java | 23 ++-- .../beam/sdk/io/PubsubUnboundedSource.java | 47 ++++---- .../java/org/apache/beam/sdk/io/Read.java | 4 +- .../java/org/apache/beam/sdk/io/Sink.java | 3 +- .../java/org/apache/beam/sdk/io/Source.java | 8 +- .../java/org/apache/beam/sdk/io/TextIO.java | 29 ++--- .../apache/beam/sdk/io/UnboundedSource.java | 11 +- .../java/org/apache/beam/sdk/io/Write.java | 11 +- .../java/org/apache/beam/sdk/io/XmlSink.java | 14 +-- .../org/apache/beam/sdk/io/XmlSource.java | 15 +-- .../org/apache/beam/sdk/io/range/ByteKey.java | 1 - .../beam/sdk/io/range/ByteKeyRange.java | 6 +- .../sdk/io/range/ByteKeyRangeTracker.java | 4 +- .../beam/sdk/io/range/OffsetRangeTracker.java | 4 +- .../apache/beam/sdk/options/GcpOptions.java | 18 ++- .../apache/beam/sdk/options/GcsOptions.java | 15 +-- .../sdk/options/GoogleApiDebugOptions.java | 1 - .../beam/sdk/options/PipelineOptionSpec.java | 4 +- .../beam/sdk/options/PipelineOptions.java | 19 ++- .../sdk/options/PipelineOptionsFactory.java | 26 ++-- .../sdk/options/PipelineOptionsReflector.java | 4 +- .../sdk/options/PipelineOptionsRegistrar.java | 1 - .../sdk/options/PipelineOptionsValidator.java | 6 +- .../sdk/options/ProxyInvocationHandler.java | 38 +++--- .../sdk/runners/PipelineRunnerRegistrar.java | 1 - .../beam/sdk/runners/TransformHierarchy.java | 9 +- .../beam/sdk/runners/TransformTreeNode.java | 14 +-- .../beam/sdk/testing/CoderProperties.java | 23 ++-- .../beam/sdk/testing/FileChecksumMatcher.java | 17 ++- .../beam/sdk/testing/MatcherDeserializer.java | 7 +- .../beam/sdk/testing/MatcherSerializer.java | 7 +- .../org/apache/beam/sdk/testing/PAssert.java | 24 ++-- .../beam/sdk/testing/PaneExtractors.java | 5 +- .../beam/sdk/testing/SerializableMatcher.java | 3 +- .../sdk/testing/SerializableMatchers.java | 18 ++- .../beam/sdk/testing/SourceTestUtils.java | 27 ++--- .../beam/sdk/testing/StaticWindows.java | 10 +- .../apache/beam/sdk/testing/TestPipeline.java | 32 +++-- .../beam/sdk/testing/TestPipelineOptions.java | 1 - .../apache/beam/sdk/testing/TestStream.java | 24 ++-- .../beam/sdk/testing/WindowFnTestUtils.java | 19 ++- .../beam/sdk/testing/WindowSupplier.java | 10 +- .../sdk/transforms/AppliedPTransform.java | 5 +- .../sdk/transforms/ApproximateQuantiles.java | 29 ++--- .../sdk/transforms/ApproximateUnique.java | 23 ++-- .../apache/beam/sdk/transforms/Combine.java | 27 ++--- .../beam/sdk/transforms/CombineFnBase.java | 10 +- .../beam/sdk/transforms/CombineFns.java | 39 +++--- .../org/apache/beam/sdk/transforms/Count.java | 13 +- .../apache/beam/sdk/transforms/Create.java | 32 +++-- .../org/apache/beam/sdk/transforms/DoFn.java | 18 ++- .../beam/sdk/transforms/DoFnAdapters.java | 4 +- .../beam/sdk/transforms/DoFnTester.java | 26 ++-- .../beam/sdk/transforms/FlatMapElements.java | 3 +- .../IntraBundleParallelization.java | 11 +- .../org/apache/beam/sdk/transforms/Max.java | 5 +- .../org/apache/beam/sdk/transforms/Mean.java | 12 +- .../org/apache/beam/sdk/transforms/Min.java | 5 +- .../apache/beam/sdk/transforms/OldDoFn.java | 19 ++- .../beam/sdk/transforms/PTransform.java | 7 +- .../org/apache/beam/sdk/transforms/ParDo.java | 10 +- .../apache/beam/sdk/transforms/Partition.java | 3 +- .../apache/beam/sdk/transforms/Sample.java | 7 +- .../org/apache/beam/sdk/transforms/Top.java | 20 ++-- .../org/apache/beam/sdk/transforms/View.java | 5 +- .../apache/beam/sdk/transforms/ViewFn.java | 3 +- .../beam/sdk/transforms/WithTimestamps.java | 1 - .../sdk/transforms/display/DisplayData.java | 21 ++-- .../beam/sdk/transforms/join/CoGbkResult.java | 30 ++--- .../transforms/join/CoGbkResultSchema.java | 10 +- .../sdk/transforms/join/CoGroupByKey.java | 5 +- .../join/KeyedPCollectionTuple.java | 7 +- .../beam/sdk/transforms/join/UnionCoder.java | 14 +-- .../sdk/transforms/reflect/DoFnInvokers.java | 33 +++--- .../sdk/transforms/reflect/DoFnSignature.java | 5 +- .../transforms/reflect/DoFnSignatures.java | 8 +- .../sdk/transforms/windowing/AfterAll.java | 9 +- .../windowing/AfterDelayFromFirstElement.java | 16 +-- .../sdk/transforms/windowing/AfterEach.java | 9 +- .../sdk/transforms/windowing/AfterFirst.java | 9 +- .../sdk/transforms/windowing/AfterPane.java | 9 +- .../windowing/AfterProcessingTime.java | 9 +- .../AfterSynchronizedProcessingTime.java | 10 +- .../transforms/windowing/AfterWatermark.java | 12 +- .../transforms/windowing/BoundedWindow.java | 3 +- .../transforms/windowing/CalendarWindows.java | 1 - .../transforms/windowing/DefaultTrigger.java | 4 +- .../transforms/windowing/FixedWindows.java | 4 +- .../transforms/windowing/GlobalWindow.java | 6 +- .../transforms/windowing/GlobalWindows.java | 6 +- .../transforms/windowing/IntervalWindow.java | 10 +- .../transforms/windowing/InvalidWindows.java | 4 +- .../beam/sdk/transforms/windowing/Never.java | 4 +- .../windowing/OrFinallyTrigger.java | 7 +- .../transforms/windowing/OutputTimeFn.java | 7 +- .../transforms/windowing/OutputTimeFns.java | 7 +- .../sdk/transforms/windowing/PaneInfo.java | 14 +-- .../windowing/PartitioningWindowFn.java | 3 +- .../sdk/transforms/windowing/Repeatedly.java | 6 +- .../sdk/transforms/windowing/Sessions.java | 8 +- .../transforms/windowing/SlidingWindows.java | 10 +- .../sdk/transforms/windowing/Trigger.java | 16 +-- .../beam/sdk/transforms/windowing/Window.java | 4 +- .../sdk/transforms/windowing/WindowFn.java | 6 +- .../apache/beam/sdk/util/ActiveWindowSet.java | 6 +- .../org/apache/beam/sdk/util/ApiSurface.java | 6 +- .../beam/sdk/util/AppliedCombineFn.java | 6 +- ...temptAndTimeBoundedExponentialBackOff.java | 4 +- .../org/apache/beam/sdk/util/AvroUtils.java | 7 +- .../beam/sdk/util/BaseExecutionContext.java | 9 +- .../org/apache/beam/sdk/util/BitSetCoder.java | 7 +- .../beam/sdk/util/BucketingFunction.java | 3 +- .../BufferedElementCountingOutputStream.java | 4 +- .../apache/beam/sdk/util/CloudKnownType.java | 1 - .../org/apache/beam/sdk/util/CloudObject.java | 2 - .../org/apache/beam/sdk/util/CoderUtils.java | 17 ++- .../apache/beam/sdk/util/CombineFnUtil.java | 7 +- .../beam/sdk/util/CredentialFactory.java | 1 - .../org/apache/beam/sdk/util/Credentials.java | 9 +- .../beam/sdk/util/DirectSideInputReader.java | 5 +- .../beam/sdk/util/ExecutableTrigger.java | 7 +- .../beam/sdk/util/ExecutionContext.java | 5 +- .../sdk/util/ExposedByteArrayInputStream.java | 3 +- .../util/ExposedByteArrayOutputStream.java | 3 +- .../beam/sdk/util/FileIOChannelFactory.java | 6 +- .../beam/sdk/util/FinishedTriggersSet.java | 1 - .../beam/sdk/util/GcpCredentialFactory.java | 6 +- .../beam/sdk/util/GcsIOChannelFactory.java | 5 +- .../beam/sdk/util/GcsPathValidator.java | 3 +- .../org/apache/beam/sdk/util/GcsUtil.java | 16 +-- .../apache/beam/sdk/util/IOChannelUtils.java | 5 +- .../beam/sdk/util/IdentityWindowFn.java | 6 +- .../apache/beam/sdk/util/InstanceBuilder.java | 5 +- .../beam/sdk/util/KeyedWorkItemCoder.java | 17 ++- .../apache/beam/sdk/util/KeyedWorkItems.java | 4 +- .../beam/sdk/util/MergingActiveWindowSet.java | 21 ++-- .../apache/beam/sdk/util/MovingFunction.java | 3 +- .../beam/sdk/util/MutationDetectors.java | 5 +- .../sdk/util/NonMergingActiveWindowSet.java | 6 +- .../beam/sdk/util/NoopCredentialFactory.java | 4 +- .../beam/sdk/util/NullSideInputReader.java | 6 +- .../beam/sdk/util/PCollectionViewWindow.java | 3 +- .../beam/sdk/util/PCollectionViews.java | 25 ++-- .../java/org/apache/beam/sdk/util/PTuple.java | 3 +- .../beam/sdk/util/PerKeyCombineFnRunner.java | 5 +- .../beam/sdk/util/PerKeyCombineFnRunners.java | 6 +- .../apache/beam/sdk/util/PubsubClient.java | 5 +- .../beam/sdk/util/PubsubGrpcClient.java | 8 +- .../beam/sdk/util/PubsubJsonClient.java | 5 +- .../beam/sdk/util/PubsubTestClient.java | 5 +- .../org/apache/beam/sdk/util/ReleaseInfo.java | 6 +- .../org/apache/beam/sdk/util/Reshuffle.java | 1 - .../beam/sdk/util/ReshuffleTrigger.java | 4 +- .../sdk/util/RetryHttpRequestInitializer.java | 7 +- .../beam/sdk/util/SerializableUtils.java | 13 +- .../org/apache/beam/sdk/util/Serializer.java | 2 - .../apache/beam/sdk/util/SideInputReader.java | 3 +- .../org/apache/beam/sdk/util/StringUtils.java | 4 +- .../org/apache/beam/sdk/util/Structs.java | 2 - .../beam/sdk/util/SystemDoFnInternal.java | 3 +- .../apache/beam/sdk/util/TestCredential.java | 1 - .../apache/beam/sdk/util/TimerInternals.java | 26 ++-- .../java/org/apache/beam/sdk/util/Timers.java | 4 +- .../org/apache/beam/sdk/util/Transport.java | 8 +- .../beam/sdk/util/TriggerContextFactory.java | 18 ++- .../beam/sdk/util/UnownedInputStream.java | 1 - .../beam/sdk/util/UnownedOutputStream.java | 1 - .../sdk/util/UploadIdResponseInterceptor.java | 4 +- .../beam/sdk/util/ValueWithRecordId.java | 13 +- .../java/org/apache/beam/sdk/util/Values.java | 1 - .../apache/beam/sdk/util/WindowedValue.java | 32 +++-- .../beam/sdk/util/WindowingInternals.java | 6 +- .../beam/sdk/util/WindowingStrategy.java | 11 +- .../org/apache/beam/sdk/util/ZipFiles.java | 4 +- .../beam/sdk/util/common/ReflectHelpers.java | 3 - .../apache/beam/sdk/util/gcsfs/GcsPath.java | 2 - .../CopyOnAccessInMemoryStateInternals.java | 16 +-- .../util/state/InMemoryStateInternals.java | 13 +- .../sdk/util/state/MergingStateAccessor.java | 3 +- .../beam/sdk/util/state/StateContexts.java | 3 +- .../sdk/util/state/StateInternalsFactory.java | 3 +- .../beam/sdk/util/state/StateMerging.java | 9 +- .../beam/sdk/util/state/StateNamespaces.java | 10 +- .../beam/sdk/util/state/StateTable.java | 4 +- .../apache/beam/sdk/util/state/StateTag.java | 5 +- .../apache/beam/sdk/util/state/StateTags.java | 10 +- .../sdk/util/state/WatermarkHoldState.java | 1 - .../java/org/apache/beam/sdk/values/KV.java | 10 +- .../org/apache/beam/sdk/values/PBegin.java | 5 +- .../beam/sdk/values/PCollectionList.java | 12 +- .../beam/sdk/values/PCollectionTuple.java | 12 +- .../beam/sdk/values/PCollectionView.java | 3 +- .../org/apache/beam/sdk/values/PDone.java | 5 +- .../org/apache/beam/sdk/values/PInput.java | 3 +- .../org/apache/beam/sdk/values/POutput.java | 3 +- .../apache/beam/sdk/values/PValueBase.java | 5 +- .../beam/sdk/values/TimestampedValue.java | 12 +- .../org/apache/beam/sdk/values/TupleTag.java | 13 +- .../apache/beam/sdk/values/TupleTagList.java | 4 +- .../beam/sdk/values/TypeDescriptor.java | 2 - .../beam/sdk/values/TypeDescriptors.java | 1 - .../dataflow/util/GcsPathValidatorTest.java | 1 - .../sdk/AggregatorPipelineExtractorTest.java | 15 +-- .../org/apache/beam/sdk/DataflowMatchers.java | 4 +- .../org/apache/beam/sdk/PipelineTest.java | 4 +- .../java/org/apache/beam/sdk/TestUtils.java | 10 +- .../org/apache/beam/sdk/WindowMatchers.java | 9 +- .../apache/beam/sdk/WindowMatchersTest.java | 4 +- .../apache/beam/sdk/coders/AvroCoderTest.java | 60 +++++----- .../beam/sdk/coders/BigDecimalCoderTest.java | 6 +- .../sdk/coders/BigEndianIntegerCoderTest.java | 6 +- .../sdk/coders/BigEndianLongCoderTest.java | 6 +- .../beam/sdk/coders/BigIntegerCoderTest.java | 6 +- .../beam/sdk/coders/ByteArrayCoderTest.java | 10 +- .../apache/beam/sdk/coders/ByteCoderTest.java | 6 +- .../beam/sdk/coders/ByteStringCoderTest.java | 11 +- .../beam/sdk/coders/CoderFactoriesTest.java | 5 +- .../beam/sdk/coders/CoderProvidersTest.java | 4 +- .../beam/sdk/coders/CoderRegistryTest.java | 31 +++-- .../org/apache/beam/sdk/coders/CoderTest.java | 4 +- .../beam/sdk/coders/CollectionCoderTest.java | 14 +-- .../beam/sdk/coders/CustomCoderTest.java | 12 +- .../beam/sdk/coders/DefaultCoderTest.java | 7 +- .../beam/sdk/coders/DelegateCoderTest.java | 11 +- .../beam/sdk/coders/DoubleCoderTest.java | 6 +- .../beam/sdk/coders/DurationCoderTest.java | 9 +- .../beam/sdk/coders/InstantCoderTest.java | 15 +-- .../beam/sdk/coders/IterableCoderTest.java | 10 +- .../apache/beam/sdk/coders/JAXBCoderTest.java | 16 +-- .../apache/beam/sdk/coders/KvCoderTest.java | 13 +- .../apache/beam/sdk/coders/ListCoderTest.java | 10 +- .../apache/beam/sdk/coders/MapCoderTest.java | 17 ++- .../beam/sdk/coders/NullableCoderTest.java | 13 +- .../beam/sdk/coders/PrintBase64Encodings.java | 4 +- .../sdk/coders/SerializableCoderTest.java | 14 +-- .../apache/beam/sdk/coders/SetCoderTest.java | 12 +- .../beam/sdk/coders/StandardCoderTest.java | 16 +-- .../sdk/coders/StringDelegateCoderTest.java | 8 +- .../beam/sdk/coders/StringUtf8CoderTest.java | 6 +- .../sdk/coders/TableRowJsonCoderTest.java | 9 +- .../sdk/coders/TextualIntegerCoderTest.java | 6 +- .../beam/sdk/coders/VarIntCoderTest.java | 6 +- .../beam/sdk/coders/VarLongCoderTest.java | 6 +- .../sdk/coders/protobuf/ProtoCoderTest.java | 14 +-- .../sdk/coders/protobuf/ProtobufUtilTest.java | 10 +- .../beam/sdk/io/AvroIOGeneratedClassTest.java | 26 ++-- .../org/apache/beam/sdk/io/AvroIOTest.java | 30 ++--- .../apache/beam/sdk/io/AvroSourceTest.java | 46 ++++---- .../BoundedReadFromUnboundedSourceTest.java | 11 +- .../beam/sdk/io/CompressedSourceTest.java | 41 +++---- .../apache/beam/sdk/io/CountingInputTest.java | 2 - .../beam/sdk/io/CountingSourceTest.java | 6 +- .../apache/beam/sdk/io/FileBasedSinkTest.java | 22 ++-- .../beam/sdk/io/FileBasedSourceTest.java | 32 +++-- .../beam/sdk/io/OffsetBasedSourceTest.java | 11 +- .../org/apache/beam/sdk/io/PubsubIOTest.java | 5 +- .../beam/sdk/io/PubsubUnboundedSinkTest.java | 13 +- .../sdk/io/PubsubUnboundedSourceTest.java | 19 ++- .../java/org/apache/beam/sdk/io/ReadTest.java | 12 +- .../org/apache/beam/sdk/io/TextIOTest.java | 47 ++++---- .../org/apache/beam/sdk/io/WriteTest.java | 29 ++--- .../org/apache/beam/sdk/io/XmlSinkTest.java | 27 ++--- .../org/apache/beam/sdk/io/XmlSourceTest.java | 29 ++--- .../ByteKeyRangeEstimateFractionTest.java | 1 - .../range/ByteKeyRangeInterpolateKeyTest.java | 1 - .../beam/sdk/io/range/ByteKeyRangeTest.java | 6 +- .../apache/beam/sdk/io/range/ByteKeyTest.java | 3 +- .../beam/sdk/options/GcpOptionsTest.java | 16 +-- .../options/GoogleApiDebugOptionsTest.java | 8 +- .../options/PipelineOptionsFactoryTest.java | 27 ++--- .../options/PipelineOptionsReflectorTest.java | 6 +- .../beam/sdk/options/PipelineOptionsTest.java | 12 +- .../options/PipelineOptionsValidatorTest.java | 1 - .../options/ProxyInvocationHandlerTest.java | 28 ++--- .../beam/sdk/runners/PipelineRunnerTest.java | 1 - .../beam/sdk/runners/TransformTreeTest.java | 8 +- .../runners/dataflow/TestCountingSource.java | 13 +- .../dataflow/TestCountingSourceTest.java | 4 +- .../beam/sdk/testing/CoderPropertiesTest.java | 11 +- .../beam/sdk/testing/CrashingRunnerTest.java | 1 - .../apache/beam/sdk/testing/ExpectedLogs.java | 12 +- .../beam/sdk/testing/ExpectedLogsTest.java | 13 +- .../sdk/testing/FastNanoClockAndSleeper.java | 1 - .../testing/FastNanoClockAndSleeperTest.java | 3 +- .../sdk/testing/FileChecksumMatcherTest.java | 9 +- .../apache/beam/sdk/testing/PAssertTest.java | 17 ++- .../sdk/testing/PCollectionViewTesting.java | 15 +-- .../beam/sdk/testing/PaneExtractorsTest.java | 4 +- .../sdk/testing/RestoreSystemProperties.java | 5 +- .../sdk/testing/SerializableMatchersTest.java | 12 +- .../beam/sdk/testing/SourceTestUtilsTest.java | 9 +- .../beam/sdk/testing/StaticWindowsTest.java | 4 +- .../sdk/testing/SystemNanoTimeSleeper.java | 1 - .../testing/SystemNanoTimeSleeperTest.java | 1 - .../beam/sdk/testing/TestPipelineTest.java | 12 +- .../beam/sdk/testing/TestStreamTest.java | 4 +- .../beam/sdk/testing/WindowSupplierTest.java | 13 +- .../transforms/ApproximateQuantilesTest.java | 13 +- .../sdk/transforms/ApproximateUniqueTest.java | 18 ++- .../beam/sdk/transforms/CombineFnsTest.java | 18 ++- .../beam/sdk/transforms/CombineTest.java | 38 +++--- .../apache/beam/sdk/transforms/CountTest.java | 7 +- .../beam/sdk/transforms/CreateTest.java | 26 ++-- .../DoFnDelegatingAggregatorTest.java | 1 - .../apache/beam/sdk/transforms/DoFnTest.java | 4 +- .../beam/sdk/transforms/DoFnTesterTest.java | 4 +- .../beam/sdk/transforms/FilterTest.java | 5 +- .../sdk/transforms/FlatMapElementsTest.java | 16 +-- .../beam/sdk/transforms/FlattenTest.java | 17 ++- .../beam/sdk/transforms/GroupByKeyTest.java | 30 ++--- .../IntraBundleParallelizationTest.java | 7 +- .../apache/beam/sdk/transforms/KeysTest.java | 4 +- .../beam/sdk/transforms/KvSwapTest.java | 4 +- .../beam/sdk/transforms/MapElementsTest.java | 7 +- .../apache/beam/sdk/transforms/MaxTest.java | 5 +- .../apache/beam/sdk/transforms/MeanTest.java | 10 +- .../apache/beam/sdk/transforms/MinTest.java | 5 +- .../beam/sdk/transforms/NoOpOldDoFn.java | 1 - .../beam/sdk/transforms/OldDoFnTest.java | 9 +- .../beam/sdk/transforms/PTransformTest.java | 1 - .../sdk/transforms/ParDoLifecycleTest.java | 6 +- .../apache/beam/sdk/transforms/ParDoTest.java | 23 ++-- .../beam/sdk/transforms/PartitionTest.java | 9 +- .../sdk/transforms/RemoveDuplicatesTest.java | 10 +- .../beam/sdk/transforms/SampleTest.java | 21 ++-- .../sdk/transforms/SimpleStatsFnsTest.java | 7 +- .../apache/beam/sdk/transforms/SumTest.java | 5 +- .../apache/beam/sdk/transforms/TopTest.java | 13 +- .../beam/sdk/transforms/ValuesTest.java | 4 +- .../apache/beam/sdk/transforms/ViewTest.java | 31 ++--- .../beam/sdk/transforms/WithKeysTest.java | 6 +- .../sdk/transforms/WithTimestampsTest.java | 4 +- .../display/DisplayDataEvaluator.java | 8 +- .../display/DisplayDataEvaluatorTest.java | 7 +- .../display/DisplayDataMatchers.java | 7 +- .../display/DisplayDataMatchersTest.java | 2 - .../transforms/display/DisplayDataTest.java | 26 ++-- .../transforms/join/CoGbkResultCoderTest.java | 4 +- .../sdk/transforms/join/CoGbkResultTest.java | 6 +- .../sdk/transforms/join/CoGroupByKeyTest.java | 15 +-- .../sdk/transforms/join/UnionCoderTest.java | 4 +- .../transforms/reflect/DoFnInvokersTest.java | 1 - .../reflect/DoFnSignaturesTest.java | 9 +- .../transforms/windowing/AfterAllTest.java | 1 - .../transforms/windowing/AfterEachTest.java | 1 - .../transforms/windowing/AfterFirstTest.java | 1 - .../transforms/windowing/AfterPaneTest.java | 1 - .../windowing/AfterProcessingTimeTest.java | 1 - .../AfterSynchronizedProcessingTimeTest.java | 1 - .../windowing/AfterWatermarkTest.java | 1 - .../windowing/CalendarWindowsTest.java | 13 +- .../windowing/DefaultTriggerTest.java | 1 - .../windowing/FixedWindowsTest.java | 11 +- .../windowing/IntervalWindowTest.java | 7 +- .../sdk/transforms/windowing/NeverTest.java | 1 - .../windowing/OrFinallyTriggerTest.java | 1 - .../transforms/windowing/PaneInfoTest.java | 1 - .../transforms/windowing/RepeatedlyTest.java | 1 - .../transforms/windowing/SessionsTest.java | 16 +-- .../windowing/SlidingWindowsTest.java | 11 +- .../sdk/transforms/windowing/StubTrigger.java | 4 +- .../sdk/transforms/windowing/TriggerTest.java | 5 +- .../sdk/transforms/windowing/WindowTest.java | 5 +- .../transforms/windowing/WindowingTest.java | 10 +- .../apache/beam/sdk/util/ApiSurfaceTest.java | 10 +- ...tAndTimeBoundedExponentialBackOffTest.java | 4 +- .../AttemptBoundedExponentialBackOffTest.java | 1 - .../apache/beam/sdk/util/AvroUtilsTest.java | 20 ++-- .../beam/sdk/util/BucketingFunctionTest.java | 1 - ...fferedElementCountingOutputStreamTest.java | 19 ++- .../apache/beam/sdk/util/CoderUtilsTest.java | 7 +- .../beam/sdk/util/CombineFnUtilTest.java | 13 +- .../beam/sdk/util/ExecutableTriggerTest.java | 6 +- .../util/ExposedByteArrayInputStreamTest.java | 5 +- .../ExposedByteArrayOutputStreamTest.java | 5 +- .../sdk/util/FileIOChannelFactoryTest.java | 16 ++- .../sdk/util/FinishedTriggersSetTest.java | 3 +- .../beam/sdk/util/GatherAllPanesTest.java | 7 +- .../sdk/util/GcsIOChannelFactoryTest.java | 1 - .../org/apache/beam/sdk/util/GcsUtilTest.java | 23 ++-- .../beam/sdk/util/IOChannelUtilsTest.java | 6 +- .../sdk/util/IdentitySideInputWindowFn.java | 5 +- .../beam/sdk/util/InstanceBuilderTest.java | 1 - .../beam/sdk/util/KeyedWorkItemCoderTest.java | 4 +- .../sdk/util/MergingActiveWindowSetTest.java | 13 +- .../beam/sdk/util/MovingFunctionTest.java | 1 - .../beam/sdk/util/MutationDetectorsTest.java | 17 ++- .../org/apache/beam/sdk/util/PTupleTest.java | 1 - .../beam/sdk/util/PubsubClientTest.java | 7 +- .../beam/sdk/util/PubsubGrpcClientTest.java | 16 +-- .../beam/sdk/util/PubsubJsonClientTest.java | 15 +-- .../beam/sdk/util/PubsubTestClientTest.java | 17 ++- .../apache/beam/sdk/util/ReshuffleTest.java | 7 +- .../beam/sdk/util/ReshuffleTriggerTest.java | 1 - .../util/RetryHttpRequestInitializerTest.java | 12 +- .../beam/sdk/util/SerializableUtilsTest.java | 19 ++- .../apache/beam/sdk/util/StreamUtilsTest.java | 9 +- .../apache/beam/sdk/util/StringUtilsTest.java | 1 - .../org/apache/beam/sdk/util/StructsTest.java | 11 +- .../beam/sdk/util/TimerInternalsTest.java | 1 - .../apache/beam/sdk/util/TriggerTester.java | 31 ++--- .../beam/sdk/util/UnownedInputStreamTest.java | 3 +- .../sdk/util/UnownedOutputStreamTest.java | 3 +- .../util/UploadIdResponseInterceptorTest.java | 7 +- .../beam/sdk/util/UserCodeExceptionTest.java | 3 +- .../org/apache/beam/sdk/util/VarIntTest.java | 9 +- .../beam/sdk/util/WindowedValueTest.java | 9 +- .../apache/beam/sdk/util/ZipFilesTest.java | 14 +-- .../sdk/util/common/ReflectHelpersTest.java | 6 +- .../beam/sdk/util/gcsfs/GcsPathTest.java | 11 +- ...opyOnAccessInMemoryStateInternalsTest.java | 1 - .../state/InMemoryStateInternalsTest.java | 4 +- .../sdk/util/state/StateNamespacesTest.java | 1 - .../beam/sdk/util/state/StateTagTest.java | 1 - .../org/apache/beam/sdk/values/KVTest.java | 4 +- .../beam/sdk/values/PCollectionListTest.java | 3 +- .../beam/sdk/values/PCollectionTupleTest.java | 8 +- .../org/apache/beam/sdk/values/PDoneTest.java | 4 +- .../beam/sdk/values/TypeDescriptorTest.java | 10 +- .../beam/sdk/values/TypeDescriptorsTest.java | 6 +- .../beam/sdk/values/TypedPValueTest.java | 1 - .../extensions/joinlibrary/InnerJoinTest.java | 6 +- .../joinlibrary/OuterLeftJoinTest.java | 6 +- .../joinlibrary/OuterRightJoinTest.java | 6 +- .../io/gcp/bigquery/BigQueryAvroUtils.java | 7 +- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 111 +++++++++--------- .../sdk/io/gcp/bigquery/BigQueryServices.java | 5 +- .../io/gcp/bigquery/BigQueryServicesImpl.java | 20 ++-- .../gcp/bigquery/BigQueryTableInserter.java | 14 +-- .../bigquery/BigQueryTableRowIterator.java | 12 +- .../beam/sdk/io/gcp/bigtable/BigtableIO.java | 43 ++++--- .../sdk/io/gcp/bigtable/BigtableService.java | 6 +- .../io/gcp/bigtable/BigtableServiceImpl.java | 12 +- .../sdk/io/gcp/datastore/DatastoreV1.java | 47 ++++---- .../gcp/bigquery/BigQueryAvroUtilsTest.java | 11 +- .../sdk/io/gcp/bigquery/BigQueryIOTest.java | 73 ++++++------ .../bigquery/BigQueryServicesImplTest.java | 29 ++--- .../bigquery/BigQueryTableInserterTest.java | 20 ++-- .../BigQueryTableRowIteratorTest.java | 10 +- .../sdk/io/gcp/bigquery/BigQueryUtilTest.java | 19 ++- .../sdk/io/gcp/bigtable/BigtableIOTest.java | 53 ++++----- .../sdk/io/gcp/bigtable/BigtableReadIT.java | 6 +- .../sdk/io/gcp/bigtable/BigtableWriteIT.java | 29 ++--- .../sdk/io/gcp/datastore/DatastoreV1Test.java | 53 ++++----- .../beam/sdk/io/gcp/datastore/V1ReadIT.java | 13 +- .../sdk/io/gcp/datastore/V1TestOptions.java | 3 +- .../beam/sdk/io/gcp/datastore/V1TestUtil.java | 17 ++- .../beam/sdk/io/gcp/datastore/V1WriteIT.java | 4 +- .../beam/sdk/io/hdfs/AvroHDFSFileSource.java | 23 ++-- .../beam/sdk/io/hdfs/AvroWrapperCoder.java | 16 ++- .../apache/beam/sdk/io/hdfs/HDFSFileSink.java | 17 ++- .../beam/sdk/io/hdfs/HDFSFileSource.java | 30 ++--- .../beam/sdk/io/hdfs/WritableCoder.java | 12 +- .../SimpleAuthAvroHDFSFileSource.java | 12 +- .../simpleauth/SimpleAuthHDFSFileSink.java | 3 +- .../simpleauth/SimpleAuthHDFSFileSource.java | 12 +- .../sdk/io/hdfs/AvroWrapperCoderTest.java | 5 +- .../beam/sdk/io/hdfs/HDFSFileSourceTest.java | 12 +- .../beam/sdk/io/hdfs/WritableCoderTest.java | 1 - .../beam/sdk/io/jms/JmsCheckpointMark.java | 9 +- .../org/apache/beam/sdk/io/jms/JmsIO.java | 36 +++--- .../org/apache/beam/sdk/io/jms/JmsRecord.java | 1 - .../org/apache/beam/sdk/io/jms/JmsIOTest.java | 25 ++-- .../sdk/io/kafka/KafkaCheckpointMark.java | 8 +- .../org/apache/beam/sdk/io/kafka/KafkaIO.java | 59 +++++----- .../apache/beam/sdk/io/kafka/KafkaRecord.java | 3 +- .../beam/sdk/io/kafka/KafkaRecordCoder.java | 14 +-- .../apache/beam/sdk/io/kafka/KafkaIOTest.java | 33 +++--- .../beam/sdk/transforms/CombineJava8Test.java | 7 +- .../beam/sdk/transforms/FilterJava8Test.java | 4 +- .../transforms/FlatMapElementsJava8Test.java | 9 +- .../sdk/transforms/MapElementsJava8Test.java | 4 +- .../sdk/transforms/PartitionJava8Test.java | 4 +- .../transforms/RemoveDuplicatesJava8Test.java | 11 +- .../sdk/transforms/WithKeysJava8Test.java | 1 - .../transforms/WithTimestampsJava8Test.java | 4 +- .../src/main/java/DebuggingWordCount.java | 9 +- .../src/main/java/WindowedWordCount.java | 19 ++- .../java/common/DataflowExampleUtils.java | 25 ++-- .../common/ExampleBigQueryTableOptions.java | 3 +- .../main/java/common/PubsubFileInjector.java | 14 +-- .../src/test/java/DebuggingWordCountTest.java | 6 +- .../src/test/java/WordCountTest.java | 5 +- .../src/main/java/StarterPipeline.java | 1 - .../src/main/java/it/pkg/StarterPipeline.java | 1 - .../coders/AvroCoderBenchmark.java | 6 +- .../coders/ByteArrayCoderBenchmark.java | 6 +- .../coders/CoderBenchmarking.java | 3 +- .../coders/StringUtf8CoderBenchmark.java | 6 +- 856 files changed, 3599 insertions(+), 5533 deletions(-) diff --git a/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java index c1b273ca9bec..4a9aba9da5e8 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java @@ -17,6 +17,9 @@ */ package org.apache.beam.examples; +import java.util.Arrays; +import java.util.List; +import java.util.regex.Pattern; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.options.Default; @@ -29,14 +32,9 @@ import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Arrays; -import java.util.List; -import java.util.regex.Pattern; - /** * An example that verifies word counts in Shakespeare and includes Dataflow best practices. diff --git a/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java index 842cb54b40fd..6d69f1488526 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java @@ -17,6 +17,13 @@ */ package org.apache.beam.examples; +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableReference; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; import org.apache.beam.examples.common.ExampleBigQueryTableOptions; import org.apache.beam.examples.common.ExampleOptions; import org.apache.beam.examples.common.ExampleUtils; @@ -33,19 +40,9 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - -import com.google.api.services.bigquery.model.TableFieldSchema; -import com.google.api.services.bigquery.model.TableReference; -import com.google.api.services.bigquery.model.TableRow; -import com.google.api.services.bigquery.model.TableSchema; - import org.joda.time.Duration; import org.joda.time.Instant; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - /** * An example that counts words in text, and can run over either unbounded or bounded input diff --git a/examples/java/src/main/java/org/apache/beam/examples/WordCount.java b/examples/java/src/main/java/org/apache/beam/examples/WordCount.java index 42d30bb5fa94..d42d6214973d 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/WordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/WordCount.java @@ -17,6 +17,9 @@ */ package org.apache.beam.examples; +import com.google.common.base.Strings; +import com.google.common.io.Resources; +import java.io.IOException; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.options.Default; @@ -37,11 +40,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; -import com.google.common.base.Strings; -import com.google.common.io.Resources; - -import java.io.IOException; - /** * An example that counts words in Shakespeare and includes Beam best practices. * diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/ExampleBigQueryTableOptions.java b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleBigQueryTableOptions.java index 54cc99ea32fc..2eef525faed0 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/common/ExampleBigQueryTableOptions.java +++ b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleBigQueryTableOptions.java @@ -17,14 +17,13 @@ */ package org.apache.beam.examples.common; +import com.google.api.services.bigquery.model.TableSchema; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.DefaultValueFactory; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.GcpOptions; import org.apache.beam.sdk.options.PipelineOptions; -import com.google.api.services.bigquery.model.TableSchema; - /** * Options that can be used to configure BigQuery tables in Beam examples. * The project defaults to the project being used to run the example. diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/ExampleOptions.java b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleOptions.java index 43afeb4c6112..a7dcc7c17517 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/common/ExampleOptions.java +++ b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleOptions.java @@ -17,21 +17,18 @@ */ package org.apache.beam.examples.common; +import com.google.common.base.MoreObjects; +import java.util.concurrent.ThreadLocalRandom; import org.apache.beam.sdk.options.ApplicationNameOptions; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.DefaultValueFactory; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.PipelineOptions; - -import com.google.common.base.MoreObjects; - import org.joda.time.DateTimeUtils; import org.joda.time.DateTimeZone; import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; -import java.util.concurrent.ThreadLocalRandom; - /** * Options that can be used to configure the Beam examples. */ diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/ExampleUtils.java b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleUtils.java index 7f03fc0d1c6f..eadb580a257c 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/common/ExampleUtils.java +++ b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleUtils.java @@ -17,13 +17,6 @@ */ package org.apache.beam.examples.common; -import org.apache.beam.sdk.PipelineResult; -import org.apache.beam.sdk.options.BigQueryOptions; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.options.PubsubOptions; -import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; -import org.apache.beam.sdk.util.Transport; - import com.google.api.client.googleapis.json.GoogleJsonResponseException; import com.google.api.client.googleapis.services.AbstractGoogleClientRequest; import com.google.api.client.util.BackOff; @@ -43,12 +36,17 @@ import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.google.common.util.concurrent.Uninterruptibles; - import java.io.IOException; import java.util.Collection; import java.util.List; import java.util.Set; import java.util.concurrent.TimeUnit; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.options.BigQueryOptions; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PubsubOptions; +import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; +import org.apache.beam.sdk.util.Transport; /** * The utility class that sets up and tears down external resources, diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/PubsubFileInjector.java b/examples/java/src/main/java/org/apache/beam/examples/common/PubsubFileInjector.java index 0a93521821d8..e6a1495e545d 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/common/PubsubFileInjector.java +++ b/examples/java/src/main/java/org/apache/beam/examples/common/PubsubFileInjector.java @@ -17,6 +17,12 @@ */ package org.apache.beam.examples.common; +import com.google.api.services.pubsub.Pubsub; +import com.google.api.services.pubsub.model.PublishRequest; +import com.google.api.services.pubsub.model.PubsubMessage; +import com.google.common.collect.ImmutableMap; +import java.io.IOException; +import java.util.Arrays; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.options.Description; @@ -28,14 +34,6 @@ import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.util.Transport; -import com.google.api.services.pubsub.Pubsub; -import com.google.api.services.pubsub.model.PublishRequest; -import com.google.api.services.pubsub.model.PubsubMessage; -import com.google.common.collect.ImmutableMap; - -import java.io.IOException; -import java.util.Arrays; - /** * A batch Dataflow pipeline for injecting a set of GCS files into * a PubSub topic line by line. Empty lines are skipped. diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java b/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java index 120c64fae54a..56c7855e45e0 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java @@ -21,6 +21,21 @@ import static com.google.datastore.v1.client.DatastoreHelper.makeKey; import static com.google.datastore.v1.client.DatastoreHelper.makeValue; +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableReference; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import com.google.common.base.MoreObjects; +import com.google.datastore.v1.Entity; +import com.google.datastore.v1.Key; +import com.google.datastore.v1.Value; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; import org.apache.beam.examples.common.ExampleBigQueryTableOptions; import org.apache.beam.examples.common.ExampleOptions; import org.apache.beam.examples.common.ExampleUtils; @@ -53,26 +68,8 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; - -import com.google.api.services.bigquery.model.TableFieldSchema; -import com.google.api.services.bigquery.model.TableReference; -import com.google.api.services.bigquery.model.TableRow; -import com.google.api.services.bigquery.model.TableSchema; -import com.google.common.base.MoreObjects; -import com.google.datastore.v1.Entity; -import com.google.datastore.v1.Key; -import com.google.datastore.v1.Value; - import org.joda.time.Duration; -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.regex.Matcher; -import java.util.regex.Pattern; - /** * An example that computes the most popular hash tags * for every prefix, which can be used for auto-completion. diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java b/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java index 3f30f2150504..c4ad35bcc818 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java @@ -17,6 +17,11 @@ */ package org.apache.beam.examples.complete; +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import java.io.IOException; +import java.util.ArrayList; import org.apache.beam.examples.common.ExampleBigQueryTableOptions; import org.apache.beam.examples.common.ExampleOptions; import org.apache.beam.examples.common.ExampleUtils; @@ -31,13 +36,6 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; -import com.google.api.services.bigquery.model.TableFieldSchema; -import com.google.api.services.bigquery.model.TableRow; -import com.google.api.services.bigquery.model.TableSchema; - -import java.io.IOException; -import java.util.ArrayList; - /** * A streaming Dataflow Example using BigQuery output. * diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java index 76b6b6a06008..a5a939263ee4 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java @@ -17,6 +17,12 @@ */ package org.apache.beam.examples.complete; +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.HashSet; +import java.util.Set; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; @@ -51,17 +57,9 @@ import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.TupleTag; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.File; -import java.io.IOException; -import java.net.URI; -import java.net.URISyntaxException; -import java.util.HashSet; -import java.util.Set; - /** * An example that computes a basic TF-IDF search table for a directory or GCS prefix. * diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java index aff41ccb5a4e..4c5472bf6f68 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java @@ -17,6 +17,8 @@ */ package org.apache.beam.examples.complete; +import com.google.api.services.bigquery.model.TableRow; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.TableRowJsonCoder; import org.apache.beam.sdk.io.TextIO; @@ -38,14 +40,9 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - -import com.google.api.services.bigquery.model.TableRow; - import org.joda.time.Duration; import org.joda.time.Instant; -import java.util.List; - /** * An example that reads Wikipedia edit data from Cloud Storage and computes the user with * the longest string of edits separated by no more than an hour within each month. diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java index 394b4327025c..f7ff035dbf50 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java @@ -17,6 +17,14 @@ */ package org.apache.beam.examples.complete; +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableReference; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import org.apache.avro.reflect.Nullable; import org.apache.beam.examples.common.ExampleBigQueryTableOptions; import org.apache.beam.examples.common.ExampleOptions; import org.apache.beam.examples.common.ExampleUtils; @@ -39,22 +47,11 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; - -import com.google.api.services.bigquery.model.TableFieldSchema; -import com.google.api.services.bigquery.model.TableReference; -import com.google.api.services.bigquery.model.TableRow; -import com.google.api.services.bigquery.model.TableSchema; - -import org.apache.avro.reflect.Nullable; import org.joda.time.Duration; import org.joda.time.Instant; import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - /** * A Dataflow Example that runs in both batch and streaming modes with traffic sensor data. * You can configure the running mode by setting {@literal --streaming} to true or false. diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java index ef716e9dd832..12eaeaa97257 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java @@ -17,6 +17,19 @@ */ package org.apache.beam.examples.complete; +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableReference; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import com.google.common.collect.Lists; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.Hashtable; +import java.util.List; +import java.util.Map; +import org.apache.avro.reflect.Nullable; import org.apache.beam.examples.common.ExampleBigQueryTableOptions; import org.apache.beam.examples.common.ExampleOptions; import org.apache.beam.examples.common.ExampleUtils; @@ -38,27 +51,11 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; - -import com.google.api.services.bigquery.model.TableFieldSchema; -import com.google.api.services.bigquery.model.TableReference; -import com.google.api.services.bigquery.model.TableRow; -import com.google.api.services.bigquery.model.TableSchema; -import com.google.common.collect.Lists; - -import org.apache.avro.reflect.Nullable; import org.joda.time.Duration; import org.joda.time.Instant; import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.Hashtable; -import java.util.List; -import java.util.Map; - /** * A Dataflow Example that runs in both batch and streaming modes with traffic sensor data. * You can configure the running mode by setting {@literal --streaming} to true or false. diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java index 09d9c29734e9..439cf020aaa9 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java @@ -17,6 +17,11 @@ */ package org.apache.beam.examples.cookbook; +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import java.util.ArrayList; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO; import org.apache.beam.sdk.options.Default; @@ -31,13 +36,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; -import com.google.api.services.bigquery.model.TableFieldSchema; -import com.google.api.services.bigquery.model.TableRow; -import com.google.api.services.bigquery.model.TableSchema; - -import java.util.ArrayList; -import java.util.List; - /** * An example that reads the public samples of weather data from BigQuery, counts the number of * tornadoes that occur in each month, and writes the results to BigQuery. diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java index 67918a3f7428..1d280a6e154b 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java @@ -17,6 +17,11 @@ */ package org.apache.beam.examples.cookbook; +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import java.util.ArrayList; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO; import org.apache.beam.sdk.options.Default; @@ -34,13 +39,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; -import com.google.api.services.bigquery.model.TableFieldSchema; -import com.google.api.services.bigquery.model.TableRow; -import com.google.api.services.bigquery.model.TableSchema; - -import java.util.ArrayList; -import java.util.List; - /** * An example that reads the public 'Shakespeare' data, and for each word in * the dataset that is over a given length, generates a string containing the diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java index 215e2ffc7898..9a9e79968670 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java @@ -22,6 +22,14 @@ import static com.google.datastore.v1.client.DatastoreHelper.makeKey; import static com.google.datastore.v1.client.DatastoreHelper.makeValue; +import com.google.datastore.v1.Entity; +import com.google.datastore.v1.Key; +import com.google.datastore.v1.PropertyFilter; +import com.google.datastore.v1.Query; +import com.google.datastore.v1.Value; +import java.util.Map; +import java.util.UUID; +import javax.annotation.Nullable; import org.apache.beam.examples.WordCount; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.TextIO; @@ -36,16 +44,6 @@ import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.ParDo; -import com.google.datastore.v1.Entity; -import com.google.datastore.v1.Key; -import com.google.datastore.v1.PropertyFilter; -import com.google.datastore.v1.Query; -import com.google.datastore.v1.Value; - -import java.util.Map; -import java.util.UUID; -import javax.annotation.Nullable; - /** * A WordCount example using DatastoreIO. * diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java index 9a0f7a2a5493..6c42520ff7c6 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java @@ -17,6 +17,12 @@ */ package org.apache.beam.examples.cookbook; +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import java.util.ArrayList; +import java.util.List; +import java.util.logging.Logger; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO; import org.apache.beam.sdk.options.Default; @@ -32,14 +38,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; -import com.google.api.services.bigquery.model.TableFieldSchema; -import com.google.api.services.bigquery.model.TableRow; -import com.google.api.services.bigquery.model.TableSchema; - -import java.util.ArrayList; -import java.util.List; -import java.util.logging.Logger; - /** * This is an example that demonstrates several approaches to filtering, and use of the Mean * transform. It shows how to dynamically set parameters by defining and using new pipeline options, diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java index 5ff2ce22a0d9..1b91bf1e0615 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java @@ -17,6 +17,7 @@ */ package org.apache.beam.examples.cookbook; +import com.google.api.services.bigquery.model.TableRow; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO; @@ -33,8 +34,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TupleTag; -import com.google.api.services.bigquery.model.TableRow; - /** * This example shows how to do a join on two collections. * It uses a sample of the GDELT 'world event' data (http://goo.gl/OB6oin), joining the event diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java index 4f266d30ac96..3772a7bc5b86 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java @@ -17,6 +17,11 @@ */ package org.apache.beam.examples.cookbook; +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import java.util.ArrayList; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO; import org.apache.beam.sdk.options.Default; @@ -31,13 +36,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; -import com.google.api.services.bigquery.model.TableFieldSchema; -import com.google.api.services.bigquery.model.TableRow; -import com.google.api.services.bigquery.model.TableSchema; - -import java.util.ArrayList; -import java.util.List; - /** * An example that reads the public samples of weather data from BigQuery, and finds * the maximum temperature ('mean_temp') for each month. diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java index 04ac2c363ed6..db5943555545 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java @@ -17,6 +17,13 @@ */ package org.apache.beam.examples.cookbook; +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableReference; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.TimeUnit; import org.apache.beam.examples.common.ExampleBigQueryTableOptions; import org.apache.beam.examples.common.ExampleOptions; import org.apache.beam.examples.common.ExampleUtils; @@ -42,19 +49,9 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; - -import com.google.api.services.bigquery.model.TableFieldSchema; -import com.google.api.services.bigquery.model.TableReference; -import com.google.api.services.bigquery.model.TableRow; -import com.google.api.services.bigquery.model.TableSchema; - import org.joda.time.Duration; import org.joda.time.Instant; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.TimeUnit; - /** * This example illustrates the basic concepts behind triggering. It shows how to use different * trigger definitions to produce partial (speculative) results before all the data is processed and diff --git a/examples/java/src/test/java/org/apache/beam/examples/DebuggingWordCountTest.java b/examples/java/src/test/java/org/apache/beam/examples/DebuggingWordCountTest.java index f463b1e651db..c1bd5d45e38b 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/DebuggingWordCountTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/DebuggingWordCountTest.java @@ -18,16 +18,14 @@ package org.apache.beam.examples; import com.google.common.io.Files; - +import java.io.File; +import java.nio.charset.StandardCharsets; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.File; -import java.nio.charset.StandardCharsets; - /** * Tests for {@link DebuggingWordCount}. */ diff --git a/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java b/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java index f93dc2b2f779..ca0c9d6bda09 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java +++ b/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java @@ -18,6 +18,7 @@ package org.apache.beam.examples; +import java.util.Date; import org.apache.beam.examples.WordCount.WordCountOptions; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.PipelineOptionsFactory; @@ -25,13 +26,10 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.testing.TestPipelineOptions; import org.apache.beam.sdk.util.IOChannelUtils; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Date; - /** * End-to-end tests of WordCount. */ diff --git a/examples/java/src/test/java/org/apache/beam/examples/WordCountTest.java b/examples/java/src/test/java/org/apache/beam/examples/WordCountTest.java index 9d36a3e3e970..98c5b17621be 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/WordCountTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/WordCountTest.java @@ -17,6 +17,8 @@ */ package org.apache.beam.examples; +import java.util.Arrays; +import java.util.List; import org.apache.beam.examples.WordCount.CountWords; import org.apache.beam.examples.WordCount.ExtractWordsFn; import org.apache.beam.examples.WordCount.FormatAsTextFn; @@ -30,7 +32,6 @@ import org.apache.beam.sdk.transforms.DoFnTester; import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.values.PCollection; - import org.hamcrest.CoreMatchers; import org.junit.Assert; import org.junit.Test; @@ -38,9 +39,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** * Tests of WordCount. */ diff --git a/examples/java/src/test/java/org/apache/beam/examples/complete/AutoCompleteTest.java b/examples/java/src/test/java/org/apache/beam/examples/complete/AutoCompleteTest.java index 6f28dec0ecdb..b6751c528bc8 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/complete/AutoCompleteTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/complete/AutoCompleteTest.java @@ -17,6 +17,11 @@ */ package org.apache.beam.examples.complete; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; import org.apache.beam.examples.complete.AutoComplete.CompletionCandidate; import org.apache.beam.examples.complete.AutoComplete.ComputeTopCompletions; import org.apache.beam.sdk.Pipeline; @@ -33,19 +38,12 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.List; - /** * Tests of AutoComplete. */ diff --git a/examples/java/src/test/java/org/apache/beam/examples/complete/TfIdfTest.java b/examples/java/src/test/java/org/apache/beam/examples/complete/TfIdfTest.java index c7ce67e0c695..c2d654ec18d9 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/complete/TfIdfTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/complete/TfIdfTest.java @@ -17,6 +17,8 @@ */ package org.apache.beam.examples.complete; +import java.net.URI; +import java.util.Arrays; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.StringDelegateCoder; import org.apache.beam.sdk.testing.PAssert; @@ -27,15 +29,11 @@ import org.apache.beam.sdk.transforms.RemoveDuplicates; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.net.URI; -import java.util.Arrays; - /** * Tests of {@link TfIdf}. */ diff --git a/examples/java/src/test/java/org/apache/beam/examples/complete/TopWikipediaSessionsTest.java b/examples/java/src/test/java/org/apache/beam/examples/complete/TopWikipediaSessionsTest.java index d19998ee3896..42fb06a031b3 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/complete/TopWikipediaSessionsTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/complete/TopWikipediaSessionsTest.java @@ -17,22 +17,19 @@ */ package org.apache.beam.examples.complete; +import com.google.api.services.bigquery.model.TableRow; +import java.util.Arrays; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.RunnableOnService; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.values.PCollection; - -import com.google.api.services.bigquery.model.TableRow; - import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; - /** Unit tests for {@link TopWikipediaSessions}. */ @RunWith(JUnit4.class) public class TopWikipediaSessionsTest { diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryTornadoesIT.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryTornadoesIT.java index fbd775cf50c8..8bcab4a70522 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryTornadoesIT.java +++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryTornadoesIT.java @@ -21,7 +21,6 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.testing.TestPipelineOptions; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryTornadoesTest.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryTornadoesTest.java index b986c0bdfa3b..87e1614ee0ca 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryTornadoesTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryTornadoesTest.java @@ -17,21 +17,18 @@ */ package org.apache.beam.examples.cookbook; +import com.google.api.services.bigquery.model.TableRow; +import java.util.List; import org.apache.beam.examples.cookbook.BigQueryTornadoes.ExtractTornadoesFn; import org.apache.beam.examples.cookbook.BigQueryTornadoes.FormatCountsFn; import org.apache.beam.sdk.transforms.DoFnTester; import org.apache.beam.sdk.values.KV; - -import com.google.api.services.bigquery.model.TableRow; - import org.hamcrest.CoreMatchers; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.List; - /** * Test case for {@link BigQueryTornadoes}. */ diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/CombinePerKeyExamplesTest.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/CombinePerKeyExamplesTest.java index 6d0b16793865..34e06799e640 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/cookbook/CombinePerKeyExamplesTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/CombinePerKeyExamplesTest.java @@ -17,21 +17,18 @@ */ package org.apache.beam.examples.cookbook; +import com.google.api.services.bigquery.model.TableRow; +import java.util.List; import org.apache.beam.examples.cookbook.CombinePerKeyExamples.ExtractLargeWordsFn; import org.apache.beam.examples.cookbook.CombinePerKeyExamples.FormatShakespeareOutputFn; import org.apache.beam.sdk.transforms.DoFnTester; import org.apache.beam.sdk.values.KV; - -import com.google.api.services.bigquery.model.TableRow; - import org.hamcrest.CoreMatchers; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.List; - /** Unit tests for {@link CombinePerKeyExamples}. */ @RunWith(JUnit4.class) public class CombinePerKeyExamplesTest { diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/DeDupExampleTest.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/DeDupExampleTest.java index 20e247062c49..c725e4f6bf02 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/cookbook/DeDupExampleTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/DeDupExampleTest.java @@ -17,6 +17,8 @@ */ package org.apache.beam.examples.cookbook; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.testing.PAssert; @@ -25,15 +27,11 @@ import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.RemoveDuplicates; import org.apache.beam.sdk.values.PCollection; - import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** Unit tests for {@link DeDupExample}. */ @RunWith(JUnit4.class) public class DeDupExampleTest { diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/FilterExamplesTest.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/FilterExamplesTest.java index 2598a971dd2f..279478c50a24 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/cookbook/FilterExamplesTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/FilterExamplesTest.java @@ -17,21 +17,18 @@ */ package org.apache.beam.examples.cookbook; +import com.google.api.services.bigquery.model.TableRow; +import java.util.Arrays; +import java.util.List; import org.apache.beam.examples.cookbook.FilterExamples.FilterSingleMonthDataFn; import org.apache.beam.examples.cookbook.FilterExamples.ProjectionFn; import org.apache.beam.sdk.transforms.DoFnTester; - -import com.google.api.services.bigquery.model.TableRow; - import org.hamcrest.CoreMatchers; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** Unit tests for {@link FilterExamples}. */ @RunWith(JUnit4.class) public class FilterExamplesTest { diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/JoinExamplesTest.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/JoinExamplesTest.java index 9b0466740755..60f71a2120f5 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/cookbook/JoinExamplesTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/JoinExamplesTest.java @@ -17,6 +17,9 @@ */ package org.apache.beam.examples.cookbook; +import com.google.api.services.bigquery.model.TableRow; +import java.util.Arrays; +import java.util.List; import org.apache.beam.examples.cookbook.JoinExamples.ExtractCountryInfoFn; import org.apache.beam.examples.cookbook.JoinExamples.ExtractEventDataFn; import org.apache.beam.sdk.Pipeline; @@ -27,9 +30,6 @@ import org.apache.beam.sdk.transforms.DoFnTester; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - -import com.google.api.services.bigquery.model.TableRow; - import org.hamcrest.CoreMatchers; import org.junit.Assert; import org.junit.Test; @@ -37,9 +37,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** Unit tests for {@link JoinExamples}. */ @RunWith(JUnit4.class) public class JoinExamplesTest { diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/MaxPerKeyExamplesTest.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/MaxPerKeyExamplesTest.java index 1d5bcf473c1c..b5ea0fc4bf0e 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/cookbook/MaxPerKeyExamplesTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/MaxPerKeyExamplesTest.java @@ -17,22 +17,19 @@ */ package org.apache.beam.examples.cookbook; +import com.google.api.services.bigquery.model.TableRow; +import com.google.common.collect.ImmutableList; +import java.util.List; import org.apache.beam.examples.cookbook.MaxPerKeyExamples.ExtractTempFn; import org.apache.beam.examples.cookbook.MaxPerKeyExamples.FormatMaxesFn; import org.apache.beam.sdk.transforms.DoFnTester; import org.apache.beam.sdk.values.KV; - -import com.google.api.services.bigquery.model.TableRow; -import com.google.common.collect.ImmutableList; - import org.hamcrest.CoreMatchers; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.List; - /** Unit tests for {@link MaxPerKeyExamples}. */ @RunWith(JUnit4.class) public class MaxPerKeyExamplesTest { diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/TriggerExampleTest.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/TriggerExampleTest.java index fee3c141358d..3848ca1135e8 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/cookbook/TriggerExampleTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/TriggerExampleTest.java @@ -17,6 +17,13 @@ */ package org.apache.beam.examples.cookbook; +import com.google.api.services.bigquery.model.TableRow; +import com.google.common.base.Joiner; +import com.google.common.collect.Lists; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; import org.apache.beam.examples.cookbook.TriggerExample.ExtractFlowInfo; import org.apache.beam.examples.cookbook.TriggerExample.TotalFlow; import org.apache.beam.sdk.Pipeline; @@ -32,11 +39,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; - -import com.google.api.services.bigquery.model.TableRow; -import com.google.common.base.Joiner; -import com.google.common.collect.Lists; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Assert; @@ -45,11 +47,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Map; - /** * Unit Tests for {@link TriggerExample}. * The results generated by triggers are by definition non-deterministic and hence hard to test. diff --git a/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java b/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java index ff8ca552b57c..a49da7bdfbb6 100644 --- a/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java +++ b/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java @@ -17,6 +17,7 @@ */ package org.apache.beam.examples; +import java.util.Arrays; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.options.PipelineOptions; @@ -28,8 +29,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TypeDescriptors; -import java.util.Arrays; - /** * An example that counts words in Shakespeare, using Java 8 language features. * diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java index 01ffb1de6883..f9957ebc0e3b 100644 --- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java +++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java @@ -17,6 +17,9 @@ */ package org.apache.beam.examples.complete.game; +import java.util.HashMap; +import java.util.Map; +import java.util.TimeZone; import org.apache.beam.examples.common.ExampleUtils; import org.apache.beam.examples.complete.game.utils.WriteWindowedToBigQuery; import org.apache.beam.sdk.Pipeline; @@ -45,7 +48,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TypeDescriptors; - import org.joda.time.DateTimeZone; import org.joda.time.Duration; import org.joda.time.Instant; @@ -54,10 +56,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.HashMap; -import java.util.Map; -import java.util.TimeZone; - /** * This class is the fourth in a series of four pipelines that tell a story in a 'gaming' * domain, following {@link UserScore}, {@link HourlyTeamScore}, and {@link LeaderBoard}. diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java index e489607dee4d..d408e2132dab 100644 --- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java +++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java @@ -17,6 +17,9 @@ */ package org.apache.beam.examples.complete.game; +import java.util.HashMap; +import java.util.Map; +import java.util.TimeZone; import org.apache.beam.examples.complete.game.utils.WriteWindowedToBigQuery; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.TextIO; @@ -30,17 +33,12 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; - import org.joda.time.DateTimeZone; import org.joda.time.Duration; import org.joda.time.Instant; import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; -import java.util.HashMap; -import java.util.Map; -import java.util.TimeZone; - /** * This class is the second in a series of four pipelines that tell a story in a 'gaming' * domain, following {@link UserScore}. In addition to the concepts introduced in {@link UserScore}, diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java index bd223059e305..8dd4e39bee01 100644 --- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java +++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java @@ -17,6 +17,9 @@ */ package org.apache.beam.examples.complete.game; +import java.util.HashMap; +import java.util.Map; +import java.util.TimeZone; import org.apache.beam.examples.common.ExampleOptions; import org.apache.beam.examples.common.ExampleUtils; import org.apache.beam.examples.complete.game.utils.WriteToBigQuery; @@ -39,17 +42,12 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - import org.joda.time.DateTimeZone; import org.joda.time.Duration; import org.joda.time.Instant; import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; -import java.util.HashMap; -import java.util.Map; -import java.util.TimeZone; - /** * This class is the third in a series of four pipelines that tell a story in a 'gaming' domain, * following {@link UserScore} and {@link HourlyTeamScore}. Concepts include: processing unbounded diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java index c97eb4152991..65036cee6b92 100644 --- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java +++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java @@ -17,6 +17,9 @@ */ package org.apache.beam.examples.complete.game; +import java.util.HashMap; +import java.util.Map; +import org.apache.avro.reflect.Nullable; import org.apache.beam.examples.complete.game.utils.WriteToBigQuery; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.AvroCoder; @@ -36,14 +39,9 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptors; - -import org.apache.avro.reflect.Nullable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.HashMap; -import java.util.Map; - /** * This class is the first in a series of four pipelines that tell a story in a 'gaming' domain. * Concepts: batch processing; reading input from Google Cloud Storage and writing output to diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/Injector.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/Injector.java index 034a18646292..8f8bd9febc31 100644 --- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/Injector.java +++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/Injector.java @@ -21,11 +21,6 @@ import com.google.api.services.pubsub.model.PublishRequest; import com.google.api.services.pubsub.model.PubsubMessage; import com.google.common.collect.ImmutableMap; - -import org.joda.time.DateTimeZone; -import org.joda.time.format.DateTimeFormat; -import org.joda.time.format.DateTimeFormatter; - import java.io.BufferedOutputStream; import java.io.FileOutputStream; import java.io.IOException; @@ -36,6 +31,9 @@ import java.util.List; import java.util.Random; import java.util.TimeZone; +import org.joda.time.DateTimeZone; +import org.joda.time.format.DateTimeFormat; +import org.joda.time.format.DateTimeFormatter; /** diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/InjectorUtils.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/InjectorUtils.java index 53e644d67fe7..8cba6c2d05f0 100644 --- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/InjectorUtils.java +++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/InjectorUtils.java @@ -29,7 +29,6 @@ import com.google.api.services.pubsub.Pubsub; import com.google.api.services.pubsub.PubsubScopes; import com.google.api.services.pubsub.model.Topic; - import java.io.IOException; class InjectorUtils { diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/RetryHttpInitializerWrapper.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/RetryHttpInitializerWrapper.java index 45be28791fc7..059999cae7d2 100644 --- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/RetryHttpInitializerWrapper.java +++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/RetryHttpInitializerWrapper.java @@ -28,7 +28,6 @@ import com.google.api.client.http.HttpUnsuccessfulResponseHandler; import com.google.api.client.util.ExponentialBackOff; import com.google.api.client.util.Sleeper; - import java.io.IOException; import java.util.logging.Logger; diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteToBigQuery.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteToBigQuery.java index 6af6e15a5321..40c4286f3afe 100644 --- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteToBigQuery.java +++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteToBigQuery.java @@ -17,6 +17,14 @@ */ package org.apache.beam.examples.complete.game.utils; +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableReference; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; import org.apache.beam.examples.complete.game.UserScore; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO; @@ -31,16 +39,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; -import com.google.api.services.bigquery.model.TableFieldSchema; -import com.google.api.services.bigquery.model.TableReference; -import com.google.api.services.bigquery.model.TableRow; -import com.google.api.services.bigquery.model.TableSchema; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; - /** * Generate, format, and write BigQuery table row information. Use provided information about * the field names and types, as well as lambda functions that describe how to generate their diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteWindowedToBigQuery.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteWindowedToBigQuery.java index c59fd61f68a9..09f3b6cf90d9 100644 --- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteWindowedToBigQuery.java +++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteWindowedToBigQuery.java @@ -17,6 +17,8 @@ */ package org.apache.beam.examples.complete.game.utils; +import com.google.api.services.bigquery.model.TableRow; +import java.util.Map; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition; @@ -27,10 +29,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; -import com.google.api.services.bigquery.model.TableRow; - -import java.util.Map; - /** * Generate, format, and write BigQuery table row information. Subclasses {@link WriteToBigQuery} * to require windowing; so this subclass may be used for writes that require access to the diff --git a/examples/java8/src/test/java/org/apache/beam/examples/MinimalWordCountJava8Test.java b/examples/java8/src/test/java/org/apache/beam/examples/MinimalWordCountJava8Test.java index 4dfa474c7cf5..85841a780147 100644 --- a/examples/java8/src/test/java/org/apache/beam/examples/MinimalWordCountJava8Test.java +++ b/examples/java8/src/test/java/org/apache/beam/examples/MinimalWordCountJava8Test.java @@ -17,6 +17,15 @@ */ package org.apache.beam.examples; +import com.google.common.collect.ImmutableList; +import java.io.IOException; +import java.io.Serializable; +import java.nio.channels.FileChannel; +import java.nio.channels.SeekableByteChannel; +import java.nio.file.Files; +import java.nio.file.StandardOpenOption; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.options.GcsOptions; @@ -29,9 +38,6 @@ import org.apache.beam.sdk.util.gcsfs.GcsPath; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TypeDescriptors; - -import com.google.common.collect.ImmutableList; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -39,15 +45,6 @@ import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import java.io.IOException; -import java.io.Serializable; -import java.nio.channels.FileChannel; -import java.nio.channels.SeekableByteChannel; -import java.nio.file.Files; -import java.nio.file.StandardOpenOption; -import java.util.Arrays; -import java.util.List; - /** * To keep {@link MinimalWordCountJava8} simple, it is not factored or testable. This test * file should be maintained with a copy of its code for a basic smoke test. diff --git a/examples/java8/src/test/java/org/apache/beam/examples/complete/game/GameStatsTest.java b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/GameStatsTest.java index df8800ddb5f6..7cd03f365b34 100644 --- a/examples/java8/src/test/java/org/apache/beam/examples/complete/game/GameStatsTest.java +++ b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/GameStatsTest.java @@ -17,6 +17,9 @@ */ package org.apache.beam.examples.complete.game; +import java.io.Serializable; +import java.util.Arrays; +import java.util.List; import org.apache.beam.examples.complete.game.GameStats.CalculateSpammyUsers; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.PAssert; @@ -25,16 +28,11 @@ import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.Arrays; -import java.util.List; - /** * Tests of GameStats. * Because the pipeline was designed for easy readability and explanations, it lacks good diff --git a/examples/java8/src/test/java/org/apache/beam/examples/complete/game/HourlyTeamScoreTest.java b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/HourlyTeamScoreTest.java index b917b4cf535f..f9fefb61f35c 100644 --- a/examples/java8/src/test/java/org/apache/beam/examples/complete/game/HourlyTeamScoreTest.java +++ b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/HourlyTeamScoreTest.java @@ -17,6 +17,9 @@ */ package org.apache.beam.examples.complete.game; +import java.io.Serializable; +import java.util.Arrays; +import java.util.List; import org.apache.beam.examples.complete.game.UserScore.GameActionInfo; import org.apache.beam.examples.complete.game.UserScore.ParseEventFn; import org.apache.beam.sdk.Pipeline; @@ -31,17 +34,12 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptors; - import org.joda.time.Instant; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.Arrays; -import java.util.List; - /** * Tests of HourlyTeamScore. * Because the pipeline was designed for easy readability and explanations, it lacks good diff --git a/examples/java8/src/test/java/org/apache/beam/examples/complete/game/UserScoreTest.java b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/UserScoreTest.java index 75d371a8caa4..7c86adf3754f 100644 --- a/examples/java8/src/test/java/org/apache/beam/examples/complete/game/UserScoreTest.java +++ b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/UserScoreTest.java @@ -17,6 +17,9 @@ */ package org.apache.beam.examples.complete.game; +import java.io.Serializable; +import java.util.Arrays; +import java.util.List; import org.apache.beam.examples.complete.game.UserScore.ExtractAndSumScore; import org.apache.beam.examples.complete.game.UserScore.GameActionInfo; import org.apache.beam.examples.complete.game.UserScore.ParseEventFn; @@ -32,17 +35,12 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptors; - import org.junit.Assert; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.Arrays; -import java.util.List; - /** * Tests of UserScore. */ diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java index 65502517219c..a97d3f306d0e 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java @@ -17,6 +17,14 @@ */ package org.apache.beam.runners.core; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.SetCoder; import org.apache.beam.sdk.transforms.Combine; @@ -31,15 +39,6 @@ import org.apache.beam.sdk.util.state.ValueState; import org.apache.beam.sdk.values.PCollectionView; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import javax.annotation.Nullable; - /** * Generic side input handler that uses {@link StateInternals} to store all data. Both the actual * side-input data and data about the windows for which we have side inputs available are stored diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSource.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSource.java index 3ce0c060c061..73688d45592e 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSource.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSource.java @@ -17,11 +17,25 @@ */ package org.apache.beam.runners.core; -import static org.apache.beam.sdk.util.StringUtils.approximateSimpleName; - import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; +import static org.apache.beam.sdk.util.StringUtils.approximateSimpleName; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Function; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.ListCoder; @@ -40,29 +54,10 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.TimestampedValue; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.Arrays; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.NoSuchElementException; - -import javax.annotation.Nullable; - /** * {@link PTransform} that converts a {@link BoundedSource} as an {@link UnboundedSource}. * diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/AssignWindowsDoFn.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/AssignWindowsDoFn.java index 739db45e667a..7e26253a829a 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/AssignWindowsDoFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/AssignWindowsDoFn.java @@ -19,18 +19,15 @@ import static com.google.common.base.Preconditions.checkNotNull; +import com.google.common.collect.Iterables; +import java.util.Collection; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.OldDoFn.RequiresWindowAccess; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.WindowFn; - -import com.google.common.collect.Iterables; - import org.joda.time.Instant; -import java.util.Collection; - /** * {@link OldDoFn} that tags elements of a {@link PCollection} with windows, according to the * provided {@link WindowFn}. diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/BatchTimerInternals.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/BatchTimerInternals.java index 0dd03ba085e5..f3e84a6861cb 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/BatchTimerInternals.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/BatchTimerInternals.java @@ -19,17 +19,13 @@ import static com.google.common.base.Preconditions.checkState; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; - import com.google.common.base.MoreObjects; - -import org.joda.time.Instant; - import java.util.HashSet; import java.util.PriorityQueue; import java.util.Set; - import javax.annotation.Nullable; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.joda.time.Instant; /** * TimerInternals that uses priority queues to manage the timers that are ready to fire. diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java index ed9ec10f0c68..8a0f6bf868d9 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java @@ -19,6 +19,17 @@ import static com.google.common.base.Preconditions.checkNotNull; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import java.io.IOException; +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.options.PipelineOptions; @@ -37,19 +48,8 @@ import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.common.collect.Sets; import org.joda.time.Instant; import org.joda.time.format.PeriodFormat; -import java.io.IOException; -import java.util.Collection; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Set; /** * A base implementation of {@link DoFnRunner}. diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunners.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunners.java index 60892281c7b7..c4df7b222036 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunners.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunners.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.util; +import java.util.List; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory; import org.apache.beam.sdk.transforms.OldDoFn; @@ -26,8 +27,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; -import java.util.List; - /** * Static utility methods that provide {@link DoFnRunner} implementations. */ diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFn.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFn.java index f872ffcf1bb0..cc418daec7e9 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFn.java @@ -17,18 +17,15 @@ */ package org.apache.beam.sdk.util; +import com.google.common.collect.Iterables; +import java.util.List; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.util.state.StateInternalsFactory; import org.apache.beam.sdk.values.KV; - -import com.google.common.collect.Iterables; - import org.joda.time.Instant; -import java.util.List; - /** * The default batch {@link GroupAlsoByWindowsDoFn} implementation, if no specialized "fast path" * implementation is applicable. diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupByKeyViaGroupByKeyOnly.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupByKeyViaGroupByKeyOnly.java index f0f900744e73..fdad17a8bb0c 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupByKeyViaGroupByKeyOnly.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupByKeyViaGroupByKeyOnly.java @@ -19,6 +19,10 @@ import static com.google.common.base.Preconditions.checkArgument; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.coders.KvCoder; @@ -32,11 +36,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.List; - /** * An implementation of {@link GroupByKey} built on top of a lower-level {@link GroupByKeyOnly} * primitive. diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunner.java index 8b3ba24bb21c..08c670e4753f 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunner.java @@ -17,16 +17,14 @@ */ package org.apache.beam.sdk.util; -import org.apache.beam.sdk.transforms.Aggregator; -import org.apache.beam.sdk.transforms.OldDoFn; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.values.KV; - import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Predicate; import com.google.common.collect.Iterables; - +import org.apache.beam.sdk.transforms.Aggregator; +import org.apache.beam.sdk.transforms.OldDoFn; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.values.KV; import org.joda.time.Instant; /** diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/PaneInfoTracker.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/PaneInfoTracker.java index 0c5849e99cdc..90c10b5c7449 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/PaneInfoTracker.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/PaneInfoTracker.java @@ -19,6 +19,8 @@ import static com.google.common.base.Preconditions.checkState; +import com.google.common.annotations.VisibleForTesting; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import org.apache.beam.sdk.transforms.windowing.AfterWatermark; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.PaneInfo.PaneInfoCoder; @@ -28,11 +30,7 @@ import org.apache.beam.sdk.util.state.StateTag; import org.apache.beam.sdk.util.state.StateTags; import org.apache.beam.sdk.util.state.ValueState; - -import com.google.common.annotations.VisibleForTesting; - import org.joda.time.Instant; -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; /** * Determine the timing and other properties of a new pane for a given computation, key and window. diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunner.java index b1442dd1e24a..d9f1fbffccb3 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunner.java @@ -17,16 +17,14 @@ */ package org.apache.beam.sdk.util; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.values.PCollectionView; - import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; - import java.util.Collection; import java.util.Collections; import java.util.HashSet; import java.util.Set; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.values.PCollectionView; /** * A {@link DoFnRunner} that can refuse to process elements that are not ready, instead returning diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFn.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFn.java index c5ee1e1f940d..8135a5beca53 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFn.java @@ -17,16 +17,14 @@ */ package org.apache.beam.sdk.util; +import java.io.Serializable; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.state.MergingStateAccessor; import org.apache.beam.sdk.util.state.ReadableState; import org.apache.beam.sdk.util.state.StateAccessor; - import org.joda.time.Instant; -import java.io.Serializable; - /** * Specification for processing to happen after elements have been grouped by key. * diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnContextFactory.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnContextFactory.java index 2d865086f74d..5d27d51ad2f4 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnContextFactory.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnContextFactory.java @@ -20,6 +20,10 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; +import com.google.common.collect.ImmutableMap; +import java.util.Collection; +import java.util.Map; +import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; @@ -36,16 +40,8 @@ import org.apache.beam.sdk.util.state.StateNamespaces; import org.apache.beam.sdk.util.state.StateNamespaces.WindowNamespace; import org.apache.beam.sdk.util.state.StateTag; - -import com.google.common.collect.ImmutableMap; - import org.joda.time.Instant; -import java.util.Collection; -import java.util.Map; - -import javax.annotation.Nullable; - /** * Factory for creating instances of the various {@link ReduceFn} contexts. */ diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java index 1fa083097aad..61e5b21ebfd9 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java @@ -20,6 +20,17 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableMap; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import javax.annotation.Nullable; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.OldDoFn; @@ -41,24 +52,9 @@ import org.apache.beam.sdk.util.state.StateNamespaces.WindowNamespace; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.ImmutableMap; - import org.joda.time.Duration; import org.joda.time.Instant; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import javax.annotation.Nullable; - /** * Manages the execution of a {@link ReduceFn} after a {@link GroupByKeyOnly} has partitioned the * {@link PCollection} by key. diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/SimpleDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/SimpleDoFnRunner.java index a0cdb40f779f..6c1cf451d61a 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/SimpleDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/SimpleDoFnRunner.java @@ -17,13 +17,13 @@ */ package org.apache.beam.sdk.util; +import java.util.List; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.util.DoFnRunners.OutputManager; import org.apache.beam.sdk.util.ExecutionContext.StepContext; import org.apache.beam.sdk.values.TupleTag; -import java.util.List; /** * Runs a {@link OldDoFn} by constructing the appropriate contexts and passing them in. diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/SystemReduceFn.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/SystemReduceFn.java index f7dca9433ec8..28177a811cd3 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/SystemReduceFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/SystemReduceFn.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.util; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.Combine.KeyedCombineFn; @@ -34,8 +35,6 @@ import org.apache.beam.sdk.util.state.StateTag; import org.apache.beam.sdk.util.state.StateTags; -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; - /** * {@link ReduceFn} implementing the default reduction behaviors of {@link GroupByKey}. * diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/TriggerRunner.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/TriggerRunner.java index e00b843e5341..a53fb8c17513 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/TriggerRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/TriggerRunner.java @@ -19,6 +19,12 @@ import static com.google.common.base.Preconditions.checkState; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableMap; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import java.util.BitSet; +import java.util.Collection; +import java.util.Map; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.DefaultTrigger; import org.apache.beam.sdk.transforms.windowing.Trigger; @@ -27,18 +33,8 @@ import org.apache.beam.sdk.util.state.StateTag; import org.apache.beam.sdk.util.state.StateTags; import org.apache.beam.sdk.util.state.ValueState; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.ImmutableMap; - import org.joda.time.Instant; -import java.util.BitSet; -import java.util.Collection; -import java.util.Map; - -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; - /** * Executes a trigger while managing persistence of information about which subtriggers are * finished. Subtriggers include all recursive trigger expressions as well as the entire trigger. diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java index 5c1700992c29..7404e1bc8ba3 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java @@ -19,6 +19,10 @@ import static com.google.common.base.Preconditions.checkState; +import com.google.common.annotations.VisibleForTesting; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import java.io.Serializable; +import javax.annotation.Nullable; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; import org.apache.beam.sdk.transforms.windowing.OutputTimeFns; @@ -30,17 +34,9 @@ import org.apache.beam.sdk.util.state.StateTag; import org.apache.beam.sdk.util.state.StateTags; import org.apache.beam.sdk.util.state.WatermarkHoldState; - -import com.google.common.annotations.VisibleForTesting; - import org.joda.time.Duration; import org.joda.time.Instant; -import java.io.Serializable; -import javax.annotation.Nullable; - -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; - /** * Implements the logic to hold the output watermark for a computation back * until it has seen all the elements it needs based on the input watermark for the diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SideInputHandlerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SideInputHandlerTest.java index 641e25e5e0f5..0bf5e90d0d6a 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SideInputHandlerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SideInputHandlerTest.java @@ -21,6 +21,7 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import com.google.common.collect.ImmutableList; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.testing.PCollectionViewTesting; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; @@ -32,9 +33,6 @@ import org.apache.beam.sdk.util.state.InMemoryStateInternals; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; - -import com.google.common.collect.ImmutableList; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Assert; diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSourceTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSourceTest.java index 45c1414e2a57..efc446e008fa 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSourceTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSourceTest.java @@ -21,6 +21,18 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.nio.channels.ReadableByteChannel; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.Random; import org.apache.beam.runners.core.UnboundedReadFromBoundedSource.BoundedToUnboundedSourceAdapter; import org.apache.beam.runners.core.UnboundedReadFromBoundedSource.BoundedToUnboundedSourceAdapter.Checkpoint; import org.apache.beam.runners.core.UnboundedReadFromBoundedSource.BoundedToUnboundedSourceAdapter.CheckpointCoder; @@ -45,11 +57,6 @@ import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; - import org.joda.time.Instant; import org.junit.Rule; import org.junit.Test; @@ -59,16 +66,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.io.OutputStream; -import java.nio.ByteBuffer; -import java.nio.channels.ReadableByteChannel; -import java.util.List; -import java.util.NoSuchElementException; -import java.util.Random; - /** * Unit tests for {@link UnboundedReadFromBoundedSource}. */ diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/BatchTimerInternalsTest.java b/runners/core-java/src/test/java/org/apache/beam/sdk/util/BatchTimerInternalsTest.java index 3e1528f3ac50..20a9852b0958 100644 --- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/BatchTimerInternalsTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/sdk/util/BatchTimerInternalsTest.java @@ -20,7 +20,6 @@ import org.apache.beam.sdk.util.TimerInternals.TimerData; import org.apache.beam.sdk.util.state.StateNamespace; import org.apache.beam.sdk.util.state.StateNamespaceForTest; - import org.joda.time.Instant; import org.junit.Before; import org.junit.Test; diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsProperties.java b/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsProperties.java index 43c287e43415..215cd4c38cb3 100644 --- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsProperties.java +++ b/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsProperties.java @@ -23,6 +23,14 @@ import static org.hamcrest.Matchers.hasSize; import static org.junit.Assert.assertThat; +import com.google.common.cache.CacheBuilder; +import com.google.common.cache.CacheLoader; +import com.google.common.cache.LoadingCache; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.DoFnTester; import org.apache.beam.sdk.transforms.DoFnTester.CloningBehavior; @@ -38,20 +46,9 @@ import org.apache.beam.sdk.util.state.StateInternalsFactory; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TimestampedValue; - -import com.google.common.cache.CacheBuilder; -import com.google.common.cache.CacheLoader; -import com.google.common.cache.LoadingCache; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; - import org.joda.time.Duration; import org.joda.time.Instant; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; - /** * Properties of {@link GroupAlsoByWindowsDoFn}. * diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFnTest.java b/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFnTest.java index 1f02a8f997b9..a1586c8bf2d9 100644 --- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFnTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFnTest.java @@ -22,7 +22,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.GroupAlsoByWindowsProperties.GroupAlsoByWindowsDoFnFactory; import org.apache.beam.sdk.util.state.StateInternalsFactory; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunnerTest.java index d929d3943011..c63e43ef55b0 100644 --- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunnerTest.java @@ -22,16 +22,15 @@ import static org.junit.Assert.assertThat; import static org.mockito.Mockito.when; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import java.util.Arrays; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.LateDataDroppingDoFnRunner.LateDataFilter; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Before; @@ -41,8 +40,6 @@ import org.mockito.Mock; import org.mockito.MockitoAnnotations; -import java.util.Arrays; - /** * Unit tests for {@link LateDataDroppingDoFnRunner}. */ diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunnerTest.java index 88851187bb65..f8ad291d9bc9 100644 --- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunnerTest.java @@ -24,6 +24,9 @@ import static org.junit.Assert.assertThat; import static org.mockito.Mockito.when; +import com.google.common.collect.ImmutableList; +import java.util.ArrayList; +import java.util.List; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.Sum; @@ -34,9 +37,6 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; - -import com.google.common.collect.ImmutableList; - import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Before; @@ -47,9 +47,6 @@ import org.mockito.Mockito; import org.mockito.MockitoAnnotations; -import java.util.ArrayList; -import java.util.List; - /** * Tests for {@link PushbackSideInputDoFnRunner}. */ diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java index 8d604cb2b9e0..647495cc344a 100644 --- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java @@ -17,11 +17,9 @@ */ package org.apache.beam.sdk.util; +import static com.google.common.base.Preconditions.checkArgument; import static org.apache.beam.sdk.WindowMatchers.isSingleWindowedValue; import static org.apache.beam.sdk.WindowMatchers.isWindowedValue; - -import static com.google.common.base.Preconditions.checkArgument; - import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.emptyIterable; @@ -37,6 +35,9 @@ import static org.mockito.Mockito.when; import static org.mockito.Mockito.withSettings; +import com.google.common.collect.Iterables; +import java.util.Iterator; +import java.util.List; import org.apache.beam.sdk.WindowMatchers; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.options.PipelineOptions; @@ -67,9 +68,6 @@ import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TimestampedValue; - -import com.google.common.collect.Iterables; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Before; @@ -83,9 +81,6 @@ import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import java.util.Iterator; -import java.util.List; - /** * Tests for {@link ReduceFnRunner}. These tests instantiate a full "stack" of * {@link ReduceFnRunner} with enclosed {@link ReduceFn}, down to the installed {@link Trigger} diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java b/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java index feba191267e4..24e33ddf310c 100644 --- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java +++ b/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java @@ -19,11 +19,30 @@ import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import com.google.common.base.Function; +import com.google.common.base.MoreObjects; +import com.google.common.collect.FluentIterable; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; +import com.google.common.collect.Sets; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.PriorityQueue; +import java.util.Set; +import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.coders.IterableCoder; @@ -57,33 +76,9 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; - -import com.google.common.base.Function; -import com.google.common.base.MoreObjects; -import com.google.common.collect.FluentIterable; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Iterables; -import com.google.common.collect.Sets; - import org.joda.time.Duration; import org.joda.time.Instant; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.PriorityQueue; -import java.util.Set; - -import javax.annotation.Nullable; - /** * Test utility that runs a {@link ReduceFn}, {@link WindowFn}, {@link Trigger} using in-memory stub * implementations to provide the {@link TimerInternals} and {@link WindowingInternals} needed to diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/SimpleDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/sdk/util/SimpleDoFnRunnerTest.java index f0c52b99a377..156b4a92ca17 100644 --- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/SimpleDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/sdk/util/SimpleDoFnRunnerTest.java @@ -20,19 +20,17 @@ import static org.hamcrest.Matchers.is; import static org.mockito.Mockito.mock; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.util.BaseExecutionContext.StepContext; import org.apache.beam.sdk.values.TupleTag; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** * Tests for base {@link DoFnRunnerBase} functionality. */ diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AggregatorContainer.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AggregatorContainer.java index 75e6558bb7ee..06490dc55355 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AggregatorContainer.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AggregatorContainer.java @@ -17,10 +17,6 @@ */ package org.apache.beam.runners.direct; -import org.apache.beam.sdk.transforms.Aggregator; -import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory; -import org.apache.beam.sdk.transforms.Combine.CombineFn; -import org.apache.beam.sdk.util.ExecutionContext; import com.google.auto.value.AutoValue; import com.google.common.base.Preconditions; import java.util.Arrays; @@ -30,6 +26,10 @@ import java.util.concurrent.ConcurrentMap; import javax.annotation.Nullable; import javax.annotation.concurrent.GuardedBy; +import org.apache.beam.sdk.transforms.Aggregator; +import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory; +import org.apache.beam.sdk.transforms.Combine.CombineFn; +import org.apache.beam.sdk.util.ExecutionContext; /** * AccumT container for the current values associated with {@link Aggregator Aggregators}. diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java index 0c4b7fd9bc80..2b15ad002eb5 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java @@ -17,6 +17,12 @@ */ package org.apache.beam.runners.direct; +import java.io.IOException; +import java.util.Queue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ConcurrentMap; +import javax.annotation.Nullable; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; import org.apache.beam.sdk.io.BoundedSource; @@ -28,14 +34,6 @@ import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; -import java.io.IOException; -import java.util.Queue; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.ConcurrentMap; - -import javax.annotation.Nullable; - /** * A {@link TransformEvaluatorFactory} that produces {@link TransformEvaluator TransformEvaluators} * for the {@link Bounded Read.Bounded} primitive {@link PTransform}. diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CommittedResult.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CommittedResult.java index 7e0cd8e1d49a..5fcf7b313b7f 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CommittedResult.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CommittedResult.java @@ -18,15 +18,12 @@ package org.apache.beam.runners.direct; -import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; -import org.apache.beam.sdk.transforms.AppliedPTransform; -import org.apache.beam.sdk.transforms.View.CreatePCollectionView; - import com.google.auto.value.AutoValue; - import java.util.Set; - import javax.annotation.Nullable; +import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; +import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.transforms.View.CreatePCollectionView; /** * A {@link TransformResult} that has been committed. diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitor.java index 3300723ad298..4fdfea071702 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitor.java @@ -19,6 +19,12 @@ import static com.google.common.base.Preconditions.checkState; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.Pipeline.PipelineVisitor; import org.apache.beam.sdk.runners.PipelineRunner; @@ -29,13 +35,6 @@ import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.PValue; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; - /** * Tracks the {@link AppliedPTransform AppliedPTransforms} that consume each {@link PValue} in the * {@link Pipeline}. This is used to schedule consuming {@link PTransform PTransforms} to consume diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRegistrar.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRegistrar.java index 772777e0c835..9a34d3d9dfb4 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRegistrar.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRegistrar.java @@ -17,14 +17,13 @@ */ package org.apache.beam.runners.direct; +import com.google.auto.service.AutoService; +import com.google.common.collect.ImmutableList; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsRegistrar; import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.runners.PipelineRunnerRegistrar; -import com.google.auto.service.AutoService; -import com.google.common.collect.ImmutableList; - /** * Contains the {@link PipelineRunnerRegistrar} and {@link PipelineOptionsRegistrar} for the * {@link org.apache.beam.runners.direct.DirectRunner}. 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 68184dec7d85..b2d61c30aa3d 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 @@ -17,6 +17,17 @@ */ package org.apache.beam.runners.direct; +import com.google.common.base.MoreObjects; +import com.google.common.base.Supplier; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import java.io.IOException; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly; import org.apache.beam.runners.direct.DirectRunner.DirectPipelineResult; import org.apache.beam.runners.direct.TestStreamEvaluatorFactory.DirectTestStreamFactory; @@ -46,23 +57,9 @@ import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.PValue; - -import com.google.common.base.MoreObjects; -import com.google.common.base.Supplier; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; - import org.joda.time.Duration; import org.joda.time.Instant; -import java.io.IOException; -import java.util.Collection; -import java.util.HashMap; -import java.util.Map; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; - /** * An In-Memory implementation of the Dataflow Programming Model. Supports Unbounded * {@link PCollection PCollections}. diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java index a4705ddce40d..400398361385 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java @@ -17,15 +17,13 @@ */ package org.apache.beam.runners.direct; +import javax.annotation.Nullable; import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate; import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate.TimerUpdateBuilder; import org.apache.beam.runners.direct.WatermarkManager.TransformWatermarks; import org.apache.beam.sdk.util.TimerInternals; - import org.joda.time.Instant; -import javax.annotation.Nullable; - /** * An implementation of {@link TimerInternals} where all relevant data exists in memory. */ diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java index 3f4f2c6a880d..0e15c18c8110 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java @@ -18,21 +18,18 @@ package org.apache.beam.runners.direct; -import org.apache.beam.sdk.runners.PipelineRunner; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.OldDoFn; -import org.apache.beam.sdk.util.SerializableUtils; - import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.util.ArrayList; import java.util.Collection; import java.util.Iterator; +import org.apache.beam.sdk.runners.PipelineRunner; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.OldDoFn; +import org.apache.beam.sdk.util.SerializableUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Manages {@link DoFn} setup, teardown, and serialization. diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluator.java index 523273cdd1ec..faa06151c31b 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluator.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluator.java @@ -19,7 +19,6 @@ package org.apache.beam.runners.direct; import org.apache.beam.sdk.util.WindowedValue; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java index b9f159a2d598..5af25bc5e806 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java @@ -19,6 +19,17 @@ import static com.google.common.base.Preconditions.checkNotNull; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import com.google.common.util.concurrent.MoreExecutors; +import java.util.Collection; +import java.util.EnumSet; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import javax.annotation.Nullable; import org.apache.beam.runners.direct.CommittedResult.OutputType; import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; @@ -42,23 +53,8 @@ import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; -import com.google.common.util.concurrent.MoreExecutors; - import org.joda.time.Instant; -import java.util.Collection; -import java.util.EnumSet; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; - -import javax.annotation.Nullable; - /** * The evaluation context for a specific pipeline being executed by the * {@link DirectRunner}. Contains state shared within the execution across all diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java index 35b6239de81c..401ed7fb80c4 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java @@ -17,20 +17,6 @@ */ package org.apache.beam.runners.direct; -import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; -import org.apache.beam.runners.direct.WatermarkManager.FiredTimers; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.transforms.AppliedPTransform; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.util.KeyedWorkItem; -import org.apache.beam.sdk.util.KeyedWorkItems; -import org.apache.beam.sdk.util.TimeDomain; -import org.apache.beam.sdk.util.TimerInternals.TimerData; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.sdk.values.PValue; - import com.google.auto.value.AutoValue; import com.google.common.base.MoreObjects; import com.google.common.base.Optional; @@ -39,10 +25,6 @@ import com.google.common.cache.LoadingCache; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -56,8 +38,22 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; - import javax.annotation.Nullable; +import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; +import org.apache.beam.runners.direct.WatermarkManager.FiredTimers; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.KeyedWorkItem; +import org.apache.beam.sdk.util.KeyedWorkItems; +import org.apache.beam.sdk.util.TimeDomain; +import org.apache.beam.sdk.util.TimerInternals.TimerData; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.PValue; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * An {@link PipelineExecutor} that uses an underlying {@link ExecutorService} and diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java index d16ffa0bf517..2da70bbe5628 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java @@ -17,6 +17,8 @@ */ package org.apache.beam.runners.direct; +import com.google.common.collect.ImmutableMap; +import java.util.Collections; import org.apache.beam.runners.core.GroupAlsoByWindowViaWindowSetDoFn; import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext; import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupAlsoByWindow; @@ -40,10 +42,6 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; -import com.google.common.collect.ImmutableMap; - -import java.util.Collections; - /** * The {@link DirectRunner} {@link TransformEvaluatorFactory} for the * {@link GroupByKeyOnly} {@link PTransform}. diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java index dbdbdaf4ea34..f085a39fda9b 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java @@ -17,9 +17,14 @@ */ package org.apache.beam.runners.direct; -import static org.apache.beam.sdk.util.CoderUtils.encodeToByteArray; import static com.google.common.base.Preconditions.checkState; +import static org.apache.beam.sdk.util.CoderUtils.encodeToByteArray; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; @@ -37,12 +42,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - /** * The {@link DirectRunner} {@link TransformEvaluatorFactory} for the * {@link GroupByKeyOnly} {@link PTransform}. diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java index 8be12fdb3673..d5c0f0c354e6 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java @@ -19,6 +19,8 @@ import static com.google.common.base.Preconditions.checkNotNull; +import com.google.common.collect.HashMultimap; +import com.google.common.collect.SetMultimap; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; import org.apache.beam.sdk.coders.Coder; @@ -29,10 +31,6 @@ import org.apache.beam.sdk.util.MutationDetectors; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.HashMultimap; -import com.google.common.collect.SetMultimap; - import org.joda.time.Instant; /** diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactory.java index b0eb38f13add..1602f68ab3a9 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactory.java @@ -17,6 +17,8 @@ */ package org.apache.beam.runners.direct; +import java.util.IdentityHashMap; +import java.util.Map; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; @@ -27,9 +29,6 @@ import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue; -import java.util.IdentityHashMap; -import java.util.Map; - /** * {@link ModelEnforcement} that enforces elements are not modified over the course of processing * an element. diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutableListBundleFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutableListBundleFactory.java index 25a0d05cb128..e79da7baa751 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutableListBundleFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutableListBundleFactory.java @@ -19,15 +19,13 @@ import static com.google.common.base.Preconditions.checkState; +import com.google.common.base.MoreObjects; +import com.google.common.collect.ImmutableList; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.base.MoreObjects; -import com.google.common.collect.ImmutableList; - import org.joda.time.Instant; /** diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java index 2fea00a6e5d4..7c4376a2c4b4 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java @@ -19,15 +19,14 @@ import static com.google.common.base.Preconditions.checkState; +import java.util.HashSet; +import java.util.Set; import org.apache.beam.sdk.Pipeline.PipelineVisitor; import org.apache.beam.sdk.runners.TransformTreeNode; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.PValue; -import java.util.HashSet; -import java.util.Set; - /** * A pipeline visitor that tracks all keyed {@link PValue PValues}. A {@link PValue} is keyed if it * is the result of a {@link PTransform} that produces keyed outputs. A {@link PTransform} that diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/NanosOffsetClock.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/NanosOffsetClock.java index 77fa1967c434..5a2b18da0f24 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/NanosOffsetClock.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/NanosOffsetClock.java @@ -17,9 +17,8 @@ */ package org.apache.beam.runners.direct; -import org.joda.time.Instant; - import java.util.concurrent.TimeUnit; +import org.joda.time.Instant; /** * A {@link Clock} that uses {@link System#nanoTime()} to track the progress of time. diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java index 6ef0ffe29c4a..85a1c6af1385 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java @@ -17,6 +17,12 @@ */ package org.apache.beam.runners.direct; +import com.google.common.collect.ImmutableList; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; @@ -34,14 +40,6 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; -import com.google.common.collect.ImmutableList; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - class ParDoEvaluator implements TransformEvaluator { public static ParDoEvaluator create( EvaluationContext evaluationContext, diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java index 2d05e68b6659..6a41adffe6a9 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java @@ -17,6 +17,10 @@ */ package org.apache.beam.runners.direct; +import com.google.common.cache.CacheBuilder; +import com.google.common.cache.CacheLoader; +import com.google.common.cache.LoadingCache; +import java.util.Map; import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.sdk.transforms.AppliedPTransform; @@ -26,16 +30,9 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.TupleTag; - -import com.google.common.cache.CacheBuilder; -import com.google.common.cache.CacheLoader; -import com.google.common.cache.LoadingCache; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Map; - /** * The {@link DirectRunner} {@link TransformEvaluatorFactory} for the * {@link BoundMulti} primitive {@link PTransform}. diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java index 97cbfa7f7b2f..4bb740be012f 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java @@ -17,6 +17,11 @@ */ package org.apache.beam.runners.direct; +import com.google.common.cache.CacheBuilder; +import com.google.common.cache.CacheLoader; +import com.google.common.cache.LoadingCache; +import com.google.common.collect.ImmutableMap; +import java.util.Collections; import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.sdk.transforms.AppliedPTransform; @@ -25,17 +30,9 @@ import org.apache.beam.sdk.transforms.ParDo.Bound; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TupleTag; - -import com.google.common.cache.CacheBuilder; -import com.google.common.cache.CacheLoader; -import com.google.common.cache.LoadingCache; -import com.google.common.collect.ImmutableMap; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Collections; - /** * The {@link DirectRunner} {@link TransformEvaluatorFactory} for the * {@link Bound ParDo.Bound} primitive {@link PTransform}. diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PipelineExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PipelineExecutor.java index 76df11cc2588..01a5c5483163 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PipelineExecutor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PipelineExecutor.java @@ -17,12 +17,11 @@ */ package org.apache.beam.runners.direct; +import java.util.Collection; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; -import java.util.Collection; - /** * An executor that schedules and executes {@link AppliedPTransform AppliedPTransforms} for both * source and intermediate {@link PTransform PTransforms}. diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SideInputContainer.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SideInputContainer.java index 6458215503b9..cd459e4c0f5d 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SideInputContainer.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SideInputContainer.java @@ -19,15 +19,6 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.PCollectionViewWindow; -import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; -import org.apache.beam.sdk.util.SideInputReader; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; -import org.apache.beam.sdk.values.PCollectionView; - import com.google.common.base.MoreObjects; import com.google.common.base.Optional; import com.google.common.cache.CacheBuilder; @@ -36,7 +27,6 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.google.common.collect.Sets; - import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -44,8 +34,15 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicReference; - import javax.annotation.Nullable; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.util.PCollectionViewWindow; +import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; +import org.apache.beam.sdk.util.SideInputReader; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.WindowingStrategy; +import org.apache.beam.sdk.values.PCollectionView; /** * An in-process container for {@link PCollectionView PCollectionViews}, which provides methods for diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepAndKey.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepAndKey.java index 18fe04ffe943..e18b2ac9b6aa 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepAndKey.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepAndKey.java @@ -17,11 +17,9 @@ */ package org.apache.beam.runners.direct; -import org.apache.beam.sdk.transforms.AppliedPTransform; - import com.google.common.base.MoreObjects; - import java.util.Objects; +import org.apache.beam.sdk.transforms.AppliedPTransform; /** * A (Step, Key) pair. This is useful as a map key or cache key for things that are available diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepTransformResult.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepTransformResult.java index 12b18cbb8a7a..1829e4aea92d 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepTransformResult.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepTransformResult.java @@ -17,6 +17,12 @@ */ package org.apache.beam.runners.direct; +import com.google.auto.value.AutoValue; +import com.google.common.collect.ImmutableList; +import java.util.Collection; +import java.util.EnumSet; +import java.util.Set; +import javax.annotation.Nullable; import org.apache.beam.runners.direct.CommittedResult.OutputType; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate; @@ -24,18 +30,8 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals; - -import com.google.auto.value.AutoValue; -import com.google.common.collect.ImmutableList; - import org.joda.time.Instant; -import java.util.Collection; -import java.util.EnumSet; -import java.util.Set; - -import javax.annotation.Nullable; - /** * An immutable {@link TransformResult}. */ diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java index 90a83b0b840f..e9f37bab7567 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java @@ -20,6 +20,11 @@ import static com.google.common.base.Preconditions.checkState; +import com.google.common.base.Supplier; +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; +import javax.annotation.Nullable; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; import org.apache.beam.sdk.Pipeline; @@ -41,18 +46,9 @@ import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.TimestampedValue; - -import com.google.common.base.Supplier; - import org.joda.time.Duration; import org.joda.time.Instant; -import java.util.List; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicReference; - -import javax.annotation.Nullable; - /** * The {@link TransformEvaluatorFactory} for the {@link TestStream} primitive. */ diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorFactory.java index 3655d26dd553..e9fa06b107f7 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorFactory.java @@ -17,6 +17,7 @@ */ package org.apache.beam.runners.direct; +import javax.annotation.Nullable; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.Read; @@ -24,8 +25,6 @@ import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.PTransform; -import javax.annotation.Nullable; - /** * A factory for creating instances of {@link TransformEvaluator} for the application of a * {@link PTransform}. diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java index c35e8b14c84a..9edc50f8f4c6 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java @@ -19,6 +19,12 @@ import static com.google.common.base.Preconditions.checkState; +import com.google.common.collect.ImmutableMap; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; +import javax.annotation.Nullable; import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupAlsoByWindow; import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; @@ -29,19 +35,9 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.windowing.Window; - -import com.google.common.collect.ImmutableMap; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Map; -import java.util.concurrent.atomic.AtomicBoolean; - -import javax.annotation.Nullable; - /** * A {@link TransformEvaluatorFactory} that delegates to primitive {@link TransformEvaluatorFactory} * implementations based on the type of {@link PTransform} of the application. diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutor.java index d873bf581a8a..cc6b5b7a93ca 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutor.java @@ -19,16 +19,14 @@ import static com.google.common.base.Preconditions.checkState; -import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; -import org.apache.beam.sdk.transforms.AppliedPTransform; -import org.apache.beam.sdk.util.WindowedValue; - import java.util.ArrayList; import java.util.Collection; import java.util.concurrent.Callable; import java.util.concurrent.atomic.AtomicReference; - import javax.annotation.Nullable; +import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; +import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.util.WindowedValue; /** * A {@link Callable} responsible for constructing a {@link TransformEvaluator} from a diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutorServices.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutorServices.java index ea15f03df1b5..876da9d5cd2e 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutorServices.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutorServices.java @@ -18,7 +18,6 @@ package org.apache.beam.runners.direct; import com.google.common.base.MoreObjects; - import java.util.Queue; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ExecutorService; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformResult.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformResult.java index c01fa5647a92..0b08294ae4b4 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformResult.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformResult.java @@ -17,6 +17,8 @@ */ package org.apache.beam.runners.direct; +import java.util.Set; +import javax.annotation.Nullable; import org.apache.beam.runners.direct.CommittedResult.OutputType; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate; @@ -25,13 +27,8 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals; - import org.joda.time.Instant; -import java.util.Set; - -import javax.annotation.Nullable; - /** * The result of evaluating an {@link AppliedPTransform} with a {@link TransformEvaluator}. */ diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadDeduplicator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadDeduplicator.java index 0246236e2b4f..2371d3b9d3da 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadDeduplicator.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadDeduplicator.java @@ -17,18 +17,15 @@ */ package org.apache.beam.runners.direct; -import org.apache.beam.sdk.coders.ByteArrayCoder; -import org.apache.beam.sdk.io.Read.Unbounded; -import org.apache.beam.sdk.transforms.PTransform; - import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; - -import org.joda.time.Duration; - import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.beam.sdk.coders.ByteArrayCoder; +import org.apache.beam.sdk.io.Read.Unbounded; +import org.apache.beam.sdk.transforms.PTransform; +import org.joda.time.Duration; /** * Provides methods to determine if a record is a duplicate within the evaluation of a diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java index c4d408b232f0..9f485e0f65ab 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java @@ -17,6 +17,12 @@ */ package org.apache.beam.runners.direct; +import com.google.common.annotations.VisibleForTesting; +import java.io.IOException; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ConcurrentMap; +import javax.annotation.Nullable; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; import org.apache.beam.sdk.io.Read.Unbounded; @@ -29,18 +35,8 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.annotations.VisibleForTesting; - import org.joda.time.Instant; -import java.io.IOException; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.ConcurrentMap; - -import javax.annotation.Nullable; - /** * A {@link TransformEvaluatorFactory} that produces {@link TransformEvaluator TransformEvaluators} * for the {@link Unbounded Read.Unbounded} primitive {@link PTransform}. diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UncommittedBundleOutputManager.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UncommittedBundleOutputManager.java index 570dc9031e97..41f7e8d64b39 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UncommittedBundleOutputManager.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UncommittedBundleOutputManager.java @@ -17,14 +17,13 @@ */ package org.apache.beam.runners.direct; +import java.util.Map; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; import org.apache.beam.sdk.util.DoFnRunners.OutputManager; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; -import java.util.Map; - /** * An {@link OutputManager} that outputs to {@link CommittedBundle Bundles} used by the * {@link DirectRunner}. diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java index 3b0de4b6f8df..40ac7f09ac2c 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java @@ -17,6 +17,8 @@ */ package org.apache.beam.runners.direct; +import java.util.ArrayList; +import java.util.List; import org.apache.beam.runners.direct.CommittedResult.OutputType; import org.apache.beam.runners.direct.DirectRunner.PCollectionViewWriter; import org.apache.beam.runners.direct.StepTransformResult.Builder; @@ -34,9 +36,6 @@ import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; -import java.util.ArrayList; -import java.util.List; - /** * The {@link DirectRunner} {@link TransformEvaluatorFactory} for the * {@link CreatePCollectionView} primitive {@link PTransform}. diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkCallbackExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkCallbackExecutor.java index 0f73b1da9349..7961f24c6125 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkCallbackExecutor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkCallbackExecutor.java @@ -17,19 +17,16 @@ */ package org.apache.beam.runners.direct; -import org.apache.beam.sdk.transforms.AppliedPTransform; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowingStrategy; - import com.google.common.collect.ComparisonChain; import com.google.common.collect.Ordering; - -import org.joda.time.Instant; - import java.util.PriorityQueue; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.Executor; +import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.WindowingStrategy; +import org.joda.time.Instant; /** * Executes callbacks that occur based on the progression of the watermark per-step. diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java index a44fa501c434..ff7428dda2cc 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java @@ -19,18 +19,6 @@ import static com.google.common.base.Preconditions.checkNotNull; -import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.transforms.AppliedPTransform; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.TimeDomain; -import org.apache.beam.sdk.util.TimerInternals; -import org.apache.beam.sdk.util.TimerInternals.TimerData; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PValue; - import com.google.auto.value.AutoValue; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; @@ -43,9 +31,6 @@ import com.google.common.collect.Ordering; import com.google.common.collect.SortedMultiset; import com.google.common.collect.TreeMultiset; - -import org.joda.time.Instant; - import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -61,8 +46,19 @@ import java.util.TreeSet; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.atomic.AtomicReference; - import javax.annotation.Nullable; +import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.TimeDomain; +import org.apache.beam.sdk.util.TimerInternals; +import org.apache.beam.sdk.util.TimerInternals.TimerData; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PValue; +import org.joda.time.Instant; /** * Manages watermarks of {@link PCollection PCollections} and input and output watermarks of diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java index f2e62cb0445d..19c1a98e7ef5 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java @@ -17,6 +17,9 @@ */ package org.apache.beam.runners.direct; +import com.google.common.collect.Iterables; +import java.util.Collection; +import javax.annotation.Nullable; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; import org.apache.beam.sdk.transforms.AppliedPTransform; @@ -28,15 +31,8 @@ import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.Iterables; - import org.joda.time.Instant; -import java.util.Collection; - -import javax.annotation.Nullable; - /** * The {@link DirectRunner} {@link TransformEvaluatorFactory} for the * {@link Bound Window.Bound} primitive {@link PTransform}. diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java index 1ab3403f45bc..d74cd56735b6 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java @@ -20,6 +20,8 @@ import static com.google.common.base.Preconditions.checkArgument; +import com.google.common.annotations.VisibleForTesting; +import java.util.concurrent.ThreadLocalRandom; import org.apache.beam.sdk.io.Write; import org.apache.beam.sdk.io.Write.Bound; import org.apache.beam.sdk.transforms.Count; @@ -39,13 +41,8 @@ import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; - -import com.google.common.annotations.VisibleForTesting; - import org.joda.time.Duration; -import java.util.concurrent.ThreadLocalRandom; - /** * A {@link PTransformOverrideFactory} that overrides {@link Write} {@link PTransform PTransforms} * with an unspecified number of shards with a write with a specified number of shards. The number diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/AggregatorContainerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/AggregatorContainerTest.java index b131b4c524c2..c8310c90e79a 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/AggregatorContainerTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/AggregatorContainerTest.java @@ -21,10 +21,12 @@ import static org.junit.Assert.assertThat; import static org.mockito.Mockito.when; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.Sum.SumIntegerFn; import org.apache.beam.sdk.util.ExecutionContext.StepContext; - import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -34,10 +36,6 @@ import org.mockito.Mock; import org.mockito.MockitoAnnotations; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.TimeUnit; - /** * Tests for {@link AggregatorContainer}. */ diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java index 9bc4f7b7904b..cbeb733a1daf 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java @@ -25,6 +25,11 @@ import static org.junit.Assert.assertThat; import static org.mockito.Mockito.when; +import com.google.common.collect.ImmutableList; +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.NoSuchElementException; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; import org.apache.beam.sdk.coders.BigEndianLongCoder; @@ -40,9 +45,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.ImmutableList; - import org.joda.time.Instant; import org.junit.Before; import org.junit.Test; @@ -51,11 +53,6 @@ import org.mockito.Mock; import org.mockito.MockitoAnnotations; -import java.io.IOException; -import java.util.Arrays; -import java.util.List; -import java.util.NoSuchElementException; - /** * Tests for {@link BoundedReadEvaluatorFactory}. */ diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java index a8c647e64682..efc6d2f81abd 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java @@ -21,6 +21,11 @@ import static org.hamcrest.Matchers.nullValue; import static org.junit.Assert.assertThat; +import com.google.common.collect.ImmutableList; +import java.io.Serializable; +import java.util.Collections; +import java.util.EnumSet; +import java.util.List; import org.apache.beam.runners.direct.CommittedResult.OutputType; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.AppliedPTransform; @@ -31,20 +36,12 @@ import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; - -import com.google.common.collect.ImmutableList; - import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.Collections; -import java.util.EnumSet; -import java.util.List; - /** * Tests for {@link CommittedResult}. */ diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitorTest.java index 529316cd3027..1c9b5a6da6d6 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitorTest.java @@ -20,6 +20,8 @@ import static org.hamcrest.Matchers.emptyIterable; import static org.junit.Assert.assertThat; +import java.io.Serializable; +import java.util.List; import org.apache.beam.sdk.io.CountingInput; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.AppliedPTransform; @@ -35,7 +37,6 @@ import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.PValue; - import org.hamcrest.Matchers; import org.junit.Rule; import org.junit.Test; @@ -43,9 +44,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.List; - /** * Tests for {@link ConsumerTrackingPipelineVisitor}. */ diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRegistrarTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRegistrarTest.java index d5cdee657180..603e43e30f6b 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRegistrarTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRegistrarTest.java @@ -20,20 +20,17 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import java.util.ServiceLoader; import org.apache.beam.runners.direct.DirectRegistrar.Options; import org.apache.beam.runners.direct.DirectRegistrar.Runner; import org.apache.beam.sdk.options.PipelineOptionsRegistrar; import org.apache.beam.sdk.runners.PipelineRunnerRegistrar; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.ServiceLoader; - /** Tests for {@link DirectRegistrar}. */ @RunWith(JUnit4.class) public class DirectRegistrarTest { diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java index 9739adb0f366..c7efac388b68 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java @@ -22,6 +22,13 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; +import com.fasterxml.jackson.annotation.JsonValue; +import com.google.common.collect.ImmutableMap; +import java.io.Serializable; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.beam.runners.direct.DirectRunner.DirectPipelineResult; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.CoderException; @@ -47,8 +54,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.TypeDescriptor; -import com.google.common.collect.ImmutableMap; -import com.fasterxml.jackson.annotation.JsonValue; import org.hamcrest.Matchers; import org.junit.Rule; import org.junit.Test; @@ -56,11 +61,6 @@ import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.concurrent.atomic.AtomicInteger; /** * Tests for basic {@link DirectRunner} functionality. diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectTimerInternalsTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectTimerInternalsTest.java index c038910a4759..51cfeedbaca0 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectTimerInternalsTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectTimerInternalsTest.java @@ -29,7 +29,6 @@ import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.TimerInternals.TimerData; import org.apache.beam.sdk.util.state.StateNamespaces; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Before; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java index 67f4ff47789f..2e4fee24760e 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java @@ -25,18 +25,16 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; +import java.util.ArrayList; +import java.util.List; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.util.WindowedValue; - import org.hamcrest.Matchers; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.ArrayList; -import java.util.List; - /** * Tests for {@link DoFnLifecycleManagerRemovingTransformEvaluator}. */ diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java index 9da46f265b7d..1f0af9913acf 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java @@ -25,13 +25,6 @@ import static org.hamcrest.Matchers.theInstance; import static org.junit.Assert.assertThat; -import org.apache.beam.sdk.transforms.OldDoFn; - -import org.hamcrest.Matchers; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - import java.util.ArrayList; import java.util.List; import java.util.concurrent.Callable; @@ -40,6 +33,11 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; +import org.apache.beam.sdk.transforms.OldDoFn; +import org.hamcrest.Matchers; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; /** * Tests for {@link DoFnLifecycleManager}. diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagersTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagersTest.java index 8be3d52763be..39a4a9d8f39c 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagersTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagersTest.java @@ -20,10 +20,10 @@ import static org.hamcrest.Matchers.equalTo; -import org.apache.beam.sdk.transforms.OldDoFn; - import com.google.common.collect.ImmutableList; - +import java.util.ArrayList; +import java.util.Collection; +import org.apache.beam.sdk.transforms.OldDoFn; import org.hamcrest.BaseMatcher; import org.hamcrest.Description; import org.hamcrest.Matcher; @@ -34,9 +34,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.ArrayList; -import java.util.Collection; - /** * Tests for {@link DoFnLifecycleManagers}. */ diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactoryTest.java index b903ef1eb42f..e0ccbe546172 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactoryTest.java @@ -19,6 +19,10 @@ import static org.hamcrest.Matchers.isA; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Collections; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.Coder; @@ -31,7 +35,6 @@ import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; - import org.joda.time.Instant; import org.junit.Rule; import org.junit.Test; @@ -39,11 +42,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.Collections; - /** * Tests for {@link EncodabilityEnforcementFactory}. */ diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java index 7ac0caadc55b..f59dbbaf8568 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java @@ -25,6 +25,13 @@ import static org.hamcrest.Matchers.not; import static org.junit.Assert.assertThat; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import java.util.Collection; +import java.util.Collections; +import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.PCollectionViewWriter; @@ -61,10 +68,6 @@ import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; - import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Before; @@ -72,12 +75,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Collection; -import java.util.Collections; -import java.util.Map; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; - /** * Tests for {@link EvaluationContext}. */ diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java index 0bc3036f6399..1c46c2411414 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java @@ -33,7 +33,6 @@ import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; - import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Test; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ForwardingPTransformTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ForwardingPTransformTest.java index 9ea71d70878d..6abaf921943e 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ForwardingPTransformTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ForwardingPTransformTest.java @@ -27,7 +27,6 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.values.PCollection; - import org.junit.Before; import org.junit.Rule; import org.junit.Test; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java index 78736c4f8087..8d1f8bdfaf68 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java @@ -22,6 +22,9 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import com.google.common.collect.HashMultiset; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Multiset; import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; @@ -36,11 +39,6 @@ import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.HashMultiset; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Multiset; - import org.hamcrest.BaseMatcher; import org.hamcrest.Description; import org.joda.time.Instant; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java index 4afd64b52ad4..9f1e916c28cf 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java @@ -22,6 +22,9 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import com.google.common.collect.HashMultiset; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Multiset; import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; @@ -36,11 +39,6 @@ import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.HashMultiset; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Multiset; - import org.hamcrest.BaseMatcher; import org.hamcrest.Description; import org.joda.time.Instant; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java index db934e542f61..d44151ac6556 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java @@ -33,7 +33,6 @@ import org.apache.beam.sdk.util.IllegalMutationException; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; - import org.joda.time.Instant; import org.junit.Before; import org.junit.Rule; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java index e1be12001225..713ae3510942 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java @@ -17,6 +17,8 @@ */ package org.apache.beam.runners.direct; +import java.io.Serializable; +import java.util.Collections; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.AppliedPTransform; @@ -27,7 +29,6 @@ import org.apache.beam.sdk.util.IllegalMutationException; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; - import org.joda.time.Instant; import org.junit.Before; import org.junit.Rule; @@ -36,9 +37,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.Collections; - /** * Tests for {@link ImmutabilityEnforcementFactory}. */ diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java index 21e4bcbc0819..43108f817661 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java @@ -21,6 +21,10 @@ import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertThat; +import com.google.common.collect.ImmutableList; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; import org.apache.beam.sdk.coders.ByteArrayCoder; @@ -36,9 +40,6 @@ import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.ImmutableList; - import org.hamcrest.Matcher; import org.hamcrest.Matchers; import org.joda.time.Instant; @@ -49,10 +50,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; - /** * Tests for {@link ImmutableListBundleFactory}. */ diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java index 9e273ad84292..ee6b2b4ed2e8 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java @@ -21,6 +21,9 @@ import static org.hamcrest.Matchers.not; import static org.junit.Assert.assertThat; +import com.google.common.collect.ImmutableSet; +import java.util.Collections; +import java.util.Set; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.coders.KvCoder; @@ -35,9 +38,6 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.ImmutableSet; - import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -45,9 +45,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Collections; -import java.util.Set; - /** * Tests for {@link KeyedPValueTrackingVisitor}. */ diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java index 3208841f61e0..2a54ecb8aada 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java @@ -23,6 +23,13 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import javax.annotation.Nullable; import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; @@ -45,9 +52,6 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Iterables; import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Before; @@ -57,10 +61,6 @@ import org.mockito.Mock; import org.mockito.Mockito; import org.mockito.MockitoAnnotations; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; -import javax.annotation.Nullable; /** * Tests for {@link ParDoEvaluator}. diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java index 19094cbfe284..555219626d72 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java @@ -25,6 +25,7 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import java.io.Serializable; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate; @@ -59,7 +60,6 @@ import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; /** * Tests for {@link ParDoMultiEvaluatorFactory}. diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java index a4fd570d8624..60b6dd990b98 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java @@ -25,6 +25,7 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import java.io.Serializable; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate; @@ -56,7 +57,6 @@ import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; /** * Tests for {@link ParDoSingleEvaluatorFactory}. diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java index ec589da6eadb..cc7d88a39e7e 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java @@ -25,6 +25,13 @@ import static org.junit.Assert.fail; import static org.mockito.Mockito.doAnswer; +import com.google.common.collect.ImmutableList; +import java.util.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.testing.TestPipeline; @@ -44,9 +51,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; - -import com.google.common.collect.ImmutableList; - import org.joda.time.Instant; import org.junit.Before; import org.junit.Rule; @@ -60,13 +64,6 @@ import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import java.util.Map; -import java.util.concurrent.Callable; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; - /** * Tests for {@link SideInputContainer}. */ diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StepTransformResultTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StepTransformResultTest.java index cfc69bc61d0f..c06eff960646 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StepTransformResultTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StepTransformResultTest.java @@ -29,7 +29,6 @@ import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.values.PCollection; - import org.hamcrest.Matchers; import org.junit.Before; import org.junit.Test; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StructuralKeyTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StructuralKeyTest.java index 26514f004ec5..18aeac688d8a 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StructuralKeyTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StructuralKeyTest.java @@ -27,7 +27,6 @@ import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorServicesTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorServicesTest.java index 04aa96ffdfd1..b085723a3ccb 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorServicesTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorServicesTest.java @@ -22,7 +22,7 @@ import static org.mockito.Mockito.verify; import com.google.common.util.concurrent.MoreExecutors; - +import java.util.concurrent.ExecutorService; import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -30,8 +30,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.concurrent.ExecutorService; - /** * Tests for {@link TransformExecutorServices}. */ diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java index f6cb8d133d8e..5af568f9f07f 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java @@ -27,6 +27,16 @@ import static org.junit.Assert.assertThat; import static org.mockito.Mockito.when; +import com.google.common.util.concurrent.MoreExecutors; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.EnumSet; +import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicBoolean; import org.apache.beam.runners.direct.CommittedResult.OutputType; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.sdk.coders.ByteArrayCoder; @@ -39,9 +49,6 @@ import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.util.concurrent.MoreExecutors; - import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Before; @@ -53,16 +60,6 @@ import org.mockito.Mock; import org.mockito.MockitoAnnotations; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.EnumSet; -import java.util.List; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.atomic.AtomicBoolean; - /** * Tests for {@link TransformExecutor}. diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadDeduplicatorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadDeduplicatorTest.java index b3c9012b60c2..7d2a95cd167c 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadDeduplicatorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadDeduplicatorTest.java @@ -21,18 +21,16 @@ import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertThat; -import org.apache.beam.runners.direct.UnboundedReadDeduplicator.CachedIdDeduplicator; -import org.apache.beam.runners.direct.UnboundedReadDeduplicator.NeverDeduplicator; - -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import org.apache.beam.runners.direct.UnboundedReadDeduplicator.CachedIdDeduplicator; +import org.apache.beam.runners.direct.UnboundedReadDeduplicator.NeverDeduplicator; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; /** * Tests for {@link UnboundedReadDeduplicator}. diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java index 839badfd6f9e..3a6add6aac97 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java @@ -25,6 +25,17 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import com.google.common.collect.ContiguousSet; +import com.google.common.collect.DiscreteDomain; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Range; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Arrays; +import java.util.List; +import java.util.NoSuchElementException; +import javax.annotation.Nullable; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; import org.apache.beam.sdk.coders.AtomicCoder; @@ -43,12 +54,6 @@ import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.ContiguousSet; -import com.google.common.collect.DiscreteDomain; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Range; - import org.hamcrest.Matchers; import org.joda.time.DateTime; import org.joda.time.Instant; @@ -57,15 +62,6 @@ import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; - -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.Arrays; -import java.util.List; -import java.util.NoSuchElementException; - -import javax.annotation.Nullable; /** * Tests for {@link UnboundedReadEvaluatorFactory}. */ diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java index 68207922855c..d3ab81d8e7d6 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java @@ -23,6 +23,7 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import com.google.common.collect.ImmutableList; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.PCollectionViewWriter; import org.apache.beam.sdk.coders.KvCoder; @@ -38,9 +39,6 @@ import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; - -import com.google.common.collect.ImmutableList; - import org.joda.time.Instant; import org.junit.Test; import org.junit.runner.RunWith; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkCallbackExecutorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkCallbackExecutorTest.java index b6b2bf58af9a..1be9a9884a8c 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkCallbackExecutorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkCallbackExecutorTest.java @@ -20,6 +20,9 @@ import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertThat; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Create; @@ -31,7 +34,6 @@ import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollection; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Before; @@ -39,10 +41,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.Executors; -import java.util.concurrent.TimeUnit; - /** * Tests for {@link WatermarkCallbackExecutor}. */ diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java index 22f148a4a3f7..d9dc404cdf07 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java @@ -25,6 +25,15 @@ import static org.hamcrest.Matchers.nullValue; import static org.junit.Assert.assertThat; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import java.io.Serializable; +import java.util.Collection; +import java.util.Collections; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.Map; +import javax.annotation.Nullable; import org.apache.beam.runners.direct.CommittedResult.OutputType; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; @@ -56,10 +65,6 @@ import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TimestampedValue; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; - import org.hamcrest.BaseMatcher; import org.hamcrest.Description; import org.hamcrest.Matcher; @@ -70,14 +75,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.Collection; -import java.util.Collections; -import java.util.EnumSet; -import java.util.HashMap; -import java.util.Map; -import javax.annotation.Nullable; - /** * Tests for {@link WatermarkManager}. */ diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java index 65dcfebe39c4..63800cfb907b 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java @@ -19,11 +19,15 @@ import static org.apache.beam.sdk.WindowMatchers.isSingleWindowedValue; import static org.apache.beam.sdk.WindowMatchers.isWindowedValue; - import static org.hamcrest.Matchers.containsInAnyOrder; import static org.junit.Assert.assertThat; import static org.mockito.Mockito.when; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; +import java.util.Collection; +import java.util.Collections; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; import org.apache.beam.sdk.coders.Coder; @@ -44,11 +48,6 @@ import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Iterables; - import org.hamcrest.Matchers; import org.joda.time.Duration; import org.joda.time.Instant; @@ -59,9 +58,6 @@ import org.mockito.Mock; import org.mockito.MockitoAnnotations; -import java.util.Collection; -import java.util.Collections; - /** * Tests for {@link WindowEvaluatorFactory}. */ diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java index 20b2776890e9..2dd477dd30e4 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java @@ -28,6 +28,18 @@ import static org.hamcrest.Matchers.not; import static org.junit.Assert.assertThat; +import com.google.common.base.Function; +import com.google.common.collect.Iterables; +import java.io.File; +import java.io.FileReader; +import java.io.Reader; +import java.nio.CharBuffer; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.ThreadLocalRandom; import org.apache.beam.runners.direct.WriteWithShardingFactory.KeyBasedOnCountFn; import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.io.Sink; @@ -46,27 +58,12 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PDone; - -import com.google.common.base.Function; -import com.google.common.collect.Iterables; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.File; -import java.io.FileReader; -import java.io.Reader; -import java.nio.CharBuffer; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.UUID; -import java.util.concurrent.ThreadLocalRandom; - /** * Tests for {@link WriteWithShardingFactory}. */ diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/TFIDF.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/TFIDF.java index 4deca12d14fa..0ca94a13463b 100644 --- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/TFIDF.java +++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/TFIDF.java @@ -17,6 +17,12 @@ */ package org.apache.beam.runners.flink.examples; +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.HashSet; +import java.util.Set; import org.apache.beam.runners.flink.FlinkPipelineOptions; import org.apache.beam.runners.flink.FlinkRunner; import org.apache.beam.sdk.Pipeline; @@ -53,17 +59,9 @@ import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.TupleTag; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.File; -import java.io.IOException; -import java.net.URI; -import java.net.URISyntaxException; -import java.util.HashSet; -import java.util.Set; - /** * An example that computes a basic TF-IDF search table for a directory or GCS prefix. * diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java index fdffd391ba0d..ab9297f72d8e 100644 --- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java +++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java @@ -28,7 +28,6 @@ import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/AutoComplete.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/AutoComplete.java index aff1a357c8ec..9b5e31d08bbe 100644 --- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/AutoComplete.java +++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/AutoComplete.java @@ -17,6 +17,8 @@ */ package org.apache.beam.runners.flink.examples.streaming; +import java.io.IOException; +import java.util.List; import org.apache.beam.runners.flink.FlinkRunner; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSocketSource; import org.apache.beam.sdk.Pipeline; @@ -47,12 +49,8 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; - import org.joda.time.Duration; -import java.io.IOException; -import java.util.List; - /** * To run the example, first open a socket on a terminal by executing the command: *

  • diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/JoinExamples.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/JoinExamples.java index 458a26333183..bf5dfc453e19 100644 --- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/JoinExamples.java +++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/JoinExamples.java @@ -35,7 +35,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TupleTag; - import org.joda.time.Duration; /** diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaIOExamples.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaIOExamples.java index 68a9edcc5a8b..27faefe6d745 100644 --- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaIOExamples.java +++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaIOExamples.java @@ -17,6 +17,11 @@ */ package org.apache.beam.runners.flink.examples.streaming; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.Serializable; +import java.util.Properties; import org.apache.beam.runners.flink.FlinkPipelineOptions; import org.apache.beam.runners.flink.FlinkRunner; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedFlinkSink; @@ -33,7 +38,6 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; - import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08; @@ -42,12 +46,6 @@ import org.apache.flink.streaming.util.serialization.SerializationSchema; import org.apache.flink.streaming.util.serialization.SimpleStringSchema; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.Serializable; -import java.util.Properties; - /** * Recipes/Examples that demonstrate how to read/write data from/to Kafka. */ diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java index 39ce22542c0b..365fb7b48672 100644 --- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java +++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java @@ -17,6 +17,7 @@ */ package org.apache.beam.runners.flink.examples.streaming; +import java.util.Properties; import org.apache.beam.runners.flink.FlinkRunner; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedFlinkSource; import org.apache.beam.sdk.Pipeline; @@ -35,13 +36,10 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08; import org.apache.flink.streaming.util.serialization.SimpleStringSchema; import org.joda.time.Duration; -import java.util.Properties; - public class KafkaWindowedWordCountExample { static final String KAFKA_TOPIC = "test"; // Default kafka topic to read from diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java index fe8e627d1335..f3361c571697 100644 --- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java +++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java @@ -17,6 +17,7 @@ */ package org.apache.beam.runners.flink.examples.streaming; +import java.io.IOException; import org.apache.beam.runners.flink.FlinkRunner; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSocketSource; import org.apache.beam.sdk.Pipeline; @@ -35,13 +36,10 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; - /** * To run the example, first open a socket on a terminal by executing the command: *
  • diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/DefaultParallelismFactory.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/DefaultParallelismFactory.java index e512db0e3eaa..2fe4569e3e40 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/DefaultParallelismFactory.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/DefaultParallelismFactory.java @@ -19,7 +19,6 @@ import org.apache.beam.sdk.options.DefaultValueFactory; import org.apache.beam.sdk.options.PipelineOptions; - import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.GlobalConfiguration; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java index f4d4ea6d546d..d1977a445adf 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java @@ -19,13 +19,13 @@ import static com.google.common.base.Preconditions.checkNotNull; +import java.util.List; import org.apache.beam.runners.flink.translation.FlinkBatchPipelineTranslator; import org.apache.beam.runners.flink.translation.FlinkPipelineTranslator; import org.apache.beam.runners.flink.translation.FlinkStreamingPipelineTranslator; import org.apache.beam.runners.flink.translation.PipelineTranslationOptimizer; import org.apache.beam.runners.flink.translation.TranslationMode; import org.apache.beam.sdk.Pipeline; - import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.java.CollectionEnvironment; import org.apache.flink.api.java.ExecutionEnvironment; @@ -34,8 +34,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.List; - /** * The class that instantiates and manages the execution of a given job. * Depending on if the job is a Streaming or Batch processing one, it creates diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java index c40473e10853..6d1a8d040f1a 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java @@ -18,21 +18,19 @@ package org.apache.beam.runners.flink; +import com.fasterxml.jackson.annotation.JsonIgnore; +import java.util.List; import org.apache.beam.sdk.options.ApplicationNameOptions; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.DefaultValueFactory; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.StreamingOptions; - -import com.fasterxml.jackson.annotation.JsonIgnore; import org.joda.time.DateTimeUtils; import org.joda.time.DateTimeZone; import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; -import java.util.List; - /** * Options which can be used to configure a Flink PipelineRunner. */ diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java index b0e88b7db7e2..8b1f42e5447c 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java @@ -17,6 +17,16 @@ */ package org.apache.beam.runners.flink; +import com.google.common.base.Joiner; +import com.google.common.collect.ImmutableMap; +import java.io.File; +import java.net.URISyntaxException; +import java.net.URL; +import java.net.URLClassLoader; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; @@ -37,23 +47,10 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; - -import com.google.common.base.Joiner; -import com.google.common.collect.ImmutableMap; - import org.apache.flink.api.common.JobExecutionResult; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.File; -import java.net.URISyntaxException; -import java.net.URL; -import java.net.URLClassLoader; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Map; - /** * A {@link PipelineRunner} that executes the operations in the * pipeline by first translating them to a Flink Plan and then executing them either locally diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java index 52b2a8ddb5d0..f328279bfba0 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java @@ -18,14 +18,13 @@ package org.apache.beam.runners.flink; +import com.google.auto.service.AutoService; +import com.google.common.collect.ImmutableList; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsRegistrar; import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.runners.PipelineRunnerRegistrar; -import com.google.auto.service.AutoService; -import com.google.common.collect.ImmutableList; - /** * AuteService registrar - will register FlinkRunner and FlinkOptions diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java index 923d54c56631..dd0733a671c9 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java @@ -17,17 +17,15 @@ */ package org.apache.beam.runners.flink; -import org.apache.beam.sdk.PipelineResult; +import java.io.IOException; +import java.util.Collections; +import java.util.Map; import org.apache.beam.sdk.AggregatorRetrievalException; import org.apache.beam.sdk.AggregatorValues; +import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.transforms.Aggregator; - import org.joda.time.Duration; -import java.io.IOException; -import java.util.Collections; -import java.util.Map; - /** * Result of executing a {@link org.apache.beam.sdk.Pipeline} with Flink. This * has methods to query to job runtime and the final values of diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java index 6a4f990cda33..dd231d6e6bdd 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java @@ -26,8 +26,6 @@ import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; -import org.apache.flink.runtime.client.JobExecutionException; - public class TestFlinkRunner extends PipelineRunner { private FlinkRunner delegate; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java index 8f9a37a6c223..66c48b07e23b 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java @@ -22,7 +22,6 @@ import org.apache.beam.sdk.runners.TransformTreeNode; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; - import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.io.DiscardingOutputFormat; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java index 01a3ab25b312..935a9ac439b9 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java @@ -17,6 +17,13 @@ */ package org.apache.beam.runners.flink.translation; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import org.apache.beam.runners.flink.translation.functions.FlinkAssignWindows; import org.apache.beam.runners.flink.translation.functions.FlinkDoFnFunction; import org.apache.beam.runners.flink.translation.functions.FlinkMergingNonShuffleReduceFunction; @@ -39,9 +46,9 @@ import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.CombineFnBase; -import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.transforms.GroupByKey; +import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.View; @@ -60,10 +67,6 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; - -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; - import org.apache.flink.api.common.functions.FilterFunction; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -77,12 +80,6 @@ import org.apache.flink.api.java.operators.SingleInputUdfOperator; import org.apache.flink.util.Collector; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - /** * Translators for transforming {@link PTransform PTransforms} to * Flink {@link DataSet DataSets}. diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java index a73bf135a2b1..835648e43ea8 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java @@ -17,6 +17,8 @@ */ package org.apache.beam.runners.flink.translation; +import java.util.HashMap; +import java.util.Map; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; @@ -29,14 +31,10 @@ import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.PValue; - import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; -import java.util.HashMap; -import java.util.Map; - /** * Helper for {@link FlinkBatchPipelineTranslator} and translators in * {@link FlinkBatchTransformTranslators}. diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java index 3bb8c5916089..b12745512c93 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java @@ -22,7 +22,6 @@ import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.PValue; - import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java index 6c2c703aace9..07e2191d4e0c 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java @@ -18,6 +18,17 @@ package org.apache.beam.runners.flink.translation; +import com.google.api.client.util.Maps; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; import org.apache.beam.runners.flink.FlinkRunner; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; import org.apache.beam.runners.flink.translation.types.FlinkCoder; @@ -60,11 +71,6 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; - -import com.google.api.client.util.Maps; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; - import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.functions.RichFlatMapFunction; @@ -85,15 +91,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; - /** * This class contains all the mappings between Beam and Flink * streaming transformations. The {@link FlinkStreamingPipelineTranslator} diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTranslationContext.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTranslationContext.java index a75ef034c850..bc80d42834ff 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTranslationContext.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTranslationContext.java @@ -19,6 +19,8 @@ import static com.google.common.base.Preconditions.checkNotNull; +import java.util.HashMap; +import java.util.Map; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; @@ -29,14 +31,10 @@ import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.PValue; - import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import java.util.HashMap; -import java.util.Map; - /** * Helper for keeping track of which {@link DataStream DataStreams} map * to which {@link PTransform PTransforms}. diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignContext.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignContext.java index 6abb8ffcbb86..447b1e507e1a 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignContext.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignContext.java @@ -19,16 +19,12 @@ import static com.google.common.base.Preconditions.checkArgument; +import com.google.common.collect.Iterables; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.WindowedValue; - -import com.google.common.collect.Iterables; - import org.joda.time.Instant; -import java.util.Collection; - /** * {@link org.apache.beam.sdk.transforms.windowing.WindowFn.AssignContext} for * Flink functions. diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignWindows.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignWindows.java index e07e49a2f060..f241ad0dc5c0 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignWindows.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignWindows.java @@ -17,15 +17,13 @@ */ package org.apache.beam.runners.flink.translation.functions; +import java.util.Collection; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.WindowedValue; - import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.util.Collector; -import java.util.Collection; - /** * Flink {@link FlatMapFunction} for implementing * {@link org.apache.beam.sdk.transforms.windowing.Window.Bound}. diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java index 733d3d46d9c0..ac5b345db678 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java @@ -17,19 +17,17 @@ */ package org.apache.beam.runners.flink.translation.functions; +import java.util.Map; import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollectionView; - import org.apache.flink.api.common.functions.RichMapPartitionFunction; import org.apache.flink.configuration.Configuration; import org.apache.flink.util.Collector; -import java.util.Map; - /** * Encapsulates a {@link OldDoFn} * inside a Flink {@link org.apache.flink.api.common.functions.RichMapPartitionFunction}. diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java index 2d360436e52f..dbaab1706ec5 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java @@ -17,6 +17,13 @@ */ package org.apache.beam.runners.flink.translation.functions; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.CombineFnBase; @@ -31,20 +38,10 @@ import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; - -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; - import org.apache.flink.api.common.functions.RichGroupReduceFunction; import org.apache.flink.util.Collector; import org.joda.time.Instant; -import java.util.Collections; -import java.util.Comparator; -import java.util.Iterator; -import java.util.List; -import java.util.Map; - /** * Special version of {@link FlinkReduceFunction} that supports merging windows. This * assumes that the windows are {@link IntervalWindow IntervalWindows} and exhibits the diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingPartialReduceFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingPartialReduceFunction.java index c12e4204a3f0..bc09bdf05c2f 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingPartialReduceFunction.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingPartialReduceFunction.java @@ -17,6 +17,13 @@ */ package org.apache.beam.runners.flink.translation.functions; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.CombineFnBase; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; @@ -29,19 +36,9 @@ import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; - -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; - import org.apache.flink.util.Collector; import org.joda.time.Instant; -import java.util.Collections; -import java.util.Comparator; -import java.util.Iterator; -import java.util.List; -import java.util.Map; - /** * Special version of {@link FlinkPartialReduceFunction} that supports merging windows. This * assumes that the windows are {@link IntervalWindow IntervalWindows} and exhibits the diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingReduceFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingReduceFunction.java index 07d1c9741533..4050f47a2ec4 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingReduceFunction.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingReduceFunction.java @@ -17,6 +17,15 @@ */ package org.apache.beam.runners.flink.translation.functions; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.CombineFnBase; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; @@ -29,21 +38,9 @@ import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; - import org.apache.flink.util.Collector; import org.joda.time.Instant; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.Iterator; -import java.util.List; -import java.util.Map; - /** * Special version of {@link FlinkReduceFunction} that supports merging windows. This * assumes that the windows are {@link IntervalWindow IntervalWindows} and exhibits the diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputDoFnFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputDoFnFunction.java index ef7587876aa8..9cc84ca65925 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputDoFnFunction.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputDoFnFunction.java @@ -17,6 +17,7 @@ */ package org.apache.beam.runners.flink.translation.functions; +import java.util.Map; import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.OldDoFn; @@ -25,13 +26,10 @@ import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; - import org.apache.flink.api.common.functions.RichMapPartitionFunction; import org.apache.flink.configuration.Configuration; import org.apache.flink.util.Collector; -import java.util.Map; - /** * Encapsulates a {@link OldDoFn} that can emit to multiple * outputs inside a Flink {@link org.apache.flink.api.common.functions.RichMapPartitionFunction}. diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputProcessContext.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputProcessContext.java index fab3c856bfea..153a2d7bdd7f 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputProcessContext.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputProcessContext.java @@ -17,6 +17,8 @@ */ package org.apache.beam.runners.flink.translation.functions; +import java.util.Collection; +import java.util.Map; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.join.RawUnionValue; @@ -26,14 +28,10 @@ import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; - import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.util.Collector; import org.joda.time.Instant; -import java.util.Collection; -import java.util.Map; - /** * {@link OldDoFn.ProcessContext} for {@link FlinkMultiOutputDoFnFunction} that supports * side outputs. diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java index 9205a5520f82..b72750a366d7 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java @@ -19,7 +19,6 @@ import org.apache.beam.sdk.transforms.join.RawUnionValue; import org.apache.beam.sdk.util.WindowedValue; - import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.util.Collector; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNoElementAssignContext.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNoElementAssignContext.java index 98446f9c7ba4..c89027262cd9 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNoElementAssignContext.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNoElementAssignContext.java @@ -20,7 +20,6 @@ import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; - import org.joda.time.Instant; /** diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java index 2db4b7b53b36..fa2ce4d46a05 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java @@ -17,6 +17,13 @@ */ package org.apache.beam.runners.flink.translation.functions; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.Map; import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.CombineFnBase; @@ -30,20 +37,10 @@ import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; - -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; - import org.apache.flink.api.common.functions.RichGroupCombineFunction; import org.apache.flink.util.Collector; import org.joda.time.Instant; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.Iterator; -import java.util.Map; - /** * This is is the first step for executing a {@link org.apache.beam.sdk.transforms.Combine.PerKey} * on Flink. The second part is {@link FlinkReduceFunction}. This function performs a local diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContext.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContext.java index 64b93c89513e..fa5eb1ab10ee 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContext.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContext.java @@ -19,6 +19,12 @@ import static com.google.common.base.Preconditions.checkNotNull; +import com.google.common.collect.Iterables; +import java.io.IOException; +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.Map; import org.apache.beam.runners.flink.translation.wrappers.SerializableFnAggregatorWrapper; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; @@ -34,19 +40,10 @@ import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; - -import com.google.common.collect.Iterables; - import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.util.Collector; import org.joda.time.Instant; -import java.io.IOException; -import java.util.Collection; -import java.util.Collections; -import java.util.Iterator; -import java.util.Map; - /** * {@link OldDoFn.ProcessContext} for our Flink Wrappers. */ diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java index b1729a42a40f..c9b24b456c03 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java @@ -17,6 +17,15 @@ */ package org.apache.beam.runners.flink.translation.functions; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.CombineFnBase; @@ -30,22 +39,10 @@ import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; - import org.apache.flink.api.common.functions.RichGroupReduceFunction; import org.apache.flink.util.Collector; import org.joda.time.Instant; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.Iterator; -import java.util.List; -import java.util.Map; - /** * This is the second part for executing a {@link org.apache.beam.sdk.transforms.Combine.PerKey} * on Flink, the second part is {@link FlinkReduceFunction}. This function performs the final diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/SideInputInitializer.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/SideInputInitializer.java index a577b684b849..12222b499d74 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/SideInputInitializer.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/SideInputInitializer.java @@ -17,16 +17,14 @@ */ package org.apache.beam.runners.flink.translation.functions; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.values.PCollectionView; - -import org.apache.flink.api.common.functions.BroadcastVariableInitializer; - import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.PCollectionView; +import org.apache.flink.api.common.functions.BroadcastVariableInitializer; /** * {@link BroadcastVariableInitializer} that initializes the broadcast input as a {@code Map} diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java index 61abf9af5ea2..9b449aabc8b5 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java @@ -20,7 +20,6 @@ import static com.google.common.base.Preconditions.checkNotNull; import org.apache.beam.sdk.coders.Coder; - import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.AtomicType; import org.apache.flink.api.common.typeinfo.TypeInformation; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java index 46219511213b..4eda357f64eb 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java @@ -17,21 +17,17 @@ */ package org.apache.beam.runners.flink.translation.types; +import java.io.EOFException; +import java.io.IOException; import org.apache.beam.runners.flink.translation.wrappers.DataInputViewWrapper; import org.apache.beam.runners.flink.translation.wrappers.DataOutputViewWrapper; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.util.CoderUtils; - import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; -import java.io.ByteArrayInputStream; -import java.io.EOFException; -import java.io.IOException; -import java.io.ObjectInputStream; - /** * Flink {@link org.apache.flink.api.common.typeutils.TypeSerializer} for * Dataflow {@link org.apache.beam.sdk.coders.Coder Coders}. diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueComparator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueComparator.java index 69bcb415c317..667ef4591a3f 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueComparator.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueComparator.java @@ -17,16 +17,14 @@ */ package org.apache.beam.runners.flink.translation.types; +import java.io.IOException; +import java.util.Arrays; import org.apache.beam.sdk.coders.Coder; - import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.core.memory.MemorySegment; -import java.io.IOException; -import java.util.Arrays; - /** * Flink {@link org.apache.flink.api.common.typeutils.TypeComparator} for Beam values that have * been encoded to byte data by a {@link Coder}. diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueSerializer.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueSerializer.java index 33af8d957723..f3e667d575a5 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueSerializer.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueSerializer.java @@ -17,14 +17,12 @@ */ package org.apache.beam.runners.flink.translation.types; +import java.io.IOException; import org.apache.beam.sdk.coders.Coder; - import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; -import java.io.IOException; - /** * {@link TypeSerializer} for values that were encoded using a {@link Coder}. */ diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueTypeInformation.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueTypeInformation.java index 46c854f087b8..0315ae3704fb 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueTypeInformation.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueTypeInformation.java @@ -18,16 +18,12 @@ package org.apache.beam.runners.flink.translation.types; import org.apache.beam.sdk.coders.Coder; - import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.AtomicType; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.base.array.BytePrimitiveArraySerializer; - -import java.util.Objects; /** * Flink {@link TypeInformation} for Beam values that have been encoded to byte data diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/FlinkCoder.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/FlinkCoder.java index 3b1e66e8e8ff..8b90c73a26fb 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/FlinkCoder.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/FlinkCoder.java @@ -17,6 +17,11 @@ */ package org.apache.beam.runners.flink.translation.types; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Collections; +import java.util.List; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StandardCoder; import org.apache.flink.api.common.ExecutionConfig; @@ -25,12 +30,6 @@ import org.apache.flink.core.memory.DataInputViewStreamWrapper; import org.apache.flink.core.memory.DataOutputViewStreamWrapper; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.Collections; -import java.util.List; - /** * A Coder that uses Flink's serialization system. * @param The type of the value to be encoded diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvKeySelector.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvKeySelector.java index 80d20cae9d9b..9df683620ec1 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvKeySelector.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvKeySelector.java @@ -21,7 +21,6 @@ import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; - import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java index 44af0ea406a7..0c6cea8dfd29 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java @@ -20,13 +20,11 @@ import static com.google.common.base.Preconditions.checkNotNull; -import org.apache.beam.sdk.options.PipelineOptions; - import com.fasterxml.jackson.databind.ObjectMapper; - import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.Serializable; +import org.apache.beam.sdk.options.PipelineOptions; /** * Encapsulates the PipelineOptions in serialized form to ship them to the cluster. diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataInputViewWrapper.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataInputViewWrapper.java index f1b8c7387e40..82a2c4ed2b9f 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataInputViewWrapper.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataInputViewWrapper.java @@ -17,11 +17,10 @@ */ package org.apache.beam.runners.flink.translation.wrappers; -import org.apache.flink.core.memory.DataInputView; - import java.io.EOFException; import java.io.IOException; import java.io.InputStream; +import org.apache.flink.core.memory.DataInputView; /** * Wrapper for {@link DataInputView}. We need this because Flink reads data using a diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java index 148f9607c21e..2cb9b188ff06 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java @@ -17,10 +17,9 @@ */ package org.apache.beam.runners.flink.translation.wrappers; -import org.apache.flink.core.memory.DataOutputView; - import java.io.IOException; import java.io.OutputStream; +import org.apache.flink.core.memory.DataOutputView; /** * Wrapper for {@link org.apache.flink.core.memory.DataOutputView}. We need this because diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java index 82d3fb8ffae3..25d777a3b125 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java @@ -17,15 +17,12 @@ */ package org.apache.beam.runners.flink.translation.wrappers; -import org.apache.beam.sdk.transforms.Aggregator; -import org.apache.beam.sdk.transforms.Combine; - import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; - -import org.apache.flink.api.common.accumulators.Accumulator; - import java.io.Serializable; +import org.apache.beam.sdk.transforms.Aggregator; +import org.apache.beam.sdk.transforms.Combine; +import org.apache.flink.api.common.accumulators.Accumulator; /** * Wrapper that wraps a {@link org.apache.beam.sdk.transforms.Combine.CombineFn} diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java index 1d06b1ac2fc9..443378f8d1f7 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java @@ -17,6 +17,8 @@ */ package org.apache.beam.runners.flink.translation.wrappers; +import java.io.IOException; +import java.util.List; import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.Source; @@ -24,7 +26,6 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowedValue; - import org.apache.flink.api.common.io.DefaultInputSplitAssigner; import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.api.common.io.statistics.BaseStatistics; @@ -34,9 +35,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.List; - /** * Wrapper for executing a {@link Source} as a Flink {@link InputFormat}. diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputSplit.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputSplit.java index c3672c0c3ce7..e4a738695be0 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputSplit.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputSplit.java @@ -18,7 +18,6 @@ package org.apache.beam.runners.flink.translation.wrappers; import org.apache.beam.sdk.io.Source; - import org.apache.flink.core.io.InputSplit; /** diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java index 092a22665f8c..000d69f34823 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java @@ -18,7 +18,15 @@ package org.apache.beam.runners.flink.translation.wrappers.streaming; import avro.shaded.com.google.common.base.Preconditions; - +import com.google.common.collect.Iterables; +import java.io.IOException; +import java.io.Serializable; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import org.apache.beam.runners.core.SideInputHandler; import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; import org.apache.beam.runners.flink.translation.wrappers.SerializableFnAggregatorWrapper; @@ -44,9 +52,6 @@ import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; - -import com.google.common.collect.Iterables; - import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.state.ListState; @@ -69,15 +74,6 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.tasks.StreamTaskState; -import java.io.IOException; -import java.io.Serializable; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - /** * Flink operator for executing {@link DoFn DoFns}. * diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkStateInternals.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkStateInternals.java index a3cf2e268740..2e1040059108 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkStateInternals.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkStateInternals.java @@ -17,6 +17,11 @@ */ package org.apache.beam.runners.flink.translation.wrappers.streaming; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.Map; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; @@ -38,20 +43,12 @@ import org.apache.beam.sdk.util.state.StateTag; import org.apache.beam.sdk.util.state.ValueState; import org.apache.beam.sdk.util.state.WatermarkHoldState; - -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; - import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.common.typeutils.base.StringSerializer; import org.apache.flink.runtime.state.AbstractStateBackend; import org.joda.time.Instant; -import java.nio.ByteBuffer; -import java.util.HashMap; -import java.util.Map; - /** * {@link StateInternals} that uses a Flink {@link AbstractStateBackend} to * manage state. diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItem.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItem.java index 94bf3af451ce..5751aac78123 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItem.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItem.java @@ -17,12 +17,11 @@ */ package org.apache.beam.runners.flink.translation.wrappers.streaming; +import java.util.Collections; import org.apache.beam.sdk.util.KeyedWorkItem; import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.WindowedValue; -import java.util.Collections; - public class SingletonKeyedWorkItem implements KeyedWorkItem { final K key; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java index 323f5722dafa..5e583e9013c2 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java @@ -19,6 +19,13 @@ import static com.google.common.base.Preconditions.checkArgument; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.List; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.StandardCoder; @@ -28,16 +35,6 @@ import org.apache.beam.sdk.util.PropertyNames; import org.apache.beam.sdk.util.WindowedValue; -import com.google.common.collect.ImmutableList; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.List; - public class SingletonKeyedWorkItemCoder extends StandardCoder> { /** * Create a new {@link KeyedWorkItemCoder} with the provided key coder, element coder, and window diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java index 73c1eed96136..c6dde5197dce 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java @@ -17,6 +17,22 @@ */ package org.apache.beam.runners.flink.translation.wrappers.streaming; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.PriorityQueue; +import java.util.Queue; +import java.util.Set; +import javax.annotation.Nullable; import org.apache.beam.runners.core.GroupAlsoByWindowViaWindowSetDoFn; import org.apache.beam.runners.flink.translation.wrappers.DataInputViewWrapper; import org.apache.beam.sdk.coders.Coder; @@ -37,7 +53,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; - import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.core.memory.DataInputView; @@ -47,23 +62,6 @@ import org.apache.flink.streaming.runtime.tasks.StreamTaskState; import org.joda.time.Instant; -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.nio.ByteBuffer; -import java.util.Collection; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.PriorityQueue; -import java.util.Queue; -import java.util.Set; -import javax.annotation.Nullable; - /** * Flink operator for executing window {@link DoFn DoFns}. * diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WorkItemKeySelector.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WorkItemKeySelector.java index 2bbed58f63e7..51d9e0c9399c 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WorkItemKeySelector.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WorkItemKeySelector.java @@ -17,19 +17,16 @@ */ package org.apache.beam.runners.flink.translation.wrappers.streaming; +import java.nio.ByteBuffer; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.KeyedWorkItem; import org.apache.beam.sdk.util.WindowedValue; - import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.typeutils.GenericTypeInfo; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; -import java.nio.ByteBuffer; - /** * {@link KeySelector} that retrieves a key from a {@link KeyedWorkItem}. This will return * the key as encoded by the provided {@link Coder} in a {@link ByteBuffer}. This ensures diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/BoundedSourceWrapper.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/BoundedSourceWrapper.java index 9d983b06cc6a..3cb93c0519b3 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/BoundedSourceWrapper.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/BoundedSourceWrapper.java @@ -17,15 +17,15 @@ */ package org.apache.beam.runners.flink.translation.wrappers.streaming.io; +import com.google.common.annotations.VisibleForTesting; +import java.util.ArrayList; +import java.util.List; import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowedValue; - -import com.google.common.annotations.VisibleForTesting; - import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; import org.apache.flink.streaming.api.operators.StreamSource; import org.apache.flink.streaming.api.watermark.Watermark; @@ -33,9 +33,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.ArrayList; -import java.util.List; - /** * Wrapper for executing {@link BoundedSource UnboundedSources} as a Flink Source. */ diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSink.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSink.java index 098473dec230..2117e9d19353 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSink.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSink.java @@ -18,23 +18,19 @@ package org.apache.beam.runners.flink.translation.wrappers.streaming.io; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Collection; +import java.util.List; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.io.Sink; import org.apache.beam.sdk.io.UnboundedSource; -import org.apache.beam.sdk.io.Write; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.CloudObject; import org.apache.beam.sdk.util.common.ElementByteSizeObserver; import org.apache.flink.streaming.api.functions.sink.SinkFunction; -import org.apache.flink.streaming.api.functions.source.SourceFunction; - -import javax.annotation.Nullable; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.Collection; -import java.util.List; /** * A wrapper translating Flink sinks implementing the {@link SinkFunction} interface, into diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSource.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSource.java index 716ca304e539..c6e0825648f4 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSource.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSource.java @@ -19,18 +19,15 @@ import static com.google.common.base.Preconditions.checkNotNull; +import java.util.List; +import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.options.PipelineOptions; - import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; import org.apache.flink.streaming.api.functions.IngestionTimeExtractor; import org.apache.flink.streaming.api.functions.source.SourceFunction; -import java.util.List; - -import javax.annotation.Nullable; - /** * A wrapper translating Flink Sources implementing the {@link SourceFunction} interface, into * unbounded Beam sources (see {@link UnboundedSource}). diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java index 08bdb509da70..8d37fe70dfb8 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java @@ -19,15 +19,6 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.io.UnboundedSource; -import org.apache.beam.sdk.options.PipelineOptions; - -import org.joda.time.Instant; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.BufferedReader; import java.io.IOException; import java.io.InputStreamReader; @@ -37,8 +28,14 @@ import java.util.Collections; import java.util.List; import java.util.NoSuchElementException; - import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.io.UnboundedSource; +import org.apache.beam.sdk.options.PipelineOptions; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * An example unbounded Beam source that reads input from a socket. This is used mainly for testing and debugging. diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java index 7f26a658bc11..8647322dfb86 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java @@ -17,6 +17,12 @@ */ package org.apache.beam.runners.flink.translation.wrappers.streaming.io; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Function; +import com.google.common.collect.Lists; +import java.io.ByteArrayInputStream; +import java.util.ArrayList; +import java.util.List; import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; @@ -29,11 +35,6 @@ import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TypeDescriptor; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; -import com.google.common.collect.Lists; - import org.apache.commons.io.output.ByteArrayOutputStream; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.checkpoint.Checkpointed; @@ -46,10 +47,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.ByteArrayInputStream; -import java.util.ArrayList; -import java.util.List; - /** * Wrapper for executing {@link UnboundedSource UnboundedSources} as a Flink Source. */ diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/EncodedValueComparatorTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/EncodedValueComparatorTest.java index 68ede89954cb..10d6d9d357ff 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/EncodedValueComparatorTest.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/EncodedValueComparatorTest.java @@ -22,7 +22,6 @@ import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.util.CoderUtils; - import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeutils.ComparatorTestBase; import org.apache.flink.api.common.typeutils.TypeComparator; diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlinkRunnerRegistrarTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlinkRunnerRegistrarTest.java index ff1025f3ded7..d9d174c8248e 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlinkRunnerRegistrarTest.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlinkRunnerRegistrarTest.java @@ -22,7 +22,6 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; - import org.junit.Test; /** diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java index 77e8a477e76a..32339dce5165 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java @@ -21,6 +21,8 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; +import java.util.Collections; +import java.util.HashMap; import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; import org.apache.beam.runners.flink.translation.wrappers.streaming.DoFnOperator; import org.apache.beam.sdk.options.Default; @@ -34,7 +36,6 @@ import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; - import org.apache.commons.lang.SerializationUtils; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.TypeHint; @@ -46,9 +47,6 @@ import org.junit.BeforeClass; import org.junit.Test; -import java.util.Collections; -import java.util.HashMap; - /** * Tests for serialization and deserialization of {@link PipelineOptions} in {@link DoFnOperator}. */ diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ReadSourceITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ReadSourceITCase.java index 516c7bae75e5..44c90178fefc 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ReadSourceITCase.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ReadSourceITCase.java @@ -17,20 +17,17 @@ */ package org.apache.beam.runners.flink; +import com.google.common.base.Joiner; +import java.io.File; +import java.net.URI; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.CountingInput; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.base.Joiner; - import org.apache.flink.test.util.JavaProgramTestBase; -import java.io.File; -import java.net.URI; - /** * Reads from a bounded source in batch execution. */ diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ReadSourceStreamingITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ReadSourceStreamingITCase.java index ea58d0dd8657..79b78826ede9 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ReadSourceStreamingITCase.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ReadSourceStreamingITCase.java @@ -17,14 +17,12 @@ */ package org.apache.beam.runners.flink; +import com.google.common.base.Joiner; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.CountingInput; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; - -import com.google.common.base.Joiner; - import org.apache.flink.streaming.util.StreamingProgramTestBase; /** diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java index f1d9097b5924..09881463f6c2 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java @@ -20,6 +20,11 @@ import static org.junit.Assert.assertNotNull; +import com.google.common.base.Joiner; +import java.io.File; +import java.io.IOException; +import java.io.PrintWriter; +import java.net.URI; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StringUtf8Coder; @@ -27,18 +32,10 @@ import org.apache.beam.sdk.io.Write; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Create; - -import com.google.common.base.Joiner; - import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.Path; import org.apache.flink.test.util.JavaProgramTestBase; -import java.io.File; -import java.io.IOException; -import java.io.PrintWriter; -import java.net.URI; - /** * Tests the translation of custom Write.Bound sinks. */ diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/DoFnOperatorTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/DoFnOperatorTest.java index fb1b1e8d1afc..5f1b066d620b 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/DoFnOperatorTest.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/DoFnOperatorTest.java @@ -20,6 +20,10 @@ import static org.hamcrest.collection.IsIterableContainingInOrder.contains; import static org.junit.Assert.assertThat; +import com.google.common.collect.ImmutableList; +import java.util.Collections; +import java.util.HashMap; +import javax.annotation.Nullable; import org.apache.beam.runners.flink.FlinkPipelineOptions; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; import org.apache.beam.runners.flink.translation.wrappers.streaming.DoFnOperator; @@ -36,9 +40,6 @@ import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; - -import com.google.common.collect.ImmutableList; - import org.apache.flink.shaded.com.google.common.base.Function; import org.apache.flink.shaded.com.google.common.base.Predicate; import org.apache.flink.shaded.com.google.common.collect.FluentIterable; @@ -53,10 +54,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Collections; -import java.util.HashMap; -import javax.annotation.Nullable; - /** * Tests for {@link DoFnOperator}. */ diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java index 627f5458dc1f..711ae0001f79 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java @@ -22,6 +22,8 @@ import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertThat; +import java.nio.ByteBuffer; +import java.util.Arrays; import org.apache.beam.runners.flink.translation.wrappers.streaming.FlinkStateInternals; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.StringUtf8Coder; @@ -42,7 +44,6 @@ import org.apache.beam.sdk.util.state.StateTags; import org.apache.beam.sdk.util.state.ValueState; import org.apache.beam.sdk.util.state.WatermarkHoldState; - import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.java.typeutils.GenericTypeInfo; import org.apache.flink.runtime.operators.testutils.DummyEnvironment; @@ -54,9 +55,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.nio.ByteBuffer; -import java.util.Arrays; - /** * Tests for {@link FlinkStateInternals}. This is based on the tests for {@code InMemoryStateInternals}. */ diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java index ca183a855b18..ab98c27733b3 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java @@ -17,28 +17,25 @@ */ package org.apache.beam.runners.flink.streaming; +import com.google.common.base.Joiner; +import java.io.Serializable; +import java.util.Arrays; import org.apache.beam.runners.flink.FlinkTestPipeline; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.GroupByKey; +import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.windowing.AfterWatermark; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.base.Joiner; - import org.apache.flink.streaming.util.StreamingProgramTestBase; import org.joda.time.Duration; import org.joda.time.Instant; -import java.io.Serializable; -import java.util.Arrays; - public class GroupByNullKeyTest extends StreamingProgramTestBase implements Serializable { diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/TestCountingSource.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/TestCountingSource.java index 3ced02ee3038..9251d42f4404 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/TestCountingSource.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/TestCountingSource.java @@ -19,6 +19,11 @@ import static org.apache.beam.sdk.util.CoderUtils.encodeToByteArray; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ThreadLocalRandom; +import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.DelegateCoder; import org.apache.beam.sdk.coders.KvCoder; @@ -26,17 +31,10 @@ import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.values.KV; - import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.ThreadLocalRandom; -import javax.annotation.Nullable; - /** * An unbounded source for testing the unbounded sources framework code. * diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java index 7912aee2409f..64f978fa0f23 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java @@ -17,6 +17,10 @@ */ package org.apache.beam.runners.flink.streaming; +import com.google.api.services.bigquery.model.TableRow; +import com.google.common.base.Joiner; +import java.io.Serializable; +import java.util.Arrays; import org.apache.beam.runners.flink.FlinkTestPipeline; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.TextIO; @@ -28,17 +32,10 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - -import com.google.api.services.bigquery.model.TableRow; -import com.google.common.base.Joiner; - import org.apache.flink.streaming.util.StreamingProgramTestBase; import org.joda.time.Duration; import org.joda.time.Instant; -import java.io.Serializable; -import java.util.Arrays; - /** * Session window test diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceWrapperTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceWrapperTest.java index 11221797456b..a70ad49b9f73 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceWrapperTest.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceWrapperTest.java @@ -23,12 +23,14 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSourceWrapper; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; - import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.configuration.Configuration; @@ -43,10 +45,6 @@ import org.apache.flink.streaming.runtime.tasks.StreamTask; import org.junit.Test; -import java.util.Collections; -import java.util.HashSet; -import java.util.Set; - /** * Tests for {@link UnboundedSourceWrapper}. */ diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BlockingDataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BlockingDataflowRunner.java index e7cd67e6b873..d26536136a79 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BlockingDataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BlockingDataflowRunner.java @@ -17,6 +17,8 @@ */ package org.apache.beam.runners.dataflow; +import java.io.IOException; +import javax.annotation.Nullable; import org.apache.beam.runners.dataflow.options.BlockingDataflowPipelineOptions; import org.apache.beam.runners.dataflow.util.MonitoringUtil; import org.apache.beam.sdk.Pipeline; @@ -28,15 +30,10 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; - import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; - -import javax.annotation.Nullable; - /** * A {@link PipelineRunner} that's like {@link DataflowRunner} * but that waits for the launched job to finish. diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobException.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobException.java index 74c0f80684a2..6e32fdecaf9b 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobException.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobException.java @@ -18,7 +18,6 @@ package org.apache.beam.runners.dataflow; import java.util.Objects; - import javax.annotation.Nullable; /** diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java index 3d0f1458fbae..9a515fa6140f 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java @@ -19,17 +19,6 @@ import static org.apache.beam.runners.dataflow.util.TimeUtil.fromCloudTime; -import org.apache.beam.runners.dataflow.internal.DataflowAggregatorTransforms; -import org.apache.beam.runners.dataflow.internal.DataflowMetricUpdateExtractor; -import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; -import org.apache.beam.runners.dataflow.util.MonitoringUtil; -import org.apache.beam.sdk.AggregatorRetrievalException; -import org.apache.beam.sdk.AggregatorValues; -import org.apache.beam.sdk.PipelineResult; -import org.apache.beam.sdk.transforms.Aggregator; -import org.apache.beam.sdk.util.AttemptAndTimeBoundedExponentialBackOff; -import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; - import com.google.api.client.googleapis.json.GoogleJsonResponseException; import com.google.api.client.util.BackOff; import com.google.api.client.util.BackOffUtils; @@ -41,18 +30,25 @@ import com.google.api.services.dataflow.model.MetricUpdate; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.MoreObjects; - -import org.joda.time.Duration; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.IOException; import java.net.SocketTimeoutException; import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; - import javax.annotation.Nullable; +import org.apache.beam.runners.dataflow.internal.DataflowAggregatorTransforms; +import org.apache.beam.runners.dataflow.internal.DataflowMetricUpdateExtractor; +import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; +import org.apache.beam.runners.dataflow.util.MonitoringUtil; +import org.apache.beam.sdk.AggregatorRetrievalException; +import org.apache.beam.sdk.AggregatorValues; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.transforms.Aggregator; +import org.apache.beam.sdk.util.AttemptAndTimeBoundedExponentialBackOff; +import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; +import org.joda.time.Duration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * A DataflowPipelineJob represents a job submitted to Dataflow using diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrar.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrar.java index 4c88c4e0f010..5090a8a9b616 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrar.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrar.java @@ -17,6 +17,8 @@ */ package org.apache.beam.runners.dataflow; +import com.google.auto.service.AutoService; +import com.google.common.collect.ImmutableList; import org.apache.beam.runners.dataflow.options.BlockingDataflowPipelineOptions; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.sdk.options.PipelineOptions; @@ -24,9 +26,6 @@ import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.runners.PipelineRunnerRegistrar; -import com.google.auto.service.AutoService; -import com.google.common.collect.ImmutableList; - /** * Contains the {@link PipelineOptionsRegistrar} and {@link PipelineRunnerRegistrar} for the * {@link DataflowRunner}. diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java index b8d94452872c..0d7288122995 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java @@ -17,6 +17,10 @@ */ package org.apache.beam.runners.dataflow; +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.base.Preconditions.checkState; +import static com.google.common.base.Strings.isNullOrEmpty; import static org.apache.beam.sdk.util.SerializableUtils.serializeToByteArray; import static org.apache.beam.sdk.util.StringUtils.byteArrayToJsonString; import static org.apache.beam.sdk.util.StringUtils.jsonStringToByteArray; @@ -28,11 +32,24 @@ import static org.apache.beam.sdk.util.Structs.addString; import static org.apache.beam.sdk.util.Structs.getString; -import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Preconditions.checkNotNull; -import static com.google.common.base.Preconditions.checkState; -import static com.google.common.base.Strings.isNullOrEmpty; - +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.api.services.dataflow.model.AutoscalingSettings; +import com.google.api.services.dataflow.model.DataflowPackage; +import com.google.api.services.dataflow.model.Disk; +import com.google.api.services.dataflow.model.Environment; +import com.google.api.services.dataflow.model.Job; +import com.google.api.services.dataflow.model.Step; +import com.google.api.services.dataflow.model.WorkerPool; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import javax.annotation.Nullable; import org.apache.beam.runners.dataflow.DataflowRunner.GroupByKeyAndSortValuesOnly; import org.apache.beam.runners.dataflow.internal.ReadTranslator; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; @@ -71,30 +88,9 @@ import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TypedPValue; - -import com.google.api.services.dataflow.model.AutoscalingSettings; -import com.google.api.services.dataflow.model.DataflowPackage; -import com.google.api.services.dataflow.model.Disk; -import com.google.api.services.dataflow.model.Environment; -import com.google.api.services.dataflow.model.Job; -import com.google.api.services.dataflow.model.Step; -import com.google.api.services.dataflow.model.WorkerPool; - -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectMapper; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import javax.annotation.Nullable; - /** * {@link DataflowPipelineTranslator} knows how to translate {@link Pipeline} objects * into Cloud Dataflow Service API {@link Job}s. diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index 7f632a1a7202..a0e24b14a575 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -17,13 +17,61 @@ */ package org.apache.beam.runners.dataflow; -import static org.apache.beam.sdk.util.StringUtils.approximatePTransformName; -import static org.apache.beam.sdk.util.StringUtils.approximateSimpleName; -import static org.apache.beam.sdk.util.WindowedValue.valueInEmptyWindows; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; import static com.google.common.base.Strings.isNullOrEmpty; +import static org.apache.beam.sdk.util.StringUtils.approximatePTransformName; +import static org.apache.beam.sdk.util.StringUtils.approximateSimpleName; +import static org.apache.beam.sdk.util.WindowedValue.valueInEmptyWindows; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.api.client.googleapis.json.GoogleJsonResponseException; +import com.google.api.services.clouddebugger.v2.Clouddebugger; +import com.google.api.services.clouddebugger.v2.model.Debuggee; +import com.google.api.services.clouddebugger.v2.model.RegisterDebuggeeRequest; +import com.google.api.services.clouddebugger.v2.model.RegisterDebuggeeResponse; +import com.google.api.services.dataflow.Dataflow; +import com.google.api.services.dataflow.model.DataflowPackage; +import com.google.api.services.dataflow.model.Job; +import com.google.api.services.dataflow.model.ListJobsResponse; +import com.google.api.services.dataflow.model.WorkerPool; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Function; +import com.google.common.base.Joiner; +import com.google.common.base.Optional; +import com.google.common.base.Utf8; +import com.google.common.collect.ForwardingMap; +import com.google.common.collect.HashMultimap; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import com.google.common.collect.Maps; +import com.google.common.collect.Multimap; +import java.io.File; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.PrintWriter; +import java.io.Serializable; +import java.net.URISyntaxException; +import java.net.URL; +import java.net.URLClassLoader; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.SortedSet; +import java.util.TreeSet; +import javax.annotation.Nullable; import org.apache.beam.runners.dataflow.DataflowPipelineTranslator.JobSpecification; import org.apache.beam.runners.dataflow.DataflowPipelineTranslator.TransformTranslator; import org.apache.beam.runners.dataflow.DataflowPipelineTranslator.TranslationContext; @@ -114,65 +162,12 @@ import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; - -import com.google.api.client.googleapis.json.GoogleJsonResponseException; -import com.google.api.services.clouddebugger.v2.Clouddebugger; -import com.google.api.services.clouddebugger.v2.model.Debuggee; -import com.google.api.services.clouddebugger.v2.model.RegisterDebuggeeRequest; -import com.google.api.services.clouddebugger.v2.model.RegisterDebuggeeResponse; -import com.google.api.services.dataflow.Dataflow; -import com.google.api.services.dataflow.model.DataflowPackage; -import com.google.api.services.dataflow.model.Job; -import com.google.api.services.dataflow.model.ListJobsResponse; -import com.google.api.services.dataflow.model.WorkerPool; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; -import com.google.common.base.Joiner; -import com.google.common.base.Optional; -import com.google.common.base.Utf8; -import com.google.common.collect.ForwardingMap; -import com.google.common.collect.HashMultimap; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Iterables; -import com.google.common.collect.Maps; -import com.google.common.collect.Multimap; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - import org.joda.time.DateTimeUtils; import org.joda.time.DateTimeZone; import org.joda.time.format.DateTimeFormat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.File; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.PrintWriter; -import java.io.Serializable; -import java.net.URISyntaxException; -import java.net.URL; -import java.net.URLClassLoader; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Random; -import java.util.Set; -import java.util.SortedSet; -import java.util.TreeSet; - -import javax.annotation.Nullable; - /** * A {@link PipelineRunner} that executes the operations in the pipeline by first translating them * to the Dataflow representation using the {@link DataflowPipelineTranslator} and then submitting diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunnerHooks.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunnerHooks.java index 05297ec52046..b67421dd386a 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunnerHooks.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunnerHooks.java @@ -17,9 +17,8 @@ */ package org.apache.beam.runners.dataflow; -import org.apache.beam.sdk.annotations.Experimental; - import com.google.api.services.dataflow.model.Environment; +import org.apache.beam.sdk.annotations.Experimental; /** * An instance of this class can be passed to the diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/CustomSources.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/CustomSources.java index 73e5da02f464..aeb81035c6d0 100755 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/CustomSources.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/CustomSources.java @@ -17,30 +17,26 @@ */ package org.apache.beam.runners.dataflow.internal; +import static com.google.api.client.util.Base64.encodeBase64String; +import static com.google.common.base.Preconditions.checkArgument; import static org.apache.beam.sdk.util.SerializableUtils.serializeToByteArray; import static org.apache.beam.sdk.util.Structs.addString; import static org.apache.beam.sdk.util.Structs.addStringList; -import static com.google.api.client.util.Base64.encodeBase64String; -import static com.google.common.base.Preconditions.checkArgument; - +import com.google.api.services.dataflow.model.SourceMetadata; +import com.google.common.annotations.VisibleForTesting; +import com.google.protobuf.ByteString; +import java.util.ArrayList; +import java.util.List; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.Source; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.CloudObject; - -import com.google.api.services.dataflow.model.SourceMetadata; -import com.google.common.annotations.VisibleForTesting; -import com.google.protobuf.ByteString; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.ArrayList; -import java.util.List; - /** * A helper class for supporting sources defined as {@code Source}. diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowAggregatorTransforms.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowAggregatorTransforms.java index 7a08fde8fc17..fb7897361fa5 100755 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowAggregatorTransforms.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowAggregatorTransforms.java @@ -17,19 +17,17 @@ */ package org.apache.beam.runners.dataflow.internal; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.transforms.Aggregator; -import org.apache.beam.sdk.transforms.AppliedPTransform; -import org.apache.beam.sdk.transforms.PTransform; - import com.google.common.collect.BiMap; import com.google.common.collect.HashBiMap; import com.google.common.collect.HashMultimap; import com.google.common.collect.Multimap; - import java.util.Collection; import java.util.HashSet; import java.util.Map; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.transforms.Aggregator; +import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.transforms.PTransform; /** * A mapping relating {@link Aggregator}s and the {@link PTransform} in which they are used. diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowMetricUpdateExtractor.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowMetricUpdateExtractor.java index 8ab59fc911d6..d715437ae68d 100755 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowMetricUpdateExtractor.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowMetricUpdateExtractor.java @@ -17,18 +17,16 @@ */ package org.apache.beam.runners.dataflow.internal; -import org.apache.beam.sdk.transforms.Aggregator; -import org.apache.beam.sdk.transforms.AppliedPTransform; -import org.apache.beam.sdk.transforms.Combine.CombineFn; -import org.apache.beam.sdk.transforms.PTransform; - import com.google.api.services.dataflow.model.MetricStructuredName; import com.google.api.services.dataflow.model.MetricUpdate; - import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; +import org.apache.beam.sdk.transforms.Aggregator; +import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.transforms.Combine.CombineFn; +import org.apache.beam.sdk.transforms.PTransform; /** * Methods for extracting the values of an {@link Aggregator} from a collection of {@link diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowUnboundedReadFromBoundedSource.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowUnboundedReadFromBoundedSource.java index 283f01221ceb..85f5e734a253 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowUnboundedReadFromBoundedSource.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowUnboundedReadFromBoundedSource.java @@ -17,10 +17,25 @@ */ package org.apache.beam.runners.dataflow.internal; -import static org.apache.beam.sdk.util.StringUtils.approximateSimpleName; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; +import static org.apache.beam.sdk.util.StringUtils.approximateSimpleName; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Function; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.ListCoder; @@ -39,30 +54,10 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.TimestampedValue; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.Arrays; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.NoSuchElementException; - -import javax.annotation.Nullable; - /** * {@link PTransform} that converts a {@link BoundedSource} as an {@link UnboundedSource}. * diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/IsmFormat.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/IsmFormat.java index d8bfe4274025..6f4a18bf9d05 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/IsmFormat.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/IsmFormat.java @@ -17,12 +17,25 @@ */ package org.apache.beam.runners.dataflow.internal; -import static org.apache.beam.sdk.util.Structs.addLong; - import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; +import static org.apache.beam.sdk.util.Structs.addLong; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.auto.value.AutoValue; +import com.google.common.collect.ImmutableList; +import com.google.common.hash.HashFunction; +import com.google.common.hash.Hashing; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.List; +import javax.annotation.Nullable; import org.apache.beam.runners.dataflow.util.RandomAccessData; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.ByteArrayCoder; @@ -38,24 +51,6 @@ import org.apache.beam.sdk.util.VarInt; import org.apache.beam.sdk.values.PCollection; -import com.google.auto.value.AutoValue; -import com.google.common.collect.ImmutableList; -import com.google.common.hash.HashFunction; -import com.google.common.hash.Hashing; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.ArrayList; -import java.util.List; - -import javax.annotation.Nullable; - /** * An Ism file is a prefix encoded composite key value file broken into shards. Each composite * key is composed of a fixed number of component keys. A fixed number of those sub keys represent diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/ReadTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/ReadTranslator.java index 373738a71709..094f4052fc87 100755 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/ReadTranslator.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/ReadTranslator.java @@ -21,6 +21,9 @@ import static org.apache.beam.sdk.util.Structs.addDictionary; import static org.apache.beam.sdk.util.Structs.addLong; +import com.google.api.services.dataflow.model.SourceMetadata; +import java.util.HashMap; +import java.util.Map; import org.apache.beam.runners.dataflow.DataflowPipelineTranslator; import org.apache.beam.runners.dataflow.DataflowPipelineTranslator.TransformTranslator; import org.apache.beam.runners.dataflow.DataflowPipelineTranslator.TranslationContext; @@ -31,11 +34,6 @@ import org.apache.beam.sdk.util.PropertyNames; import org.apache.beam.sdk.values.PValue; -import com.google.api.services.dataflow.model.SourceMetadata; - -import java.util.HashMap; -import java.util.Map; - /** * Translator for the {@code Read} {@code PTransform} for the Dataflow back-end. */ diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/CloudDebuggerOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/CloudDebuggerOptions.java index dbfbb16f3022..d1c8e7a7affd 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/CloudDebuggerOptions.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/CloudDebuggerOptions.java @@ -17,15 +17,13 @@ */ package org.apache.beam.runners.dataflow.options; +import com.google.api.services.clouddebugger.v2.model.Debuggee; +import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.Hidden; -import com.google.api.services.clouddebugger.v2.model.Debuggee; - -import javax.annotation.Nullable; - /** * Options for controlling Cloud Debugger. */ diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptions.java index ac2e0b72babf..dfe538da9f05 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptions.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptions.java @@ -17,6 +17,10 @@ */ package org.apache.beam.runners.dataflow.options; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.google.api.services.dataflow.Dataflow; +import java.util.List; +import java.util.Map; import org.apache.beam.runners.dataflow.util.DataflowTransport; import org.apache.beam.runners.dataflow.util.GcsStager; import org.apache.beam.runners.dataflow.util.Stager; @@ -28,13 +32,6 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.InstanceBuilder; -import com.google.api.services.dataflow.Dataflow; - -import com.fasterxml.jackson.annotation.JsonIgnore; - -import java.util.List; -import java.util.Map; - /** * Internal. Options used to control execution of the Dataflow SDK for * debugging and testing purposes. diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java index 841741f5562a..8ef43c5efaf6 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java @@ -20,6 +20,8 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Strings.isNullOrEmpty; +import com.google.common.base.MoreObjects; +import java.io.IOException; import org.apache.beam.runners.dataflow.DataflowRunner; import org.apache.beam.sdk.options.ApplicationNameOptions; import org.apache.beam.sdk.options.BigQueryOptions; @@ -33,16 +35,11 @@ import org.apache.beam.sdk.options.StreamingOptions; import org.apache.beam.sdk.options.Validation; import org.apache.beam.sdk.util.IOChannelUtils; - -import com.google.common.base.MoreObjects; - import org.joda.time.DateTimeUtils; import org.joda.time.DateTimeZone; import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; -import java.io.IOException; - /** * Options that can be used to configure the {@link DataflowRunner}. */ diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java index 9b7a9ceb7f74..6c59f381ac6b 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java @@ -17,6 +17,9 @@ */ package org.apache.beam.runners.dataflow.options; +import com.fasterxml.jackson.annotation.JsonIgnore; +import java.util.List; +import javax.annotation.Nullable; import org.apache.beam.runners.dataflow.DataflowRunner; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.options.Default; @@ -25,12 +28,6 @@ import org.apache.beam.sdk.options.Hidden; import org.apache.beam.sdk.options.PipelineOptions; -import com.fasterxml.jackson.annotation.JsonIgnore; - -import java.util.List; - -import javax.annotation.Nullable; - /** * Options that are used to configure the Dataflow pipeline worker pool. */ diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowProfilingOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowProfilingOptions.java index c7b4c91a0998..ae8801b49555 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowProfilingOptions.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowProfilingOptions.java @@ -17,12 +17,11 @@ */ package org.apache.beam.runners.dataflow.options; +import java.util.HashMap; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.Hidden; -import java.util.HashMap; - /** * Options for controlling profiling of pipeline execution. */ diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowWorkerLoggingOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowWorkerLoggingOptions.java index 91ac62aa0947..38c139c68274 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowWorkerLoggingOptions.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowWorkerLoggingOptions.java @@ -19,15 +19,13 @@ import static com.google.common.base.Preconditions.checkNotNull; -import org.apache.beam.sdk.options.Default; -import org.apache.beam.sdk.options.Description; -import org.apache.beam.sdk.options.PipelineOptions; - import com.fasterxml.jackson.annotation.JsonCreator; - import java.util.Arrays; import java.util.HashMap; import java.util.Map; +import org.apache.beam.sdk.options.Default; +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.PipelineOptions; /** * Options that are used to control logging configuration on the Dataflow worker. diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java index f74f4dd17039..9be773b9b084 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java @@ -19,6 +19,18 @@ import static org.hamcrest.MatcherAssert.assertThat; +import com.google.api.services.dataflow.model.JobMessage; +import com.google.api.services.dataflow.model.JobMetrics; +import com.google.api.services.dataflow.model.MetricUpdate; +import com.google.common.base.Joiner; +import com.google.common.base.Optional; +import com.google.common.base.Throwables; +import java.io.IOException; +import java.math.BigDecimal; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; import org.apache.beam.runners.dataflow.DataflowJobExecutionException; import org.apache.beam.runners.dataflow.DataflowPipelineJob; import org.apache.beam.runners.dataflow.DataflowRunner; @@ -34,25 +46,10 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; - -import com.google.api.services.dataflow.model.JobMessage; -import com.google.api.services.dataflow.model.JobMetrics; -import com.google.api.services.dataflow.model.MetricUpdate; -import com.google.common.base.Joiner; -import com.google.common.base.Optional; -import com.google.common.base.Throwables; - import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.math.BigDecimal; -import java.util.List; -import java.util.concurrent.Callable; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Future; - /** * {@link TestDataflowRunner} is a pipeline runner that wraps a * {@link DataflowRunner} when running tests against the {@link TestPipeline}. diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowTransport.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowTransport.java index f9887491555d..0391594ed57e 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowTransport.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowTransport.java @@ -20,18 +20,16 @@ import static org.apache.beam.sdk.util.Transport.getJsonFactory; import static org.apache.beam.sdk.util.Transport.getTransport; -import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; -import org.apache.beam.sdk.util.RetryHttpRequestInitializer; - import com.google.api.client.auth.oauth2.Credential; import com.google.api.client.http.HttpRequestInitializer; import com.google.api.services.clouddebugger.v2.Clouddebugger; import com.google.api.services.dataflow.Dataflow; import com.google.cloud.hadoop.util.ChainingHttpRequestInitializer; import com.google.common.collect.ImmutableList; - import java.net.MalformedURLException; import java.net.URL; +import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; +import org.apache.beam.sdk.util.RetryHttpRequestInitializer; /** * Helpers for cloud communication. diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DoFnInfo.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DoFnInfo.java index 2017313b6ee6..139db9da0ab9 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DoFnInfo.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DoFnInfo.java @@ -17,13 +17,12 @@ */ package org.apache.beam.runners.dataflow.util; +import java.io.Serializable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollectionView; -import java.io.Serializable; - /** * Wrapper class holding the necessary information to serialize a OldDoFn. * diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/GcsStager.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/GcsStager.java index bf25ce47a98e..6ca4c3f78913 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/GcsStager.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/GcsStager.java @@ -19,14 +19,12 @@ import static com.google.common.base.Preconditions.checkNotNull; +import com.google.api.services.dataflow.model.DataflowPackage; +import java.util.List; import org.apache.beam.runners.dataflow.options.DataflowPipelineDebugOptions; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.sdk.options.PipelineOptions; -import com.google.api.services.dataflow.model.DataflowPackage; - -import java.util.List; - /** * Utility class for staging files to GCS. */ diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/MonitoringUtil.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/MonitoringUtil.java index 4d12e665c745..d0146230c66f 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/MonitoringUtil.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/MonitoringUtil.java @@ -19,9 +19,6 @@ import static org.apache.beam.runners.dataflow.util.TimeUtil.fromCloudTime; -import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; -import org.apache.beam.sdk.PipelineResult.State; - import com.google.api.services.dataflow.Dataflow; import com.google.api.services.dataflow.Dataflow.Projects.Jobs.Messages; import com.google.api.services.dataflow.model.JobMessage; @@ -29,11 +26,6 @@ import com.google.common.base.MoreObjects; import com.google.common.base.Strings; import com.google.common.collect.ImmutableMap; - -import org.joda.time.Instant; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.IOException; import java.io.UnsupportedEncodingException; import java.net.URLEncoder; @@ -42,8 +34,12 @@ import java.util.Comparator; import java.util.List; import java.util.Map; - import javax.annotation.Nullable; +import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; +import org.apache.beam.sdk.PipelineResult.State; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * A helper class for monitoring jobs submitted to the service. diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java index cff7e2b9c68f..bf1f666c32c5 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java @@ -17,11 +17,7 @@ */ package org.apache.beam.runners.dataflow.util; -import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; -import org.apache.beam.sdk.util.IOChannelUtils; -import org.apache.beam.sdk.util.MimeTypes; -import org.apache.beam.sdk.util.ZipFiles; - +import com.fasterxml.jackson.core.Base64Variants; import com.google.api.client.util.BackOffUtils; import com.google.api.client.util.Sleeper; import com.google.api.services.dataflow.model.DataflowPackage; @@ -31,12 +27,6 @@ import com.google.common.hash.Hashing; import com.google.common.io.CountingOutputStream; import com.google.common.io.Files; - -import com.fasterxml.jackson.core.Base64Variants; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; @@ -47,6 +37,12 @@ import java.util.Collection; import java.util.List; import java.util.Objects; +import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; +import org.apache.beam.sdk.util.IOChannelUtils; +import org.apache.beam.sdk.util.MimeTypes; +import org.apache.beam.sdk.util.ZipFiles; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** Helper routines for packages. */ public class PackageUtil { diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/RandomAccessData.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/RandomAccessData.java index 9e102422b7db..683e16b3e9a0 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/RandomAccessData.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/RandomAccessData.java @@ -20,26 +20,22 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; -import org.apache.beam.sdk.coders.AtomicCoder; -import org.apache.beam.sdk.coders.ByteArrayCoder; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.util.VarInt; - +import com.fasterxml.jackson.annotation.JsonCreator; import com.google.common.base.MoreObjects; import com.google.common.io.ByteStreams; import com.google.common.primitives.UnsignedBytes; - -import com.fasterxml.jackson.annotation.JsonCreator; - import java.io.ByteArrayInputStream; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.util.Arrays; import java.util.Comparator; - import javax.annotation.concurrent.NotThreadSafe; +import org.apache.beam.sdk.coders.AtomicCoder; +import org.apache.beam.sdk.coders.ByteArrayCoder; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.util.VarInt; /** * An elastic-sized byte array which allows you to manipulate it as a stream, or access diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/Stager.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/Stager.java index 2172eb4711cf..232e03260fe2 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/Stager.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/Stager.java @@ -18,7 +18,6 @@ package org.apache.beam.runners.dataflow.util; import com.google.api.services.dataflow.model.DataflowPackage; - import java.util.List; /** diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/TimeUtil.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/TimeUtil.java index 434c83f7da22..bff379fc1cc7 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/TimeUtil.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/TimeUtil.java @@ -17,6 +17,9 @@ */ package org.apache.beam.runners.dataflow.util; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import javax.annotation.Nullable; import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Instant; @@ -24,11 +27,6 @@ import org.joda.time.ReadableInstant; import org.joda.time.chrono.ISOChronology; -import java.util.regex.Matcher; -import java.util.regex.Pattern; - -import javax.annotation.Nullable; - /** * A helper class for converting between Dataflow API and SDK time * representations. diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BlockingDataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BlockingDataflowRunnerTest.java index 7bdac3de46af..4572a64aeaa9 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BlockingDataflowRunnerTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BlockingDataflowRunnerTest.java @@ -34,7 +34,6 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.util.NoopPathValidator; import org.apache.beam.sdk.util.TestCredential; - import org.hamcrest.Description; import org.hamcrest.Factory; import org.hamcrest.Matcher; diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java index e6277d9c98c3..22b5400b0fdf 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java @@ -32,6 +32,21 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import com.google.api.services.dataflow.Dataflow; +import com.google.api.services.dataflow.Dataflow.Projects.Jobs.Get; +import com.google.api.services.dataflow.Dataflow.Projects.Jobs.GetMetrics; +import com.google.api.services.dataflow.Dataflow.Projects.Jobs.Messages; +import com.google.api.services.dataflow.model.Job; +import com.google.api.services.dataflow.model.JobMetrics; +import com.google.api.services.dataflow.model.MetricStructuredName; +import com.google.api.services.dataflow.model.MetricUpdate; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSetMultimap; +import java.io.IOException; +import java.math.BigDecimal; +import java.net.SocketTimeoutException; +import java.util.concurrent.TimeUnit; import org.apache.beam.runners.dataflow.internal.DataflowAggregatorTransforms; import org.apache.beam.runners.dataflow.testing.TestDataflowPipelineOptions; import org.apache.beam.runners.dataflow.util.MonitoringUtil; @@ -48,19 +63,6 @@ import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; - -import com.google.api.services.dataflow.Dataflow; -import com.google.api.services.dataflow.Dataflow.Projects.Jobs.Get; -import com.google.api.services.dataflow.Dataflow.Projects.Jobs.GetMetrics; -import com.google.api.services.dataflow.Dataflow.Projects.Jobs.Messages; -import com.google.api.services.dataflow.model.Job; -import com.google.api.services.dataflow.model.JobMetrics; -import com.google.api.services.dataflow.model.MetricStructuredName; -import com.google.api.services.dataflow.model.MetricUpdate; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSetMultimap; - import org.joda.time.Duration; import org.junit.Before; import org.junit.Rule; @@ -71,11 +73,6 @@ import org.mockito.Mock; import org.mockito.MockitoAnnotations; -import java.io.IOException; -import java.math.BigDecimal; -import java.net.SocketTimeoutException; -import java.util.concurrent.TimeUnit; - /** * Tests for DataflowPipelineJob. */ diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrarTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrarTest.java index 388a85ac50d7..f08475735e53 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrarTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrarTest.java @@ -20,20 +20,17 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import java.util.ServiceLoader; import org.apache.beam.runners.dataflow.options.BlockingDataflowPipelineOptions; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsRegistrar; import org.apache.beam.sdk.runners.PipelineRunnerRegistrar; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.ServiceLoader; - /** Tests for {@link DataflowPipelineRegistrar}. */ @RunWith(JUnit4.class) public class DataflowPipelineRegistrarTest { diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java index 2a01c0397ec1..2b7013dc9936 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java @@ -20,7 +20,6 @@ import static org.apache.beam.sdk.util.Structs.addObject; import static org.apache.beam.sdk.util.Structs.getDictionary; import static org.apache.beam.sdk.util.Structs.getString; - import static org.hamcrest.Matchers.allOf; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.hasKey; @@ -36,6 +35,23 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import com.google.api.services.dataflow.Dataflow; +import com.google.api.services.dataflow.model.DataflowPackage; +import com.google.api.services.dataflow.model.Job; +import com.google.api.services.dataflow.model.Step; +import com.google.api.services.dataflow.model.WorkerPool; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; +import java.io.IOException; +import java.io.Serializable; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; import org.apache.beam.runners.dataflow.DataflowPipelineTranslator.TranslationContext; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.runners.dataflow.options.DataflowPipelineWorkerPoolOptions; @@ -65,17 +81,6 @@ import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.TupleTag; - -import com.google.api.services.dataflow.Dataflow; -import com.google.api.services.dataflow.model.DataflowPackage; -import com.google.api.services.dataflow.model.Job; -import com.google.api.services.dataflow.model.Step; -import com.google.api.services.dataflow.model.WorkerPool; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Iterables; - import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -87,15 +92,6 @@ import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import java.io.IOException; -import java.io.Serializable; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; - /** * Tests for DataflowPipelineTranslator. */ diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java index 92a6bcb4c729..208e84c96678 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java @@ -18,7 +18,6 @@ package org.apache.beam.runners.dataflow; import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; - import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; @@ -38,6 +37,28 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import com.google.api.services.dataflow.Dataflow; +import com.google.api.services.dataflow.model.DataflowPackage; +import com.google.api.services.dataflow.model.Job; +import com.google.api.services.dataflow.model.ListJobsResponse; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import java.io.File; +import java.io.IOException; +import java.net.URL; +import java.net.URLClassLoader; +import java.nio.channels.FileChannel; +import java.nio.channels.SeekableByteChannel; +import java.nio.file.Files; +import java.nio.file.StandardOpenOption; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.regex.Pattern; import org.apache.beam.runners.dataflow.DataflowRunner.BatchViewAsList; import org.apache.beam.runners.dataflow.DataflowRunner.BatchViewAsMap; import org.apache.beam.runners.dataflow.DataflowRunner.BatchViewAsMultimap; @@ -84,15 +105,6 @@ import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; - -import com.google.api.services.dataflow.Dataflow; -import com.google.api.services.dataflow.model.DataflowPackage; -import com.google.api.services.dataflow.model.Job; -import com.google.api.services.dataflow.model.ListJobsResponse; -import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; - import org.hamcrest.Description; import org.hamcrest.Matchers; import org.hamcrest.TypeSafeMatcher; @@ -109,22 +121,6 @@ import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import java.io.File; -import java.io.IOException; -import java.net.URL; -import java.net.URLClassLoader; -import java.nio.channels.FileChannel; -import java.nio.channels.SeekableByteChannel; -import java.nio.file.Files; -import java.nio.file.StandardOpenOption; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.regex.Pattern; - /** * Tests for the {@link DataflowRunner}. */ diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/RecordingPipelineVisitor.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/RecordingPipelineVisitor.java index 8ebc4bd1f408..2090877f41ff 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/RecordingPipelineVisitor.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/RecordingPipelineVisitor.java @@ -17,14 +17,13 @@ */ package org.apache.beam.runners.dataflow; +import java.util.ArrayList; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.runners.TransformTreeNode; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.PValue; -import java.util.ArrayList; -import java.util.List; - /** * Provides a simple {@link org.apache.beam.sdk.Pipeline.PipelineVisitor} * that records the transformation tree. diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptionsTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptionsTest.java index 71b6b5721458..d376acf14b91 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptionsTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptionsTest.java @@ -22,7 +22,6 @@ import static org.junit.Assert.assertThat; import org.apache.beam.sdk.options.PipelineOptionsFactory; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptionsTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptionsTest.java index c0422ee83de6..61d39923ae5d 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptionsTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptionsTest.java @@ -26,7 +26,6 @@ import org.apache.beam.sdk.testing.RestoreSystemProperties; import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.util.NoopPathValidator; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowProfilingOptionsTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowProfilingOptionsTest.java index 58f851468460..87c74a4e37b9 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowProfilingOptionsTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowProfilingOptionsTest.java @@ -20,10 +20,8 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; -import org.apache.beam.sdk.options.PipelineOptionsFactory; - import com.fasterxml.jackson.databind.ObjectMapper; - +import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.hamcrest.Matchers; import org.junit.Test; import org.junit.runner.RunWith; diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowWorkerLoggingOptionsTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowWorkerLoggingOptionsTest.java index 5b12ad5f07ad..b463dcb86686 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowWorkerLoggingOptionsTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowWorkerLoggingOptionsTest.java @@ -18,15 +18,11 @@ package org.apache.beam.runners.dataflow.options; import static org.apache.beam.runners.dataflow.options.DataflowWorkerLoggingOptions.Level.WARN; - import static org.junit.Assert.assertEquals; -import org.apache.beam.runners.dataflow.options.DataflowWorkerLoggingOptions.WorkerLogLevelOverrides; - -import com.google.common.collect.ImmutableMap; - import com.fasterxml.jackson.databind.ObjectMapper; - +import com.google.common.collect.ImmutableMap; +import org.apache.beam.runners.dataflow.options.DataflowWorkerLoggingOptions.WorkerLogLevelOverrides; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunnerTest.java index b4bbd39c371f..24d5b2388539 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunnerTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunnerTest.java @@ -31,6 +31,22 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import com.google.api.client.http.LowLevelHttpResponse; +import com.google.api.client.json.Json; +import com.google.api.client.testing.http.MockHttpTransport; +import com.google.api.client.testing.http.MockLowLevelHttpRequest; +import com.google.api.client.testing.http.MockLowLevelHttpResponse; +import com.google.api.services.dataflow.Dataflow; +import com.google.api.services.dataflow.model.JobMessage; +import com.google.api.services.dataflow.model.JobMetrics; +import com.google.api.services.dataflow.model.MetricStructuredName; +import com.google.api.services.dataflow.model.MetricUpdate; +import com.google.common.base.Optional; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import java.io.IOException; +import java.math.BigDecimal; +import java.util.Arrays; import org.apache.beam.runners.dataflow.DataflowPipelineJob; import org.apache.beam.runners.dataflow.DataflowRunner; import org.apache.beam.runners.dataflow.util.MonitoringUtil; @@ -50,21 +66,6 @@ import org.apache.beam.sdk.util.TestCredential; import org.apache.beam.sdk.util.Transport; import org.apache.beam.sdk.values.PCollection; - -import com.google.api.client.http.LowLevelHttpResponse; -import com.google.api.client.json.Json; -import com.google.api.client.testing.http.MockHttpTransport; -import com.google.api.client.testing.http.MockLowLevelHttpRequest; -import com.google.api.client.testing.http.MockLowLevelHttpResponse; -import com.google.api.services.dataflow.Dataflow; -import com.google.api.services.dataflow.model.JobMessage; -import com.google.api.services.dataflow.model.JobMetrics; -import com.google.api.services.dataflow.model.MetricStructuredName; -import com.google.api.services.dataflow.model.MetricUpdate; -import com.google.common.base.Optional; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; - import org.hamcrest.BaseMatcher; import org.hamcrest.Description; import org.joda.time.Duration; @@ -81,10 +82,6 @@ import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import java.io.IOException; -import java.math.BigDecimal; -import java.util.Arrays; - /** Tests for {@link TestDataflowRunner}. */ @RunWith(JUnit4.class) public class TestDataflowRunnerTest { diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowGroupByKeyTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowGroupByKeyTest.java index b219ea2e3046..6d5a63ab44d7 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowGroupByKeyTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowGroupByKeyTest.java @@ -17,6 +17,8 @@ */ package org.apache.beam.runners.dataflow.transforms; +import java.util.Arrays; +import java.util.List; import org.apache.beam.runners.dataflow.DataflowRunner; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.sdk.Pipeline; @@ -35,7 +37,6 @@ import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptor; - import org.joda.time.Duration; import org.junit.Rule; import org.junit.Test; @@ -43,9 +44,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** Tests for {@link GroupByKey} for the {@link DataflowRunner}. */ @RunWith(JUnit4.class) public class DataflowGroupByKeyTest { diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowViewTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowViewTest.java index 95cbaae8ca74..2eddcf78c22f 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowViewTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowViewTest.java @@ -34,7 +34,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TypeDescriptor; - import org.hamcrest.Matchers; import org.joda.time.Duration; import org.junit.Rule; diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/MonitoringUtilTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/MonitoringUtilTest.java index da075150f7fc..98d8ab60e815 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/MonitoringUtilTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/MonitoringUtilTest.java @@ -22,17 +22,19 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import com.google.api.services.dataflow.Dataflow; +import com.google.api.services.dataflow.model.JobMessage; +import com.google.api.services.dataflow.model.ListJobMessagesResponse; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.runners.dataflow.util.MonitoringUtil.LoggingHandler; import org.apache.beam.sdk.PipelineResult.State; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.ExpectedLogs; import org.apache.beam.sdk.util.TestCredential; - -import com.google.api.services.dataflow.Dataflow; -import com.google.api.services.dataflow.model.JobMessage; -import com.google.api.services.dataflow.model.ListJobMessagesResponse; - import org.joda.time.DateTime; import org.joda.time.Instant; import org.joda.time.chrono.ISOChronology; @@ -42,11 +44,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - /** * Tests for MonitoringUtil. */ diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/PackageUtilTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/PackageUtilTest.java index 41ad05dc044b..02aceef34b5d 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/PackageUtilTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/PackageUtilTest.java @@ -32,15 +32,6 @@ import static org.mockito.Mockito.verifyNoMoreInteractions; import static org.mockito.Mockito.when; -import org.apache.beam.runners.dataflow.util.PackageUtil.PackageAttributes; -import org.apache.beam.sdk.options.GcsOptions; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.testing.ExpectedLogs; -import org.apache.beam.sdk.testing.FastNanoClockAndSleeper; -import org.apache.beam.sdk.util.GcsUtil; -import org.apache.beam.sdk.util.IOChannelUtils; -import org.apache.beam.sdk.util.gcsfs.GcsPath; - import com.google.api.client.googleapis.json.GoogleJsonError.ErrorInfo; import com.google.api.client.googleapis.json.GoogleJsonResponseException; import com.google.api.client.http.HttpRequest; @@ -62,19 +53,6 @@ import com.google.common.collect.Lists; import com.google.common.io.Files; import com.google.common.io.LineReader; - -import org.hamcrest.BaseMatcher; -import org.hamcrest.Description; -import org.hamcrest.Matchers; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; -import org.mockito.Mock; -import org.mockito.MockitoAnnotations; - import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; @@ -88,6 +66,25 @@ import java.util.regex.Pattern; import java.util.zip.ZipEntry; import java.util.zip.ZipInputStream; +import org.apache.beam.runners.dataflow.util.PackageUtil.PackageAttributes; +import org.apache.beam.sdk.options.GcsOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.testing.ExpectedLogs; +import org.apache.beam.sdk.testing.FastNanoClockAndSleeper; +import org.apache.beam.sdk.util.GcsUtil; +import org.apache.beam.sdk.util.IOChannelUtils; +import org.apache.beam.sdk.util.gcsfs.GcsPath; +import org.hamcrest.BaseMatcher; +import org.hamcrest.Description; +import org.hamcrest.Matchers; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; /** Tests for PackageUtil. */ @RunWith(JUnit4.class) diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/RandomAccessDataTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/RandomAccessDataTest.java index c5c77cfaab16..042e14578b78 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/RandomAccessDataTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/RandomAccessDataTest.java @@ -23,24 +23,21 @@ import static org.junit.Assert.assertSame; import static org.junit.Assert.assertTrue; +import com.google.common.primitives.UnsignedBytes; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.InputStream; +import java.util.Arrays; import org.apache.beam.runners.dataflow.util.RandomAccessData.RandomAccessDataCoder; import org.apache.beam.sdk.coders.Coder.Context; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.testing.CoderProperties; - -import com.google.common.primitives.UnsignedBytes; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.InputStream; -import java.util.Arrays; - /** * Tests for {@link RandomAccessData}. */ diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/TimeUtilTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/TimeUtilTest.java index c22c223e2747..e0785d424fe7 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/TimeUtilTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/TimeUtilTest.java @@ -21,7 +21,6 @@ import static org.apache.beam.runners.dataflow.util.TimeUtil.fromCloudTime; import static org.apache.beam.runners.dataflow.util.TimeUtil.toCloudDuration; import static org.apache.beam.runners.dataflow.util.TimeUtil.toCloudTime; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java index be50f70277d6..2ce1ff697e42 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java @@ -40,7 +40,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; - import org.apache.spark.SparkException; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.streaming.Duration; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerRegistrar.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerRegistrar.java index 7a317539e472..bedfda419f41 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerRegistrar.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerRegistrar.java @@ -18,14 +18,13 @@ package org.apache.beam.runners.spark; +import com.google.auto.service.AutoService; +import com.google.common.collect.ImmutableList; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsRegistrar; import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.runners.PipelineRunnerRegistrar; -import com.google.auto.service.AutoService; -import com.google.common.collect.ImmutableList; - /** * Contains the {@link PipelineRunnerRegistrar} and {@link PipelineOptionsRegistrar} for the * {@link SparkRunner}. diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java index 6ab6dc9a303f..c15e276c43db 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java @@ -18,19 +18,17 @@ package org.apache.beam.runners.spark.aggregators; -import org.apache.beam.runners.spark.translation.SparkRuntimeContext; -import org.apache.beam.sdk.coders.CannotProvideCoderException; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.transforms.Combine; - import com.google.common.collect.ImmutableList; - import java.io.IOException; import java.io.ObjectInputStream; import java.io.ObjectOutputStream; import java.io.Serializable; import java.util.Map; import java.util.TreeMap; +import org.apache.beam.runners.spark.translation.SparkRuntimeContext; +import org.apache.beam.sdk.coders.CannotProvideCoderException; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.transforms.Combine; /** * This class wraps a map of named aggregators. Spark expects that all accumulators be declared diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/CoderHelpers.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/CoderHelpers.java index 07587fcac76c..0df66c237a45 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/CoderHelpers.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/CoderHelpers.java @@ -18,20 +18,16 @@ package org.apache.beam.runners.spark.coders; -import org.apache.beam.runners.spark.util.ByteArray; -import org.apache.beam.sdk.coders.Coder; - import com.google.common.collect.Iterables; - -import org.apache.spark.api.java.function.Function; -import org.apache.spark.api.java.function.PairFunction; - import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.util.LinkedList; import java.util.List; - +import org.apache.beam.runners.spark.util.ByteArray; +import org.apache.beam.sdk.coders.Coder; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.PairFunction; import scala.Tuple2; /** diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/NullWritableCoder.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/NullWritableCoder.java index 7cff3251cad7..ebbab1a235b9 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/NullWritableCoder.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/NullWritableCoder.java @@ -18,13 +18,11 @@ package org.apache.beam.runners.spark.coders; -import org.apache.beam.sdk.coders.Coder; - import com.fasterxml.jackson.annotation.JsonCreator; -import org.apache.hadoop.io.NullWritable; - import java.io.InputStream; import java.io.OutputStream; +import org.apache.beam.sdk.coders.Coder; +import org.apache.hadoop.io.NullWritable; /** * Simple writable coder for Null. diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/WritableCoder.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/WritableCoder.java index 4719e46b23d1..b40e0229f282 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/WritableCoder.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/WritableCoder.java @@ -18,16 +18,8 @@ package org.apache.beam.runners.spark.coders; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.coders.StandardCoder; -import org.apache.beam.sdk.util.CloudObject; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.io.Writable; - import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; @@ -35,6 +27,12 @@ import java.io.OutputStream; import java.lang.reflect.InvocationTargetException; import java.util.List; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.coders.StandardCoder; +import org.apache.beam.sdk.util.CloudObject; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.Writable; /** * A {@code WritableCoder} is a {@link Coder} for a Java class that implements {@link Writable}. diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/KafkaIO.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/KafkaIO.java index 13171f3d619e..f57c114e69fd 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/KafkaIO.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/KafkaIO.java @@ -19,17 +19,15 @@ import static com.google.common.base.Preconditions.checkNotNull; +import java.util.Map; +import java.util.Set; +import kafka.serializer.Decoder; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PInput; -import java.util.Map; -import java.util.Set; - -import kafka.serializer.Decoder; - /** * Read stream from Kafka. */ diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/HadoopIO.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/HadoopIO.java index 1177a578dfe7..7b106105ee19 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/HadoopIO.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/HadoopIO.java @@ -20,6 +20,8 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; +import java.util.HashMap; +import java.util.Map; import org.apache.beam.sdk.io.ShardNameTemplate; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.WindowingStrategy; @@ -27,13 +29,9 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.PInput; - import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; -import java.util.HashMap; -import java.util.Map; - /** * Spark native HadoopIO. */ diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilder.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilder.java index 6b36427bc52c..11b4b5352879 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilder.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilder.java @@ -18,10 +18,9 @@ package org.apache.beam.runners.spark.io.hadoop; -import org.apache.hadoop.fs.Path; - import java.util.regex.Matcher; import java.util.regex.Pattern; +import org.apache.hadoop.fs.Path; /** * Shard name builder. diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateHelper.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateHelper.java index d06b0162b1e5..7f8e2978c439 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateHelper.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateHelper.java @@ -20,6 +20,7 @@ import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.replaceShardNumber; +import java.io.IOException; import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.TaskID; @@ -28,8 +29,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; - /** * Shard name template helper. */ diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedAvroKeyOutputFormat.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedAvroKeyOutputFormat.java index f747e7be3b6e..62a610ba57f2 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedAvroKeyOutputFormat.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedAvroKeyOutputFormat.java @@ -18,14 +18,13 @@ package org.apache.beam.runners.spark.io.hadoop; +import java.io.IOException; +import java.io.OutputStream; import org.apache.avro.mapreduce.AvroKeyOutputFormat; import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.TaskAttemptContext; -import java.io.IOException; -import java.io.OutputStream; - /** * Templated Avro key output format. */ diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedSequenceFileOutputFormat.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedSequenceFileOutputFormat.java index bd2ee4d20d72..ab1263b031f6 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedSequenceFileOutputFormat.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedSequenceFileOutputFormat.java @@ -18,13 +18,12 @@ package org.apache.beam.runners.spark.io.hadoop; +import java.io.IOException; import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat; -import java.io.IOException; - /** * Templated sequence file output format. */ diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedTextOutputFormat.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedTextOutputFormat.java index 8725a95c5e4e..5a6e9a938653 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedTextOutputFormat.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedTextOutputFormat.java @@ -18,13 +18,12 @@ package org.apache.beam.runners.spark.io.hadoop; +import java.io.IOException; import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat; -import java.io.IOException; - /** * Templates text output format. */ diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java index c08d1850d412..800d614e5e96 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java @@ -18,20 +18,18 @@ package org.apache.beam.runners.spark.translation; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; import org.apache.beam.runners.spark.util.BroadcastHelper; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; - import org.apache.spark.api.java.function.FlatMapFunction; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; - /** * Dataflow's Do functions correspond to Spark's FlatMap functions. * diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java index 4ccac0e1a0b2..724f54fd056e 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java @@ -20,6 +20,14 @@ import static com.google.common.base.Preconditions.checkArgument; +import com.google.common.base.Function; +import com.google.common.collect.Iterables; +import java.io.IOException; +import java.util.LinkedHashMap; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; import org.apache.beam.runners.spark.EvaluationResult; import org.apache.beam.runners.spark.coders.CoderHelpers; import org.apache.beam.sdk.AggregatorRetrievalException; @@ -38,21 +46,10 @@ import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.PValue; - -import com.google.common.base.Function; -import com.google.common.collect.Iterables; - import org.apache.spark.api.java.JavaRDDLike; import org.apache.spark.api.java.JavaSparkContext; import org.joda.time.Duration; -import java.io.IOException; -import java.util.LinkedHashMap; -import java.util.LinkedHashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - /** * Evaluation context allows us to define how pipeline instructions. diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java index abf0e8379a2e..4c44ffd04a9d 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java @@ -18,22 +18,18 @@ package org.apache.beam.runners.spark.translation; -import org.apache.beam.runners.spark.util.BroadcastHelper; -import org.apache.beam.sdk.transforms.OldDoFn; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.values.TupleTag; - import com.google.common.base.Function; import com.google.common.collect.Iterators; import com.google.common.collect.LinkedListMultimap; import com.google.common.collect.Multimap; - -import org.apache.spark.api.java.function.PairFlatMapFunction; -import org.joda.time.Instant; - import java.util.Iterator; import java.util.Map; - +import org.apache.beam.runners.spark.util.BroadcastHelper; +import org.apache.beam.sdk.transforms.OldDoFn; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.spark.api.java.function.PairFlatMapFunction; +import org.joda.time.Instant; import scala.Tuple2; /** diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java index 1cdbd92010d8..566a2729d055 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java @@ -18,6 +18,12 @@ package org.apache.beam.runners.spark.translation; +import com.google.common.collect.AbstractIterator; +import com.google.common.collect.Iterables; +import java.io.IOException; +import java.util.Collection; +import java.util.Iterator; +import java.util.Map; import org.apache.beam.runners.spark.util.BroadcastHelper; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; @@ -33,19 +39,10 @@ import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; - -import com.google.common.collect.AbstractIterator; -import com.google.common.collect.Iterables; - import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.Collection; -import java.util.Iterator; -import java.util.Map; - /** * Spark runner process context. */ diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java index c2edd023c7e1..4bc0c00973f8 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java @@ -18,6 +18,14 @@ package org.apache.beam.runners.spark.translation; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import java.io.IOException; +import java.io.Serializable; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; import org.apache.beam.runners.spark.aggregators.AggAccumParam; import org.apache.beam.runners.spark.aggregators.NamedAggregators; import org.apache.beam.sdk.AggregatorValues; @@ -32,20 +40,9 @@ import org.apache.beam.sdk.transforms.Min; import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.values.TypeDescriptor; - -import com.google.common.collect.ImmutableList; - -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.spark.Accumulator; import org.apache.spark.api.java.JavaSparkContext; -import java.io.IOException; -import java.io.Serializable; -import java.util.Collection; -import java.util.HashMap; -import java.util.Map; - /** * The SparkRuntimeContext allows us to define useful features on the client side before our diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformEvaluator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformEvaluator.java index c5c7128feb52..fbfa84d76b50 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformEvaluator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformEvaluator.java @@ -18,9 +18,8 @@ package org.apache.beam.runners.spark.translation; -import org.apache.beam.sdk.transforms.PTransform; - import java.io.Serializable; +import org.apache.beam.sdk.transforms.PTransform; /** * Describe a {@link PTransform} evaluator. diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java index c51a50037415..08e3fda7a36b 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java @@ -24,6 +24,19 @@ import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.getOutputFileTemplate; import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.replaceShardCount; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import java.io.IOException; +import java.io.Serializable; +import java.lang.reflect.Field; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import org.apache.avro.mapred.AvroKey; +import org.apache.avro.mapreduce.AvroJob; +import org.apache.avro.mapreduce.AvroKeyInputFormat; import org.apache.beam.runners.spark.coders.CoderHelpers; import org.apache.beam.runners.spark.io.hadoop.HadoopIO; import org.apache.beam.runners.spark.io.hadoop.ShardNameTemplateHelper; @@ -65,14 +78,6 @@ import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; - -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; - -import org.apache.avro.mapred.AvroKey; -import org.apache.avro.mapreduce.AvroJob; -import org.apache.avro.mapreduce.AvroKeyInputFormat; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.Text; @@ -86,15 +91,6 @@ import org.apache.spark.api.java.function.Function2; import org.apache.spark.api.java.function.PairFlatMapFunction; import org.apache.spark.api.java.function.PairFunction; - -import java.io.IOException; -import java.io.Serializable; -import java.lang.reflect.Field; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Map; - import scala.Tuple2; /** diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingEvaluationContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingEvaluationContext.java index afcca932e9fa..2e4da4467723 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingEvaluationContext.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingEvaluationContext.java @@ -18,6 +18,12 @@ package org.apache.beam.runners.spark.translation.streaming; +import java.util.LinkedHashMap; +import java.util.LinkedHashSet; +import java.util.Map; +import java.util.Queue; +import java.util.Set; +import java.util.concurrent.LinkedBlockingQueue; import org.apache.beam.runners.spark.translation.EvaluationContext; import org.apache.beam.runners.spark.translation.SparkRuntimeContext; import org.apache.beam.sdk.Pipeline; @@ -28,7 +34,6 @@ import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.PValue; - import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaRDDLike; import org.apache.spark.api.java.JavaSparkContext; @@ -37,13 +42,6 @@ import org.apache.spark.streaming.api.java.JavaDStreamLike; import org.apache.spark.streaming.api.java.JavaStreamingContext; -import java.util.LinkedHashMap; -import java.util.LinkedHashSet; -import java.util.Map; -import java.util.Queue; -import java.util.Set; -import java.util.concurrent.LinkedBlockingQueue; - /** * Streaming evaluation context helps to handle streaming. diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java index 2ce2c298bc4e..43160f550066 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java @@ -17,6 +17,17 @@ */ package org.apache.beam.runners.spark.translation.streaming; +import com.google.api.client.util.Lists; +import com.google.api.client.util.Maps; +import com.google.api.client.util.Sets; +import com.google.common.reflect.TypeToken; +import java.lang.reflect.ParameterizedType; +import java.lang.reflect.Type; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; +import kafka.serializer.Decoder; import org.apache.beam.runners.spark.io.ConsoleIO; import org.apache.beam.runners.spark.io.CreateStream; import org.apache.beam.runners.spark.io.KafkaIO; @@ -43,12 +54,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.PDone; - -import com.google.api.client.util.Lists; -import com.google.api.client.util.Maps; -import com.google.api.client.util.Sets; -import com.google.common.reflect.TypeToken; - import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.function.Function; import org.apache.spark.api.java.function.VoidFunction; @@ -59,15 +64,6 @@ import org.apache.spark.streaming.api.java.JavaPairInputDStream; import org.apache.spark.streaming.api.java.JavaStreamingContext; import org.apache.spark.streaming.kafka.KafkaUtils; - -import java.lang.reflect.ParameterizedType; -import java.lang.reflect.Type; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import kafka.serializer.Decoder; import scala.Tuple2; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/BroadcastHelper.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/BroadcastHelper.java index d3fa05a75142..29c2dd995b6e 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/BroadcastHelper.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/BroadcastHelper.java @@ -18,18 +18,16 @@ package org.apache.beam.runners.spark.util; +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.Serializable; import org.apache.beam.runners.spark.coders.CoderHelpers; import org.apache.beam.sdk.coders.Coder; - import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.broadcast.Broadcast; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.ByteArrayInputStream; -import java.io.IOException; -import java.io.Serializable; - /** * Broadcast helper. */ diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/ByteArray.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/ByteArray.java index 8c493f5bc2b9..bb0c43912c97 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/ByteArray.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/ByteArray.java @@ -18,7 +18,6 @@ package org.apache.beam.runners.spark.util; import com.google.common.primitives.UnsignedBytes; - import java.io.Serializable; import java.util.Arrays; diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/DeDupTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/DeDupTest.java index 9a167446847e..2d06a207a371 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/DeDupTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/DeDupTest.java @@ -18,6 +18,10 @@ package org.apache.beam.runners.spark; +import com.google.common.collect.ImmutableSet; +import java.util.Arrays; +import java.util.List; +import java.util.Set; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.options.PipelineOptionsFactory; @@ -25,15 +29,8 @@ import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.RemoveDuplicates; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.ImmutableSet; - import org.junit.Test; -import java.util.Arrays; -import java.util.List; -import java.util.Set; - /** * A test based on {@code DeDupExample} from the SDK. */ diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/EmptyInputTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/EmptyInputTest.java index c2e331f78364..90ce4278bdf6 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/EmptyInputTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/EmptyInputTest.java @@ -20,6 +20,9 @@ import static org.junit.Assert.assertEquals; +import com.google.common.collect.Iterables; +import java.util.Collections; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.options.PipelineOptionsFactory; @@ -27,14 +30,8 @@ import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.Iterables; - import org.junit.Test; -import java.util.Collections; -import java.util.List; - /** * Empty input test. */ diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/SimpleWordCountTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/SimpleWordCountTest.java index 441d92d2af65..f644765682f1 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/SimpleWordCountTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/SimpleWordCountTest.java @@ -21,6 +21,12 @@ import static org.hamcrest.Matchers.containsInAnyOrder; import static org.junit.Assert.assertThat; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Sets; +import java.io.File; +import java.util.Arrays; +import java.util.List; +import java.util.Set; import org.apache.beam.runners.spark.examples.WordCount; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.StringUtf8Coder; @@ -30,20 +36,11 @@ import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Sets; - import org.apache.commons.io.FileUtils; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import java.io.File; -import java.util.Arrays; -import java.util.List; -import java.util.Set; - /** * Simple word count test. */ diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/SparkRunnerRegistrarTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/SparkRunnerRegistrarTest.java index 3ca9df4ad839..4e1fd7c5afee 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/SparkRunnerRegistrarTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/SparkRunnerRegistrarTest.java @@ -21,18 +21,15 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; -import org.apache.beam.sdk.options.PipelineOptionsRegistrar; -import org.apache.beam.sdk.runners.PipelineRunnerRegistrar; - import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; - +import java.util.ServiceLoader; +import org.apache.beam.sdk.options.PipelineOptionsRegistrar; +import org.apache.beam.sdk.runners.PipelineRunnerRegistrar; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.ServiceLoader; - /** * Test {@link SparkRunnerRegistrar}. */ diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/TfIdfTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/TfIdfTest.java index 17bf6ddbe1b1..d911bfb965b0 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/TfIdfTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/TfIdfTest.java @@ -18,6 +18,8 @@ package org.apache.beam.runners.spark; +import java.net.URI; +import java.util.Arrays; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.StringDelegateCoder; import org.apache.beam.sdk.options.PipelineOptionsFactory; @@ -36,16 +38,12 @@ import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.net.URI; -import java.util.Arrays; - /** * A test based on {@code TfIdf} from the SDK. */ diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/coders/WritableCoderTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/coders/WritableCoderTest.java index f2bd4d3e76fa..538fd97a8bb0 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/coders/WritableCoderTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/coders/WritableCoderTest.java @@ -19,7 +19,6 @@ package org.apache.beam.runners.spark.coders; import org.apache.beam.sdk.testing.CoderProperties; - import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.NullWritable; import org.junit.Test; diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/AvroPipelineTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/AvroPipelineTest.java index d86242402c7c..413a71c1d7b2 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/AvroPipelineTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/AvroPipelineTest.java @@ -20,17 +20,12 @@ import static org.junit.Assert.assertEquals; -import org.apache.beam.runners.spark.EvaluationResult; -import org.apache.beam.runners.spark.SparkRunner; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.io.AvroIO; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.values.PCollection; - import com.google.common.collect.Lists; import com.google.common.io.Resources; - +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.util.List; import org.apache.avro.Schema; import org.apache.avro.file.DataFileReader; import org.apache.avro.file.DataFileWriter; @@ -38,16 +33,18 @@ import org.apache.avro.generic.GenericDatumReader; import org.apache.avro.generic.GenericDatumWriter; import org.apache.avro.generic.GenericRecord; +import org.apache.beam.runners.spark.EvaluationResult; +import org.apache.beam.runners.spark.SparkRunner; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.io.AvroIO; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.values.PCollection; import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.util.List; - /** * Avro pipeline test. */ diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/NumShardsTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/NumShardsTest.java index 9c65917ece38..81803c338b91 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/NumShardsTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/NumShardsTest.java @@ -21,6 +21,15 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import com.google.common.base.Charsets; +import com.google.common.collect.Sets; +import com.google.common.io.Files; +import java.io.File; +import java.io.FileFilter; +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Set; import org.apache.beam.runners.spark.EvaluationResult; import org.apache.beam.runners.spark.SparkPipelineOptions; import org.apache.beam.runners.spark.SparkRunner; @@ -32,23 +41,11 @@ import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.base.Charsets; -import com.google.common.collect.Sets; -import com.google.common.io.Files; - import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import java.io.File; -import java.io.FileFilter; -import java.io.IOException; -import java.util.Arrays; -import java.util.List; -import java.util.Set; - /** * Number of shards test. */ diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/HadoopFileFormatPipelineTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/HadoopFileFormatPipelineTest.java index 01aa839bde2e..b14465d7c9b3 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/HadoopFileFormatPipelineTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/HadoopFileFormatPipelineTest.java @@ -20,6 +20,8 @@ import static org.junit.Assert.assertEquals; +import java.io.File; +import java.io.IOException; import org.apache.beam.runners.spark.EvaluationResult; import org.apache.beam.runners.spark.SparkRunner; import org.apache.beam.runners.spark.coders.WritableCoder; @@ -29,7 +31,6 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.IntWritable; @@ -45,9 +46,6 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; -import java.io.File; -import java.io.IOException; - /** * Pipeline on the Hadoop file format test. */ diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilderTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilderTest.java index e1620dbc7c97..1f2cf639ba8b 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilderTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilderTest.java @@ -23,7 +23,6 @@ import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.getOutputFileTemplate; import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.replaceShardCount; import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.replaceShardNumber; - import static org.junit.Assert.assertEquals; import org.junit.Test; diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/CombineGloballyTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/CombineGloballyTest.java index e4ef7d705212..8022d064c168 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/CombineGloballyTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/CombineGloballyTest.java @@ -20,6 +20,9 @@ import static org.junit.Assert.assertEquals; +import com.google.common.collect.Iterables; +import java.util.Arrays; +import java.util.List; import org.apache.beam.runners.spark.EvaluationResult; import org.apache.beam.runners.spark.SparkPipelineOptions; import org.apache.beam.runners.spark.SparkRunner; @@ -29,14 +32,8 @@ import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.Iterables; - import org.junit.Test; -import java.util.Arrays; -import java.util.List; - /** * Combine globally test. */ diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/CombinePerKeyTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/CombinePerKeyTest.java index cdf2cfbb892f..281144f362f9 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/CombinePerKeyTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/CombinePerKeyTest.java @@ -18,6 +18,10 @@ package org.apache.beam.runners.spark.translation; +import com.google.common.collect.ImmutableList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import org.apache.beam.runners.spark.EvaluationResult; import org.apache.beam.runners.spark.SparkRunner; import org.apache.beam.sdk.Pipeline; @@ -33,16 +37,9 @@ import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.ImmutableList; - import org.junit.Assert; import org.junit.Test; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - /** * Combine per key function test. */ diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/DoFnOutputTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/DoFnOutputTest.java index e4b25bb06f74..31e0dd8d4ff0 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/DoFnOutputTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/DoFnOutputTest.java @@ -18,6 +18,7 @@ package org.apache.beam.runners.spark.translation; +import java.io.Serializable; import org.apache.beam.runners.spark.EvaluationResult; import org.apache.beam.runners.spark.SparkPipelineOptions; import org.apache.beam.runners.spark.SparkRunner; @@ -28,11 +29,8 @@ import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; - import org.junit.Test; -import java.io.Serializable; - /** * DoFN output test. */ diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/MultiOutputWordCountTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/MultiOutputWordCountTest.java index 0d0c0b41efd8..517596aab782 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/MultiOutputWordCountTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/MultiOutputWordCountTest.java @@ -18,6 +18,9 @@ package org.apache.beam.runners.spark.translation; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; +import java.util.Set; import org.apache.beam.runners.spark.EvaluationResult; import org.apache.beam.runners.spark.SparkRunner; import org.apache.beam.sdk.AggregatorValues; @@ -44,15 +47,9 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; - -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Iterables; - import org.junit.Assert; import org.junit.Test; -import java.util.Set; - /** * Multi-output word count test. */ diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SerializationTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SerializationTest.java index 019b107e0ddd..0e9121ca87f9 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SerializationTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SerializationTest.java @@ -18,6 +18,16 @@ package org.apache.beam.runners.spark.translation; +import com.google.common.base.Function; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Lists; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Arrays; +import java.util.List; +import java.util.Set; +import java.util.regex.Pattern; import org.apache.beam.runners.spark.EvaluationResult; import org.apache.beam.runners.spark.SparkPipelineOptions; import org.apache.beam.runners.spark.SparkRunner; @@ -36,21 +46,8 @@ import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.base.Function; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Lists; - import org.junit.Test; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.Arrays; -import java.util.List; -import java.util.Set; -import java.util.regex.Pattern; - /** * Serialization test. */ diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SideEffectsTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SideEffectsTest.java index 6cefa49ac04b..0d15d12d3f3b 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SideEffectsTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SideEffectsTest.java @@ -22,6 +22,8 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import java.io.Serializable; +import java.net.URI; import org.apache.beam.runners.spark.SparkPipelineOptions; import org.apache.beam.runners.spark.SparkRunner; import org.apache.beam.sdk.Pipeline; @@ -30,14 +32,10 @@ import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.ParDo; - import org.junit.After; import org.junit.Before; import org.junit.Test; -import java.io.Serializable; -import java.net.URI; - /** * Side effects test. */ diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkPipelineOptionsTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkPipelineOptionsTest.java index 59888c253ef5..2b655cbc8d74 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkPipelineOptionsTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkPipelineOptionsTest.java @@ -20,7 +20,6 @@ import org.apache.beam.runners.spark.SparkPipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; - import org.junit.Assert; import org.junit.Test; diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java index e1789f145e54..f61ad1c9a9fd 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java @@ -21,6 +21,15 @@ import static org.hamcrest.Matchers.containsInAnyOrder; import static org.junit.Assert.assertThat; +import com.google.common.base.Charsets; +import com.google.common.io.Files; +import java.io.File; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.List; +import java.util.regex.Pattern; import org.apache.beam.runners.spark.SparkRunner; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.TextIO; @@ -28,10 +37,6 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.base.Charsets; -import com.google.common.io.Files; - import org.apache.commons.io.FileUtils; import org.junit.Rule; import org.junit.Test; @@ -39,14 +44,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.File; -import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.nio.file.Paths; -import java.util.ArrayList; -import java.util.List; -import java.util.regex.Pattern; - /** * A test for the transforms registered in TransformTranslator. * Builds a regular Beam pipeline with each of the mapped diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/WindowedWordCountTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/WindowedWordCountTest.java index b70e090ddae7..69b2943e6f2d 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/WindowedWordCountTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/WindowedWordCountTest.java @@ -18,6 +18,9 @@ package org.apache.beam.runners.spark.translation; +import com.google.common.collect.ImmutableList; +import java.util.Arrays; +import java.util.List; import org.apache.beam.runners.spark.EvaluationResult; import org.apache.beam.runners.spark.SparkRunner; import org.apache.beam.runners.spark.examples.WordCount; @@ -32,15 +35,9 @@ import org.apache.beam.sdk.transforms.windowing.SlidingWindows; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.ImmutableList; - import org.joda.time.Duration; import org.junit.Test; -import java.util.Arrays; -import java.util.List; - /** * Windowed word count test. */ diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/FlattenStreamingTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/FlattenStreamingTest.java index 6f4d8fbf2d08..a6fe755c43de 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/FlattenStreamingTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/FlattenStreamingTest.java @@ -17,6 +17,9 @@ */ package org.apache.beam.runners.spark.translation.streaming; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; import org.apache.beam.runners.spark.EvaluationResult; import org.apache.beam.runners.spark.SparkPipelineOptions; import org.apache.beam.runners.spark.SparkRunner; @@ -30,14 +33,9 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; - import org.joda.time.Duration; import org.junit.Test; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - /** * Test Flatten (union) implementation for streaming. */ diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/KafkaStreamingTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/KafkaStreamingTest.java index 25271522aed5..27d6f5ea6068 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/KafkaStreamingTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/KafkaStreamingTest.java @@ -17,6 +17,12 @@ */ package org.apache.beam.runners.spark.translation.streaming; +import com.google.common.collect.ImmutableMap; +import java.io.IOException; +import java.util.Collections; +import java.util.Map; +import java.util.Properties; +import kafka.serializer.StringDecoder; import org.apache.beam.runners.spark.EvaluationResult; import org.apache.beam.runners.spark.SparkPipelineOptions; import org.apache.beam.runners.spark.SparkRunner; @@ -33,9 +39,6 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.ImmutableMap; - import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.serialization.Serializer; @@ -45,13 +48,6 @@ import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; - -import java.io.IOException; -import java.util.Collections; -import java.util.Map; -import java.util.Properties; - -import kafka.serializer.StringDecoder; /** * Test Kafka as input. */ diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/SimpleStreamingWordCountTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/SimpleStreamingWordCountTest.java index c761faed0e2e..671d227d3ac9 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/SimpleStreamingWordCountTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/SimpleStreamingWordCountTest.java @@ -18,6 +18,10 @@ package org.apache.beam.runners.spark.translation.streaming; +import java.io.Serializable; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; import org.apache.beam.runners.spark.EvaluationResult; import org.apache.beam.runners.spark.SparkPipelineOptions; import org.apache.beam.runners.spark.SparkRunner; @@ -31,16 +35,10 @@ import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.PCollection; - import org.apache.spark.streaming.Durations; import org.joda.time.Duration; import org.junit.Test; -import java.io.Serializable; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - /** * Simple word count streaming test. */ diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/EmbeddedKafkaCluster.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/EmbeddedKafkaCluster.java index cd326ef03355..d1729a4ca0a6 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/EmbeddedKafkaCluster.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/EmbeddedKafkaCluster.java @@ -17,12 +17,6 @@ */ package org.apache.beam.runners.spark.translation.streaming.utils; -import org.apache.zookeeper.server.NIOServerCnxnFactory; -import org.apache.zookeeper.server.ServerCnxnFactory; -import org.apache.zookeeper.server.ZooKeeperServer; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; @@ -33,10 +27,14 @@ import java.util.List; import java.util.Properties; import java.util.Random; - import kafka.server.KafkaConfig; import kafka.server.KafkaServer; import kafka.utils.Time; +import org.apache.zookeeper.server.NIOServerCnxnFactory; +import org.apache.zookeeper.server.ServerCnxnFactory; +import org.apache.zookeeper.server.ZooKeeperServer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Embedded Kafka cluster. diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/PAssertStreaming.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/PAssertStreaming.java index f85c4409df09..97425bd32c8b 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/PAssertStreaming.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/PAssertStreaming.java @@ -20,6 +20,7 @@ import static org.hamcrest.Matchers.containsInAnyOrder; import static org.junit.Assert.assertThat; +import java.io.Serializable; import org.apache.beam.runners.spark.EvaluationResult; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.MapElements; @@ -27,11 +28,8 @@ import org.apache.beam.sdk.transforms.Values; import org.apache.beam.sdk.transforms.WithKeys; import org.apache.beam.sdk.values.PCollection; - import org.junit.Assert; -import java.io.Serializable; - /** * Since PAssert doesn't propagate assert exceptions, use Aggregators to assert streaming * success/failure counters. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/AggregatorPipelineExtractor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/AggregatorPipelineExtractor.java index ac215c9ed1f0..0e79abe9683e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/AggregatorPipelineExtractor.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/AggregatorPipelineExtractor.java @@ -17,6 +17,11 @@ */ package org.apache.beam.sdk; +import com.google.common.collect.HashMultimap; +import com.google.common.collect.SetMultimap; +import java.util.Collection; +import java.util.Collections; +import java.util.Map; import org.apache.beam.sdk.Pipeline.PipelineVisitor; import org.apache.beam.sdk.runners.TransformTreeNode; import org.apache.beam.sdk.transforms.Aggregator; @@ -25,13 +30,6 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PValue; -import com.google.common.collect.HashMultimap; -import com.google.common.collect.SetMultimap; - -import java.util.Collection; -import java.util.Collections; -import java.util.Map; - /** * Retrieves {@link Aggregator Aggregators} at each {@link ParDo} and returns a {@link Map} of * {@link Aggregator} to the {@link PTransform PTransforms} in which it is present. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/AggregatorValues.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/AggregatorValues.java index efaad85b67b5..6297085319e2 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/AggregatorValues.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/AggregatorValues.java @@ -17,13 +17,12 @@ */ package org.apache.beam.sdk; +import java.util.Collection; +import java.util.Map; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.OldDoFn; -import java.util.Collection; -import java.util.Map; - /** * A collection of values associated with an {@link Aggregator}. Aggregators declared in a * {@link OldDoFn} are emitted on a per-{@code OldDoFn}-application basis. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java index 1bbc56f1fad2..53f46f6f06ab 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java @@ -19,6 +19,15 @@ import static com.google.common.base.Preconditions.checkState; +import com.google.common.collect.HashMultimap; +import com.google.common.collect.Iterables; +import com.google.common.collect.Multimap; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.options.PipelineOptions; @@ -36,21 +45,9 @@ import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.PValue; - -import com.google.common.collect.HashMultimap; -import com.google.common.collect.Iterables; -import com.google.common.collect.Multimap; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - /** * A {@link Pipeline} manages a directed acyclic graph of {@link PTransform PTransforms}, and the * {@link PCollection PCollections} that the {@link PTransform}s consume and produce. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/PipelineResult.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/PipelineResult.java index edfc9248f527..d9cdc16a3da4 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/PipelineResult.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/PipelineResult.java @@ -17,12 +17,10 @@ */ package org.apache.beam.sdk; +import java.io.IOException; import org.apache.beam.sdk.transforms.Aggregator; - import org.joda.time.Duration; -import java.io.IOException; - /** * Result of {@link Pipeline#run()}. */ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AvroCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AvroCoder.java index da4db93e1953..7894d14ccd1d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AvroCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AvroCoder.java @@ -19,11 +19,22 @@ import static org.apache.beam.sdk.util.Structs.addString; -import org.apache.beam.sdk.util.CloudObject; -import org.apache.beam.sdk.values.TypeDescriptor; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.lang.reflect.Field; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.SortedMap; +import java.util.SortedSet; +import javax.annotation.Nullable; import org.apache.avro.Schema; import org.apache.avro.generic.GenericDatumReader; import org.apache.avro.generic.GenericDatumWriter; @@ -45,22 +56,8 @@ import org.apache.avro.specific.SpecificData; import org.apache.avro.util.ClassUtils; import org.apache.avro.util.Utf8; - -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.Serializable; -import java.lang.reflect.Field; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.SortedMap; -import java.util.SortedSet; - -import javax.annotation.Nullable; +import org.apache.beam.sdk.util.CloudObject; +import org.apache.beam.sdk.values.TypeDescriptor; /** * A {@link Coder} using Avro binary format. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigDecimalCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigDecimalCoder.java index c4b7cd115510..e2628821c96d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigDecimalCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigDecimalCoder.java @@ -20,7 +20,6 @@ import static com.google.common.base.Preconditions.checkNotNull; import com.fasterxml.jackson.annotation.JsonCreator; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianIntegerCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianIntegerCoder.java index bd3bcf3800aa..ac8db120a022 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianIntegerCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianIntegerCoder.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.coders; import com.fasterxml.jackson.annotation.JsonCreator; - import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.EOFException; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianLongCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianLongCoder.java index 5f5793ed7723..e0053241f8d0 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianLongCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianLongCoder.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.coders; import com.fasterxml.jackson.annotation.JsonCreator; - import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.EOFException; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigIntegerCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigIntegerCoder.java index a41defed15b2..daba983a83b5 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigIntegerCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigIntegerCoder.java @@ -20,7 +20,6 @@ import static com.google.common.base.Preconditions.checkNotNull; import com.fasterxml.jackson.annotation.JsonCreator; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteArrayCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteArrayCoder.java index 4115825bf3d5..65e24da8c220 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteArrayCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteArrayCoder.java @@ -17,17 +17,14 @@ */ package org.apache.beam.sdk.coders; -import org.apache.beam.sdk.util.ExposedByteArrayOutputStream; -import org.apache.beam.sdk.util.StreamUtils; -import org.apache.beam.sdk.util.VarInt; - -import com.google.common.io.ByteStreams; - import com.fasterxml.jackson.annotation.JsonCreator; - +import com.google.common.io.ByteStreams; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import org.apache.beam.sdk.util.ExposedByteArrayOutputStream; +import org.apache.beam.sdk.util.StreamUtils; +import org.apache.beam.sdk.util.VarInt; /** * A {@link Coder} for {@code byte[]}. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteCoder.java index e146945c8e16..c912b355b149 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteCoder.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.coders; import com.fasterxml.jackson.annotation.JsonCreator; - import java.io.EOFException; import java.io.IOException; import java.io.InputStream; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteStringCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteStringCoder.java index b03f98a3e174..c70b9db978e5 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteStringCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteStringCoder.java @@ -17,16 +17,13 @@ */ package org.apache.beam.sdk.coders; -import org.apache.beam.sdk.util.VarInt; - +import com.fasterxml.jackson.annotation.JsonCreator; import com.google.common.io.ByteStreams; import com.google.protobuf.ByteString; - -import com.fasterxml.jackson.annotation.JsonCreator; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import org.apache.beam.sdk.util.VarInt; /** * A {@link Coder} for {@link ByteString} objects based on their encoded Protocol Buffer form. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java index 136c00a21796..688d1f793ab1 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java @@ -19,15 +19,9 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.annotations.Experimental; -import org.apache.beam.sdk.annotations.Experimental.Kind; -import org.apache.beam.sdk.util.CloudObject; -import org.apache.beam.sdk.util.common.ElementByteSizeObserver; - import com.google.common.base.Joiner; import com.google.common.base.MoreObjects; import com.google.common.base.Objects; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -35,8 +29,11 @@ import java.util.Arrays; import java.util.Collection; import java.util.List; - import javax.annotation.Nullable; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.annotations.Experimental.Kind; +import org.apache.beam.sdk.util.CloudObject; +import org.apache.beam.sdk.util.common.ElementByteSizeObserver; /** * A {@link Coder Coder<T>} defines how to encode and decode values of type {@code T} into diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderFactories.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderFactories.java index 480f424182d8..e1a202a26467 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderFactories.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderFactories.java @@ -17,8 +17,6 @@ */ package org.apache.beam.sdk.coders; -import org.apache.beam.sdk.values.TypeDescriptor; - import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.lang.reflect.Modifier; @@ -26,6 +24,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import org.apache.beam.sdk.values.TypeDescriptor; /** * Static utility methods for creating and working with {@link Coder}s. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderProviders.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderProviders.java index 3b852e2ab1f0..c072008d5f28 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderProviders.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderProviders.java @@ -19,15 +19,13 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.util.InstanceBuilder; -import org.apache.beam.sdk.values.TypeDescriptor; - import com.google.common.base.Joiner; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; - import java.lang.reflect.InvocationTargetException; import java.util.List; +import org.apache.beam.sdk.util.InstanceBuilder; +import org.apache.beam.sdk.values.TypeDescriptor; /** * Static utility methods for working with {@link CoderProvider CoderProviders}. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java index 411ebe2bbb58..9110de058af6 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java @@ -19,24 +19,11 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.coders.CannotProvideCoderException.ReasonCode; -import org.apache.beam.sdk.coders.protobuf.ProtoCoder; -import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.TimestampedValue; -import org.apache.beam.sdk.values.TypeDescriptor; - import com.google.api.services.bigquery.model.TableRow; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import com.google.protobuf.ByteString; - -import org.joda.time.Instant; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.lang.reflect.ParameterizedType; import java.lang.reflect.Type; import java.lang.reflect.TypeVariable; @@ -49,8 +36,17 @@ import java.util.List; import java.util.Map; import java.util.Set; - import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.CannotProvideCoderException.ReasonCode; +import org.apache.beam.sdk.coders.protobuf.ProtoCoder; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.util.CoderUtils; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * A {@link CoderRegistry} allows registering the default {@link Coder} to use for a Java class, diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CollectionCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CollectionCoder.java index f40d5f029ffb..7c61e882c35b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CollectionCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CollectionCoder.java @@ -19,13 +19,11 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.util.PropertyNames; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; - import java.util.Collection; import java.util.List; +import org.apache.beam.sdk.util.PropertyNames; /** * A {@link CollectionCoder} encodes {@link Collection Collections} in the format diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CustomCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CustomCoder.java index 69412e5849aa..2614cc178c02 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CustomCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CustomCoder.java @@ -17,23 +17,19 @@ */ package org.apache.beam.sdk.coders; +import static com.google.common.base.Preconditions.checkNotNull; import static org.apache.beam.sdk.util.Structs.addString; import static org.apache.beam.sdk.util.Structs.addStringList; -import static com.google.common.base.Preconditions.checkNotNull; - -import org.apache.beam.sdk.util.CloudObject; -import org.apache.beam.sdk.util.PropertyNames; -import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.sdk.util.StringUtils; - -import com.google.common.collect.Lists; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; - +import com.google.common.collect.Lists; import java.io.Serializable; import java.util.Collection; +import org.apache.beam.sdk.util.CloudObject; +import org.apache.beam.sdk.util.PropertyNames; +import org.apache.beam.sdk.util.SerializableUtils; +import org.apache.beam.sdk.util.StringUtils; /** * An abstract base class for writing a {@link Coder} class that encodes itself via Java diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DefaultCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DefaultCoder.java index 3eb1253c54cb..9a976f9d4cc4 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DefaultCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DefaultCoder.java @@ -17,13 +17,12 @@ */ package org.apache.beam.sdk.coders; -import org.apache.beam.sdk.values.PCollection; - import java.lang.annotation.Documented; import java.lang.annotation.ElementType; import java.lang.annotation.Retention; import java.lang.annotation.RetentionPolicy; import java.lang.annotation.Target; +import org.apache.beam.sdk.values.PCollection; /** * The {@link DefaultCoder} annotation diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DelegateCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DelegateCoder.java index 385c14996367..e7f9d4ac11f1 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DelegateCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DelegateCoder.java @@ -20,7 +20,6 @@ import com.google.common.base.MoreObjects; import com.google.common.base.Objects; import com.google.common.collect.Lists; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DoubleCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DoubleCoder.java index cb204ecb71ac..4e56914a18ad 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DoubleCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DoubleCoder.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.coders; import com.fasterxml.jackson.annotation.JsonCreator; - import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.EOFException; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DurationCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DurationCoder.java index c34ce66eb019..a2458f249cf3 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DurationCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DurationCoder.java @@ -17,15 +17,13 @@ */ package org.apache.beam.sdk.coders; -import org.apache.beam.sdk.util.common.ElementByteSizeObserver; - import com.fasterxml.jackson.annotation.JsonCreator; -import org.joda.time.Duration; -import org.joda.time.ReadableDuration; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import org.apache.beam.sdk.util.common.ElementByteSizeObserver; +import org.joda.time.Duration; +import org.joda.time.ReadableDuration; /** * A {@link Coder} that encodes a joda {@link Duration} as a {@link Long} using the format of diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/InstantCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/InstantCoder.java index d41bd1f21bd0..c0409fba46cc 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/InstantCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/InstantCoder.java @@ -17,18 +17,14 @@ */ package org.apache.beam.sdk.coders; -import org.apache.beam.sdk.util.common.ElementByteSizeObserver; - -import com.google.common.base.Converter; - import com.fasterxml.jackson.annotation.JsonCreator; -import org.joda.time.Instant; - +import com.google.common.base.Converter; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; - -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import org.apache.beam.sdk.util.common.ElementByteSizeObserver; +import org.joda.time.Instant; /** * A {@link Coder} for joda {@link Instant} that encodes it as a big endian {@link Long} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/IterableCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/IterableCoder.java index 2e5a8cc361c9..11fb1720deb7 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/IterableCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/IterableCoder.java @@ -17,17 +17,14 @@ */ package org.apache.beam.sdk.coders; -import static org.apache.beam.sdk.util.Structs.addBoolean; - import static com.google.common.base.Preconditions.checkArgument; - -import org.apache.beam.sdk.util.CloudObject; -import org.apache.beam.sdk.util.PropertyNames; +import static org.apache.beam.sdk.util.Structs.addBoolean; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; - import java.util.List; +import org.apache.beam.sdk.util.CloudObject; +import org.apache.beam.sdk.util.PropertyNames; /** * An {@link IterableCoder} encodes any {@link Iterable} in the format diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/IterableLikeCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/IterableLikeCoder.java index 9417d8561201..8680552bddc5 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/IterableLikeCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/IterableLikeCoder.java @@ -19,11 +19,6 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.util.BufferedElementCountingOutputStream; -import org.apache.beam.sdk.util.VarInt; -import org.apache.beam.sdk.util.common.ElementByteSizeObservableIterable; -import org.apache.beam.sdk.util.common.ElementByteSizeObserver; - import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; @@ -35,6 +30,10 @@ import java.util.List; import java.util.Observable; import java.util.Observer; +import org.apache.beam.sdk.util.BufferedElementCountingOutputStream; +import org.apache.beam.sdk.util.VarInt; +import org.apache.beam.sdk.util.common.ElementByteSizeObservableIterable; +import org.apache.beam.sdk.util.common.ElementByteSizeObserver; /** * An abstract base class with functionality for assembling a diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/JAXBCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/JAXBCoder.java index 689f993a1cde..748b07d8f6f9 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/JAXBCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/JAXBCoder.java @@ -17,25 +17,21 @@ */ package org.apache.beam.sdk.coders; -import org.apache.beam.sdk.util.CloudObject; -import org.apache.beam.sdk.util.Structs; -import org.apache.beam.sdk.util.VarInt; - -import com.google.common.io.ByteStreams; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; - +import com.google.common.io.ByteStreams; import java.io.FilterInputStream; import java.io.FilterOutputStream; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; - import javax.xml.bind.JAXBContext; import javax.xml.bind.JAXBException; import javax.xml.bind.Marshaller; import javax.xml.bind.Unmarshaller; +import org.apache.beam.sdk.util.CloudObject; +import org.apache.beam.sdk.util.Structs; +import org.apache.beam.sdk.util.VarInt; /** * A coder for JAXB annotated objects. This coder uses JAXB marshalling/unmarshalling mechanisms diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/KvCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/KvCoder.java index beeb9c5267a6..ad13226b05c4 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/KvCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/KvCoder.java @@ -17,23 +17,20 @@ */ package org.apache.beam.sdk.coders; -import static org.apache.beam.sdk.util.Structs.addBoolean; - import static com.google.common.base.Preconditions.checkArgument; - -import org.apache.beam.sdk.util.CloudObject; -import org.apache.beam.sdk.util.PropertyNames; -import org.apache.beam.sdk.util.common.ElementByteSizeObserver; -import org.apache.beam.sdk.values.KV; +import static org.apache.beam.sdk.util.Structs.addBoolean; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.util.Arrays; import java.util.List; +import org.apache.beam.sdk.util.CloudObject; +import org.apache.beam.sdk.util.PropertyNames; +import org.apache.beam.sdk.util.common.ElementByteSizeObserver; +import org.apache.beam.sdk.values.KV; /** * A {@code KvCoder} encodes {@link KV}s. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ListCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ListCoder.java index f8437a4c929a..78785017d49c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ListCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ListCoder.java @@ -19,12 +19,10 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.util.PropertyNames; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; - import java.util.List; +import org.apache.beam.sdk.util.PropertyNames; /** * A {@link Coder} for {@link List}, using the format of {@link IterableLikeCoder}. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/MapCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/MapCoder.java index 044fff1e413a..ebe705156ffb 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/MapCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/MapCoder.java @@ -19,14 +19,9 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.util.PropertyNames; -import org.apache.beam.sdk.util.common.ElementByteSizeObserver; - -import com.google.common.collect.Maps; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; - +import com.google.common.collect.Maps; import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; @@ -36,6 +31,8 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; +import org.apache.beam.sdk.util.PropertyNames; +import org.apache.beam.sdk.util.common.ElementByteSizeObserver; /** * A {@link Coder} for {@link Map Maps} that encodes them according to provided diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/NullableCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/NullableCoder.java index 4001b811db18..44aadbdd88cc 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/NullableCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/NullableCoder.java @@ -19,21 +19,17 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.util.PropertyNames; -import org.apache.beam.sdk.util.common.ElementByteSizeObserver; - -import com.google.common.base.Optional; -import com.google.common.collect.ImmutableList; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; - +import com.google.common.base.Optional; +import com.google.common.collect.ImmutableList; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.util.List; - import javax.annotation.Nullable; +import org.apache.beam.sdk.util.PropertyNames; +import org.apache.beam.sdk.util.common.ElementByteSizeObserver; /** * A {@link NullableCoder} encodes nullable values of type {@code T} using a nested diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SerializableCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SerializableCoder.java index 0995bdc976e0..46777b9d835e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SerializableCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SerializableCoder.java @@ -17,12 +17,8 @@ */ package org.apache.beam.sdk.coders; -import org.apache.beam.sdk.util.CloudObject; -import org.apache.beam.sdk.values.TypeDescriptor; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; - import java.io.IOException; import java.io.InputStream; import java.io.ObjectInputStream; @@ -30,6 +26,8 @@ import java.io.ObjectStreamClass; import java.io.OutputStream; import java.io.Serializable; +import org.apache.beam.sdk.util.CloudObject; +import org.apache.beam.sdk.values.TypeDescriptor; /** * A {@link Coder} for Java classes that implement {@link Serializable}. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SetCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SetCoder.java index fcbb3797a8a2..0d1b0176ae05 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SetCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SetCoder.java @@ -19,14 +19,12 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.util.PropertyNames; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; - import java.util.HashSet; import java.util.List; import java.util.Set; +import org.apache.beam.sdk.util.PropertyNames; /** * A {@link SetCoder} encodes any {@link Set} using the format of {@link IterableLikeCoder}. The diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StandardCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StandardCoder.java index 8b7b78890d4c..0e57ed22ea37 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StandardCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StandardCoder.java @@ -17,26 +17,23 @@ */ package org.apache.beam.sdk.coders; +import static com.google.common.base.Preconditions.checkNotNull; import static org.apache.beam.sdk.util.Structs.addList; import static org.apache.beam.sdk.util.Structs.addString; import static org.apache.beam.sdk.util.Structs.addStringList; -import static com.google.common.base.Preconditions.checkNotNull; - -import org.apache.beam.sdk.util.CloudObject; -import org.apache.beam.sdk.util.PropertyNames; -import org.apache.beam.sdk.util.common.ElementByteSizeObserver; - import com.google.common.collect.Lists; import com.google.common.io.ByteStreams; import com.google.common.io.CountingOutputStream; - import java.io.ByteArrayOutputStream; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.List; +import org.apache.beam.sdk.util.CloudObject; +import org.apache.beam.sdk.util.PropertyNames; +import org.apache.beam.sdk.util.common.ElementByteSizeObserver; /** * An abstract base class to implement a {@link Coder} that defines equality, hashing, and printing diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringDelegateCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringDelegateCoder.java index c498a8aa2715..80bcae30b386 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringDelegateCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringDelegateCoder.java @@ -17,14 +17,13 @@ */ package org.apache.beam.sdk.coders; -import org.apache.beam.sdk.coders.DelegateCoder.CodingFunction; -import org.apache.beam.sdk.coders.protobuf.ProtoCoder; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.lang.reflect.InvocationTargetException; import java.util.Collection; +import org.apache.beam.sdk.coders.DelegateCoder.CodingFunction; +import org.apache.beam.sdk.coders.protobuf.ProtoCoder; /** * A {@link Coder} that wraps a {@code Coder} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringUtf8Coder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringUtf8Coder.java index 807c00190cdb..e01dfd8a1219 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringUtf8Coder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringUtf8Coder.java @@ -17,16 +17,10 @@ */ package org.apache.beam.sdk.coders; -import org.apache.beam.sdk.util.ExposedByteArrayOutputStream; -import org.apache.beam.sdk.util.StreamUtils; -import org.apache.beam.sdk.util.VarInt; - +import com.fasterxml.jackson.annotation.JsonCreator; import com.google.common.base.Utf8; import com.google.common.io.ByteStreams; import com.google.common.io.CountingOutputStream; - -import com.fasterxml.jackson.annotation.JsonCreator; - import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.EOFException; @@ -35,6 +29,9 @@ import java.io.OutputStream; import java.io.UTFDataFormatException; import java.nio.charset.StandardCharsets; +import org.apache.beam.sdk.util.ExposedByteArrayOutputStream; +import org.apache.beam.sdk.util.StreamUtils; +import org.apache.beam.sdk.util.VarInt; /** * A {@link Coder} that encodes {@link String Strings} in UTF-8 encoding. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/TableRowJsonCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/TableRowJsonCoder.java index 42a6dc96cdd9..a2562f28baf6 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/TableRowJsonCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/TableRowJsonCoder.java @@ -17,12 +17,10 @@ */ package org.apache.beam.sdk.coders; -import com.google.api.services.bigquery.model.TableRow; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.SerializationFeature; - +import com.google.api.services.bigquery.model.TableRow; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/TextualIntegerCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/TextualIntegerCoder.java index 156160eb0e7e..6258b2101c40 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/TextualIntegerCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/TextualIntegerCoder.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.coders; import com.fasterxml.jackson.annotation.JsonCreator; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VarIntCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VarIntCoder.java index 20ce7d4c8cad..baf3be88af59 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VarIntCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VarIntCoder.java @@ -17,15 +17,13 @@ */ package org.apache.beam.sdk.coders; -import org.apache.beam.sdk.util.VarInt; - import com.fasterxml.jackson.annotation.JsonCreator; - import java.io.EOFException; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.io.UTFDataFormatException; +import org.apache.beam.sdk.util.VarInt; /** * A {@link Coder} that encodes {@link Integer Integers} using between 1 and 5 bytes. Negative diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VarLongCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VarLongCoder.java index f1a5ea078411..ee3c501966fe 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VarLongCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VarLongCoder.java @@ -17,15 +17,13 @@ */ package org.apache.beam.sdk.coders; -import org.apache.beam.sdk.util.VarInt; - import com.fasterxml.jackson.annotation.JsonCreator; - import java.io.EOFException; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.io.UTFDataFormatException; +import org.apache.beam.sdk.util.VarInt; /** * A {@link Coder} that encodes {@link Long Longs} using between 1 and 10 bytes. Negative diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VoidCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VoidCoder.java index d67d82fbb6a0..6bd8a0536ebe 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VoidCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VoidCoder.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.coders; import com.fasterxml.jackson.annotation.JsonCreator; - import java.io.InputStream; import java.io.OutputStream; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/protobuf/ProtoCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/protobuf/ProtoCoder.java index 67cec884b8b6..79fb37305ea6 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/protobuf/ProtoCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/protobuf/ProtoCoder.java @@ -19,27 +19,14 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.coders.AtomicCoder; -import org.apache.beam.sdk.coders.CannotProvideCoderException; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.coders.CoderProvider; -import org.apache.beam.sdk.coders.CoderRegistry; -import org.apache.beam.sdk.util.CloudObject; -import org.apache.beam.sdk.util.Structs; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.TypeDescriptor; - +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.google.protobuf.ExtensionRegistry; import com.google.protobuf.Message; import com.google.protobuf.Parser; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -52,8 +39,17 @@ import java.util.Set; import java.util.SortedSet; import java.util.TreeSet; - import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.AtomicCoder; +import org.apache.beam.sdk.coders.CannotProvideCoderException; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.coders.CoderProvider; +import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.util.CloudObject; +import org.apache.beam.sdk.util.Structs; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TypeDescriptor; /** * A {@link Coder} using Google Protocol Buffers binary format. {@link ProtoCoder} supports both diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/protobuf/ProtobufUtil.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/protobuf/ProtobufUtil.java index b03b748bfa18..77afb47db49c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/protobuf/ProtobufUtil.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/protobuf/ProtobufUtil.java @@ -19,8 +19,6 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.coders.Coder.NonDeterministicException; - import com.google.protobuf.Descriptors.Descriptor; import com.google.protobuf.Descriptors.FieldDescriptor; import com.google.protobuf.Descriptors.FileDescriptor.Syntax; @@ -28,10 +26,10 @@ import com.google.protobuf.ExtensionRegistry; import com.google.protobuf.ExtensionRegistry.ExtensionInfo; import com.google.protobuf.Message; - import java.lang.reflect.InvocationTargetException; import java.util.HashSet; import java.util.Set; +import org.apache.beam.sdk.coders.Coder.NonDeterministicException; /** * Utility functions for reflecting and analyzing Protocol Buffers classes. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java index 28c283d97daf..e7c302bed1d7 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java @@ -20,6 +20,16 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; +import com.google.common.annotations.VisibleForTesting; +import java.io.IOException; +import java.nio.channels.Channels; +import java.nio.channels.WritableByteChannel; +import java.util.regex.Pattern; +import javax.annotation.Nullable; +import org.apache.avro.Schema; +import org.apache.avro.file.DataFileWriter; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.reflect.ReflectData; import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.VoidCoder; @@ -34,20 +44,6 @@ import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.PInput; -import com.google.common.annotations.VisibleForTesting; - -import org.apache.avro.Schema; -import org.apache.avro.file.DataFileWriter; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.reflect.ReflectData; - -import java.io.IOException; -import java.nio.channels.Channels; -import java.nio.channels.WritableByteChannel; -import java.util.regex.Pattern; - -import javax.annotation.Nullable; - /** * {@link PTransform}s for reading and writing Avro files. * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java index d1e0c4dc5e39..6ef02aa2d501 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java @@ -21,29 +21,6 @@ import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; -import org.apache.beam.sdk.annotations.Experimental; -import org.apache.beam.sdk.coders.AvroCoder; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.runners.PipelineRunner; -import org.apache.beam.sdk.util.AvroUtils; -import org.apache.beam.sdk.util.AvroUtils.AvroMetadata; -import org.apache.beam.sdk.values.PCollection; - -import org.apache.avro.Schema; -import org.apache.avro.file.CodecFactory; -import org.apache.avro.file.DataFileConstants; -import org.apache.avro.generic.GenericDatumReader; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.io.BinaryDecoder; -import org.apache.avro.io.DatumReader; -import org.apache.avro.io.DecoderFactory; -import org.apache.avro.reflect.ReflectData; -import org.apache.avro.reflect.ReflectDatumReader; -import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream; -import org.apache.commons.compress.compressors.snappy.SnappyCompressorInputStream; -import org.apache.commons.compress.compressors.xz.XZCompressorInputStream; -import org.apache.commons.compress.utils.CountingInputStream; - import java.io.ByteArrayInputStream; import java.io.EOFException; import java.io.IOException; @@ -57,8 +34,28 @@ import java.util.Collection; import java.util.zip.Inflater; import java.util.zip.InflaterInputStream; - import javax.annotation.concurrent.GuardedBy; +import org.apache.avro.Schema; +import org.apache.avro.file.CodecFactory; +import org.apache.avro.file.DataFileConstants; +import org.apache.avro.generic.GenericDatumReader; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.io.BinaryDecoder; +import org.apache.avro.io.DatumReader; +import org.apache.avro.io.DecoderFactory; +import org.apache.avro.reflect.ReflectData; +import org.apache.avro.reflect.ReflectDatumReader; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.runners.PipelineRunner; +import org.apache.beam.sdk.util.AvroUtils; +import org.apache.beam.sdk.util.AvroUtils.AvroMetadata; +import org.apache.beam.sdk.values.PCollection; +import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream; +import org.apache.commons.compress.compressors.snappy.SnappyCompressorInputStream; +import org.apache.commons.compress.compressors.xz.XZCompressorInputStream; +import org.apache.commons.compress.utils.CountingInputStream; // CHECKSTYLE.OFF: JavadocStyle /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BlockBasedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BlockBasedSource.java index 997c77a1273f..83336ff73913 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BlockBasedSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BlockBasedSource.java @@ -17,13 +17,11 @@ */ package org.apache.beam.sdk.io; -import org.apache.beam.sdk.annotations.Experimental; -import org.apache.beam.sdk.options.PipelineOptions; - import java.io.IOException; import java.util.NoSuchElementException; - import javax.annotation.Nullable; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.options.PipelineOptions; /** * A {@code BlockBasedSource} is a {@link FileBasedSource} where a file consists of blocks of diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java index cfdd5814e273..ede65a96cc99 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java @@ -19,6 +19,13 @@ import static org.apache.beam.sdk.util.StringUtils.approximateSimpleName; +import com.google.api.client.util.BackOff; +import com.google.common.util.concurrent.Uninterruptibles; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.concurrent.TimeUnit; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; @@ -31,19 +38,9 @@ import org.apache.beam.sdk.util.ValueWithRecordId; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PInput; - -import com.google.api.client.util.BackOff; -import com.google.common.util.concurrent.Uninterruptibles; - import org.joda.time.Duration; import org.joda.time.Instant; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.NoSuchElementException; -import java.util.concurrent.TimeUnit; - /** * {@link PTransform} that reads a bounded amount of data from an {@link UnboundedSource}, diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedSource.java index 394afa4bb3c8..5fd7b8adc948 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedSource.java @@ -17,20 +17,17 @@ */ package org.apache.beam.sdk.io; +import java.io.IOException; +import java.util.List; +import java.util.NoSuchElementException; +import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.io.range.OffsetRangeTracker; import org.apache.beam.sdk.io.range.RangeTracker; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; - import org.joda.time.Instant; -import java.io.IOException; -import java.util.List; -import java.util.NoSuchElementException; - -import javax.annotation.Nullable; - /** * A {@link Source} that reads a finite amount of input and, because of that, supports * some additional operations. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CompressedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CompressedSource.java index 11ff90f66af9..3cd097ca507b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CompressedSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CompressedSource.java @@ -20,17 +20,8 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; -import org.apache.beam.sdk.annotations.Experimental; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.transforms.display.DisplayData; - import com.google.common.io.ByteStreams; import com.google.common.primitives.Ints; - -import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream; -import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream; - import java.io.IOException; import java.io.InputStream; import java.io.PushbackInputStream; @@ -42,8 +33,13 @@ import java.util.zip.GZIPInputStream; import java.util.zip.ZipEntry; import java.util.zip.ZipInputStream; - import javax.annotation.concurrent.GuardedBy; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream; +import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream; /** * A Source that reads from compressed files. A {@code CompressedSources} wraps a delegate diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingInput.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingInput.java index a21bc2418fa3..f47921529644 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingInput.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingInput.java @@ -20,6 +20,7 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; +import com.google.common.base.Optional; import org.apache.beam.sdk.io.CountingSource.NowTimestampFn; import org.apache.beam.sdk.io.Read.Unbounded; import org.apache.beam.sdk.transforms.PTransform; @@ -28,9 +29,6 @@ import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; - -import com.google.common.base.Optional; - import org.joda.time.Duration; import org.joda.time.Instant; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingSource.java index 403d22eba319..59a8df8ac9d9 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingSource.java @@ -20,6 +20,10 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; +import com.google.common.collect.ImmutableList; +import java.io.IOException; +import java.util.List; +import java.util.NoSuchElementException; import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.DefaultCoder; @@ -29,16 +33,9 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.ImmutableList; - import org.joda.time.Duration; import org.joda.time.Instant; -import java.io.IOException; -import java.util.List; -import java.util.NoSuchElementException; - /** * A source that produces longs. When used as a {@link BoundedSource}, {@link CountingSource} * starts at {@code 0} and counts up to a specified maximum. When used as an diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java index 761d49c9f23b..ea95f2f5629c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java @@ -21,23 +21,7 @@ import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.SerializableCoder; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.util.FileIOChannelFactory; -import org.apache.beam.sdk.util.GcsIOChannelFactory; -import org.apache.beam.sdk.util.GcsUtil; -import org.apache.beam.sdk.util.GcsUtil.GcsUtilFactory; -import org.apache.beam.sdk.util.IOChannelFactory; -import org.apache.beam.sdk.util.IOChannelUtils; -import org.apache.beam.sdk.util.MimeTypes; - import com.google.common.collect.Ordering; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.File; import java.io.IOException; import java.io.Serializable; @@ -50,6 +34,19 @@ import java.util.Collection; import java.util.HashSet; import java.util.List; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.SerializableCoder; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.util.FileIOChannelFactory; +import org.apache.beam.sdk.util.GcsIOChannelFactory; +import org.apache.beam.sdk.util.GcsUtil; +import org.apache.beam.sdk.util.GcsUtil.GcsUtilFactory; +import org.apache.beam.sdk.util.IOChannelFactory; +import org.apache.beam.sdk.util.IOChannelUtils; +import org.apache.beam.sdk.util.MimeTypes; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Abstract {@link Sink} for file-based output. An implementation of FileBasedSink writes file-based diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSource.java index f000f6a71eda..b07323660536 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSource.java @@ -20,22 +20,12 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.util.IOChannelFactory; -import org.apache.beam.sdk.util.IOChannelUtils; - import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; - -import org.joda.time.Instant; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.IOException; import java.nio.channels.ReadableByteChannel; import java.nio.channels.SeekableByteChannel; @@ -48,6 +38,13 @@ import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executors; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.util.IOChannelFactory; +import org.apache.beam.sdk.util.IOChannelUtils; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * A common base class for all file-based {@link Source}s. Extend this class to implement your own diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/OffsetBasedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/OffsetBasedSource.java index 8cbcd1f868da..6c685ffc89a0 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/OffsetBasedSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/OffsetBasedSource.java @@ -19,19 +19,17 @@ import static com.google.common.base.Preconditions.checkArgument; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.NoSuchElementException; import org.apache.beam.sdk.io.range.OffsetRangeTracker; import org.apache.beam.sdk.io.range.RangeTracker; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.display.DisplayData; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.NoSuchElementException; - /** * A {@link BoundedSource} that uses offsets to define starting and ending positions. * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java index 2b2717574e55..b137f1551010 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java @@ -19,6 +19,14 @@ import static com.google.common.base.Preconditions.checkState; +import com.google.common.base.Strings; +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VoidCoder; @@ -41,23 +49,11 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.PInput; - -import com.google.common.base.Strings; - import org.joda.time.Duration; import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.List; -import java.util.regex.Matcher; -import java.util.regex.Pattern; - -import javax.annotation.Nullable; - /** * Read and Write {@link PTransform}s for Cloud Pub/Sub streams. These transforms create * and consume unbounded {@link PCollection PCollections}. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubUnboundedSink.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubUnboundedSink.java index 301475198b02..179abf6da4f6 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubUnboundedSink.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubUnboundedSink.java @@ -20,6 +20,16 @@ import static com.google.common.base.Preconditions.checkState; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.hash.Hashing; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.ThreadLocalRandom; +import javax.annotation.Nullable; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.BigEndianLongCoder; import org.apache.beam.sdk.coders.ByteArrayCoder; @@ -52,21 +62,8 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.hash.Hashing; - import org.joda.time.Duration; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.ArrayList; -import java.util.List; -import java.util.UUID; -import java.util.concurrent.ThreadLocalRandom; -import javax.annotation.Nullable; - /** * A PTransform which streams messages to Pubsub. *
      diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubUnboundedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubUnboundedSource.java index f99b47142425..36f154f9c987 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubUnboundedSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubUnboundedSource.java @@ -22,6 +22,28 @@ import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; +import com.google.api.client.util.Clock; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Charsets; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.security.GeneralSecurityException; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Queue; +import java.util.Set; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.atomic.AtomicInteger; +import javax.annotation.Nullable; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; @@ -49,36 +71,11 @@ import org.apache.beam.sdk.util.PubsubClient.TopicPath; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; - -import com.google.api.client.util.Clock; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Charsets; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; - import org.joda.time.Duration; import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.security.GeneralSecurityException; -import java.util.ArrayDeque; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashSet; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.NoSuchElementException; -import java.util.Queue; -import java.util.Set; -import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.atomic.AtomicInteger; -import javax.annotation.Nullable; - /** * A PTransform which streams messages from Pubsub. *
        diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java index e13ff06dc39d..f99877d84d29 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java @@ -19,6 +19,7 @@ import static org.apache.beam.sdk.util.StringUtils.approximateSimpleName; +import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.display.DisplayData; @@ -27,11 +28,8 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.PInput; - import org.joda.time.Duration; -import javax.annotation.Nullable; - /** * A {@link PTransform} for reading from a {@link Source}. * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Sink.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Sink.java index 20b1631306d5..1abcc3d1166d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Sink.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Sink.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.io; +import java.io.Serializable; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; @@ -24,8 +25,6 @@ import org.apache.beam.sdk.transforms.display.HasDisplayData; import org.apache.beam.sdk.values.PCollection; -import java.io.Serializable; - /** * A {@code Sink} represents a resource that can be written to using the {@link Write} transform. * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Source.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Source.java index de000351538b..542d91ca791e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Source.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Source.java @@ -17,17 +17,15 @@ */ package org.apache.beam.sdk.io; +import java.io.IOException; +import java.io.Serializable; +import java.util.NoSuchElementException; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.display.HasDisplayData; - import org.joda.time.Instant; -import java.io.IOException; -import java.io.Serializable; -import java.util.NoSuchElementException; - /** * Base class for defining input formats and creating a {@code Source} for reading the input. * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java index 64db3f76311c..ed9a62790afe 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java @@ -20,23 +20,8 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.Coder.Context; -import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.coders.VoidCoder; -import org.apache.beam.sdk.io.Read.Bounded; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.util.IOChannelUtils; -import org.apache.beam.sdk.util.MimeTypes; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PDone; -import org.apache.beam.sdk.values.PInput; - import com.google.common.annotations.VisibleForTesting; import com.google.protobuf.ByteString; - import java.io.IOException; import java.io.OutputStream; import java.nio.ByteBuffer; @@ -47,8 +32,20 @@ import java.nio.charset.StandardCharsets; import java.util.NoSuchElementException; import java.util.regex.Pattern; - import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.Coder.Context; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.coders.VoidCoder; +import org.apache.beam.sdk.io.Read.Bounded; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.util.IOChannelUtils; +import org.apache.beam.sdk.util.MimeTypes; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PDone; +import org.apache.beam.sdk.values.PInput; /** * {@link PTransform}s for reading and writing text files. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/UnboundedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/UnboundedSource.java index dded8e2493b4..043f2fcefb19 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/UnboundedSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/UnboundedSource.java @@ -17,17 +17,14 @@ */ package org.apache.beam.sdk.io; -import org.apache.beam.sdk.annotations.Experimental; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.options.PipelineOptions; - -import org.joda.time.Instant; - import java.io.IOException; import java.util.List; import java.util.NoSuchElementException; - import javax.annotation.Nullable; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.options.PipelineOptions; +import org.joda.time.Instant; /** * A {@link Source} that reads an unbounded amount of input and, because of that, supports diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java index a474528e04e6..9d0beb769c44 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java @@ -20,6 +20,10 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; +import com.google.common.collect.Lists; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.ThreadLocalRandom; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.coders.Coder; @@ -44,16 +48,9 @@ import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PDone; - -import com.google.common.collect.Lists; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.List; -import java.util.UUID; -import java.util.concurrent.ThreadLocalRandom; - /** * A {@link PTransform} that writes to a {@link Sink}. A write begins with a sequential global * initialization of a sink, followed by a parallel write, and ends with a sequential finalization diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSink.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSink.java index ef50a1c322f5..983eed2468a5 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSink.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSink.java @@ -19,6 +19,12 @@ import static com.google.common.base.Preconditions.checkNotNull; +import java.io.OutputStream; +import java.nio.channels.Channels; +import java.nio.channels.WritableByteChannel; +import javax.xml.bind.JAXBContext; +import javax.xml.bind.JAXBException; +import javax.xml.bind.Marshaller; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.io.FileBasedSink.FileBasedWriteOperation; import org.apache.beam.sdk.io.FileBasedSink.FileBasedWriter; @@ -27,14 +33,6 @@ import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.values.PCollection; -import java.io.OutputStream; -import java.nio.channels.Channels; -import java.nio.channels.WritableByteChannel; - -import javax.xml.bind.JAXBContext; -import javax.xml.bind.JAXBException; -import javax.xml.bind.Marshaller; - // CHECKSTYLE.OFF: JavadocStyle /** * A {@link Sink} that outputs records as XML-formatted elements. Writes a {@link PCollection} of diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSource.java index 83443a32107e..e00857ea2bf8 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSource.java @@ -19,14 +19,6 @@ import static com.google.common.base.Preconditions.checkNotNull; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.JAXBCoder; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.runners.PipelineRunner; -import org.apache.beam.sdk.transforms.display.DisplayData; - -import org.codehaus.stax2.XMLInputFactory2; - import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -39,7 +31,6 @@ import java.nio.channels.ReadableByteChannel; import java.nio.charset.StandardCharsets; import java.util.NoSuchElementException; - import javax.xml.bind.JAXBContext; import javax.xml.bind.JAXBElement; import javax.xml.bind.JAXBException; @@ -51,6 +42,12 @@ import javax.xml.stream.XMLStreamConstants; import javax.xml.stream.XMLStreamException; import javax.xml.stream.XMLStreamReader; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.JAXBCoder; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.runners.PipelineRunner; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.codehaus.stax2.XMLInputFactory2; // CHECKSTYLE.OFF: JavadocStyle /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKey.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKey.java index f82a11d532cd..e4129ff56512 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKey.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKey.java @@ -21,7 +21,6 @@ import com.google.protobuf.ByteString; import com.google.protobuf.ByteString.ByteIterator; - import java.io.Serializable; /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRange.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRange.java index cfd924d3841e..288124b310ec 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRange.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRange.java @@ -24,16 +24,14 @@ import com.google.common.base.MoreObjects; import com.google.common.collect.ImmutableList; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.Serializable; import java.math.BigDecimal; import java.math.BigInteger; import java.util.Arrays; import java.util.List; import java.util.Objects; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * A class representing a range of {@link ByteKey ByteKeys}. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRangeTracker.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRangeTracker.java index 24bf6e474b4d..7c0f1c0a6d38 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRangeTracker.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRangeTracker.java @@ -20,13 +20,11 @@ import static com.google.common.base.MoreObjects.toStringHelper; import static com.google.common.base.Preconditions.checkState; +import javax.annotation.Nullable; import org.apache.beam.sdk.io.BoundedSource.BoundedReader; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import javax.annotation.Nullable; - /** * A {@link RangeTracker} for {@link ByteKey ByteKeys} in {@link ByteKeyRange ByteKeyRanges}. * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/OffsetRangeTracker.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/OffsetRangeTracker.java index a8d00ee1d16b..51e2b1ac2a1c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/OffsetRangeTracker.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/OffsetRangeTracker.java @@ -19,10 +19,8 @@ import static com.google.common.base.Preconditions.checkState; -import org.apache.beam.sdk.io.BoundedSource.BoundedReader; - import com.google.common.annotations.VisibleForTesting; - +import org.apache.beam.sdk.io.BoundedSource.BoundedReader; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GcpOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GcpOptions.java index b2df96e7474f..b4ae31406448 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GcpOptions.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GcpOptions.java @@ -17,21 +17,12 @@ */ package org.apache.beam.sdk.options; -import org.apache.beam.sdk.util.CredentialFactory; -import org.apache.beam.sdk.util.GcpCredentialFactory; -import org.apache.beam.sdk.util.InstanceBuilder; -import org.apache.beam.sdk.util.PathValidator; - +import com.fasterxml.jackson.annotation.JsonIgnore; import com.google.api.client.auth.oauth2.Credential; import com.google.api.client.googleapis.auth.oauth2.GoogleOAuthConstants; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Strings; import com.google.common.io.Files; - -import com.fasterxml.jackson.annotation.JsonIgnore; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.File; import java.io.IOException; import java.nio.charset.StandardCharsets; @@ -40,8 +31,13 @@ import java.util.Map; import java.util.regex.Matcher; import java.util.regex.Pattern; - import javax.annotation.Nullable; +import org.apache.beam.sdk.util.CredentialFactory; +import org.apache.beam.sdk.util.GcpCredentialFactory; +import org.apache.beam.sdk.util.InstanceBuilder; +import org.apache.beam.sdk.util.PathValidator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Options used to configure Google Cloud Platform project and credentials. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GcsOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GcsOptions.java index 1b3436bf3118..29a29d9f4acd 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GcsOptions.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GcsOptions.java @@ -17,22 +17,19 @@ */ package org.apache.beam.sdk.options; -import org.apache.beam.sdk.util.AppEngineEnvironment; -import org.apache.beam.sdk.util.GcsPathValidator; -import org.apache.beam.sdk.util.GcsUtil; -import org.apache.beam.sdk.util.InstanceBuilder; -import org.apache.beam.sdk.util.PathValidator; - +import com.fasterxml.jackson.annotation.JsonIgnore; import com.google.cloud.hadoop.util.AbstractGoogleAsyncWriteChannel; import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.ThreadFactoryBuilder; - -import com.fasterxml.jackson.annotation.JsonIgnore; - import java.util.concurrent.ExecutorService; import java.util.concurrent.SynchronousQueue; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; +import org.apache.beam.sdk.util.AppEngineEnvironment; +import org.apache.beam.sdk.util.GcsPathValidator; +import org.apache.beam.sdk.util.GcsUtil; +import org.apache.beam.sdk.util.InstanceBuilder; +import org.apache.beam.sdk.util.PathValidator; /** * Options used to configure Google Cloud Storage. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GoogleApiDebugOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GoogleApiDebugOptions.java index 39cd40e715f8..4932b16ef49f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GoogleApiDebugOptions.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GoogleApiDebugOptions.java @@ -20,7 +20,6 @@ import com.google.api.client.googleapis.services.AbstractGoogleClient; import com.google.api.client.googleapis.services.AbstractGoogleClientRequest; import com.google.api.client.googleapis.services.GoogleClientRequestInitializer; - import java.io.IOException; import java.util.HashMap; import java.util.Map; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionSpec.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionSpec.java index 9a88f7076232..1220e6bc14e2 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionSpec.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionSpec.java @@ -17,10 +17,8 @@ */ package org.apache.beam.sdk.options; -import com.google.auto.value.AutoValue; - import com.fasterxml.jackson.annotation.JsonIgnore; - +import com.google.auto.value.AutoValue; import java.lang.reflect.Method; /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java index 4595fc871d22..db54d0ac76b6 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java @@ -17,6 +17,14 @@ */ package org.apache.beam.sdk.options; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import com.fasterxml.jackson.databind.annotation.JsonSerialize; +import com.google.auto.service.AutoService; +import java.lang.reflect.Proxy; +import java.util.ServiceLoader; +import javax.annotation.concurrent.ThreadSafe; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.options.GoogleApiDebugOptions.GoogleApiTracer; import org.apache.beam.sdk.options.ProxyInvocationHandler.Deserializer; @@ -26,17 +34,6 @@ import org.apache.beam.sdk.transforms.DoFn.Context; import org.apache.beam.sdk.transforms.display.HasDisplayData; -import com.google.auto.service.AutoService; - -import com.fasterxml.jackson.annotation.JsonIgnore; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.annotation.JsonDeserialize; -import com.fasterxml.jackson.databind.annotation.JsonSerialize; - -import java.lang.reflect.Proxy; -import java.util.ServiceLoader; -import javax.annotation.concurrent.ThreadSafe; - /** * PipelineOptions are used to configure Pipelines. You can extend {@link PipelineOptions} * to create custom configuration options specific to your {@link Pipeline}, diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java index a795fcd024ff..43927bc71621 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java @@ -20,13 +20,9 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; -import org.apache.beam.sdk.options.Validation.Required; -import org.apache.beam.sdk.runners.PipelineRunner; -import org.apache.beam.sdk.runners.PipelineRunnerRegistrar; -import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.util.StringUtils; -import org.apache.beam.sdk.util.common.ReflectHelpers; - +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.databind.JavaType; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Joiner; @@ -49,13 +45,6 @@ import com.google.common.collect.SortedSetMultimap; import com.google.common.collect.TreeBasedTable; import com.google.common.collect.TreeMultimap; - -import com.fasterxml.jackson.annotation.JsonIgnore; -import com.fasterxml.jackson.databind.JavaType; -import com.fasterxml.jackson.databind.ObjectMapper; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.beans.BeanInfo; import java.beans.IntrospectionException; import java.beans.Introspector; @@ -81,8 +70,15 @@ import java.util.SortedSet; import java.util.TreeMap; import java.util.TreeSet; - import javax.annotation.Nonnull; +import org.apache.beam.sdk.options.Validation.Required; +import org.apache.beam.sdk.runners.PipelineRunner; +import org.apache.beam.sdk.runners.PipelineRunnerRegistrar; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.util.StringUtils; +import org.apache.beam.sdk.util.common.ReflectHelpers; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Constructs a {@link PipelineOptions} or any derived interface that is composable to any other diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsReflector.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsReflector.java index 607bddae6578..26c65aec890d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsReflector.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsReflector.java @@ -17,16 +17,14 @@ */ package org.apache.beam.sdk.options; -import org.apache.beam.sdk.util.common.ReflectHelpers; - import com.google.common.collect.HashMultimap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Multimap; - import java.beans.Introspector; import java.lang.reflect.Method; import java.util.Map; import java.util.Set; +import org.apache.beam.sdk.util.common.ReflectHelpers; /** * Utilities to reflect over {@link PipelineOptions}. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsRegistrar.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsRegistrar.java index 16cf7cd73002..2ddff42f8eb4 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsRegistrar.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsRegistrar.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.options; import com.google.auto.service.AutoService; - import java.util.ServiceLoader; /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsValidator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsValidator.java index 70be65fa4acd..bd54ec39bd74 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsValidator.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsValidator.java @@ -20,17 +20,15 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; -import org.apache.beam.sdk.options.Validation.Required; -import org.apache.beam.sdk.util.common.ReflectHelpers; - import com.google.common.collect.Collections2; import com.google.common.collect.Ordering; import com.google.common.collect.SortedSetMultimap; import com.google.common.collect.TreeMultimap; - import java.lang.reflect.Method; import java.lang.reflect.Proxy; import java.util.Collection; +import org.apache.beam.sdk.options.Validation.Required; +import org.apache.beam.sdk.util.common.ReflectHelpers; /** * Validates that the {@link PipelineOptions} conforms to all the {@link Validation} criteria. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java index fe67f1685da3..204ad97e7184 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java @@ -20,25 +20,6 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; -import org.apache.beam.sdk.options.PipelineOptionsFactory.JsonIgnorePredicate; -import org.apache.beam.sdk.options.PipelineOptionsFactory.Registration; -import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.transforms.display.HasDisplayData; -import org.apache.beam.sdk.util.InstanceBuilder; -import org.apache.beam.sdk.util.common.ReflectHelpers; - -import com.google.auto.value.AutoValue; -import com.google.common.base.Defaults; -import com.google.common.base.Function; -import com.google.common.collect.ClassToInstanceMap; -import com.google.common.collect.FluentIterable; -import com.google.common.collect.HashMultimap; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.common.collect.Multimap; -import com.google.common.collect.MutableClassToInstanceMap; - import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.core.JsonParser; @@ -51,7 +32,17 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.SerializerProvider; import com.fasterxml.jackson.databind.node.ObjectNode; - +import com.google.auto.value.AutoValue; +import com.google.common.base.Defaults; +import com.google.common.base.Function; +import com.google.common.collect.ClassToInstanceMap; +import com.google.common.collect.FluentIterable; +import com.google.common.collect.HashMultimap; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Multimap; +import com.google.common.collect.MutableClassToInstanceMap; import java.beans.PropertyDescriptor; import java.io.IOException; import java.lang.annotation.Annotation; @@ -70,9 +61,14 @@ import java.util.SortedMap; import java.util.TreeMap; import java.util.concurrent.ThreadLocalRandom; - import javax.annotation.Nullable; import javax.annotation.concurrent.ThreadSafe; +import org.apache.beam.sdk.options.PipelineOptionsFactory.JsonIgnorePredicate; +import org.apache.beam.sdk.options.PipelineOptionsFactory.Registration; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.transforms.display.HasDisplayData; +import org.apache.beam.sdk.util.InstanceBuilder; +import org.apache.beam.sdk.util.common.ReflectHelpers; /** * Represents and {@link InvocationHandler} for a {@link Proxy}. The invocation handler uses bean diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunnerRegistrar.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunnerRegistrar.java index 949f5daf9ec4..d72c44f9a37d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunnerRegistrar.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunnerRegistrar.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.runners; import com.google.auto.service.AutoService; - import java.util.ServiceLoader; /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java index 6b93bbd8a2ad..0a4bb08a315d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java @@ -19,16 +19,15 @@ import static com.google.common.base.Preconditions.checkState; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.values.PInput; -import org.apache.beam.sdk.values.POutput; -import org.apache.beam.sdk.values.PValue; - import java.util.Deque; import java.util.HashMap; import java.util.LinkedList; import java.util.Map; import java.util.Set; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.values.PInput; +import org.apache.beam.sdk.values.POutput; +import org.apache.beam.sdk.values.PValue; /** * Captures information about a collection of transformations and their diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformTreeNode.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformTreeNode.java index b5a250b57f1f..d16b8281dc66 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformTreeNode.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformTreeNode.java @@ -20,21 +20,19 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; -import org.apache.beam.sdk.Pipeline.PipelineVisitor; -import org.apache.beam.sdk.Pipeline.PipelineVisitor.CompositeBehavior; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.values.PInput; -import org.apache.beam.sdk.values.POutput; -import org.apache.beam.sdk.values.PValue; - import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.Set; - import javax.annotation.Nullable; +import org.apache.beam.sdk.Pipeline.PipelineVisitor; +import org.apache.beam.sdk.Pipeline.PipelineVisitor.CompositeBehavior; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.PInput; +import org.apache.beam.sdk.values.POutput; +import org.apache.beam.sdk.values.PValue; /** * Provides internal tracking of transform relationships with helper methods diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java index e56f01ff7a3e..910b9395aae8 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java @@ -26,24 +26,12 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.Coder.NonDeterministicException; -import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.PropertyNames; -import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.sdk.util.Serializer; -import org.apache.beam.sdk.util.Structs; -import org.apache.beam.sdk.util.UnownedInputStream; -import org.apache.beam.sdk.util.UnownedOutputStream; -import org.apache.beam.sdk.util.common.ElementByteSizeObserver; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.io.ByteStreams; import com.google.common.io.CountingInputStream; import com.google.common.io.CountingOutputStream; - import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -51,6 +39,17 @@ import java.util.Collection; import java.util.Collections; import java.util.List; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.Coder.NonDeterministicException; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.util.CoderUtils; +import org.apache.beam.sdk.util.PropertyNames; +import org.apache.beam.sdk.util.SerializableUtils; +import org.apache.beam.sdk.util.Serializer; +import org.apache.beam.sdk.util.Structs; +import org.apache.beam.sdk.util.UnownedInputStream; +import org.apache.beam.sdk.util.UnownedOutputStream; +import org.apache.beam.sdk.util.common.ElementByteSizeObserver; /** * Properties for use in {@link Coder} tests. These are implemented with junit assertions diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FileChecksumMatcher.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FileChecksumMatcher.java index 303efcb392b8..de6cea383e8c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FileChecksumMatcher.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FileChecksumMatcher.java @@ -20,20 +20,10 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.PipelineResult; -import org.apache.beam.sdk.util.IOChannelFactory; -import org.apache.beam.sdk.util.IOChannelUtils; - import com.google.common.base.Strings; import com.google.common.hash.HashCode; import com.google.common.hash.Hashing; import com.google.common.io.CharStreams; - -import org.hamcrest.Description; -import org.hamcrest.TypeSafeMatcher; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.IOException; import java.io.Reader; import java.nio.channels.Channels; @@ -41,6 +31,13 @@ import java.util.ArrayList; import java.util.Collection; import java.util.List; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.util.IOChannelFactory; +import org.apache.beam.sdk.util.IOChannelUtils; +import org.hamcrest.Description; +import org.hamcrest.TypeSafeMatcher; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Matcher to verify file checksum in E2E test. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherDeserializer.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherDeserializer.java index 84984709bc64..6ca07badac92 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherDeserializer.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherDeserializer.java @@ -17,17 +17,14 @@ */ package org.apache.beam.sdk.testing; -import org.apache.beam.sdk.util.SerializableUtils; - -import com.google.api.client.util.Base64; - import com.fasterxml.jackson.core.JsonParser; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.DeserializationContext; import com.fasterxml.jackson.databind.JsonDeserializer; import com.fasterxml.jackson.databind.node.ObjectNode; - +import com.google.api.client.util.Base64; import java.io.IOException; +import org.apache.beam.sdk.util.SerializableUtils; /** * MatcherDeserializer is used with Jackson to enable deserialization of SerializableMatchers. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherSerializer.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherSerializer.java index 845248604e2b..2b4584c8cdac 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherSerializer.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherSerializer.java @@ -17,16 +17,13 @@ */ package org.apache.beam.sdk.testing; -import org.apache.beam.sdk.util.SerializableUtils; - -import com.google.api.client.util.Base64; - import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.JsonSerializer; import com.fasterxml.jackson.databind.SerializerProvider; - +import com.google.api.client.util.Base64; import java.io.IOException; +import org.apache.beam.sdk.util.SerializableUtils; /** * MatcherSerializer is used with Jackson to enable serialization of SerializableMatchers. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java index f02bbe0ed493..56a1b7c418da 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java @@ -22,6 +22,16 @@ import static org.hamcrest.Matchers.not; import static org.junit.Assert.assertThat; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import java.io.IOException; +import java.io.Serializable; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Map; +import java.util.NoSuchElementException; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; @@ -63,24 +73,10 @@ import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PDone; - -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; - import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.io.Serializable; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.Map; -import java.util.NoSuchElementException; - -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; - /** * An assertion on the contents of a {@link PCollection} incorporated into the pipeline. Such an * assertion can be checked no matter what kind of {@link PipelineRunner} is used. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PaneExtractors.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PaneExtractors.java index 899612b3ef2a..db72a0cd4c22 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PaneExtractors.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PaneExtractors.java @@ -19,6 +19,8 @@ import static com.google.common.base.Preconditions.checkState; +import java.util.ArrayList; +import java.util.List; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.SimpleFunction; import org.apache.beam.sdk.transforms.windowing.PaneInfo; @@ -27,9 +29,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptor; -import java.util.ArrayList; -import java.util.List; - /** * {@link PTransform PTransforms} which take an {@link Iterable} of {@link WindowedValue * WindowedValues} and outputs an {@link Iterable} of all values in the specified pane, dropping the diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatcher.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatcher.java index a465bbec32aa..4e4299d13b2b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatcher.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatcher.java @@ -19,9 +19,8 @@ import com.fasterxml.jackson.databind.annotation.JsonDeserialize; import com.fasterxml.jackson.databind.annotation.JsonSerialize; -import org.hamcrest.Matcher; - import java.io.Serializable; +import org.hamcrest.Matcher; /** * A {@link Matcher} that is also {@link Serializable}. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatchers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatchers.java index 62a42e4176e8..bd44c4801dcb 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatchers.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatchers.java @@ -17,28 +17,24 @@ */ package org.apache.beam.sdk.testing; +import com.google.common.base.MoreObjects; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import java.io.Serializable; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.ListCoder; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.values.KV; - -import com.google.common.base.MoreObjects; - import org.hamcrest.BaseMatcher; import org.hamcrest.Description; import org.hamcrest.Matcher; import org.hamcrest.Matchers; -import java.io.Serializable; -import java.util.Arrays; -import java.util.Collection; -import java.util.List; -import javax.annotation.Nullable; - -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; - /** * Static class for building and using {@link SerializableMatcher} instances. * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SourceTestUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SourceTestUtils.java index 9ce9c5e518da..e38e1af45663 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SourceTestUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SourceTestUtils.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.testing; import static com.google.common.base.Preconditions.checkNotNull; - import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertEquals; @@ -27,21 +26,7 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.io.BoundedSource; -import org.apache.beam.sdk.io.BoundedSource.BoundedReader; -import org.apache.beam.sdk.io.Source; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.values.KV; - import com.google.common.collect.ImmutableList; - -import org.joda.time.Instant; -import org.junit.Assert; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -52,8 +37,18 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; - import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.io.BoundedSource.BoundedReader; +import org.apache.beam.sdk.io.Source; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.values.KV; +import org.joda.time.Instant; +import org.junit.Assert; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Helper functions and test harnesses for checking correctness of {@link Source} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/StaticWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/StaticWindows.java index 08d23559bb50..949ecac43c81 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/StaticWindows.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/StaticWindows.java @@ -19,17 +19,15 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.NonMergingWindowFn; -import org.apache.beam.sdk.transforms.windowing.WindowFn; - import com.google.common.base.Supplier; import com.google.common.collect.Iterables; - import java.util.Collection; import java.util.Collections; import java.util.Objects; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.NonMergingWindowFn; +import org.apache.beam.sdk.transforms.windowing.WindowFn; /** * A {@link WindowFn} that assigns all elements to a static collection of diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java index 98cdeba0bb50..f1bf09d9c215 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java @@ -17,35 +17,31 @@ */ package org.apache.beam.sdk.testing; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.PipelineResult; -import org.apache.beam.sdk.options.ApplicationNameOptions; -import org.apache.beam.sdk.options.GcpOptions; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.options.PipelineOptions.CheckEnabled; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.runners.PipelineRunner; -import org.apache.beam.sdk.util.IOChannelUtils; -import org.apache.beam.sdk.util.TestCredential; - -import com.google.common.base.Optional; -import com.google.common.base.Strings; -import com.google.common.collect.Iterators; - import com.fasterxml.jackson.core.JsonParser; import com.fasterxml.jackson.core.TreeNode; import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.ObjectNode; -import org.junit.experimental.categories.Category; - +import com.google.common.base.Optional; +import com.google.common.base.Strings; +import com.google.common.collect.Iterators; import java.io.IOException; import java.lang.reflect.Method; import java.util.ArrayList; import java.util.Iterator; import java.util.Map.Entry; - import javax.annotation.Nullable; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.options.ApplicationNameOptions; +import org.apache.beam.sdk.options.GcpOptions; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptions.CheckEnabled; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.runners.PipelineRunner; +import org.apache.beam.sdk.util.IOChannelUtils; +import org.apache.beam.sdk.util.TestCredential; +import org.junit.experimental.categories.Category; /** * A creator of test pipelines that can be used inside of tests that can be diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipelineOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipelineOptions.java index c4596c152c5a..ff553bafa85e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipelineOptions.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipelineOptions.java @@ -21,7 +21,6 @@ import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.DefaultValueFactory; import org.apache.beam.sdk.options.PipelineOptions; - import org.hamcrest.BaseMatcher; import org.hamcrest.Description; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestStream.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestStream.java index e2eda3222855..e2730edfe6f4 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestStream.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestStream.java @@ -21,6 +21,16 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.auto.value.AutoValue; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Collections; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.DurationCoder; @@ -36,24 +46,10 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TimestampedValue.TimestampedValueCoder; - -import com.google.auto.value.AutoValue; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.ImmutableList; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - import org.joda.time.Duration; import org.joda.time.Instant; import org.joda.time.ReadableDuration; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.Collections; -import java.util.List; - /** * A testing input that generates an unbounded {@link PCollection} of elements, advancing the * watermark and processing time as elements are emitted. After all of the specified elements are diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/WindowFnTestUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/WindowFnTestUtils.java index 127721affcdd..63e7903b8e64 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/WindowFnTestUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/WindowFnTestUtils.java @@ -22,16 +22,6 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.transforms.windowing.IntervalWindow; -import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; -import org.apache.beam.sdk.transforms.windowing.OutputTimeFns; -import org.apache.beam.sdk.transforms.windowing.WindowFn; - -import org.joda.time.Instant; -import org.joda.time.ReadableInstant; - import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -41,8 +31,15 @@ import java.util.List; import java.util.Map; import java.util.Set; - import javax.annotation.Nullable; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.GlobalWindow; +import org.apache.beam.sdk.transforms.windowing.IntervalWindow; +import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; +import org.apache.beam.sdk.transforms.windowing.OutputTimeFns; +import org.apache.beam.sdk.transforms.windowing.WindowFn; +import org.joda.time.Instant; +import org.joda.time.ReadableInstant; /** * A utility class for testing {@link WindowFn}s. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/WindowSupplier.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/WindowSupplier.java index 62bc09ffc275..96091ef87fde 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/WindowSupplier.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/WindowSupplier.java @@ -17,17 +17,15 @@ */ package org.apache.beam.sdk.testing; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.CoderUtils; - import com.google.common.base.Supplier; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; - import java.io.Serializable; import java.util.Collection; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.CoderUtils; /** * A {@link Supplier} that returns a static set of {@link BoundedWindow BoundedWindows}. The diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AppliedPTransform.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AppliedPTransform.java index fdbde74e4b13..d80c116c175c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AppliedPTransform.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AppliedPTransform.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.transforms; -import org.apache.beam.sdk.values.PInput; -import org.apache.beam.sdk.values.POutput; - import com.google.common.base.MoreObjects; import com.google.common.base.Objects; +import org.apache.beam.sdk.values.PInput; +import org.apache.beam.sdk.values.POutput; /** * Represents the application of a {@link PTransform} to a specific input to produce diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java index fa87ed89fb0c..656bd7bbac36 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java @@ -19,24 +19,9 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.coders.BigEndianIntegerCoder; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.coders.CoderRegistry; -import org.apache.beam.sdk.coders.CustomCoder; -import org.apache.beam.sdk.coders.ListCoder; -import org.apache.beam.sdk.transforms.Combine.AccumulatingCombineFn; -import org.apache.beam.sdk.transforms.Combine.AccumulatingCombineFn.Accumulator; -import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.util.WeightedValue; -import org.apache.beam.sdk.util.common.ElementByteSizeObserver; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PCollection; - import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import com.google.common.collect.UnmodifiableIterator; - import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; @@ -50,8 +35,20 @@ import java.util.Iterator; import java.util.List; import java.util.PriorityQueue; - import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.BigEndianIntegerCoder; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.coders.CustomCoder; +import org.apache.beam.sdk.coders.ListCoder; +import org.apache.beam.sdk.transforms.Combine.AccumulatingCombineFn; +import org.apache.beam.sdk.transforms.Combine.AccumulatingCombineFn.Accumulator; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.util.WeightedValue; +import org.apache.beam.sdk.util.common.ElementByteSizeObserver; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; /** * {@code PTransform}s for getting an idea of a {@code PCollection}'s diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateUnique.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateUnique.java index 2fa24277661c..71c2158fa42c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateUnique.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateUnique.java @@ -17,6 +17,16 @@ */ package org.apache.beam.sdk.transforms; +import com.google.common.hash.Hashing; +import com.google.common.hash.HashingOutputStream; +import com.google.common.io.ByteStreams; +import java.io.IOException; +import java.io.Serializable; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.PriorityQueue; +import org.apache.avro.reflect.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.Coder.Context; import org.apache.beam.sdk.coders.CoderException; @@ -28,19 +38,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; -import com.google.common.hash.Hashing; -import com.google.common.hash.HashingOutputStream; -import com.google.common.io.ByteStreams; - -import org.apache.avro.reflect.Nullable; - -import java.io.IOException; -import java.io.Serializable; -import java.util.Arrays; -import java.util.Iterator; -import java.util.List; -import java.util.PriorityQueue; - /** * {@code PTransform}s for estimating the number of distinct elements * in a {@code PCollection}, or the number of distinct values diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java index 56c0bc4e5c6e..26f0f660f074 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java @@ -19,6 +19,18 @@ import static com.google.common.base.Preconditions.checkState; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.ThreadLocalRandom; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; @@ -58,21 +70,6 @@ import org.apache.beam.sdk.values.TupleTagList; import org.apache.beam.sdk.values.TypeDescriptor; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Iterator; -import java.util.List; -import java.util.concurrent.ThreadLocalRandom; - /** * {@code PTransform}s for combining {@code PCollection} elements * globally and per-key. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFnBase.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFnBase.java index c73ba542c359..f98ec96b782e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFnBase.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFnBase.java @@ -17,6 +17,10 @@ */ package org.apache.beam.sdk.transforms; +import com.google.common.collect.ImmutableMap; +import java.io.Serializable; +import java.lang.reflect.Type; +import java.lang.reflect.TypeVariable; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; @@ -28,12 +32,6 @@ import org.apache.beam.sdk.transforms.display.HasDisplayData; import org.apache.beam.sdk.values.TypeDescriptor; -import com.google.common.collect.ImmutableMap; - -import java.io.Serializable; -import java.lang.reflect.Type; -import java.lang.reflect.TypeVariable; - /** * This class contains the shared interfaces and abstract classes for different types of combine * functions. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFns.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFns.java index 61f4888ebb2d..9fa8ded6b240 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFns.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFns.java @@ -19,6 +19,24 @@ import static com.google.common.base.Preconditions.checkArgument; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Multimap; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; @@ -37,27 +55,6 @@ import org.apache.beam.sdk.util.PropertyNames; import org.apache.beam.sdk.values.TupleTag; -import com.google.common.collect.ArrayListMultimap; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.common.collect.Multimap; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.Serializable; -import java.util.Collection; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; - /** * Static utility methods that create combine function instances. */ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Count.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Count.java index 195c5d17ed88..b393a303951d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Count.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Count.java @@ -17,6 +17,12 @@ */ package org.apache.beam.sdk.transforms; +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.UTFDataFormatException; +import java.util.Iterator; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.CoderRegistry; @@ -26,13 +32,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; -import java.io.EOFException; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.UTFDataFormatException; -import java.util.Iterator; - /** * {@code PTransorm}s to count the elements in a {@link PCollection}. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java index 08d0a7ae356b..e261db2b30f9 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java @@ -19,6 +19,20 @@ import static com.google.common.base.Preconditions.checkArgument; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Function; +import com.google.common.base.Optional; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Objects; +import javax.annotation.Nullable; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; @@ -37,26 +51,8 @@ import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TimestampedValue.TimestampedValueCoder; import org.apache.beam.sdk.values.TypeDescriptor; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; -import com.google.common.base.Optional; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; - import org.joda.time.Instant; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.NoSuchElementException; -import java.util.Objects; - -import javax.annotation.Nullable; - /** * {@code Create} takes a collection of elements of type {@code T} * known when the pipeline is constructed and returns a diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java index 2348783f9f61..9f898261c7bb 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java @@ -21,6 +21,14 @@ import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; +import java.io.Serializable; +import java.lang.annotation.Documented; +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; +import java.util.HashMap; +import java.util.Map; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.OldDoFn.DelegatingAggregator; @@ -32,19 +40,9 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TypeDescriptor; - import org.joda.time.Duration; import org.joda.time.Instant; -import java.io.Serializable; -import java.lang.annotation.Documented; -import java.lang.annotation.ElementType; -import java.lang.annotation.Retention; -import java.lang.annotation.RetentionPolicy; -import java.lang.annotation.Target; -import java.util.HashMap; -import java.util.Map; - /** * The argument to {@link ParDo} providing the code to use to process * elements of the input diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java index 71a148ffafe9..4803d77f57f1 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.transforms; +import java.io.IOException; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; @@ -28,12 +29,9 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TypeDescriptor; - import org.joda.time.Duration; import org.joda.time.Instant; -import java.io.IOException; - /** * Utility class containing adapters for running a {@link DoFn} as an {@link OldDoFn}. * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java index 4cd410a19597..82c12938fa68 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java @@ -17,6 +17,19 @@ */ package org.apache.beam.sdk.transforms; +import com.google.common.base.Function; +import com.google.common.base.MoreObjects; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; @@ -34,20 +47,7 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; -import com.google.common.base.Function; -import com.google.common.base.MoreObjects; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; import org.joda.time.Instant; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; /** * A harness for unit-testing a {@link OldDoFn}. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/FlatMapElements.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/FlatMapElements.java index 2837c40cc3a6..b590d459f9e1 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/FlatMapElements.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/FlatMapElements.java @@ -17,13 +17,12 @@ */ package org.apache.beam.sdk.transforms; +import java.lang.reflect.ParameterizedType; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.sdk.values.TypeDescriptors; -import java.lang.reflect.ParameterizedType; - /** * {@code PTransform}s for mapping a simple function that returns iterables over the elements of a * {@link PCollection} and merging the results. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/IntraBundleParallelization.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/IntraBundleParallelization.java index b5fe60f330d0..1eef0e1b0647 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/IntraBundleParallelization.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/IntraBundleParallelization.java @@ -19,6 +19,10 @@ import static com.google.common.base.Preconditions.checkArgument; +import com.google.common.base.Throwables; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Semaphore; +import java.util.concurrent.atomic.AtomicReference; import org.apache.beam.sdk.options.GcsOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Combine.CombineFn; @@ -30,15 +34,8 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TypeDescriptor; - -import com.google.common.base.Throwables; - import org.joda.time.Instant; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Semaphore; -import java.util.concurrent.atomic.AtomicReference; - /** * Provides multi-threading of {@link OldDoFn}s, using threaded execution to * process multiple elements concurrently within a bundle. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Max.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Max.java index eed13fbb70b4..c44d9b634671 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Max.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Max.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.transforms; -import org.apache.beam.sdk.transforms.Combine.BinaryCombineFn; -import org.apache.beam.sdk.transforms.display.DisplayData; - import java.io.Serializable; import java.util.Comparator; +import org.apache.beam.sdk.transforms.Combine.BinaryCombineFn; +import org.apache.beam.sdk.transforms.display.DisplayData; /** * {@code PTransform}s for computing the maximum of the elements in a {@code PCollection}, or the diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Mean.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Mean.java index 5005f6bafb57..1a0791f99309 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Mean.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Mean.java @@ -17,6 +17,11 @@ */ package org.apache.beam.sdk.transforms; +import com.google.common.base.MoreObjects; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Objects; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.BigEndianLongCoder; import org.apache.beam.sdk.coders.Coder; @@ -25,13 +30,6 @@ import org.apache.beam.sdk.coders.DoubleCoder; import org.apache.beam.sdk.transforms.Combine.AccumulatingCombineFn.Accumulator; -import com.google.common.base.MoreObjects; - -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.Objects; - /** * {@code PTransform}s for computing the arithmetic mean * (a.k.a. average) of the elements in a {@code PCollection}, or the diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Min.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Min.java index 9c9d14f3ad01..f04677903833 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Min.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Min.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.transforms; -import org.apache.beam.sdk.transforms.Combine.BinaryCombineFn; -import org.apache.beam.sdk.transforms.display.DisplayData; - import java.io.Serializable; import java.util.Comparator; +import org.apache.beam.sdk.transforms.Combine.BinaryCombineFn; +import org.apache.beam.sdk.transforms.display.DisplayData; /** * {@code PTransform}s for computing the minimum of the elements in a {@code PCollection}, or the diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java index 84cd9978c621..474efef2c3d3 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java @@ -21,6 +21,14 @@ import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; +import com.google.common.base.MoreObjects; +import java.io.Serializable; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import java.util.UUID; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.options.PipelineOptions; @@ -33,20 +41,9 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TypeDescriptor; - -import com.google.common.base.MoreObjects; - import org.joda.time.Duration; import org.joda.time.Instant; -import java.io.Serializable; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; -import java.util.Objects; -import java.util.UUID; - /** * The argument to {@link ParDo} providing the code to use to process * elements of the input diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java index 12ab54d0e370..19abef90cea1 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java @@ -17,6 +17,9 @@ */ package org.apache.beam.sdk.transforms; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; @@ -27,10 +30,6 @@ import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.TypedPValue; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import java.io.Serializable; - /** * A {@code PTransform} is an operation that takes an * {@code InputT} (some subtype of {@link PInput}) and produces an diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java index af500ba0c9c3..5efbe9f19024 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java @@ -17,6 +17,10 @@ */ package org.apache.beam.sdk.transforms; +import com.google.common.collect.ImmutableList; +import java.io.Serializable; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.coders.CannotProvideCoderException; @@ -34,12 +38,6 @@ import org.apache.beam.sdk.values.TupleTagList; import org.apache.beam.sdk.values.TypedPValue; -import com.google.common.collect.ImmutableList; - -import java.io.Serializable; -import java.util.Arrays; -import java.util.List; - /** * {@link ParDo} is the core element-wise transform in Google Cloud * Dataflow, invoking a user-specified function on each of the elements of the input diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Partition.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Partition.java index 05c94700245a..9247942f36de 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Partition.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Partition.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.transforms; +import java.io.Serializable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.values.PCollection; @@ -25,8 +26,6 @@ import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; -import java.io.Serializable; - /** * {@code Partition} takes a {@code PCollection} and a * {@code PartitionFn}, uses the {@code PartitionFn} to split the diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sample.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sample.java index 12ff2b90b495..eca987a94227 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sample.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sample.java @@ -19,6 +19,9 @@ import static com.google.common.base.Preconditions.checkArgument; +import java.util.ArrayList; +import java.util.List; +import java.util.Random; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; @@ -31,10 +34,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; -import java.util.ArrayList; -import java.util.List; -import java.util.Random; - /** * {@code PTransform}s for taking samples of the elements in a * {@code PCollection}, or samples of the values associated with each diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Top.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Top.java index 04cf9e33f6f9..992a341059a6 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Top.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Top.java @@ -19,6 +19,15 @@ import static com.google.common.base.Preconditions.checkArgument; +import com.google.common.collect.Lists; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.List; +import java.util.PriorityQueue; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.CoderRegistry; @@ -34,17 +43,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; -import com.google.common.collect.Lists; - -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Comparator; -import java.util.List; -import java.util.PriorityQueue; - /** * {@code PTransform}s for finding the largest (or smallest) set * of elements in a {@code PCollection}, or the largest (or smallest) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/View.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/View.java index 7a97c13d336c..d750efc9fedc 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/View.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/View.java @@ -17,15 +17,14 @@ */ package org.apache.beam.sdk.transforms; +import java.util.List; +import java.util.Map; import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.util.PCollectionViews; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; -import java.util.List; -import java.util.Map; - /** * Transforms for creating {@link PCollectionView PCollectionViews} from * {@link PCollection PCollections} (to read them as side inputs). diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ViewFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ViewFn.java index aa3cb0d628e6..767e58e243d4 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ViewFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ViewFn.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.transforms; +import java.io.Serializable; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; -import java.io.Serializable; - /** * A function to adapt a primitive "view" of a {@link PCollection} - some materialization * specified in the Beam model and implemented by the runner - to a user-facing view type diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithTimestamps.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithTimestamps.java index 7b395f5e8dbf..00ac8e4a392d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithTimestamps.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithTimestamps.java @@ -24,7 +24,6 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.values.PCollection; - import org.joda.time.Duration; import org.joda.time.Instant; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java index 5dcaec892379..0b92d9fce394 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java @@ -20,27 +20,24 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; -import org.apache.beam.sdk.transforms.PTransform; - +import com.fasterxml.jackson.annotation.JsonGetter; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonValue; import com.google.auto.value.AutoValue; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import com.google.common.collect.Sets; - -import com.fasterxml.jackson.annotation.JsonGetter; -import com.fasterxml.jackson.annotation.JsonInclude; -import com.fasterxml.jackson.annotation.JsonValue; -import org.apache.avro.reflect.Nullable; -import org.joda.time.Duration; -import org.joda.time.Instant; -import org.joda.time.format.DateTimeFormatter; -import org.joda.time.format.ISODateTimeFormat; - import java.io.Serializable; import java.util.Collection; import java.util.Map; import java.util.Objects; import java.util.Set; +import org.apache.avro.reflect.Nullable; +import org.apache.beam.sdk.transforms.PTransform; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.joda.time.format.DateTimeFormatter; +import org.joda.time.format.ISODateTimeFormat; /** * Static display data associated with a pipeline component. Display data is useful for diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java index aa26cbbf68e2..10ba3c9489da 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java @@ -17,10 +17,21 @@ */ package org.apache.beam.sdk.transforms.join; -import static org.apache.beam.sdk.util.Structs.addObject; - import static com.google.common.base.Preconditions.checkArgument; +import static org.apache.beam.sdk.util.Structs.addObject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.Iterators; +import com.google.common.collect.PeekingIterator; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.Objects; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.IterableCoder; @@ -31,24 +42,9 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; - -import com.google.common.collect.Iterators; -import com.google.common.collect.PeekingIterator; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Iterator; -import java.util.List; -import java.util.Objects; - /** * A row result of a {@link CoGroupByKey}. This is a tuple of {@link Iterable}s produced for * a given key, and these can be accessed in different ways. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResultSchema.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResultSchema.java index 29a0c7d731d4..276cccb3a053 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResultSchema.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResultSchema.java @@ -19,18 +19,16 @@ import static org.apache.beam.sdk.util.Structs.addList; -import org.apache.beam.sdk.util.CloudObject; -import org.apache.beam.sdk.util.PropertyNames; -import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.sdk.values.TupleTagList; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; - import java.io.Serializable; import java.util.ArrayList; import java.util.HashMap; import java.util.List; +import org.apache.beam.sdk.util.CloudObject; +import org.apache.beam.sdk.util.PropertyNames; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TupleTagList; /** * A schema for the results of a {@link CoGroupByKey}. This maintains the full diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGroupByKey.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGroupByKey.java index cb06f95354fe..9546b291d4fc 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGroupByKey.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGroupByKey.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.transforms.join; +import java.util.ArrayList; +import java.util.List; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.transforms.DoFn; @@ -30,9 +32,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; -import java.util.ArrayList; -import java.util.List; - /** * A {@link PTransform} that performs a {@link CoGroupByKey} on a tuple * of tables. A {@link CoGroupByKey} groups results from all diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/KeyedPCollectionTuple.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/KeyedPCollectionTuple.java index 0852c9c25b29..67b819fddc2c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/KeyedPCollectionTuple.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/KeyedPCollectionTuple.java @@ -17,6 +17,9 @@ */ package org.apache.beam.sdk.transforms.join; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; @@ -29,10 +32,6 @@ import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; - /** * An immutable tuple of keyed {@link PCollection PCollections} * with key type K. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/UnionCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/UnionCoder.java index 29240e7bb863..72ac6e881fff 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/UnionCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/UnionCoder.java @@ -17,20 +17,18 @@ */ package org.apache.beam.sdk.transforms.join; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.coders.StandardCoder; -import org.apache.beam.sdk.util.PropertyNames; -import org.apache.beam.sdk.util.VarInt; -import org.apache.beam.sdk.util.common.ElementByteSizeObserver; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.util.List; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.coders.StandardCoder; +import org.apache.beam.sdk.util.PropertyNames; +import org.apache.beam.sdk.util.VarInt; +import org.apache.beam.sdk.util.common.ElementByteSizeObserver; /** * A UnionCoder encodes RawUnionValues. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java index 73874d767d3c..68e2ca99886e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java @@ -20,14 +20,14 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.DoFn.FinishBundle; -import org.apache.beam.sdk.transforms.DoFn.ProcessElement; -import org.apache.beam.sdk.transforms.DoFn.Setup; -import org.apache.beam.sdk.transforms.DoFn.StartBundle; -import org.apache.beam.sdk.transforms.DoFn.Teardown; -import org.apache.beam.sdk.util.UserCodeException; - +import java.lang.reflect.Constructor; +import java.lang.reflect.InvocationTargetException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.EnumMap; +import java.util.LinkedHashMap; +import java.util.Map; +import javax.annotation.Nullable; import net.bytebuddy.ByteBuddy; import net.bytebuddy.NamingStrategy; import net.bytebuddy.description.field.FieldDescription; @@ -53,16 +53,13 @@ import net.bytebuddy.jar.asm.MethodVisitor; import net.bytebuddy.jar.asm.Opcodes; import net.bytebuddy.matcher.ElementMatchers; - -import java.lang.reflect.Constructor; -import java.lang.reflect.InvocationTargetException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.EnumMap; -import java.util.LinkedHashMap; -import java.util.Map; - -import javax.annotation.Nullable; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.DoFn.FinishBundle; +import org.apache.beam.sdk.transforms.DoFn.ProcessElement; +import org.apache.beam.sdk.transforms.DoFn.Setup; +import org.apache.beam.sdk.transforms.DoFn.StartBundle; +import org.apache.beam.sdk.transforms.DoFn.Teardown; +import org.apache.beam.sdk.util.UserCodeException; /** Dynamically generates {@link DoFnInvoker} instances for invoking a {@link DoFn}. */ public class DoFnInvokers { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java index 6730140ac309..181c08882049 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java @@ -17,15 +17,12 @@ */ package org.apache.beam.sdk.transforms.reflect; -import org.apache.beam.sdk.transforms.DoFn; - import com.google.auto.value.AutoValue; - import java.lang.reflect.Method; import java.util.Collections; import java.util.List; - import javax.annotation.Nullable; +import org.apache.beam.sdk.transforms.DoFn; /** * Describes the signature of a {@link DoFn}, in particular, which features it uses, which extra diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java index 80b3b4f5b825..7e482d5df6e3 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java @@ -20,14 +20,9 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.common.ReflectHelpers; - import com.google.common.annotations.VisibleForTesting; import com.google.common.reflect.TypeParameter; import com.google.common.reflect.TypeToken; - import java.lang.annotation.Annotation; import java.lang.reflect.Method; import java.lang.reflect.Modifier; @@ -41,6 +36,9 @@ import java.util.LinkedHashSet; import java.util.List; import java.util.Map; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.common.ReflectHelpers; /** * Parses a {@link DoFn} and computes its {@link DoFnSignature}. See {@link #getOrParseSignature}. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterAll.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterAll.java index 27dd1293ad32..cc8c97fa14bf 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterAll.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterAll.java @@ -19,17 +19,14 @@ import static com.google.common.base.Preconditions.checkArgument; +import com.google.common.base.Joiner; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger; import org.apache.beam.sdk.util.ExecutableTrigger; - -import com.google.common.base.Joiner; - import org.joda.time.Instant; -import java.util.Arrays; -import java.util.List; - /** * Create a {@link Trigger} that fires and finishes once after all of its sub-triggers have fired. */ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterDelayFromFirstElement.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterDelayFromFirstElement.java index a0fc04d64288..c4bc94615dc3 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterDelayFromFirstElement.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterDelayFromFirstElement.java @@ -17,6 +17,12 @@ */ package org.apache.beam.sdk.transforms.windowing; +import com.google.common.collect.ImmutableList; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import java.util.List; +import java.util.Locale; +import java.util.Objects; +import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.coders.InstantCoder; import org.apache.beam.sdk.transforms.Combine; @@ -31,21 +37,11 @@ import org.apache.beam.sdk.util.state.StateMerging; import org.apache.beam.sdk.util.state.StateTag; import org.apache.beam.sdk.util.state.StateTags; - -import com.google.common.collect.ImmutableList; - import org.joda.time.Duration; import org.joda.time.Instant; import org.joda.time.format.PeriodFormat; import org.joda.time.format.PeriodFormatter; -import java.util.List; -import java.util.Locale; -import java.util.Objects; -import javax.annotation.Nullable; - -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; - /** * A base class for triggers that happen after a processing time delay from the arrival * of the first element in a pane. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterEach.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterEach.java index dc1e74b09958..629c640eabdd 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterEach.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterEach.java @@ -19,15 +19,12 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.annotations.Experimental; -import org.apache.beam.sdk.util.ExecutableTrigger; - import com.google.common.base.Joiner; - -import org.joda.time.Instant; - import java.util.Arrays; import java.util.List; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.util.ExecutableTrigger; +import org.joda.time.Instant; /** * A composite {@link Trigger} that executes its sub-triggers in order. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterFirst.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterFirst.java index 1034cac7d8b9..6b06cfa09eb3 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterFirst.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterFirst.java @@ -19,17 +19,14 @@ import static com.google.common.base.Preconditions.checkArgument; +import com.google.common.base.Joiner; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger; import org.apache.beam.sdk.util.ExecutableTrigger; - -import com.google.common.base.Joiner; - import org.joda.time.Instant; -import java.util.Arrays; -import java.util.List; - /** * Create a composite {@link Trigger} that fires once after at least one of its sub-triggers have * fired. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterPane.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterPane.java index 73289877c00d..8c128dd32bc2 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterPane.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterPane.java @@ -17,6 +17,9 @@ */ package org.apache.beam.sdk.transforms.windowing; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import java.util.List; +import java.util.Objects; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.transforms.Sum; @@ -27,14 +30,8 @@ import org.apache.beam.sdk.util.state.StateMerging; import org.apache.beam.sdk.util.state.StateTag; import org.apache.beam.sdk.util.state.StateTags; - import org.joda.time.Instant; -import java.util.List; -import java.util.Objects; - -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; - /** * {@link Trigger}s that fire based on properties of the elements in the current pane. */ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTime.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTime.java index 4c792df51829..f551118b22c0 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTime.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTime.java @@ -17,17 +17,14 @@ */ package org.apache.beam.sdk.transforms.windowing; +import java.util.List; +import java.util.Objects; +import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.util.TimeDomain; - import org.joda.time.Instant; -import java.util.List; -import java.util.Objects; - -import javax.annotation.Nullable; - /** * {@code AfterProcessingTime} triggers fire based on the current processing time. They operate in * the real-time domain. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTime.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTime.java index e8e0fb6f707e..59ece1073c20 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTime.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTime.java @@ -17,17 +17,13 @@ */ package org.apache.beam.sdk.transforms.windowing; -import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.sdk.util.TimeDomain; - import com.google.common.base.Objects; - -import org.joda.time.Instant; - import java.util.Collections; import java.util.List; - import javax.annotation.Nullable; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.util.TimeDomain; +import org.joda.time.Instant; class AfterSynchronizedProcessingTime extends AfterDelayFromFirstElement { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java index 0d2a878e56b2..9690be838526 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java @@ -19,20 +19,16 @@ import static com.google.common.base.Preconditions.checkNotNull; +import com.google.common.collect.ImmutableList; +import java.util.List; +import java.util.Objects; +import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger; import org.apache.beam.sdk.util.ExecutableTrigger; import org.apache.beam.sdk.util.TimeDomain; - -import com.google.common.collect.ImmutableList; - import org.joda.time.Instant; -import java.util.List; -import java.util.Objects; - -import javax.annotation.Nullable; - /** *

        {@code AfterWatermark} triggers fire based on progress of the system watermark. This time is a * lower-bound, sometimes heuristically established, on event times that have been fully processed diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java index 50ee9b5552c4..3654074174c1 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java @@ -17,9 +17,8 @@ */ package org.apache.beam.sdk.transforms.windowing; -import org.joda.time.Instant; - import java.util.concurrent.TimeUnit; +import org.joda.time.Instant; /** * A {@code BoundedWindow} represents a finite grouping of elements, with an diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/CalendarWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/CalendarWindows.java index 9653ae84ea94..fada50ad5a4d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/CalendarWindows.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/CalendarWindows.java @@ -19,7 +19,6 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.display.DisplayData; - import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.joda.time.Days; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/DefaultTrigger.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/DefaultTrigger.java index fcea3337b8a4..d6b72ef60352 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/DefaultTrigger.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/DefaultTrigger.java @@ -17,13 +17,11 @@ */ package org.apache.beam.sdk.transforms.windowing; +import java.util.List; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.util.TimeDomain; - import org.joda.time.Instant; -import java.util.List; - /** * A trigger that is equivalent to {@code Repeatedly.forever(AfterWatermark.pastEndOfWindow())}. * See {@link Repeatedly#forever} and {@link AfterWatermark#pastEndOfWindow} for more details. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/FixedWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/FixedWindows.java index 1e439ff389b3..8683a600e204 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/FixedWindows.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/FixedWindows.java @@ -17,14 +17,12 @@ */ package org.apache.beam.sdk.transforms.windowing; +import java.util.Objects; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.display.DisplayData; - import org.joda.time.Duration; import org.joda.time.Instant; -import java.util.Objects; - /** * A {@link WindowFn} that windows values into fixed-size timestamp-based windows. * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindow.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindow.java index a4712fef7636..58b059aab67c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindow.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindow.java @@ -17,14 +17,12 @@ */ package org.apache.beam.sdk.transforms.windowing; +import java.io.InputStream; +import java.io.OutputStream; import org.apache.beam.sdk.coders.AtomicCoder; - import org.joda.time.Duration; import org.joda.time.Instant; -import java.io.InputStream; -import java.io.OutputStream; - /** * The default window into which all data is placed (via {@link GlobalWindows}). */ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindows.java index 002bf2e0b419..aba00a312291 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindows.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindows.java @@ -17,12 +17,10 @@ */ package org.apache.beam.sdk.transforms.windowing; -import org.apache.beam.sdk.coders.Coder; - -import org.joda.time.Instant; - import java.util.Collection; import java.util.Collections; +import org.apache.beam.sdk.coders.Coder; +import org.joda.time.Instant; /** * Default {@link WindowFn} that assigns all data to the same window. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/IntervalWindow.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/IntervalWindow.java index 45898e00d26b..af987966be92 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/IntervalWindow.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/IntervalWindow.java @@ -17,21 +17,19 @@ */ package org.apache.beam.sdk.transforms.windowing; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.DurationCoder; import org.apache.beam.sdk.coders.InstantCoder; - -import com.fasterxml.jackson.annotation.JsonCreator; import org.joda.time.Duration; import org.joda.time.Instant; import org.joda.time.ReadableDuration; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; - /** * An implementation of {@link BoundedWindow} that represents an interval from * {@link #start} (inclusive) to {@link #end} (exclusive). diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/InvalidWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/InvalidWindows.java index e0d65e5e98e5..cbbc937eecb6 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/InvalidWindows.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/InvalidWindows.java @@ -17,12 +17,10 @@ */ package org.apache.beam.sdk.transforms.windowing; +import java.util.Collection; import org.apache.beam.sdk.coders.Coder; - import org.joda.time.Instant; -import java.util.Collection; - /** * A {@link WindowFn} that represents an invalid pipeline state. * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Never.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Never.java index 7267d002715b..e1f5d4da0c82 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Never.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Never.java @@ -17,13 +17,11 @@ */ package org.apache.beam.sdk.transforms.windowing; +import java.util.List; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger; - import org.joda.time.Instant; -import java.util.List; - /** * A trigger which never fires. * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTrigger.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTrigger.java index c48f5f47daca..25b7b34b520b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTrigger.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTrigger.java @@ -17,14 +17,11 @@ */ package org.apache.beam.sdk.transforms.windowing; -import org.apache.beam.sdk.util.ExecutableTrigger; - import com.google.common.annotations.VisibleForTesting; - -import org.joda.time.Instant; - import java.util.Arrays; import java.util.List; +import org.apache.beam.sdk.util.ExecutableTrigger; +import org.joda.time.Instant; /** * Executes the {@code actual} trigger until it finishes or until the {@code until} trigger fires. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OutputTimeFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OutputTimeFn.java index 497a6fbd0c98..0efd278a644a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OutputTimeFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OutputTimeFn.java @@ -17,14 +17,11 @@ */ package org.apache.beam.sdk.transforms.windowing; -import org.apache.beam.sdk.annotations.Experimental; - import com.google.common.collect.Ordering; - -import org.joda.time.Instant; - import java.io.Serializable; import java.util.Objects; +import org.apache.beam.sdk.annotations.Experimental; +import org.joda.time.Instant; /** * (Experimental) A function from timestamps of input values to the timestamp for a diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OutputTimeFns.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OutputTimeFns.java index 363181b22df2..2bcd3192f5ea 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OutputTimeFns.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OutputTimeFns.java @@ -19,14 +19,11 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.annotations.Experimental; - import com.google.common.collect.Iterables; import com.google.common.collect.Ordering; - -import org.joda.time.Instant; - import javax.annotation.Nullable; +import org.apache.beam.sdk.annotations.Experimental; +import org.joda.time.Instant; /** * (Experimental) Static utility methods and provided implementations for diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PaneInfo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PaneInfo.java index 6ec17f991c0c..0c87e2271ec9 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PaneInfo.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PaneInfo.java @@ -20,20 +20,18 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; -import org.apache.beam.sdk.coders.AtomicCoder; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.transforms.GroupByKey; -import org.apache.beam.sdk.transforms.OldDoFn; -import org.apache.beam.sdk.util.VarInt; - import com.google.common.base.MoreObjects; import com.google.common.collect.ImmutableMap; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.util.Objects; +import org.apache.beam.sdk.coders.AtomicCoder; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.transforms.GroupByKey; +import org.apache.beam.sdk.transforms.OldDoFn; +import org.apache.beam.sdk.util.VarInt; /** * Provides information about the pane an element belongs to. Every pane is implicitly associated diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PartitioningWindowFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PartitioningWindowFn.java index da2f38ce6cdc..b84f891464be 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PartitioningWindowFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PartitioningWindowFn.java @@ -17,10 +17,9 @@ */ package org.apache.beam.sdk.transforms.windowing; -import org.joda.time.Instant; - import java.util.Arrays; import java.util.Collection; +import org.joda.time.Instant; /** * A {@link WindowFn} that places each value into exactly one window based on its timestamp and diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Repeatedly.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Repeatedly.java index 591bbf06a0ea..88587983dfc2 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Repeatedly.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Repeatedly.java @@ -17,12 +17,10 @@ */ package org.apache.beam.sdk.transforms.windowing; -import org.apache.beam.sdk.util.ExecutableTrigger; - -import org.joda.time.Instant; - import java.util.Arrays; import java.util.List; +import org.apache.beam.sdk.util.ExecutableTrigger; +import org.joda.time.Instant; /** * Repeat a trigger, either until some condition is met or forever. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Sessions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Sessions.java index 875db12918ae..14d68f972381 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Sessions.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Sessions.java @@ -17,14 +17,12 @@ */ package org.apache.beam.sdk.transforms.windowing; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.transforms.display.DisplayData; - -import org.joda.time.Duration; - import java.util.Arrays; import java.util.Collection; import java.util.Objects; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.joda.time.Duration; /** * A {@link WindowFn} windowing values into sessions separated by {@link #gapDuration}-long diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/SlidingWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/SlidingWindows.java index 99367f5557f4..ec21723384f4 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/SlidingWindows.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/SlidingWindows.java @@ -17,19 +17,17 @@ */ package org.apache.beam.sdk.transforms.windowing; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Objects; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.display.DisplayData; - import org.joda.time.Duration; import org.joda.time.Instant; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; -import java.util.Objects; - /** * A {@link WindowFn} that windows values into possibly overlapping fixed-size * timestamp-based windows. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Trigger.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Trigger.java index 86801e817799..a960aa4b7a00 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Trigger.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Trigger.java @@ -17,22 +17,18 @@ */ package org.apache.beam.sdk.transforms.windowing; -import org.apache.beam.sdk.annotations.Experimental; -import org.apache.beam.sdk.util.ExecutableTrigger; -import org.apache.beam.sdk.util.TimeDomain; -import org.apache.beam.sdk.util.state.MergingStateAccessor; -import org.apache.beam.sdk.util.state.StateAccessor; - import com.google.common.base.Joiner; - -import org.joda.time.Instant; - import java.io.Serializable; import java.util.ArrayList; import java.util.List; import java.util.Objects; - import javax.annotation.Nullable; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.util.ExecutableTrigger; +import org.apache.beam.sdk.util.TimeDomain; +import org.apache.beam.sdk.util.state.MergingStateAccessor; +import org.apache.beam.sdk.util.state.StateAccessor; +import org.joda.time.Instant; /** * {@code Trigger}s control when the elements for a specific key and window are output. As elements diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java index 9dd069cf9952..52b78584fcb4 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.transforms.windowing; +import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.coders.Coder; @@ -29,11 +30,8 @@ import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode; import org.apache.beam.sdk.values.PCollection; - import org.joda.time.Duration; -import javax.annotation.Nullable; - /** * {@code Window} logically divides up or groups the elements of a * {@link PCollection} into finite windows according to a {@link WindowFn}. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/WindowFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/WindowFn.java index d84866b6fb72..abb4ee083f20 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/WindowFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/WindowFn.java @@ -17,17 +17,15 @@ */ package org.apache.beam.sdk.transforms.windowing; +import java.io.Serializable; +import java.util.Collection; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.display.HasDisplayData; - import org.joda.time.Instant; -import java.io.Serializable; -import java.util.Collection; - /** * The argument to the {@link Window} transform used to assign elements into * windows and to determine how windows are merged. See {@link Window} for more diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ActiveWindowSet.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ActiveWindowSet.java index 02c12c040b2d..2e0af29732a1 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ActiveWindowSet.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ActiveWindowSet.java @@ -17,13 +17,11 @@ */ package org.apache.beam.sdk.util; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.WindowFn; - import com.google.common.annotations.VisibleForTesting; - import java.util.Collection; import java.util.Set; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.WindowFn; /** * Track which windows are active, and the state address window(s) under which their diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ApiSurface.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ApiSurface.java index 0d6874026cb2..3914bb0e10a1 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ApiSurface.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ApiSurface.java @@ -29,10 +29,6 @@ import com.google.common.reflect.Invokable; import com.google.common.reflect.Parameter; import com.google.common.reflect.TypeToken; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.IOException; import java.lang.annotation.Annotation; import java.lang.reflect.Constructor; @@ -49,6 +45,8 @@ import java.util.List; import java.util.Set; import java.util.regex.Pattern; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Represents the API surface of a package prefix. Used for accessing public classes, diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AppliedCombineFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AppliedCombineFn.java index 1e7d7c002e19..30b302c9953d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AppliedCombineFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AppliedCombineFn.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.util; +import com.google.common.annotations.VisibleForTesting; +import java.io.Serializable; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; @@ -25,10 +27,6 @@ import org.apache.beam.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext; import org.apache.beam.sdk.values.PCollectionView; -import com.google.common.annotations.VisibleForTesting; - -import java.io.Serializable; - /** * A {@link KeyedCombineFnWithContext} with a fixed accumulator coder. This is created from a * specific application of the {@link KeyedCombineFnWithContext}. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOff.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOff.java index df2897ee91e0..d8050e0a3219 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOff.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOff.java @@ -21,10 +21,8 @@ import com.google.api.client.util.BackOff; import com.google.api.client.util.NanoClock; - -import java.util.concurrent.TimeUnit; - import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import java.util.concurrent.TimeUnit; /** * Extension of {@link AttemptBoundedExponentialBackOff} that bounds the total time that the backoff diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AvroUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AvroUtils.java index 16fc6fa5b3fb..d855b344fe10 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AvroUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AvroUtils.java @@ -17,15 +17,14 @@ */ package org.apache.beam.sdk.util; -import org.apache.avro.file.DataFileConstants; -import org.apache.avro.io.BinaryDecoder; -import org.apache.avro.io.DecoderFactory; - import java.io.IOException; import java.io.InputStream; import java.nio.ByteBuffer; import java.nio.channels.Channels; import java.util.Arrays; +import org.apache.avro.file.DataFileConstants; +import org.apache.avro.io.BinaryDecoder; +import org.apache.avro.io.DecoderFactory; /** * A set of utilities for working with Avro files. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BaseExecutionContext.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BaseExecutionContext.java index dd36367f7546..dead76eb3d00 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BaseExecutionContext.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BaseExecutionContext.java @@ -17,16 +17,15 @@ */ package org.apache.beam.sdk.util; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.state.StateInternals; -import org.apache.beam.sdk.values.TupleTag; - import java.io.IOException; import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.Map; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.state.StateInternals; +import org.apache.beam.sdk.values.TupleTag; /** * Base class for implementations of {@link ExecutionContext}. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java index b406b1a56cc5..fde90af68369 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java @@ -17,14 +17,13 @@ */ package org.apache.beam.sdk.util; -import org.apache.beam.sdk.coders.AtomicCoder; -import org.apache.beam.sdk.coders.ByteArrayCoder; -import org.apache.beam.sdk.coders.CoderException; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.util.BitSet; +import org.apache.beam.sdk.coders.AtomicCoder; +import org.apache.beam.sdk.coders.ByteArrayCoder; +import org.apache.beam.sdk.coders.CoderException; /** * Coder for the BitSet used to track child-trigger finished states. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BucketingFunction.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BucketingFunction.java index e14aec862a0e..8740b2271077 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BucketingFunction.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BucketingFunction.java @@ -20,10 +20,9 @@ import static com.google.common.base.Preconditions.checkState; -import org.apache.beam.sdk.transforms.Combine; - import java.util.HashMap; import java.util.Map; +import org.apache.beam.sdk.transforms.Combine; /** * Keep track of the minimum/maximum/sum of a set of timestamped long values. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStream.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStream.java index 6335de47378d..c17d92d41f8a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStream.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStream.java @@ -17,13 +17,11 @@ */ package org.apache.beam.sdk.util; -import org.apache.beam.sdk.coders.Coder.Context; - import java.io.IOException; import java.io.OutputStream; import java.nio.ByteBuffer; - import javax.annotation.concurrent.NotThreadSafe; +import org.apache.beam.sdk.coders.Coder.Context; /** * Provides an efficient encoding for {@link Iterable}s containing small values by diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CloudKnownType.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CloudKnownType.java index ad79b989d923..b22b3c01286d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CloudKnownType.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CloudKnownType.java @@ -20,7 +20,6 @@ import java.util.Collections; import java.util.HashMap; import java.util.Map; - import javax.annotation.Nullable; /** A utility for manipulating well-known cloud types. */ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CloudObject.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CloudObject.java index 27722255e916..9cab453bdf89 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CloudObject.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CloudObject.java @@ -22,9 +22,7 @@ import com.google.api.client.json.GenericJson; import com.google.api.client.util.Key; - import java.util.Map; - import javax.annotation.Nullable; /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CoderUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CoderUtils.java index 520ba4a9a611..36bf78980e54 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CoderUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CoderUtils.java @@ -19,15 +19,6 @@ import static org.apache.beam.sdk.util.Structs.addList; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.coders.IterableCoder; -import org.apache.beam.sdk.coders.KvCoder; -import org.apache.beam.sdk.values.TypeDescriptor; - -import com.google.api.client.util.Base64; -import com.google.common.base.Throwables; - import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeInfo.As; import com.fasterxml.jackson.annotation.JsonTypeInfo.Id; @@ -38,7 +29,8 @@ import com.fasterxml.jackson.databind.jsontype.impl.TypeIdResolverBase; import com.fasterxml.jackson.databind.module.SimpleModule; import com.fasterxml.jackson.databind.type.TypeFactory; - +import com.google.api.client.util.Base64; +import com.google.common.base.Throwables; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -47,6 +39,11 @@ import java.lang.ref.SoftReference; import java.lang.reflect.ParameterizedType; import java.lang.reflect.TypeVariable; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.coders.IterableCoder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.values.TypeDescriptor; /** * Utilities for working with Coders. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineFnUtil.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineFnUtil.java index 351a158c1edf..a9a0178edf4e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineFnUtil.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineFnUtil.java @@ -17,6 +17,9 @@ */ package org.apache.beam.sdk.util; +import java.io.IOException; +import java.io.NotSerializableException; +import java.io.ObjectOutputStream; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; @@ -30,10 +33,6 @@ import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.util.state.StateContext; -import java.io.IOException; -import java.io.NotSerializableException; -import java.io.ObjectOutputStream; - /** * Static utility methods that create combine function instances. */ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CredentialFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CredentialFactory.java index 3283c8e0c528..ce29d9192aed 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CredentialFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CredentialFactory.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.util; import com.google.api.client.auth.oauth2.Credential; - import java.io.IOException; import java.security.GeneralSecurityException; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Credentials.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Credentials.java index 41065cde4114..1e77f4dc324a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Credentials.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Credentials.java @@ -19,8 +19,6 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.options.GcpOptions; - import com.google.api.client.auth.oauth2.Credential; import com.google.api.client.extensions.java6.auth.oauth2.AbstractPromptReceiver; import com.google.api.client.extensions.java6.auth.oauth2.AuthorizationCodeInstalledApp; @@ -34,10 +32,6 @@ import com.google.api.client.json.JsonFactory; import com.google.api.client.json.jackson2.JacksonFactory; import com.google.api.client.util.store.FileDataStoreFactory; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.File; import java.io.FileReader; import java.io.IOException; @@ -45,6 +39,9 @@ import java.util.Arrays; import java.util.Collection; import java.util.List; +import org.apache.beam.sdk.options.GcpOptions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Provides support for loading credentials. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DirectSideInputReader.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DirectSideInputReader.java index c8d360c09a8f..30e371656396 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DirectSideInputReader.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DirectSideInputReader.java @@ -17,14 +17,13 @@ */ package org.apache.beam.sdk.util; +import com.google.common.base.Predicate; +import com.google.common.collect.Iterables; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; -import com.google.common.base.Predicate; -import com.google.common.collect.Iterables; - /** * Basic side input reader wrapping a {@link PTuple} of side input iterables. Encapsulates * conversion according to the {@link PCollectionView} and projection to a particular diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExecutableTrigger.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExecutableTrigger.java index bb71fccabdda..088c499daa0b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExecutableTrigger.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExecutableTrigger.java @@ -20,13 +20,12 @@ import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.Trigger; -import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger; - import java.io.Serializable; import java.util.ArrayList; import java.util.List; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.Trigger; +import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger; /** * A wrapper around a trigger used during execution. While an actual trigger may appear multiple diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExecutionContext.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExecutionContext.java index 1c2f554559e8..82d900c53037 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExecutionContext.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExecutionContext.java @@ -17,14 +17,13 @@ */ package org.apache.beam.sdk.util; +import java.io.IOException; +import java.util.Collection; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.values.TupleTag; -import java.io.IOException; -import java.util.Collection; - /** * Context for the current execution. This is guaranteed to exist during processing, * but does not necessarily persist between different batches of work. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayInputStream.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayInputStream.java index eae3e40ad1a7..020c45308f97 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayInputStream.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayInputStream.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.util; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import java.io.ByteArrayInputStream; import java.io.IOException; -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; - /** * {@link ByteArrayInputStream} that allows accessing the entire internal buffer without copying. */ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStream.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStream.java index 48006ea22f4d..5a98f8419c3c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStream.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStream.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.util; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import java.io.ByteArrayOutputStream; import java.io.IOException; -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; - /** * {@link ByteArrayOutputStream} special cased to treat writes of a single byte-array specially. * When calling {@link #toByteArray()} after writing only one {@code byte[]} using diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FileIOChannelFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FileIOChannelFactory.java index 92f351bb3af2..a11231bae2f1 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FileIOChannelFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FileIOChannelFactory.java @@ -20,10 +20,6 @@ import com.google.common.base.Predicate; import com.google.common.base.Predicates; import com.google.common.collect.Iterables; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.BufferedOutputStream; import java.io.File; import java.io.FileInputStream; @@ -44,6 +40,8 @@ import java.util.LinkedList; import java.util.List; import java.util.regex.Matcher; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Implements {@link IOChannelFactory} for local files. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FinishedTriggersSet.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FinishedTriggersSet.java index 23aaa5467767..a9feb7300a17 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FinishedTriggersSet.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FinishedTriggersSet.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.util; import com.google.common.collect.Sets; - import java.util.Set; /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcpCredentialFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcpCredentialFactory.java index 98b82e2d39aa..0497e750bbe1 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcpCredentialFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcpCredentialFactory.java @@ -17,13 +17,11 @@ */ package org.apache.beam.sdk.util; -import org.apache.beam.sdk.options.GcpOptions; -import org.apache.beam.sdk.options.PipelineOptions; - import com.google.api.client.auth.oauth2.Credential; - import java.io.IOException; import java.security.GeneralSecurityException; +import org.apache.beam.sdk.options.GcpOptions; +import org.apache.beam.sdk.options.PipelineOptions; /** * Construct an oauth credential to be used by the SDK and the SDK workers. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsIOChannelFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsIOChannelFactory.java index 2122c6427bd0..14090e31c38e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsIOChannelFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsIOChannelFactory.java @@ -17,15 +17,14 @@ */ package org.apache.beam.sdk.util; -import org.apache.beam.sdk.options.GcsOptions; -import org.apache.beam.sdk.util.gcsfs.GcsPath; - import java.io.IOException; import java.nio.channels.ReadableByteChannel; import java.nio.channels.WritableByteChannel; import java.util.Collection; import java.util.LinkedList; import java.util.List; +import org.apache.beam.sdk.options.GcsOptions; +import org.apache.beam.sdk.util.gcsfs.GcsPath; /** * Implements IOChannelFactory for GCS. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsPathValidator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsPathValidator.java index 87f918109059..89363cec4cb2 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsPathValidator.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsPathValidator.java @@ -19,12 +19,11 @@ import static com.google.common.base.Preconditions.checkArgument; +import java.io.IOException; import org.apache.beam.sdk.options.GcsOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.gcsfs.GcsPath; -import java.io.IOException; - /** * GCP implementation of {@link PathValidator}. Only GCS paths are allowed. */ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java index 06685e577bf9..44a182eeb3d8 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java @@ -20,11 +20,6 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; -import org.apache.beam.sdk.options.DefaultValueFactory; -import org.apache.beam.sdk.options.GcsOptions; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.gcsfs.GcsPath; - import com.google.api.client.googleapis.batch.BatchRequest; import com.google.api.client.googleapis.batch.json.JsonBatchCallback; import com.google.api.client.googleapis.json.GoogleJsonError; @@ -50,10 +45,6 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.FileNotFoundException; import java.io.IOException; import java.nio.channels.SeekableByteChannel; @@ -70,8 +61,13 @@ import java.util.concurrent.TimeUnit; import java.util.regex.Matcher; import java.util.regex.Pattern; - import javax.annotation.Nullable; +import org.apache.beam.sdk.options.DefaultValueFactory; +import org.apache.beam.sdk.options.GcsOptions; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.util.gcsfs.GcsPath; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Provides operations on GCS. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IOChannelUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IOChannelUtils.java index 913824d821a8..16a6e952f6c4 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IOChannelUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IOChannelUtils.java @@ -17,9 +17,6 @@ */ package org.apache.beam.sdk.util; -import org.apache.beam.sdk.options.GcsOptions; -import org.apache.beam.sdk.options.PipelineOptions; - import java.io.FileNotFoundException; import java.io.IOException; import java.nio.channels.WritableByteChannel; @@ -32,6 +29,8 @@ import java.util.Set; import java.util.regex.Matcher; import java.util.regex.Pattern; +import org.apache.beam.sdk.options.GcsOptions; +import org.apache.beam.sdk.options.PipelineOptions; /** * Provides utilities for creating read and write channels. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IdentityWindowFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IdentityWindowFn.java index a3477e9f4f12..8ca1bfd77b5b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IdentityWindowFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IdentityWindowFn.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.util; +import java.util.Collection; +import java.util.Collections; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.GroupByKey; @@ -26,12 +28,8 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.values.PCollection; - import org.joda.time.Instant; -import java.util.Collection; -import java.util.Collections; - /** * A {@link WindowFn} that leaves all associations between elements and windows unchanged. * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/InstanceBuilder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/InstanceBuilder.java index 08e07ce4e640..e9e542c0853c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/InstanceBuilder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/InstanceBuilder.java @@ -20,18 +20,15 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; -import org.apache.beam.sdk.values.TypeDescriptor; - import com.google.common.base.Joiner; - import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.lang.reflect.Modifier; import java.util.LinkedList; import java.util.List; - import javax.annotation.Nullable; +import org.apache.beam.sdk.values.TypeDescriptor; /** * Utility for creating objects dynamically. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/KeyedWorkItemCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/KeyedWorkItemCoder.java index ec5d82138269..a6e3d6c10b0d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/KeyedWorkItemCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/KeyedWorkItemCoder.java @@ -19,6 +19,13 @@ import static com.google.common.base.Preconditions.checkArgument; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.List; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.IterableCoder; @@ -28,16 +35,6 @@ import org.apache.beam.sdk.util.TimerInternals.TimerDataCoder; import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; -import com.google.common.collect.ImmutableList; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.List; - /** * A {@link Coder} for {@link KeyedWorkItem KeyedWorkItems}. */ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/KeyedWorkItems.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/KeyedWorkItems.java index 11d84d083ba1..74348428331f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/KeyedWorkItems.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/KeyedWorkItems.java @@ -17,13 +17,11 @@ */ package org.apache.beam.sdk.util; -import org.apache.beam.sdk.util.TimerInternals.TimerData; - import com.google.common.base.MoreObjects; import com.google.common.collect.Iterables; - import java.util.Collections; import java.util.Objects; +import org.apache.beam.sdk.util.TimerInternals.TimerData; /** * Static utility methods that provide {@link KeyedWorkItem} implementations. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MergingActiveWindowSet.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MergingActiveWindowSet.java index 4702fe20460b..6ad63b0f1a0e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MergingActiveWindowSet.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MergingActiveWindowSet.java @@ -20,20 +20,9 @@ import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; -import org.apache.beam.sdk.coders.MapCoder; -import org.apache.beam.sdk.coders.SetCoder; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.state.StateInternals; -import org.apache.beam.sdk.util.state.StateNamespaces; -import org.apache.beam.sdk.util.state.StateTag; -import org.apache.beam.sdk.util.state.StateTags; -import org.apache.beam.sdk.util.state.ValueState; - import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Iterables; import com.google.common.collect.Sets; - import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; @@ -44,8 +33,16 @@ import java.util.Map; import java.util.Objects; import java.util.Set; - import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.MapCoder; +import org.apache.beam.sdk.coders.SetCoder; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.WindowFn; +import org.apache.beam.sdk.util.state.StateInternals; +import org.apache.beam.sdk.util.state.StateNamespaces; +import org.apache.beam.sdk.util.state.StateTag; +import org.apache.beam.sdk.util.state.StateTags; +import org.apache.beam.sdk.util.state.ValueState; /** * An {@link ActiveWindowSet} for merging {@link WindowFn} implementations. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MovingFunction.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MovingFunction.java index eb0a91af6f06..7b30d346024e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MovingFunction.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MovingFunction.java @@ -20,9 +20,8 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.transforms.Combine; - import java.util.Arrays; +import org.apache.beam.sdk.transforms.Combine; /** * Keep track of the moving minimum/maximum/sum of sampled long values. The minimum/maximum/sum diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MutationDetectors.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MutationDetectors.java index ee84c45957c2..3b593bf0e944 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MutationDetectors.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MutationDetectors.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.util; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; - import java.util.Arrays; import java.util.Objects; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; /** * Static methods for creating and working with {@link MutationDetector}. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NonMergingActiveWindowSet.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NonMergingActiveWindowSet.java index 15a4ebe0f25e..99d591bb00b0 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NonMergingActiveWindowSet.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NonMergingActiveWindowSet.java @@ -17,14 +17,12 @@ */ package org.apache.beam.sdk.util; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.WindowFn; - import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableSet; - import java.util.Collection; import java.util.Set; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.WindowFn; /** * Implementation of {@link ActiveWindowSet} used with {@link WindowFn WindowFns} that don't diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NoopCredentialFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NoopCredentialFactory.java index 09f1f9000f0b..5d9255286a77 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NoopCredentialFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NoopCredentialFactory.java @@ -17,12 +17,10 @@ */ package org.apache.beam.sdk.util; -import org.apache.beam.sdk.options.PipelineOptions; - import com.google.api.client.auth.oauth2.Credential; - import java.io.IOException; import java.security.GeneralSecurityException; +import org.apache.beam.sdk.options.PipelineOptions; /** * Construct an oauth credential to be used by the SDK and the SDK workers. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NullSideInputReader.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NullSideInputReader.java index 99319ed14234..3da2dc17b401 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NullSideInputReader.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NullSideInputReader.java @@ -17,13 +17,11 @@ */ package org.apache.beam.sdk.util; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.values.PCollectionView; - import com.google.common.collect.Sets; - import java.util.Collections; import java.util.Set; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.values.PCollectionView; /** * A {@link SideInputReader} representing a well-defined set of views, but not storing diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViewWindow.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViewWindow.java index a7311e3767bc..410c8cea6832 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViewWindow.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViewWindow.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.util; +import java.util.Objects; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.PCollectionView; -import java.util.Objects; - /** * A pair of a {@link PCollectionView} and a {@link BoundedWindow}, which can * be thought of as window "of" the view. This is a value class for use e.g. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViews.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViews.java index 581a98a1a745..14ae5c8dfcfc 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViews.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViews.java @@ -17,25 +17,12 @@ */ package org.apache.beam.sdk.util; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.IterableCoder; -import org.apache.beam.sdk.transforms.ViewFn; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.InvalidWindows; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.sdk.values.PValueBase; -import org.apache.beam.sdk.values.TupleTag; - import com.google.common.base.Function; import com.google.common.base.MoreObjects; import com.google.common.collect.HashMultimap; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Multimap; - import java.io.IOException; import java.util.Collections; import java.util.HashMap; @@ -43,8 +30,18 @@ import java.util.Map; import java.util.NoSuchElementException; import java.util.Objects; - import javax.annotation.Nullable; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.IterableCoder; +import org.apache.beam.sdk.transforms.ViewFn; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.InvalidWindows; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.PValueBase; +import org.apache.beam.sdk.values.TupleTag; /** * Implementations of {@link PCollectionView} shared across the SDK. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PTuple.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PTuple.java index eea7bcdc0f47..d03803fd6d49 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PTuple.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PTuple.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.util; -import org.apache.beam.sdk.values.TupleTag; - import java.util.Collections; import java.util.LinkedHashMap; import java.util.Map; +import org.apache.beam.sdk.values.TupleTag; /** * A {@code PTuple} is an immutable tuple of diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PerKeyCombineFnRunner.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PerKeyCombineFnRunner.java index ae3d391bd809..e8fa4f7e0c23 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PerKeyCombineFnRunner.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PerKeyCombineFnRunner.java @@ -17,14 +17,13 @@ */ package org.apache.beam.sdk.util; +import java.io.Serializable; +import java.util.Collection; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.CombineFnBase.PerKeyCombineFn; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import java.io.Serializable; -import java.util.Collection; - /** * An interface that runs a {@link PerKeyCombineFn} with unified APIs. * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PerKeyCombineFnRunners.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PerKeyCombineFnRunners.java index 87870a83f3b4..c537eb36a459 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PerKeyCombineFnRunners.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PerKeyCombineFnRunners.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.util; +import com.google.common.collect.Iterables; +import java.util.Collection; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Combine.KeyedCombineFn; import org.apache.beam.sdk.transforms.CombineFnBase.PerKeyCombineFn; @@ -26,10 +28,6 @@ import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import com.google.common.collect.Iterables; - -import java.util.Collection; - /** * Static utility methods that provide {@link PerKeyCombineFnRunner} implementations * for different keyed combine functions. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubClient.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubClient.java index 9e6c7d224983..fdcee16b69ad 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubClient.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubClient.java @@ -21,12 +21,9 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; -import org.apache.beam.sdk.options.PubsubOptions; - import com.google.api.client.util.DateTime; import com.google.common.base.Objects; import com.google.common.base.Strings; - import java.io.Closeable; import java.io.IOException; import java.io.Serializable; @@ -34,8 +31,8 @@ import java.util.List; import java.util.Map; import java.util.concurrent.ThreadLocalRandom; - import javax.annotation.Nullable; +import org.apache.beam.sdk.options.PubsubOptions; /** * An (abstract) helper class for talking to Pubsub via an underlying transport. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubGrpcClient.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubGrpcClient.java index ac157fb80309..988b90fec63c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubGrpcClient.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubGrpcClient.java @@ -20,9 +20,6 @@ import static com.google.common.base.Preconditions.checkState; -import org.apache.beam.sdk.options.GcpOptions; -import org.apache.beam.sdk.options.PubsubOptions; - import com.google.auth.oauth2.GoogleCredentials; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Strings; @@ -50,7 +47,6 @@ import com.google.pubsub.v1.SubscriberGrpc.SubscriberBlockingStub; import com.google.pubsub.v1.Subscription; import com.google.pubsub.v1.Topic; - import io.grpc.Channel; import io.grpc.ClientInterceptors; import io.grpc.ManagedChannel; @@ -58,7 +54,6 @@ import io.grpc.netty.GrpcSslContexts; import io.grpc.netty.NegotiationType; import io.grpc.netty.NettyChannelBuilder; - import java.io.IOException; import java.util.ArrayList; import java.util.Collections; @@ -66,8 +61,9 @@ import java.util.Map; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; - import javax.annotation.Nullable; +import org.apache.beam.sdk.options.GcpOptions; +import org.apache.beam.sdk.options.PubsubOptions; /** * A helper class for talking to Pubsub via grpc. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubJsonClient.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubJsonClient.java index 69c5128df8e0..bdb5c04f8cbf 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubJsonClient.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubJsonClient.java @@ -20,8 +20,6 @@ import static com.google.common.base.Preconditions.checkState; -import org.apache.beam.sdk.options.PubsubOptions; - import com.google.api.client.auth.oauth2.Credential; import com.google.api.client.http.HttpRequestInitializer; import com.google.api.services.pubsub.Pubsub; @@ -42,14 +40,13 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; - import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.TreeMap; - import javax.annotation.Nullable; +import org.apache.beam.sdk.options.PubsubOptions; /** * A Pubsub client using JSON transport. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubTestClient.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubTestClient.java index 88ae6cc8cf10..6e5ba46a7f0c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubTestClient.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubTestClient.java @@ -20,12 +20,9 @@ import static com.google.common.base.Preconditions.checkState; -import org.apache.beam.sdk.options.PubsubOptions; - import com.google.api.client.util.Clock; import com.google.common.collect.Lists; import com.google.common.collect.Sets; - import java.io.Closeable; import java.io.IOException; import java.util.ArrayList; @@ -34,8 +31,8 @@ import java.util.List; import java.util.Map; import java.util.Set; - import javax.annotation.Nullable; +import org.apache.beam.sdk.options.PubsubOptions; /** * A (partial) implementation of {@link PubsubClient} for use by unit tests. Only suitable for diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReleaseInfo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReleaseInfo.java index ad03c79e01c8..2c0ae4099c80 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReleaseInfo.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReleaseInfo.java @@ -19,13 +19,11 @@ import com.google.api.client.json.GenericJson; import com.google.api.client.util.Key; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.IOException; import java.io.InputStream; import java.util.Properties; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Utilities for working with release information. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Reshuffle.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Reshuffle.java index ad33a252e0a7..ebd44bf4ab03 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Reshuffle.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Reshuffle.java @@ -25,7 +25,6 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - import org.joda.time.Duration; /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReshuffleTrigger.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReshuffleTrigger.java index f529ce160e2f..9e2c27d1bc7f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReshuffleTrigger.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReshuffleTrigger.java @@ -17,13 +17,11 @@ */ package org.apache.beam.sdk.util; +import java.util.List; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Trigger; - import org.joda.time.Instant; -import java.util.List; - /** * The trigger used with {@link Reshuffle} which triggers on every element * and never buffers state. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RetryHttpRequestInitializer.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RetryHttpRequestInitializer.java index 3857435b08f9..fa6e9136cd28 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RetryHttpRequestInitializer.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RetryHttpRequestInitializer.java @@ -28,18 +28,15 @@ import com.google.api.client.util.ExponentialBackOff; import com.google.api.client.util.NanoClock; import com.google.api.client.util.Sleeper; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.IOException; import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashSet; import java.util.Set; - import javax.annotation.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Implements a request initializer that adds retry handlers to all diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SerializableUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SerializableUtils.java index 1e70aaf021b7..354aa5d91182 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SerializableUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SerializableUtils.java @@ -17,17 +17,10 @@ */ package org.apache.beam.sdk.util; +import static com.google.common.base.Preconditions.checkState; import static org.apache.beam.sdk.util.CoderUtils.decodeFromByteArray; import static org.apache.beam.sdk.util.CoderUtils.encodeToByteArray; -import static com.google.common.base.Preconditions.checkState; - -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; - -import org.xerial.snappy.SnappyInputStream; -import org.xerial.snappy.SnappyOutputStream; - import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -35,6 +28,10 @@ import java.io.ObjectOutputStream; import java.io.Serializable; import java.util.Arrays; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.xerial.snappy.SnappyInputStream; +import org.xerial.snappy.SnappyOutputStream; /** * Utilities for working with Serializables. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Serializer.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Serializer.java index 738577d17340..86a3b8ecc9bf 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Serializer.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Serializer.java @@ -20,12 +20,10 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.ObjectMapper; - import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; - import javax.annotation.Nullable; /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SideInputReader.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SideInputReader.java index e99eb57d9a38..e81c704cf196 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SideInputReader.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SideInputReader.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.util; +import javax.annotation.Nullable; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.PCollectionView; -import javax.annotation.Nullable; - /** * The interface to objects that provide side inputs. Particular implementations * may read a side input directly or use appropriate sorts of caching, etc. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/StringUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/StringUtils.java index bb5937357157..4f81eef4fc7c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/StringUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/StringUtils.java @@ -20,14 +20,12 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; -import org.apache.beam.sdk.transforms.PTransform; - import com.google.common.base.Joiner; - import java.util.ArrayList; import java.util.List; import java.util.regex.Matcher; import java.util.regex.Pattern; +import org.apache.beam.sdk.transforms.PTransform; /** * Utilities for working with JSON and other human-readable string formats. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Structs.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Structs.java index d6b704bf902e..d50b74a0526f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Structs.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Structs.java @@ -18,13 +18,11 @@ package org.apache.beam.sdk.util; import com.google.api.client.util.Data; - import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Map; - import javax.annotation.Nullable; /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SystemDoFnInternal.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SystemDoFnInternal.java index b8a5cd4eb51d..e9904b2d7660 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SystemDoFnInternal.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SystemDoFnInternal.java @@ -17,13 +17,12 @@ */ package org.apache.beam.sdk.util; -import org.apache.beam.sdk.transforms.OldDoFn; - import java.lang.annotation.Documented; import java.lang.annotation.ElementType; import java.lang.annotation.Retention; import java.lang.annotation.RetentionPolicy; import java.lang.annotation.Target; +import org.apache.beam.sdk.transforms.OldDoFn; /** * Annotation to mark {@link OldDoFn DoFns} as an internal component of the Dataflow SDK. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TestCredential.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TestCredential.java index 554ac1b8cb5f..4b81a0ef54f2 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TestCredential.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TestCredential.java @@ -21,7 +21,6 @@ import com.google.api.client.auth.oauth2.Credential; import com.google.api.client.auth.oauth2.TokenResponse; import com.google.api.client.testing.http.MockHttpTransport; - import java.io.IOException; /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimerInternals.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimerInternals.java index eb49b9dd6fac..dd3b77331234 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimerInternals.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimerInternals.java @@ -20,30 +20,26 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.coders.InstantCoder; -import org.apache.beam.sdk.coders.StandardCoder; -import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.state.StateNamespace; -import org.apache.beam.sdk.util.state.StateNamespaces; - -import com.google.common.base.MoreObjects; -import com.google.common.collect.ComparisonChain; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import org.joda.time.Instant; - +import com.google.common.base.MoreObjects; +import com.google.common.collect.ComparisonChain; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.util.Arrays; import java.util.List; import java.util.Objects; - import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.coders.InstantCoder; +import org.apache.beam.sdk.coders.StandardCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.state.StateNamespace; +import org.apache.beam.sdk.util.state.StateNamespaces; +import org.joda.time.Instant; /** * Encapsulate interaction with time within the execution environment. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Timers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Timers.java index e8404bda26f5..beb80ec3d51e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Timers.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Timers.java @@ -17,12 +17,10 @@ */ package org.apache.beam.sdk.util; +import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; - import org.joda.time.Instant; -import javax.annotation.Nullable; - /** * Interface for interacting with time. */ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Transport.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Transport.java index 2c68dfcd3b7a..d824207a7233 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Transport.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Transport.java @@ -17,10 +17,6 @@ */ package org.apache.beam.sdk.util; -import org.apache.beam.sdk.options.BigQueryOptions; -import org.apache.beam.sdk.options.GcsOptions; -import org.apache.beam.sdk.options.PubsubOptions; - import com.google.api.client.auth.oauth2.Credential; import com.google.api.client.googleapis.javanet.GoogleNetHttpTransport; import com.google.api.client.http.HttpRequestInitializer; @@ -32,11 +28,13 @@ import com.google.api.services.storage.Storage; import com.google.cloud.hadoop.util.ChainingHttpRequestInitializer; import com.google.common.collect.ImmutableList; - import java.io.IOException; import java.net.MalformedURLException; import java.net.URL; import java.security.GeneralSecurityException; +import org.apache.beam.sdk.options.BigQueryOptions; +import org.apache.beam.sdk.options.GcsOptions; +import org.apache.beam.sdk.options.PubsubOptions; /** * Helpers for cloud communication. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TriggerContextFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TriggerContextFactory.java index 03f1baa8b493..e09aac2eee75 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TriggerContextFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TriggerContextFactory.java @@ -17,6 +17,13 @@ */ package org.apache.beam.sdk.util; +import com.google.common.base.Predicate; +import com.google.common.collect.FluentIterable; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import java.util.Collection; +import java.util.Map; +import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Trigger; @@ -30,19 +37,8 @@ import org.apache.beam.sdk.util.state.StateNamespace; import org.apache.beam.sdk.util.state.StateNamespaces; import org.apache.beam.sdk.util.state.StateTag; - -import com.google.common.base.Predicate; -import com.google.common.collect.FluentIterable; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Iterables; - import org.joda.time.Instant; -import java.util.Collection; -import java.util.Map; - -import javax.annotation.Nullable; - /** * Factory for creating instances of the various {@link Trigger} contexts. * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UnownedInputStream.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UnownedInputStream.java index 58c6617e818a..e7e7bbd20359 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UnownedInputStream.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UnownedInputStream.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.util; import com.google.common.base.MoreObjects; - import java.io.FilterInputStream; import java.io.IOException; import java.io.InputStream; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UnownedOutputStream.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UnownedOutputStream.java index c2f30c695794..3c9b3bc91c6d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UnownedOutputStream.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UnownedOutputStream.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.util; import com.google.common.base.MoreObjects; - import java.io.FilterOutputStream; import java.io.IOException; import java.io.OutputStream; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UploadIdResponseInterceptor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UploadIdResponseInterceptor.java index caa0759fddbb..f685b6974441 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UploadIdResponseInterceptor.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UploadIdResponseInterceptor.java @@ -20,12 +20,10 @@ import com.google.api.client.http.GenericUrl; import com.google.api.client.http.HttpResponse; import com.google.api.client.http.HttpResponseInterceptor; - +import java.io.IOException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; - /** * Implements a response intercepter that logs the upload id if the upload * id header exists and it is the first request (does not have upload_id parameter in the request). diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ValueWithRecordId.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ValueWithRecordId.java index f0e48124425e..d6b5fe3b4b38 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ValueWithRecordId.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ValueWithRecordId.java @@ -19,22 +19,19 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.coders.ByteArrayCoder; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.StandardCoder; -import org.apache.beam.sdk.transforms.OldDoFn; - -import com.google.common.base.MoreObjects; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; - +import com.google.common.base.MoreObjects; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.util.Arrays; import java.util.List; import java.util.Objects; +import org.apache.beam.sdk.coders.ByteArrayCoder; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.StandardCoder; +import org.apache.beam.sdk.transforms.OldDoFn; /** * Immutable struct containing a value as well as a unique id identifying the value. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Values.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Values.java index 71110e1cbe2e..d8aa046e0790 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Values.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Values.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.util; import java.util.Map; - import javax.annotation.Nullable; /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java index 9d341a1d611e..a0b4cf526096 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java @@ -17,29 +17,14 @@ */ package org.apache.beam.sdk.util; -import static org.apache.beam.sdk.util.Structs.addBoolean; - import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; - -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.coders.CollectionCoder; -import org.apache.beam.sdk.coders.InstantCoder; -import org.apache.beam.sdk.coders.StandardCoder; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.transforms.windowing.PaneInfo.PaneInfoCoder; -import org.apache.beam.sdk.util.common.ElementByteSizeObserver; - -import com.google.common.base.MoreObjects; -import com.google.common.collect.ImmutableList; +import static org.apache.beam.sdk.util.Structs.addBoolean; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import org.joda.time.Instant; - +import com.google.common.base.MoreObjects; +import com.google.common.collect.ImmutableList; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -50,6 +35,17 @@ import java.util.List; import java.util.Objects; import java.util.Set; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.coders.CollectionCoder; +import org.apache.beam.sdk.coders.InstantCoder; +import org.apache.beam.sdk.coders.StandardCoder; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.GlobalWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.transforms.windowing.PaneInfo.PaneInfoCoder; +import org.apache.beam.sdk.util.common.ElementByteSizeObserver; +import org.joda.time.Instant; /** * An immutable triple of value, timestamp, and windows. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowingInternals.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowingInternals.java index 3a1b6543d513..54158d24e520 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowingInternals.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowingInternals.java @@ -17,18 +17,16 @@ */ package org.apache.beam.sdk.util; +import java.io.IOException; +import java.util.Collection; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; - import org.joda.time.Instant; -import java.io.IOException; -import java.util.Collection; - /** * Interface that may be required by some (internal) {@code OldDoFn}s to implement windowing. It * should not be necessary for general user code to interact with this at all. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowingStrategy.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowingStrategy.java index f5ae81263620..b332ed7c8e0c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowingStrategy.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowingStrategy.java @@ -17,6 +17,10 @@ */ package org.apache.beam.sdk.util; +import com.google.common.base.MoreObjects; +import java.io.Serializable; +import java.util.Collections; +import java.util.Objects; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.DefaultTrigger; @@ -27,16 +31,9 @@ import org.apache.beam.sdk.transforms.windowing.Trigger; import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior; import org.apache.beam.sdk.transforms.windowing.WindowFn; - -import com.google.common.base.MoreObjects; - import org.joda.time.Duration; import org.joda.time.Instant; -import java.io.Serializable; -import java.util.Collections; -import java.util.Objects; - /** * A {@code WindowingStrategy} describes the windowing behavior for a specific collection of values. * It has both a {@link WindowFn} describing how elements are assigned to windows and a diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ZipFiles.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ZipFiles.java index c93e18d7706d..c17de3722e7c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ZipFiles.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ZipFiles.java @@ -26,7 +26,7 @@ import com.google.common.io.CharSource; import com.google.common.io.Closer; import com.google.common.io.Files; - +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import java.io.BufferedOutputStream; import java.io.File; import java.io.FileOutputStream; @@ -40,8 +40,6 @@ import java.util.zip.ZipFile; import java.util.zip.ZipOutputStream; -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; - /** * Functions for zipping a directory (including a subdirectory) into a ZIP-file * or unzipping it again. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ReflectHelpers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ReflectHelpers.java index 6db532e5811f..2034ebaef6c8 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ReflectHelpers.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ReflectHelpers.java @@ -19,7 +19,6 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; - import static java.util.Arrays.asList; import com.google.common.base.Function; @@ -27,7 +26,6 @@ import com.google.common.collect.FluentIterable; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Queues; - import java.lang.reflect.GenericArrayType; import java.lang.reflect.Method; import java.lang.reflect.ParameterizedType; @@ -38,7 +36,6 @@ import java.util.Collections; import java.util.LinkedHashSet; import java.util.Queue; - import javax.annotation.Nonnull; import javax.annotation.Nullable; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/gcsfs/GcsPath.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/gcsfs/GcsPath.java index e629d643715a..bfcd6da68141 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/gcsfs/GcsPath.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/gcsfs/GcsPath.java @@ -21,7 +21,6 @@ import static com.google.common.base.Strings.isNullOrEmpty; import com.google.api.services.storage.model.StorageObject; - import java.io.File; import java.io.IOException; import java.net.URI; @@ -35,7 +34,6 @@ import java.util.Iterator; import java.util.regex.Matcher; import java.util.regex.Pattern; - import javax.annotation.Nonnull; import javax.annotation.Nullable; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/CopyOnAccessInMemoryStateInternals.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/CopyOnAccessInMemoryStateInternals.java index 3cc34a694077..da9aa3ef5ebd 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/CopyOnAccessInMemoryStateInternals.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/CopyOnAccessInMemoryStateInternals.java @@ -19,6 +19,12 @@ import static com.google.common.base.Preconditions.checkState; +import com.google.common.base.Optional; +import com.google.common.collect.Iterables; +import java.util.Collection; +import java.util.HashSet; +import java.util.Map; +import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.Combine.KeyedCombineFn; @@ -28,18 +34,8 @@ import org.apache.beam.sdk.util.CombineFnUtil; import org.apache.beam.sdk.util.state.InMemoryStateInternals.InMemoryState; import org.apache.beam.sdk.util.state.StateTag.StateBinder; - -import com.google.common.base.Optional; -import com.google.common.collect.Iterables; - import org.joda.time.Instant; -import java.util.Collection; -import java.util.HashSet; -import java.util.Map; - -import javax.annotation.Nullable; - /** * {@link StateInternals} built on top of an underlying {@link StateTable} that contains instances * of {@link InMemoryState}. Whenever state that exists in the underlying {@link StateTable} is diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryStateInternals.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryStateInternals.java index 1d5d4325dad2..efb270cfa9ec 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryStateInternals.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryStateInternals.java @@ -17,6 +17,11 @@ */ package org.apache.beam.sdk.util.state; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Objects; +import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.coders.Coder; @@ -27,16 +32,8 @@ import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; import org.apache.beam.sdk.util.CombineFnUtil; import org.apache.beam.sdk.util.state.StateTag.StateBinder; - import org.joda.time.Instant; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Objects; - -import javax.annotation.Nullable; - /** * In-memory implementation of {@link StateInternals}. Used in {@code BatchModeExecutionContext} * and for running tests that need state. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/MergingStateAccessor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/MergingStateAccessor.java index 26af7a324dcc..fb877cad78f9 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/MergingStateAccessor.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/MergingStateAccessor.java @@ -17,12 +17,11 @@ */ package org.apache.beam.sdk.util.state; +import java.util.Map; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import java.util.Map; - /** * Interface for accessing persistent state while windows are merging. * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateContexts.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateContexts.java index 77bb4eb9c4a6..d0c566d7accf 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateContexts.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateContexts.java @@ -17,13 +17,12 @@ */ package org.apache.beam.sdk.util.state; +import javax.annotation.Nullable; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowingInternals; import org.apache.beam.sdk.values.PCollectionView; -import javax.annotation.Nullable; - /** * Factory that produces {@link StateContext} based on different inputs. */ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateInternalsFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateInternalsFactory.java index 05c3b77c1384..019b69d5fd29 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateInternalsFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateInternalsFactory.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.util.state; +import java.io.Serializable; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; -import java.io.Serializable; - /** * A factory for providing {@link StateInternals} for a particular key. * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateMerging.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateMerging.java index c41a23e7115b..12baff9312c0 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateMerging.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateMerging.java @@ -19,16 +19,13 @@ import static com.google.common.base.Preconditions.checkState; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; - -import org.joda.time.Instant; - +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import java.util.ArrayList; import java.util.Collection; import java.util.List; import java.util.Map; - -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.joda.time.Instant; /** * Helpers for merging state. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateNamespaces.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateNamespaces.java index bfee9765b661..75f555cfe73a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateNamespaces.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateNamespaces.java @@ -17,16 +17,14 @@ */ package org.apache.beam.sdk.util.state; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.CoderUtils; - import com.google.common.base.Splitter; - import java.io.IOException; import java.util.List; import java.util.Objects; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.CoderUtils; /** * Factory methods for creating the {@link StateNamespace StateNamespaces}. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTable.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTable.java index 2ae651679d9e..dcd2a7eb00db 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTable.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTable.java @@ -17,13 +17,11 @@ */ package org.apache.beam.sdk.util.state; -import org.apache.beam.sdk.util.state.StateTag.StateBinder; - import com.google.common.collect.HashBasedTable; import com.google.common.collect.Table; - import java.util.Map; import java.util.Set; +import org.apache.beam.sdk.util.state.StateTag.StateBinder; /** * Table mapping {@code StateNamespace} and {@code StateTag} to a {@code State} instance. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTag.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTag.java index 388b5e1cc7f9..94cba2f5a109 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTag.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTag.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.util.state; +import java.io.IOException; +import java.io.Serializable; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.coders.Coder; @@ -27,9 +29,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; -import java.io.IOException; -import java.io.Serializable; - /** * An address for persistent state. This includes a unique identifier for the location, the * information necessary to encode the value, and details about the intended access pattern. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTags.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTags.java index e50ad8d00f9d..b0797b6fac20 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTags.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTags.java @@ -17,6 +17,10 @@ */ package org.apache.beam.sdk.util.state; +import com.google.common.base.MoreObjects; +import java.io.IOException; +import java.io.Serializable; +import java.util.Objects; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.coders.CannotProvideCoderException; @@ -28,12 +32,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; -import com.google.common.base.MoreObjects; - -import java.io.IOException; -import java.io.Serializable; -import java.util.Objects; - /** * Static utility methods for creating {@link StateTag} instances. */ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/WatermarkHoldState.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/WatermarkHoldState.java index f22e9e06f6ca..415cc6e143e5 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/WatermarkHoldState.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/WatermarkHoldState.java @@ -21,7 +21,6 @@ import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; - import org.joda.time.Instant; /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/KV.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/KV.java index 180617873637..dec9a16709e0 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/KV.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/KV.java @@ -17,17 +17,15 @@ */ package org.apache.beam.sdk.values; -import org.apache.beam.sdk.transforms.Combine; -import org.apache.beam.sdk.transforms.GroupByKey; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.SerializableComparator; - import com.google.common.base.MoreObjects; - import java.io.Serializable; import java.util.Arrays; import java.util.Comparator; import java.util.Objects; +import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.sdk.transforms.GroupByKey; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.SerializableComparator; /** * An immutable key/value pair. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PBegin.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PBegin.java index e6a2d1f0b991..f1dbb374a2c3 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PBegin.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PBegin.java @@ -17,14 +17,13 @@ */ package org.apache.beam.sdk.values; +import java.util.Collection; +import java.util.Collections; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.TextIO.Read; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.PTransform; -import java.util.Collection; -import java.util.Collections; - /** * {@link PBegin} is the "input" to a root {@link PTransform}, such as {@link Read Read} or * {@link Create}. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionList.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionList.java index 01acca7097d2..4c9e220ea0ef 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionList.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionList.java @@ -17,19 +17,17 @@ */ package org.apache.beam.sdk.values; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.transforms.AppliedPTransform; -import org.apache.beam.sdk.transforms.Flatten; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.Partition; - import com.google.common.collect.ImmutableList; - import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.Iterator; import java.util.List; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.transforms.Flatten; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.Partition; /** * A {@link PCollectionList PCollectionList<T>} is an immutable list of homogeneously diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionTuple.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionTuple.java index b44499bac1d0..f6776f07979a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionTuple.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionTuple.java @@ -17,6 +17,11 @@ */ package org.apache.beam.sdk.values; +import com.google.common.collect.ImmutableMap; +import java.util.Collection; +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.Map; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; @@ -24,13 +29,6 @@ import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollection.IsBounded; -import com.google.common.collect.ImmutableMap; - -import java.util.Collection; -import java.util.Collections; -import java.util.LinkedHashMap; -import java.util.Map; - /** * A {@link PCollectionTuple} is an immutable tuple of * heterogeneously-typed {@link PCollection PCollections}, "keyed" by diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionView.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionView.java index 20f10718b8f3..0e5f5949f242 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionView.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionView.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.values; +import java.io.Serializable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.View; @@ -24,8 +25,6 @@ import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; -import java.io.Serializable; - /** * A {@link PCollectionView PCollectionView<T>} is an immutable view of a {@link PCollection} * as a value of type {@code T} that can be accessed diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PDone.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PDone.java index 7c0570344287..83d6a9232818 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PDone.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PDone.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.values; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.transforms.PTransform; - import java.util.Collection; import java.util.Collections; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.transforms.PTransform; /** * {@link PDone} is the output of a {@link PTransform} that has a trivial result, diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PInput.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PInput.java index 3faf6b9b8097..98987cda9822 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PInput.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PInput.java @@ -17,9 +17,8 @@ */ package org.apache.beam.sdk.values; -import org.apache.beam.sdk.Pipeline; - import java.util.Collection; +import org.apache.beam.sdk.Pipeline; /** * The interface for things that might be input to a diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/POutput.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/POutput.java index 299d55dc5e56..6be9215d6df1 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/POutput.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/POutput.java @@ -17,12 +17,11 @@ */ package org.apache.beam.sdk.values; +import java.util.Collection; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; -import java.util.Collection; - /** * The interface for things that might be output from a {@link PTransform}. */ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValueBase.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValueBase.java index 8a92fa345f89..685e32fe15b7 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValueBase.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValueBase.java @@ -17,14 +17,13 @@ */ package org.apache.beam.sdk.values; +import java.util.Collection; +import java.util.Collections; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.StringUtils; -import java.util.Collection; -import java.util.Collections; - /** * A {@link PValueBase} is an abstract base class that provides * sensible default implementations for methods of {@link PValue}. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java index 69bf77de9c15..f2ad6168bb65 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java @@ -20,21 +20,19 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.InstantCoder; -import org.apache.beam.sdk.coders.StandardCoder; -import org.apache.beam.sdk.util.PropertyNames; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import org.joda.time.Instant; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.util.Arrays; import java.util.List; import java.util.Objects; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.InstantCoder; +import org.apache.beam.sdk.coders.StandardCoder; +import org.apache.beam.sdk.util.PropertyNames; +import org.joda.time.Instant; /** * An immutable pair of a value and a timestamp. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TupleTag.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TupleTag.java index b281a43e4f2b..a6b63ab1aa3a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TupleTag.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TupleTag.java @@ -20,18 +20,15 @@ import static org.apache.beam.sdk.util.Structs.addBoolean; import static org.apache.beam.sdk.util.Structs.addString; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.util.CloudObject; -import org.apache.beam.sdk.util.PropertyNames; - -import com.google.common.collect.HashMultiset; -import com.google.common.collect.Multiset; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; - +import com.google.common.collect.HashMultiset; +import com.google.common.collect.Multiset; import java.io.Serializable; import java.util.Random; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.util.CloudObject; +import org.apache.beam.sdk.util.PropertyNames; /** * A {@link TupleTag} is a typed tag to use as the key of a diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TupleTagList.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TupleTagList.java index 8a4beb3ab505..412678b5b0f5 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TupleTagList.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TupleTagList.java @@ -17,14 +17,12 @@ */ package org.apache.beam.sdk.values; -import org.apache.beam.sdk.transforms.ParDo; - import com.google.common.collect.ImmutableList; - import java.io.Serializable; import java.util.ArrayList; import java.util.Collections; import java.util.List; +import org.apache.beam.sdk.transforms.ParDo; /** * A {@link TupleTagList} is an immutable list of heterogeneously diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptor.java index eea720aa2015..724b8b6925ca 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptor.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptor.java @@ -22,7 +22,6 @@ import com.google.common.reflect.Parameter; import com.google.common.reflect.TypeParameter; import com.google.common.reflect.TypeToken; - import java.io.Serializable; import java.lang.reflect.Field; import java.lang.reflect.Method; @@ -30,7 +29,6 @@ import java.lang.reflect.Type; import java.lang.reflect.TypeVariable; import java.util.List; - import javax.annotation.Nullable; /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptors.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptors.java index 0e7b9cb4397a..84b69d7a0350 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptors.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptors.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.values; import com.google.common.reflect.TypeParameter; - import java.math.BigDecimal; import java.math.BigInteger; import java.util.List; diff --git a/sdks/java/core/src/test/java/org/apache/beam/runners/dataflow/util/GcsPathValidatorTest.java b/sdks/java/core/src/test/java/org/apache/beam/runners/dataflow/util/GcsPathValidatorTest.java index 8913916fde42..398fa6333a43 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/runners/dataflow/util/GcsPathValidatorTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/runners/dataflow/util/GcsPathValidatorTest.java @@ -30,7 +30,6 @@ import org.apache.beam.sdk.util.GcsUtil; import org.apache.beam.sdk.util.TestCredential; import org.apache.beam.sdk.util.gcsfs.GcsPath; - import org.junit.Before; import org.junit.Rule; import org.junit.Test; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/AggregatorPipelineExtractorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/AggregatorPipelineExtractorTest.java index 930fbe781b05..335d81f47bbd 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/AggregatorPipelineExtractorTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/AggregatorPipelineExtractorTest.java @@ -23,6 +23,12 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.UUID; import org.apache.beam.sdk.Pipeline.PipelineVisitor; import org.apache.beam.sdk.runners.TransformTreeNode; import org.apache.beam.sdk.transforms.Aggregator; @@ -33,10 +39,6 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.Sum; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; - import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -47,11 +49,6 @@ import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.UUID; - /** * Tests for {@link AggregatorPipelineExtractor}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/DataflowMatchers.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/DataflowMatchers.java index 2d2a373d8e9a..c76dcf8cbf2a 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/DataflowMatchers.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/DataflowMatchers.java @@ -18,12 +18,10 @@ package org.apache.beam.sdk; import com.google.protobuf.ByteString; - +import java.io.Serializable; import org.hamcrest.Description; import org.hamcrest.TypeSafeMatcher; -import java.io.Serializable; - /** * Matchers that are useful when writing Dataflow tests. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java index d7b3ac54de9d..66ae79f69a9b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java @@ -24,6 +24,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; +import com.google.common.collect.ImmutableList; import org.apache.beam.sdk.Pipeline.PipelineExecutionException; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptions.CheckEnabled; @@ -47,9 +48,6 @@ import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.TupleTag; - -import com.google.common.collect.ImmutableList; - import org.junit.Assert; import org.junit.Rule; import org.junit.Test; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/TestUtils.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/TestUtils.java index 35709ed42dac..1224f10e0e8b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/TestUtils.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/TestUtils.java @@ -19,19 +19,17 @@ import static org.junit.Assert.assertThat; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.values.KV; - import org.hamcrest.CoreMatchers; import org.hamcrest.Description; import org.hamcrest.Matcher; import org.hamcrest.TypeSafeMatcher; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - /** * Utilities for tests. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/WindowMatchers.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/WindowMatchers.java index 48c25899968d..3531a8631cbe 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/WindowMatchers.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/WindowMatchers.java @@ -17,22 +17,19 @@ */ package org.apache.beam.sdk; +import com.google.common.collect.Lists; +import java.util.Collection; +import java.util.Objects; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowedValue; - -import com.google.common.collect.Lists; - import org.hamcrest.Description; import org.hamcrest.Matcher; import org.hamcrest.Matchers; import org.hamcrest.TypeSafeMatcher; import org.joda.time.Instant; -import java.util.Collection; -import java.util.Objects; - /** * Matchers that are useful for working with Windowing, Timestamps, etc. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/WindowMatchersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/WindowMatchersTest.java index 8b108cda2e1a..89637e2af270 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/WindowMatchersTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/WindowMatchersTest.java @@ -19,12 +19,10 @@ import static org.junit.Assert.assertThat; +import com.google.common.collect.ImmutableList; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowedValue; - -import com.google.common.collect.ImmutableList; - import org.joda.time.Instant; import org.junit.Test; import org.junit.runner.RunWith; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTest.java index 3b13e351300a..a97acfb5bcd4 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTest.java @@ -23,20 +23,22 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.coders.Coder.Context; -import org.apache.beam.sdk.coders.Coder.NonDeterministicException; -import org.apache.beam.sdk.testing.CoderProperties; -import org.apache.beam.sdk.testing.NeedsRunner; -import org.apache.beam.sdk.testing.PAssert; -import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.util.CloudObject; -import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.sdk.values.PCollection; - +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.SortedMap; +import java.util.SortedSet; +import java.util.TreeMap; +import java.util.TreeSet; import org.apache.avro.AvroTypeException; import org.apache.avro.Schema; import org.apache.avro.SchemaBuilder; @@ -50,6 +52,19 @@ import org.apache.avro.reflect.Union; import org.apache.avro.specific.SpecificData; import org.apache.avro.util.Utf8; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.Coder.Context; +import org.apache.beam.sdk.coders.Coder.NonDeterministicException; +import org.apache.beam.sdk.testing.CoderProperties; +import org.apache.beam.sdk.testing.NeedsRunner; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.util.CloudObject; +import org.apache.beam.sdk.util.SerializableUtils; +import org.apache.beam.sdk.values.PCollection; import org.hamcrest.Description; import org.hamcrest.Matcher; import org.hamcrest.Matchers; @@ -60,23 +75,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.HashSet; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.SortedMap; -import java.util.SortedSet; -import java.util.TreeMap; -import java.util.TreeSet; - /** Tests for {@link AvroCoder}. */ @RunWith(JUnit4.class) public class AvroCoderTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigDecimalCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigDecimalCoderTest.java index a5e3bc26b46c..f5d56cbc11f0 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigDecimalCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigDecimalCoderTest.java @@ -20,17 +20,17 @@ import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertThat; +import com.google.common.collect.ImmutableList; +import java.math.BigDecimal; +import java.util.List; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.testing.CoderProperties.TestElementByteSizeObserver; import org.apache.beam.sdk.util.CoderUtils; -import com.google.common.collect.ImmutableList; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.math.BigDecimal; -import java.util.List; /** * Test case for {@link BigDecimalCoder}. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianIntegerCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianIntegerCoderTest.java index f4c22bb5a0f2..091fda25c164 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianIntegerCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianIntegerCoderTest.java @@ -17,18 +17,16 @@ */ package org.apache.beam.sdk.coders; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.util.CoderUtils; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** * Test case for {@link BigEndianIntegerCoder}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianLongCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianLongCoderTest.java index cf148b76f92c..3a74e301695d 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianLongCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianLongCoderTest.java @@ -17,18 +17,16 @@ */ package org.apache.beam.sdk.coders; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.util.CoderUtils; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** * Test case for {@link BigEndianLongCoder}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigIntegerCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigIntegerCoderTest.java index d04237807b9f..6ead941a3d8d 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigIntegerCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigIntegerCoderTest.java @@ -20,17 +20,17 @@ import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertThat; +import com.google.common.collect.ImmutableList; +import java.math.BigInteger; +import java.util.List; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.testing.CoderProperties.TestElementByteSizeObserver; import org.apache.beam.sdk.util.CoderUtils; -import com.google.common.collect.ImmutableList; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.math.BigInteger; -import java.util.List; /** * Test case for {@link BigIntegerCoder}. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteArrayCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteArrayCoderTest.java index aca0a8304c70..f80a409fced4 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteArrayCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteArrayCoderTest.java @@ -21,20 +21,18 @@ import static org.hamcrest.Matchers.not; import static org.junit.Assert.assertThat; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.util.CoderUtils; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.util.Arrays; -import java.util.List; - /** * Unit tests for {@link ByteArrayCoder}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteCoderTest.java index 0494fe07d615..b7673e227422 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteCoderTest.java @@ -17,18 +17,16 @@ */ package org.apache.beam.sdk.coders; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.util.CoderUtils; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** * Test case for {@link ByteCoder}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteStringCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteStringCoderTest.java index d65772cf20eb..1d4c0627f151 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteStringCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteStringCoderTest.java @@ -20,22 +20,19 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import com.google.common.collect.ImmutableList; +import com.google.protobuf.ByteString; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.coders.Coder.Context; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.util.CoderUtils; - -import com.google.common.collect.ImmutableList; -import com.google.protobuf.ByteString; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** * Test case for {@link ByteStringCoder}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderFactoriesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderFactoriesTest.java index a871404ed87b..4ffc9c1a1984 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderFactoriesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderFactoriesTest.java @@ -19,13 +19,12 @@ import static org.junit.Assert.assertEquals; +import java.util.Arrays; +import java.util.Collections; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.Collections; - /** * Tests for {@link CoderFactories}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderProvidersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderProvidersTest.java index 44d17d4f28c3..44be56daaff0 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderProvidersTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderProvidersTest.java @@ -20,16 +20,14 @@ import static org.hamcrest.Matchers.instanceOf; import static org.junit.Assert.assertThat; +import java.util.Map; import org.apache.beam.sdk.values.TypeDescriptor; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Map; - /** * Tests for {@link CoderFactories}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java index da1540585a84..d690a474ead4 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java @@ -21,6 +21,20 @@ import static org.hamcrest.Matchers.containsString; import static org.junit.Assert.assertEquals; +import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageA; +import com.google.common.collect.ImmutableList; +import com.google.protobuf.Duration; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.lang.reflect.Type; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.CoderRegistry.IncompatibleCoderException; import org.apache.beam.sdk.coders.protobuf.ProtoCoder; @@ -35,11 +49,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptor; - -import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageA; -import com.google.common.collect.ImmutableList; -import com.google.protobuf.Duration; - import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -47,18 +56,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.Serializable; -import java.lang.reflect.Type; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Set; - /** * Tests for CoderRegistry. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderTest.java index b6378fc9bd0d..ccbffdd15db1 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderTest.java @@ -23,17 +23,15 @@ import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertThat; +import java.util.Collections; import org.apache.beam.sdk.coders.Coder.Context; import org.apache.beam.sdk.coders.Coder.NonDeterministicException; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Collections; - /** Tests for constructs defined within {@link Coder}. */ @RunWith(JUnit4.class) public class CoderTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CollectionCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CollectionCoderTest.java index a2bc20dff46e..6a5d94b5ae51 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CollectionCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CollectionCoderTest.java @@ -17,22 +17,20 @@ */ package org.apache.beam.sdk.coders; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; +import java.util.TreeSet; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.util.CoderUtils; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.LinkedList; -import java.util.List; -import java.util.TreeSet; - /** * Test case for {@link CollectionCoder}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CustomCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CustomCoderTest.java index 10259d96da13..31bbdb9e60fb 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CustomCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CustomCoderTest.java @@ -17,11 +17,15 @@ */ package org.apache.beam.sdk.coders; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.values.KV; - import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -29,12 +33,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; - /** Unit tests for {@link CustomCoder}. */ @RunWith(JUnit4.class) public class CustomCoderTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DefaultCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DefaultCoderTest.java index 1faf58f94639..a8496c946b63 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DefaultCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DefaultCoderTest.java @@ -18,13 +18,13 @@ package org.apache.beam.sdk.coders; import static com.google.common.base.Preconditions.checkArgument; - import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.instanceOf; import static org.junit.Assert.assertThat; +import java.io.Serializable; +import java.util.List; import org.apache.beam.sdk.values.TypeDescriptor; - import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -32,9 +32,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.List; - /** * Tests of Coder defaults. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DelegateCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DelegateCoderTest.java index cf770aadbd16..9bb9d51ef47f 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DelegateCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DelegateCoderTest.java @@ -20,15 +20,8 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; -import org.apache.beam.sdk.testing.CoderProperties; - import com.google.common.collect.Lists; import com.google.common.collect.Sets; - -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; @@ -38,6 +31,10 @@ import java.util.HashSet; import java.util.List; import java.util.Set; +import org.apache.beam.sdk.testing.CoderProperties; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; /** Unit tests for {@link DelegateCoder}. */ @RunWith(JUnit4.class) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DoubleCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DoubleCoderTest.java index 73fb464cb69a..f43af304e388 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DoubleCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DoubleCoderTest.java @@ -17,18 +17,16 @@ */ package org.apache.beam.sdk.coders; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.util.CoderUtils; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** * Test case for {@link DoubleCoder}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DurationCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DurationCoderTest.java index e2daefcbf51d..4cb697eff93d 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DurationCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DurationCoderTest.java @@ -17,11 +17,11 @@ */ package org.apache.beam.sdk.coders; +import com.google.common.collect.Lists; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.util.CoderUtils; - -import com.google.common.collect.Lists; - import org.joda.time.Duration; import org.joda.time.ReadableDuration; import org.junit.Rule; @@ -30,9 +30,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** Unit tests for {@link DurationCoder}. */ @RunWith(JUnit4.class) public class DurationCoderTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/InstantCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/InstantCoderTest.java index c1ce7e647282..16cb703ca3ab 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/InstantCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/InstantCoderTest.java @@ -17,12 +17,14 @@ */ package org.apache.beam.sdk.coders; -import org.apache.beam.sdk.testing.CoderProperties; -import org.apache.beam.sdk.util.CoderUtils; - import com.google.common.collect.Lists; import com.google.common.primitives.UnsignedBytes; - +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import org.apache.beam.sdk.testing.CoderProperties; +import org.apache.beam.sdk.util.CoderUtils; import org.joda.time.Instant; import org.junit.Assert; import org.junit.Rule; @@ -31,11 +33,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - /** Unit tests for {@link InstantCoder}. */ @RunWith(JUnit4.class) public class InstantCoderTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/IterableCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/IterableCoderTest.java index 42397b7bf99f..15ec44bb2d76 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/IterableCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/IterableCoderTest.java @@ -20,20 +20,18 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; +import java.util.Arrays; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.util.CoderUtils; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.Collections; -import java.util.LinkedList; -import java.util.List; - /** Unit tests for {@link IterableCoder}. */ @RunWith(JUnit4.class) public class IterableCoderTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/JAXBCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/JAXBCoderTest.java index 6b59e525d965..129be61ae7c1 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/JAXBCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/JAXBCoderTest.java @@ -20,16 +20,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; -import org.apache.beam.sdk.testing.CoderProperties; -import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.SerializableUtils; - import com.google.common.collect.ImmutableList; - -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -39,8 +30,13 @@ import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; - import javax.xml.bind.annotation.XmlRootElement; +import org.apache.beam.sdk.testing.CoderProperties; +import org.apache.beam.sdk.util.CoderUtils; +import org.apache.beam.sdk.util.SerializableUtils; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; /** Unit tests for {@link JAXBCoder}. */ @RunWith(JUnit4.class) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/KvCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/KvCoderTest.java index 47fcad79a914..f0f7d22ea95a 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/KvCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/KvCoderTest.java @@ -17,23 +17,20 @@ */ package org.apache.beam.sdk.coders; +import com.google.common.collect.ImmutableMap; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.values.KV; - -import com.google.common.collect.ImmutableMap; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Map; - /** * Test case for {@link KvCoder}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ListCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ListCoderTest.java index 05b66813c397..ba9cc9d170de 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ListCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ListCoderTest.java @@ -20,20 +20,18 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; +import java.util.Arrays; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.util.CoderUtils; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.Collections; -import java.util.LinkedList; -import java.util.List; - /** Unit tests for {@link ListCoder}. */ @RunWith(JUnit4.class) public class ListCoderTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/MapCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/MapCoderTest.java index b92b656e974e..dc4a8b527f70 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/MapCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/MapCoderTest.java @@ -20,23 +20,20 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; -import org.apache.beam.sdk.testing.CoderProperties; -import org.apache.beam.sdk.util.CoderUtils; - import com.google.common.collect.ImmutableMap; - -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.TreeMap; +import org.apache.beam.sdk.testing.CoderProperties; +import org.apache.beam.sdk.util.CoderUtils; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; /** Unit tests for {@link MapCoder}. */ @RunWith(JUnit4.class) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/NullableCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/NullableCoderTest.java index 5bfbe05c1a46..61e7e41afa76 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/NullableCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/NullableCoderTest.java @@ -24,21 +24,18 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; -import org.apache.beam.sdk.testing.CoderProperties; - import com.google.common.collect.ImmutableList; - +import java.io.ByteArrayInputStream; +import java.io.InputStream; +import java.util.Arrays; +import java.util.List; +import org.apache.beam.sdk.testing.CoderProperties; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.ByteArrayInputStream; -import java.io.InputStream; -import java.util.Arrays; -import java.util.List; - /** Unit tests for {@link NullableCoder}. */ @RunWith(JUnit4.class) public class NullableCoderTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/PrintBase64Encodings.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/PrintBase64Encodings.java index 9ab66d13b479..5cdb71974e0c 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/PrintBase64Encodings.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/PrintBase64Encodings.java @@ -17,14 +17,12 @@ */ package org.apache.beam.sdk.coders; -import org.apache.beam.sdk.util.CoderUtils; - import com.google.common.base.Joiner; import com.google.common.collect.Lists; - import java.lang.reflect.Field; import java.lang.reflect.Modifier; import java.util.List; +import org.apache.beam.sdk.util.CoderUtils; /** * A command-line utility for printing the base-64 encodings of test values, for generating exact diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SerializableCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SerializableCoderTest.java index b5465fae0e99..8d344de2831d 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SerializableCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SerializableCoderTest.java @@ -20,6 +20,12 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.Serializable; +import java.util.Arrays; +import java.util.LinkedList; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.testing.NeedsRunner; @@ -32,7 +38,6 @@ import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.Serializer; import org.apache.beam.sdk.values.PCollection; - import org.hamcrest.Matchers; import org.junit.Assert; import org.junit.Test; @@ -40,13 +45,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.Serializable; -import java.util.Arrays; -import java.util.LinkedList; -import java.util.List; - /** * Tests SerializableCoder. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SetCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SetCoderTest.java index e353504d30e1..58b0b8ef580f 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SetCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SetCoderTest.java @@ -17,21 +17,19 @@ */ package org.apache.beam.sdk.coders; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Set; +import java.util.TreeSet; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.util.CoderUtils; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Set; -import java.util.TreeSet; - /** * Test case for {@link SetCoder}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StandardCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StandardCoderTest.java index 1dc221017e7e..e3d0537c8765 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StandardCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StandardCoderTest.java @@ -17,24 +17,20 @@ */ package org.apache.beam.sdk.coders; -import org.apache.beam.sdk.testing.CoderProperties; - import com.google.common.collect.ImmutableList; - -import org.hamcrest.CoreMatchers; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.util.Arrays; import java.util.Collections; import java.util.List; - import javax.annotation.Nullable; +import org.apache.beam.sdk.testing.CoderProperties; +import org.hamcrest.CoreMatchers; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; /** * Test case for {@link StandardCoder}. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StringDelegateCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StringDelegateCoderTest.java index 2a4f2e6a3143..b8a45fd80d73 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StringDelegateCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StringDelegateCoderTest.java @@ -17,17 +17,15 @@ */ package org.apache.beam.sdk.coders; +import java.net.URI; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.coders.Coder.NonDeterministicException; import org.apache.beam.sdk.testing.CoderProperties; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.net.URI; -import java.util.Arrays; -import java.util.List; - /** Unit tests for {@link StringDelegateCoder}. */ @RunWith(JUnit4.class) public class StringDelegateCoderTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StringUtf8CoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StringUtf8CoderTest.java index 5c810f72c438..cce04ec42567 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StringUtf8CoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StringUtf8CoderTest.java @@ -17,18 +17,16 @@ */ package org.apache.beam.sdk.coders; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.util.CoderUtils; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** * Test case for {@link StringUtf8Coder}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/TableRowJsonCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/TableRowJsonCoderTest.java index 03f2339741ff..3da65221a036 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/TableRowJsonCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/TableRowJsonCoderTest.java @@ -17,17 +17,14 @@ */ package org.apache.beam.sdk.coders; -import org.apache.beam.sdk.testing.CoderProperties; - import com.google.api.services.bigquery.model.TableRow; - +import java.util.Arrays; +import java.util.List; +import org.apache.beam.sdk.testing.CoderProperties; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** * Test case for {@link TableRowJsonCoder}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/TextualIntegerCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/TextualIntegerCoderTest.java index 252f8a4863ab..5012d1371d89 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/TextualIntegerCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/TextualIntegerCoderTest.java @@ -17,18 +17,16 @@ */ package org.apache.beam.sdk.coders; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.util.CoderUtils; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** * Test case for {@link TextualIntegerCoder}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarIntCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarIntCoderTest.java index 3da242d39ace..72200917f78e 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarIntCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarIntCoderTest.java @@ -17,18 +17,16 @@ */ package org.apache.beam.sdk.coders; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.util.CoderUtils; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** * Test case for {@link VarIntCoder}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarLongCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarLongCoderTest.java index 4cd221628072..615d4fcb0565 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarLongCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarLongCoderTest.java @@ -17,18 +17,16 @@ */ package org.apache.beam.sdk.coders; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.util.CoderUtils; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** * Test case for {@link VarLongCoder}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/protobuf/ProtoCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/protobuf/ProtoCoderTest.java index b35adf6308e2..8b889dab2629 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/protobuf/ProtoCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/protobuf/ProtoCoderTest.java @@ -20,6 +20,12 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; +import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages; +import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageA; +import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageB; +import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageC; +import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageWithMap; +import com.google.common.collect.ImmutableList; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.Coder.NonDeterministicException; @@ -28,14 +34,6 @@ import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.values.TypeDescriptor; - -import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages; -import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageA; -import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageB; -import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageC; -import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageWithMap; -import com.google.common.collect.ImmutableList; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/protobuf/ProtobufUtilTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/protobuf/ProtobufUtilTest.java index 14fe4d8db7fe..97368248998b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/protobuf/ProtobufUtilTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/protobuf/ProtobufUtilTest.java @@ -20,12 +20,9 @@ import static org.apache.beam.sdk.coders.protobuf.ProtobufUtil.checkProto2Syntax; import static org.apache.beam.sdk.coders.protobuf.ProtobufUtil.getRecursiveDescriptorsForClass; import static org.apache.beam.sdk.coders.protobuf.ProtobufUtil.verifyDeterministic; - import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertThat; -import org.apache.beam.sdk.coders.Coder.NonDeterministicException; - import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages; import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageA; import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageB; @@ -39,16 +36,15 @@ import com.google.protobuf.Duration; import com.google.protobuf.ExtensionRegistry; import com.google.protobuf.Message; - +import java.util.HashSet; +import java.util.Set; +import org.apache.beam.sdk.coders.Coder.NonDeterministicException; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.HashSet; -import java.util.Set; - /** * Tests for {@link ProtobufUtil}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOGeneratedClassTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOGeneratedClassTest.java index 6e26d33d92e4..ede135f19ce8 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOGeneratedClassTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOGeneratedClassTest.java @@ -21,14 +21,11 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; -import org.apache.beam.sdk.coders.AvroCoder; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.testing.NeedsRunner; -import org.apache.beam.sdk.testing.PAssert; -import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.values.PCollection; - +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; import org.apache.avro.Schema; import org.apache.avro.file.DataFileReader; import org.apache.avro.file.DataFileWriter; @@ -38,6 +35,13 @@ import org.apache.avro.io.DatumWriter; import org.apache.avro.specific.SpecificDatumReader; import org.apache.avro.specific.SpecificDatumWriter; +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.testing.NeedsRunner; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.values.PCollection; import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -46,12 +50,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.File; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - /** * Tests for AvroIO Read and Write transforms, using classes generated from {@code user.avsc}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java index 026724add21e..a8a7746f214b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.io; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.hasItem; import static org.junit.Assert.assertEquals; @@ -26,6 +25,19 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; +import com.google.common.base.MoreObjects; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterators; +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; +import java.util.Set; +import org.apache.avro.Schema; +import org.apache.avro.file.DataFileReader; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.reflect.Nullable; import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.DefaultCoder; import org.apache.beam.sdk.io.AvroIO.Write.Bound; @@ -40,15 +52,6 @@ import org.apache.beam.sdk.transforms.display.DisplayDataEvaluator; import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.base.MoreObjects; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterators; - -import org.apache.avro.Schema; -import org.apache.avro.file.DataFileReader; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.reflect.Nullable; import org.junit.BeforeClass; import org.junit.Ignore; import org.junit.Rule; @@ -58,13 +61,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.File; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Objects; -import java.util.Set; - /** * Tests for AvroIO Read and Write transforms. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroSourceTest.java index 09405abeece6..c1b532f8b4d9 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroSourceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroSourceTest.java @@ -18,26 +18,24 @@ package org.apache.beam.sdk.io; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - import static org.hamcrest.Matchers.containsInAnyOrder; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; -import org.apache.beam.sdk.coders.AvroCoder; -import org.apache.beam.sdk.coders.DefaultCoder; -import org.apache.beam.sdk.io.AvroSource.AvroReader; -import org.apache.beam.sdk.io.AvroSource.AvroReader.Seeker; -import org.apache.beam.sdk.io.BlockBasedSource.BlockBasedReader; -import org.apache.beam.sdk.io.BoundedSource.BoundedReader; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.testing.SourceTestUtils; -import org.apache.beam.sdk.transforms.display.DisplayData; - import com.google.common.base.MoreObjects; - +import java.io.ByteArrayInputStream; +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.PushbackInputStream; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.Objects; +import java.util.Random; import org.apache.avro.Schema; import org.apache.avro.file.CodecFactory; import org.apache.avro.file.DataFileConstants; @@ -47,6 +45,16 @@ import org.apache.avro.reflect.AvroDefault; import org.apache.avro.reflect.Nullable; import org.apache.avro.reflect.ReflectData; +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.io.AvroSource.AvroReader; +import org.apache.beam.sdk.io.AvroSource.AvroReader.Seeker; +import org.apache.beam.sdk.io.BlockBasedSource.BlockBasedReader; +import org.apache.beam.sdk.io.BoundedSource.BoundedReader; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.testing.SourceTestUtils; +import org.apache.beam.sdk.transforms.display.DisplayData; import org.hamcrest.Matchers; import org.junit.Rule; import org.junit.Test; @@ -55,18 +63,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.ByteArrayInputStream; -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.io.PushbackInputStream; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.NoSuchElementException; -import java.util.Objects; -import java.util.Random; - /** * Tests for AvroSource. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java index fe9415bbb3a1..7bf0f35576bd 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java @@ -18,12 +18,15 @@ package org.apache.beam.sdk.io; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; - import static org.hamcrest.Matchers.containsInAnyOrder; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.runners.dataflow.TestCountingSource; import org.apache.beam.sdk.testing.PAssert; @@ -33,18 +36,12 @@ import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - import org.joda.time.Duration; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; - /** Unit tests for {@link BoundedReadFromUnboundedSource}. */ @RunWith(JUnit4.class) public class BoundedReadFromUnboundedSourceTest implements Serializable{ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CompressedSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CompressedSourceTest.java index 4a9f95088804..d7c451d00cef 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CompressedSourceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CompressedSourceTest.java @@ -19,7 +19,6 @@ import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; - import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.not; @@ -30,6 +29,24 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import com.google.common.io.Files; +import com.google.common.primitives.Bytes; +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.nio.channels.ReadableByteChannel; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.Random; +import java.util.zip.GZIPOutputStream; +import javax.annotation.Nullable; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.SerializableCoder; @@ -46,12 +63,6 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; -import com.google.common.io.Files; -import com.google.common.primitives.Bytes; - import org.apache.commons.compress.compressors.bzip2.BZip2CompressorOutputStream; import org.apache.commons.compress.compressors.gzip.GzipCompressorOutputStream; import org.hamcrest.Matchers; @@ -64,22 +75,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.ByteArrayOutputStream; -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.io.OutputStream; -import java.nio.ByteBuffer; -import java.nio.channels.ReadableByteChannel; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.List; -import java.util.NoSuchElementException; -import java.util.Random; -import java.util.zip.GZIPOutputStream; - -import javax.annotation.Nullable; - /** * Tests for CompressedSource. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingInputTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingInputTest.java index 4ec2c9ae02da..0b92b2698ea3 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingInputTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingInputTest.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.io; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertThat; @@ -38,7 +37,6 @@ import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.values.PCollection; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Test; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingSourceTest.java index 0bd91c14846f..7c5fa1377713 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingSourceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingSourceTest.java @@ -23,6 +23,8 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; +import java.io.IOException; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.BoundedSource.BoundedReader; import org.apache.beam.sdk.io.CountingSource.CounterMark; @@ -44,7 +46,6 @@ import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Test; @@ -52,9 +53,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.util.List; - /** * Tests of {@link CountingSource}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSinkTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSinkTest.java index d3454da19f13..0fdb11f1e3f0 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSinkTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSinkTest.java @@ -24,18 +24,6 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import org.apache.beam.sdk.io.FileBasedSink.FileBasedWriteOperation; -import org.apache.beam.sdk.io.FileBasedSink.FileBasedWriteOperation.TemporaryFileRetention; -import org.apache.beam.sdk.io.FileBasedSink.FileResult; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.options.PipelineOptionsFactory; - -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - import java.io.BufferedReader; import java.io.File; import java.io.FileOutputStream; @@ -47,6 +35,16 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import org.apache.beam.sdk.io.FileBasedSink.FileBasedWriteOperation; +import org.apache.beam.sdk.io.FileBasedSink.FileBasedWriteOperation.TemporaryFileRetention; +import org.apache.beam.sdk.io.FileBasedSink.FileResult; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; /** * Tests for FileBasedSink. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSourceTest.java index c9f4079e5c1a..520891059080 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSourceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSourceTest.java @@ -21,7 +21,6 @@ import static org.apache.beam.sdk.testing.SourceTestUtils.assertSplitAtFractionFails; import static org.apache.beam.sdk.testing.SourceTestUtils.assertSplitAtFractionSucceedsAndConsistent; import static org.apache.beam.sdk.testing.SourceTestUtils.readFromSource; - import static org.hamcrest.Matchers.containsInAnyOrder; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; @@ -29,6 +28,20 @@ import static org.junit.Assert.fail; import static org.mockito.Mockito.when; +import com.google.common.collect.ImmutableList; +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.ReadableByteChannel; +import java.nio.channels.SeekableByteChannel; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.Random; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StringUtf8Coder; @@ -43,9 +56,6 @@ import org.apache.beam.sdk.util.IOChannelFactory; import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.ImmutableList; - import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -54,20 +64,6 @@ import org.junit.runners.JUnit4; import org.mockito.Mockito; -import java.io.ByteArrayOutputStream; -import java.io.File; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.nio.channels.ReadableByteChannel; -import java.nio.channels.SeekableByteChannel; -import java.nio.charset.StandardCharsets; -import java.nio.file.Files; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.NoSuchElementException; -import java.util.Random; - /** * Tests code common to all file-based sources. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/OffsetBasedSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/OffsetBasedSourceTest.java index f689f51e97c0..923b4b4ba3c5 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/OffsetBasedSourceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/OffsetBasedSourceTest.java @@ -19,29 +19,26 @@ import static org.apache.beam.sdk.testing.SourceTestUtils.assertSplitAtFractionExhaustive; import static org.apache.beam.sdk.testing.SourceTestUtils.readFromSource; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.NoSuchElementException; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.BoundedSource.BoundedReader; import org.apache.beam.sdk.io.OffsetBasedSource.OffsetBasedReader; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.NoSuchElementException; - /** * Tests code common to all offset-based sources. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubIOTest.java index 1e9ebf2d1529..4067055b25c7 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubIOTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubIOTest.java @@ -18,15 +18,14 @@ package org.apache.beam.sdk.io; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - import static org.hamcrest.Matchers.hasItem; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; +import java.util.Set; import org.apache.beam.sdk.testing.RunnableOnService; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.display.DisplayDataEvaluator; - import org.joda.time.Duration; import org.junit.Rule; import org.junit.Test; @@ -35,8 +34,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Set; - /** * Tests for PubsubIO Read and Write transforms. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSinkTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSinkTest.java index db03a5cce7a0..4edd9c106ea3 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSinkTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSinkTest.java @@ -18,6 +18,11 @@ package org.apache.beam.sdk.io; +import com.google.common.collect.ImmutableList; +import com.google.common.hash.Hashing; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.io.PubsubUnboundedSink.RecordIdMethod; import org.apache.beam.sdk.testing.CoderProperties; @@ -31,10 +36,6 @@ import org.apache.beam.sdk.util.PubsubClient.TopicPath; import org.apache.beam.sdk.util.PubsubTestClient; import org.apache.beam.sdk.util.PubsubTestClient.PubsubTestClientFactory; - -import com.google.common.collect.ImmutableList; -import com.google.common.hash.Hashing; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Test; @@ -42,10 +43,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - /** * Test PubsubUnboundedSink. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSourceTest.java index a19ccc5197e3..c46eca527dcd 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSourceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSourceTest.java @@ -26,6 +26,14 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; +import com.google.api.client.util.Clock; +import com.google.common.collect.ImmutableList; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.io.PubsubUnboundedSource.PubsubCheckpoint; import org.apache.beam.sdk.io.PubsubUnboundedSource.PubsubReader; @@ -38,23 +46,12 @@ import org.apache.beam.sdk.util.PubsubClient.SubscriptionPath; import org.apache.beam.sdk.util.PubsubTestClient; import org.apache.beam.sdk.util.PubsubTestClient.PubsubTestClientFactory; - -import com.google.api.client.util.Clock; -import com.google.common.collect.ImmutableList; - import org.joda.time.Instant; import org.junit.After; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.atomic.AtomicLong; - /** * Test PubsubUnboundedSource. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java index a47ddf2a34cf..30a8a43e90a5 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java @@ -19,14 +19,16 @@ import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; - import static org.hamcrest.MatcherAssert.assertThat; +import java.io.IOException; +import java.io.Serializable; +import java.util.List; +import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.display.DisplayData; - import org.joda.time.Duration; import org.junit.Rule; import org.junit.Test; @@ -34,12 +36,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.io.Serializable; -import java.util.List; - -import javax.annotation.Nullable; - /** * Tests for {@link Read}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java index 6fd3093f2155..358a30f3d2b8 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java @@ -23,7 +23,6 @@ import static org.apache.beam.sdk.TestUtils.NO_LINES_ARRAY; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasValue; - import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.hasItem; @@ -34,6 +33,27 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; +import com.google.common.collect.ImmutableList; +import java.io.BufferedReader; +import java.io.File; +import java.io.FileOutputStream; +import java.io.FileReader; +import java.io.IOException; +import java.io.OutputStream; +import java.io.PrintStream; +import java.nio.channels.FileChannel; +import java.nio.channels.SeekableByteChannel; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.StandardOpenOption; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Set; +import java.util.zip.GZIPOutputStream; +import java.util.zip.ZipEntry; +import java.util.zip.ZipOutputStream; +import javax.annotation.Nullable; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StringUtf8Coder; @@ -59,9 +79,6 @@ import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.util.gcsfs.GcsPath; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.ImmutableList; - import org.apache.commons.compress.compressors.bzip2.BZip2CompressorOutputStream; import org.junit.BeforeClass; import org.junit.Ignore; @@ -76,28 +93,6 @@ import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import java.io.BufferedReader; -import java.io.File; -import java.io.FileOutputStream; -import java.io.FileReader; -import java.io.IOException; -import java.io.OutputStream; -import java.io.PrintStream; -import java.nio.channels.FileChannel; -import java.nio.channels.SeekableByteChannel; -import java.nio.charset.StandardCharsets; -import java.nio.file.Files; -import java.nio.file.StandardOpenOption; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Set; -import java.util.zip.GZIPOutputStream; -import java.util.zip.ZipEntry; -import java.util.zip.ZipOutputStream; - -import javax.annotation.Nullable; - /** * Tests for TextIO Read and Write transforms. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java index b9ba53ba6e13..28651884d298 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java @@ -28,6 +28,19 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; +import com.google.common.base.MoreObjects; +import com.google.common.base.Optional; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Objects; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.SerializableCoder; @@ -52,10 +65,6 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.base.MoreObjects; -import com.google.common.base.Optional; - import org.hamcrest.Matchers; import org.joda.time.Duration; import org.junit.Rule; @@ -65,18 +74,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.Objects; -import java.util.Set; -import java.util.UUID; -import java.util.concurrent.ThreadLocalRandom; -import java.util.concurrent.atomic.AtomicInteger; - /** * Tests for the Write PTransform. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/XmlSinkTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/XmlSinkTest.java index ea0db73865d9..2788ea664621 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/XmlSinkTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/XmlSinkTest.java @@ -18,26 +18,11 @@ package org.apache.beam.sdk.io; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; -import org.apache.beam.sdk.io.XmlSink.XmlWriteOperation; -import org.apache.beam.sdk.io.XmlSink.XmlWriter; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.transforms.display.DisplayData; - import com.google.common.collect.Lists; - -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - import java.io.BufferedReader; import java.io.File; import java.io.FileOutputStream; @@ -46,10 +31,20 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; - import javax.xml.bind.annotation.XmlElement; import javax.xml.bind.annotation.XmlRootElement; import javax.xml.bind.annotation.XmlType; +import org.apache.beam.sdk.io.XmlSink.XmlWriteOperation; +import org.apache.beam.sdk.io.XmlSink.XmlWriter; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; /** * Tests for XmlSink. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/XmlSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/XmlSourceTest.java index 37e388188813..1f154d577cce 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/XmlSourceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/XmlSourceTest.java @@ -21,7 +21,6 @@ import static org.apache.beam.sdk.testing.SourceTestUtils.assertSplitAtFractionFails; import static org.apache.beam.sdk.testing.SourceTestUtils.assertSplitAtFractionSucceedsAndConsistent; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - import static org.hamcrest.Matchers.both; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.containsString; @@ -29,6 +28,18 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; +import com.google.common.collect.ImmutableList; +import java.io.BufferedWriter; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.List; +import java.util.Random; +import javax.xml.bind.annotation.XmlAttribute; +import javax.xml.bind.annotation.XmlRootElement; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.Source.Reader; import org.apache.beam.sdk.options.PipelineOptions; @@ -38,9 +49,6 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.ImmutableList; - import org.hamcrest.Matchers; import org.junit.Ignore; import org.junit.Rule; @@ -51,19 +59,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.BufferedWriter; -import java.io.File; -import java.io.FileWriter; -import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.nio.file.Files; -import java.util.ArrayList; -import java.util.List; -import java.util.Random; - -import javax.xml.bind.annotation.XmlAttribute; -import javax.xml.bind.annotation.XmlRootElement; - /** * Tests XmlSource. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeEstimateFractionTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeEstimateFractionTest.java index 50ec2721d3de..92daf89cad2f 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeEstimateFractionTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeEstimateFractionTest.java @@ -21,7 +21,6 @@ import static org.junit.Assert.assertThat; import com.google.common.collect.ImmutableList; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeInterpolateKeyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeInterpolateKeyTest.java index e8010673e13c..a6445eb537f3 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeInterpolateKeyTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeInterpolateKeyTest.java @@ -23,7 +23,6 @@ import static org.junit.Assert.assertThat; import com.google.common.collect.ImmutableList; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeTest.java index f068ec9be73a..40f6d8fe9a79 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeTest.java @@ -27,14 +27,12 @@ import com.google.common.collect.ImmutableList; import com.google.protobuf.ByteString; - +import java.util.Arrays; +import java.util.List; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** * Tests for {@link ByteKeyRange}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyTest.java index b8c85ee1292e..1117ac7ea10f 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyTest.java @@ -24,12 +24,11 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import java.util.Arrays; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; - /** * Tests of {@link ByteKey}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/GcpOptionsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/GcpOptionsTest.java index ec2902e2f538..34077a2bdfcd 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/GcpOptionsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/GcpOptionsTest.java @@ -18,20 +18,21 @@ package org.apache.beam.sdk.options; import static com.google.common.base.Strings.isNullOrEmpty; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; +import com.google.common.collect.ImmutableMap; +import com.google.common.io.Files; +import java.io.File; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.Map; import org.apache.beam.sdk.options.GcpOptions.DefaultProjectFactory; import org.apache.beam.sdk.testing.RestoreSystemProperties; import org.apache.beam.sdk.util.NoopPathValidator; - -import com.google.common.collect.ImmutableMap; -import com.google.common.io.Files; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; @@ -39,11 +40,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.File; -import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.util.Map; - /** Tests for {@link GcpOptions}. */ @RunWith(JUnit4.class) public class GcpOptionsTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/GoogleApiDebugOptionsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/GoogleApiDebugOptionsTest.java index 8e1439bd7de1..dae7208487e1 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/GoogleApiDebugOptionsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/GoogleApiDebugOptionsTest.java @@ -21,14 +21,12 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.api.services.bigquery.Bigquery.Datasets.Delete; +import com.google.api.services.storage.Storage; import org.apache.beam.sdk.options.GoogleApiDebugOptions.GoogleApiTracer; import org.apache.beam.sdk.util.TestCredential; import org.apache.beam.sdk.util.Transport; - -import com.google.api.services.bigquery.Bigquery.Datasets.Delete; -import com.google.api.services.storage.Storage; - -import com.fasterxml.jackson.databind.ObjectMapper; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java index 0c1b5963c849..70c89837ca5b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java @@ -27,6 +27,18 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.auto.service.AutoService; +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ListMultimap; +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import java.util.List; +import java.util.Map; +import java.util.Set; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.runners.PipelineRunner; @@ -34,15 +46,6 @@ import org.apache.beam.sdk.testing.CrashingRunner; import org.apache.beam.sdk.testing.ExpectedLogs; import org.apache.beam.sdk.testing.RestoreSystemProperties; - -import com.google.auto.service.AutoService; -import com.google.common.collect.ArrayListMultimap; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ListMultimap; - -import com.fasterxml.jackson.annotation.JsonIgnore; -import com.fasterxml.jackson.annotation.JsonProperty; import org.hamcrest.Matchers; import org.junit.Rule; import org.junit.Test; @@ -51,12 +54,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.ByteArrayOutputStream; -import java.io.PrintStream; -import java.util.List; -import java.util.Map; -import java.util.Set; - /** Tests for {@link PipelineOptionsFactory}. */ @RunWith(JUnit4.class) public class PipelineOptionsFactoryTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsReflectorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsReflectorTest.java index 8f801c79688a..7632b50ab279 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsReflectorTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsReflectorTest.java @@ -26,9 +26,9 @@ import static org.hamcrest.Matchers.isOneOf; import static org.hamcrest.Matchers.not; -import com.google.common.collect.ImmutableSet; - import com.fasterxml.jackson.annotation.JsonIgnore; +import com.google.common.collect.ImmutableSet; +import java.util.Set; import org.hamcrest.FeatureMatcher; import org.hamcrest.Matcher; import org.hamcrest.Matchers; @@ -36,8 +36,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Set; - /** * Unit tests for {@link PipelineOptionsReflector}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsTest.java index b2efa61f6943..012a5b04c55d 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsTest.java @@ -23,21 +23,19 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; - import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import java.io.IOException; +import java.util.List; +import java.util.Set; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.util.List; -import java.util.Set; - /** Unit tests for {@link PipelineOptions}. */ @RunWith(JUnit4.class) public class PipelineOptionsTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsValidatorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsValidatorTest.java index 2b684a8a047c..80d2a60b678d 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsValidatorTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsValidatorTest.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.options; import org.apache.beam.sdk.testing.CrashingRunner; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java index 148b59080766..1ba6b43821b6 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java @@ -32,19 +32,23 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; -import org.apache.beam.sdk.transforms.display.DisplayData; - +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.JsonMappingException; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Maps; import com.google.common.testing.EqualsTester; - -import com.fasterxml.jackson.annotation.JsonIgnore; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.databind.JsonMappingException; -import com.fasterxml.jackson.databind.ObjectMapper; - +import java.io.IOException; +import java.io.Serializable; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import org.apache.beam.sdk.transforms.display.DisplayData; import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Rule; @@ -55,14 +59,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.io.Serializable; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Set; - /** Tests for {@link ProxyInvocationHandler}. */ @RunWith(JUnit4.class) public class ProxyInvocationHandlerTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java index f9ce018fc479..e98049738615 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java @@ -27,7 +27,6 @@ import org.apache.beam.sdk.testing.CrashingRunner; import org.apache.beam.sdk.util.GcsUtil; import org.apache.beam.sdk.util.TestCredential; - import org.junit.Assert; import org.junit.Before; import org.junit.Test; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java index 9009a777a645..def3a027e9d7 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java @@ -24,6 +24,9 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import java.io.File; +import java.util.Arrays; +import java.util.EnumSet; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.VoidCoder; @@ -41,7 +44,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.PDone; - import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -49,10 +51,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.File; -import java.util.Arrays; -import java.util.EnumSet; - /** * Tests for {@link TransformTreeNode} and {@link TransformHierarchy}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/dataflow/TestCountingSource.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/dataflow/TestCountingSource.java index 10631c2aa6bc..b53d1fccefc2 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/dataflow/TestCountingSource.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/dataflow/TestCountingSource.java @@ -19,6 +19,11 @@ import static org.apache.beam.sdk.util.CoderUtils.encodeToByteArray; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ThreadLocalRandom; +import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.DelegateCoder; import org.apache.beam.sdk.coders.KvCoder; @@ -26,18 +31,10 @@ import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.values.KV; - import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.ThreadLocalRandom; - -import javax.annotation.Nullable; - /** * An unbounded source for testing the unbounded sources framework code. * diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/dataflow/TestCountingSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/dataflow/TestCountingSourceTest.java index 6ba060e8cfef..72a4585104cf 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/dataflow/TestCountingSourceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/dataflow/TestCountingSourceTest.java @@ -22,15 +22,13 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import java.io.IOException; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; - /** * Test the TestCountingSource. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CoderPropertiesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CoderPropertiesTest.java index 5bf55b08d619..f337f36215a9 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CoderPropertiesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CoderPropertiesTest.java @@ -20,13 +20,14 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; +import com.google.common.base.Strings; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; import org.apache.beam.sdk.coders.Coder.Context; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; - -import com.google.common.base.Strings; - import org.hamcrest.CoreMatchers; import org.junit.Assert; import org.junit.Rule; @@ -35,10 +36,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; - /** Unit tests for {@link CoderProperties}. */ @RunWith(JUnit4.class) public class CoderPropertiesTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CrashingRunnerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CrashingRunnerTest.java index 041a73ae2d26..c66aa50cb57f 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CrashingRunnerTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CrashingRunnerTest.java @@ -26,7 +26,6 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.transforms.Create; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ExpectedLogs.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ExpectedLogs.java index caf8fd7919d2..a8e3f9425031 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ExpectedLogs.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ExpectedLogs.java @@ -19,12 +19,6 @@ import static org.junit.Assert.fail; -import org.hamcrest.Description; -import org.hamcrest.Matcher; -import org.hamcrest.TypeSafeMatcher; -import org.junit.rules.ExternalResource; -import org.junit.rules.TestRule; - import java.util.Collection; import java.util.concurrent.ConcurrentLinkedDeque; import java.util.logging.Formatter; @@ -33,8 +27,12 @@ import java.util.logging.LogRecord; import java.util.logging.Logger; import java.util.logging.SimpleFormatter; - import javax.annotation.concurrent.ThreadSafe; +import org.hamcrest.Description; +import org.hamcrest.Matcher; +import org.hamcrest.TypeSafeMatcher; +import org.junit.rules.ExternalResource; +import org.junit.rules.TestRule; /** * This {@link TestRule} enables the ability to capture JUL logging events during test execution and diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ExpectedLogsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ExpectedLogsTest.java index 1d7e18a0db15..84d55844f825 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ExpectedLogsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ExpectedLogsTest.java @@ -19,13 +19,6 @@ import static org.apache.beam.sdk.testing.SystemNanoTimeSleeper.sleepMillis; -import org.junit.Rule; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -35,6 +28,12 @@ import java.util.concurrent.ExecutorCompletionService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** Tests for {@link FastNanoClockAndSleeper}. */ @RunWith(JUnit4.class) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FastNanoClockAndSleeper.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FastNanoClockAndSleeper.java index a97726be545f..6bfafa5a8637 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FastNanoClockAndSleeper.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FastNanoClockAndSleeper.java @@ -19,7 +19,6 @@ import com.google.api.client.util.NanoClock; import com.google.api.client.util.Sleeper; - import org.junit.rules.ExternalResource; import org.junit.rules.TestRule; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FastNanoClockAndSleeperTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FastNanoClockAndSleeperTest.java index 9d15c0fcabb6..7d209515421b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FastNanoClockAndSleeperTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FastNanoClockAndSleeperTest.java @@ -20,13 +20,12 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import java.util.concurrent.TimeUnit; import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.concurrent.TimeUnit; - /** Tests for {@link FastNanoClockAndSleeper}. */ @RunWith(JUnit4.class) public class FastNanoClockAndSleeperTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java index d94ffe2c4761..b2f2ec853573 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java @@ -20,9 +20,12 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsString; +import com.google.common.io.Files; +import java.io.File; +import java.io.IOException; +import java.nio.charset.StandardCharsets; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.util.IOChannelUtils; -import com.google.common.io.Files; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -32,10 +35,6 @@ import org.mockito.Mock; import org.mockito.Mockito; -import java.io.File; -import java.io.IOException; -import java.nio.charset.StandardCharsets; - /** Tests for {@link FileChecksumMatcher}. */ @RunWith(JUnit4.class) public class FileChecksumMatcherTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java index acc2b48f9644..54ddd3f99f1a 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java @@ -22,6 +22,13 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.google.common.collect.Iterables; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.util.regex.Pattern; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.CoderException; @@ -35,10 +42,6 @@ import org.apache.beam.sdk.util.common.ElementByteSizeObserver; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; - -import com.google.common.collect.Iterables; - -import com.fasterxml.jackson.annotation.JsonCreator; import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Rule; @@ -48,12 +51,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.Serializable; -import java.util.regex.Pattern; - /** * Test case for {@link PAssert}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PCollectionViewTesting.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PCollectionViewTesting.java index 517ed689b87a..ea8e0af5030b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PCollectionViewTesting.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PCollectionViewTesting.java @@ -17,6 +17,12 @@ */ package org.apache.beam.sdk.testing; +import com.google.common.base.Function; +import com.google.common.base.MoreObjects; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import java.util.List; +import java.util.Objects; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.transforms.ViewFn; @@ -29,18 +35,9 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValueBase; import org.apache.beam.sdk.values.TupleTag; - -import com.google.common.base.Function; -import com.google.common.base.MoreObjects; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; - import org.joda.time.Duration; import org.joda.time.Instant; -import java.util.List; -import java.util.Objects; - /** * Methods for creating and using {@link PCollectionView} instances. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PaneExtractorsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PaneExtractorsTest.java index 1487eeae0dda..ef501d495c50 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PaneExtractorsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PaneExtractorsTest.java @@ -21,14 +21,12 @@ import static org.hamcrest.Matchers.emptyIterable; import static org.junit.Assert.assertThat; +import com.google.common.collect.ImmutableList; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing; import org.apache.beam.sdk.util.WindowedValue; - -import com.google.common.collect.ImmutableList; - import org.joda.time.Instant; import org.junit.Rule; import org.junit.Test; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/RestoreSystemProperties.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/RestoreSystemProperties.java index 423026d28d23..d1d0507a7606 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/RestoreSystemProperties.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/RestoreSystemProperties.java @@ -17,12 +17,11 @@ */ package org.apache.beam.sdk.testing; -import org.junit.rules.ExternalResource; -import org.junit.rules.TestRule; - import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; +import org.junit.rules.ExternalResource; +import org.junit.rules.TestRule; /** * Saves and restores the current system properties for tests. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SerializableMatchersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SerializableMatchersTest.java index 7cbd4a8c8b92..db5ff2e3049b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SerializableMatchersTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SerializableMatchersTest.java @@ -22,17 +22,17 @@ import static org.apache.beam.sdk.testing.SerializableMatchers.containsInAnyOrder; import static org.apache.beam.sdk.testing.SerializableMatchers.kvWithKey; import static org.apache.beam.sdk.testing.SerializableMatchers.not; - import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; +import com.google.common.collect.ImmutableList; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.values.KV; - -import com.google.common.collect.ImmutableList; - import org.hamcrest.Matchers; import org.junit.Rule; import org.junit.Test; @@ -40,10 +40,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.Serializable; - /** * Test case for {@link SerializableMatchers}. * diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SourceTestUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SourceTestUtilsTest.java index f2b332bb7dd3..efb385de630a 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SourceTestUtilsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SourceTestUtilsTest.java @@ -20,21 +20,18 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; +import com.google.common.collect.Sets; +import java.util.List; +import java.util.Set; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.BoundedSource.BoundedReader; import org.apache.beam.sdk.io.CountingSource; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; - -import com.google.common.collect.Sets; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.List; -import java.util.Set; - /** * Tests for {@link SourceTestUtils}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/StaticWindowsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/StaticWindowsTest.java index fd715dc5a567..e6626190006e 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/StaticWindowsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/StaticWindowsTest.java @@ -19,13 +19,11 @@ import static org.junit.Assert.assertThat; +import com.google.common.collect.ImmutableList; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; - -import com.google.common.collect.ImmutableList; - import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Rule; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SystemNanoTimeSleeper.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SystemNanoTimeSleeper.java index 254a1d57e2b9..810b6f182aaa 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SystemNanoTimeSleeper.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SystemNanoTimeSleeper.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.testing; import com.google.api.client.util.Sleeper; - import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.LockSupport; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SystemNanoTimeSleeperTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SystemNanoTimeSleeperTest.java index 623224d83896..fe97675e7e7a 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SystemNanoTimeSleeperTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SystemNanoTimeSleeperTest.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.testing; import static org.apache.beam.sdk.testing.SystemNanoTimeSleeper.sleepMillis; - import static org.junit.Assert.assertTrue; import org.junit.Test; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java index 0bd789326d3e..ed65f158753d 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java @@ -23,14 +23,17 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertThat; +import com.fasterxml.jackson.databind.ObjectMapper; +import java.util.Arrays; +import java.util.Date; +import java.util.List; +import java.util.UUID; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.options.ApplicationNameOptions; import org.apache.beam.sdk.options.GcpOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.Create; - -import com.fasterxml.jackson.databind.ObjectMapper; import org.hamcrest.BaseMatcher; import org.hamcrest.Description; import org.junit.Rule; @@ -40,11 +43,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.Date; -import java.util.List; -import java.util.UUID; - /** Tests for {@link TestPipeline}. */ @RunWith(JUnit4.class) public class TestPipelineTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java index df37d7f967a5..6457f910308f 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java @@ -23,6 +23,7 @@ import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.junit.Assert.assertThat; +import java.io.Serializable; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; @@ -50,7 +51,6 @@ import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Rule; @@ -60,8 +60,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; - /** * Tests for {@link TestStream}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/WindowSupplierTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/WindowSupplierTest.java index 1a2fd1d56041..1ab4c275eac8 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/WindowSupplierTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/WindowSupplierTest.java @@ -19,14 +19,16 @@ import static org.junit.Assert.assertThat; +import com.google.common.collect.ImmutableList; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Collections; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.util.SerializableUtils; - -import com.google.common.collect.ImmutableList; - import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Rule; @@ -35,11 +37,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.Collections; - /** * Tests for {@link WindowSupplier}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateQuantilesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateQuantilesTest.java index fc10d4b29b37..ab1394697c4f 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateQuantilesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateQuantilesTest.java @@ -19,10 +19,14 @@ import static org.apache.beam.sdk.TestUtils.checkCombineFn; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.collection.IsIterableContainingInOrder.contains; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.KvCoder; @@ -34,7 +38,6 @@ import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - import org.hamcrest.CoreMatchers; import org.hamcrest.Description; import org.hamcrest.Matcher; @@ -44,12 +47,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Comparator; -import java.util.List; - /** * Tests for {@link ApproximateQuantiles}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateUniqueTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateUniqueTest.java index 7b6d671aa935..ba1ddfeaf2cd 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateUniqueTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateUniqueTest.java @@ -18,12 +18,18 @@ package org.apache.beam.sdk.transforms; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.TestUtils; import org.apache.beam.sdk.testing.NeedsRunner; @@ -34,21 +40,11 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; - import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - /** * Tests for the ApproximateUnique aggregator transform. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineFnsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineFnsTest.java index 95ba1aa0d8d1..35f985855dae 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineFnsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineFnsTest.java @@ -19,9 +19,15 @@ import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; - import static org.junit.Assert.assertThat; +import com.google.common.collect.ImmutableList; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.Coder; @@ -43,9 +49,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; - -import com.google.common.collect.ImmutableList; - import org.hamcrest.Matchers; import org.junit.Rule; import org.junit.Test; @@ -54,13 +57,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.Serializable; -import java.util.Arrays; -import java.util.List; - /** * Unit tests for {@link CombineFns}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java index 897d17a03638..77a1d6b03e69 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java @@ -17,18 +17,31 @@ */ package org.apache.beam.sdk.transforms; +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.base.Preconditions.checkState; import static org.apache.beam.sdk.TestUtils.checkCombineFn; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasNamespace; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; -import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Preconditions.checkNotNull; -import static com.google.common.base.Preconditions.checkState; - import static org.hamcrest.Matchers.hasItem; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.MoreObjects; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Sets; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Objects; +import java.util.Set; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.BigEndianLongCoder; @@ -66,13 +79,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.POutput; - -import com.google.common.base.MoreObjects; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Sets; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; import org.hamcrest.Matchers; import org.joda.time.Duration; import org.junit.Test; @@ -81,16 +87,6 @@ import org.junit.runners.JUnit4; import org.mockito.Mock; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.Serializable; -import java.util.Arrays; -import java.util.HashSet; -import java.util.List; -import java.util.Objects; -import java.util.Set; - /** * Tests for Combine transforms. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CountTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CountTest.java index ca898b3bc2b9..7f77ae7f64c3 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CountTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CountTest.java @@ -18,9 +18,10 @@ package org.apache.beam.sdk.transforms; import static org.apache.beam.sdk.TestUtils.NO_LINES; - import static org.junit.Assert.assertEquals; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.testing.PAssert; @@ -28,15 +29,11 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** * Tests for Count. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java index 9db01368cc66..2a89a1837df7 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java @@ -21,13 +21,23 @@ import static org.apache.beam.sdk.TestUtils.LINES_ARRAY; import static org.apache.beam.sdk.TestUtils.NO_LINES; import static org.apache.beam.sdk.TestUtils.NO_LINES_ARRAY; - import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Random; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; @@ -50,10 +60,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; - import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Rule; @@ -63,16 +69,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Random; - /** * Tests for Create. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnDelegatingAggregatorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnDelegatingAggregatorTest.java index 4e8d06c3e597..25b909aabbaf 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnDelegatingAggregatorTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnDelegatingAggregatorTest.java @@ -25,7 +25,6 @@ import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.OldDoFn.DelegatingAggregator; - import org.junit.Before; import org.junit.Rule; import org.junit.Test; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTest.java index 3fb3193f4d5a..e5f5cb6547f5 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTest.java @@ -23,13 +23,13 @@ import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertThat; +import java.io.Serializable; import org.apache.beam.sdk.Pipeline.PipelineExecutionException; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.Max.MaxIntegerFn; import org.apache.beam.sdk.transforms.display.DisplayData; - import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -37,8 +37,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; - /** Tests for {@link DoFn}. */ @RunWith(JUnit4.class) public class DoFnTest implements Serializable { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java index e379f1127923..2f1519cc4276 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java @@ -24,6 +24,7 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; +import java.util.List; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; @@ -32,15 +33,12 @@ import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TimestampedValue; - import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.List; - /** * Tests for {@link DoFnTester}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FilterTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FilterTest.java index 2edab05418e8..5221f75b956b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FilterTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FilterTest.java @@ -18,22 +18,19 @@ package org.apache.beam.sdk.transforms; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - import static org.hamcrest.MatcherAssert.assertThat; +import java.io.Serializable; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.RunnableOnService; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.values.PCollection; - import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; - /** * Tests for {@link Filter}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsTest.java index cb7892cad572..311c8de369d4 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsTest.java @@ -18,10 +18,15 @@ package org.apache.beam.sdk.transforms; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertThat; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import java.io.Serializable; +import java.util.Collections; +import java.util.List; +import java.util.Set; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.PAssert; @@ -30,10 +35,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptor; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; - import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -41,11 +42,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.Collections; -import java.util.List; -import java.util.Set; - /** * Tests for {@link FlatMapElements}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java index 3469223c38fe..d755e28b8548 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java @@ -23,6 +23,13 @@ import static org.apache.beam.sdk.TestUtils.NO_LINES; import static org.apache.beam.sdk.TestUtils.NO_LINES_ARRAY; +import com.google.common.collect.ImmutableSet; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Set; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CollectionCoder; @@ -41,9 +48,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.PCollectionView; - -import com.google.common.collect.ImmutableSet; - import org.joda.time.Duration; import org.junit.Assert; import org.junit.Rule; @@ -53,13 +57,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.List; -import java.util.Set; - /** * Tests for Flatten. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java index afe460fcdc7c..bea0e2d90185 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java @@ -19,7 +19,6 @@ import static org.apache.beam.sdk.TestUtils.KvMatcher.isKv; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.hasItem; import static org.hamcrest.Matchers.empty; @@ -27,6 +26,19 @@ import static org.hamcrest.core.Is.is; import static org.junit.Assert.assertThat; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.google.common.base.Function; +import com.google.common.collect.Iterables; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ThreadLocalRandom; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; @@ -51,11 +63,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TypeDescriptor; - -import com.google.common.base.Function; -import com.google.common.collect.Iterables; - -import com.fasterxml.jackson.annotation.JsonCreator; import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Assert; @@ -66,17 +73,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ThreadLocalRandom; - /** * Tests for GroupByKey. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/IntraBundleParallelizationTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/IntraBundleParallelizationTest.java index fa2fae96d419..b9afd35513df 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/IntraBundleParallelizationTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/IntraBundleParallelizationTest.java @@ -20,7 +20,6 @@ import static org.apache.beam.sdk.testing.SystemNanoTimeSleeper.sleepMillis; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; - import static org.hamcrest.Matchers.both; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.greaterThanOrEqualTo; @@ -31,20 +30,18 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; +import java.util.ArrayList; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.display.DisplayData; - import org.junit.Before; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.ArrayList; -import java.util.concurrent.atomic.AtomicInteger; - /** * Tests for RateLimiter. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/KeysTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/KeysTest.java index cf3094068bf2..fce5b2f205c3 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/KeysTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/KeysTest.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.transforms; +import java.util.Arrays; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.KvCoder; @@ -26,14 +27,11 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; - /** * Tests for Keys transform. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/KvSwapTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/KvSwapTest.java index 1a2d7f6ca060..359819875415 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/KvSwapTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/KvSwapTest.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.transforms; +import java.util.Arrays; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.KvCoder; @@ -26,14 +27,11 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; - /** * Tests for KvSwap transform. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MapElementsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MapElementsTest.java index 7217bca663fa..4a34c57b58b6 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MapElementsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MapElementsTest.java @@ -18,11 +18,12 @@ package org.apache.beam.sdk.transforms; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasItem; import static org.junit.Assert.assertThat; +import java.io.Serializable; +import java.util.Set; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.PAssert; @@ -33,7 +34,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptor; - import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -41,9 +41,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.Set; - /** * Tests for {@link MapElements}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MaxTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MaxTest.java index 87fa5541fa6b..5c78b3f15bb3 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MaxTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MaxTest.java @@ -19,14 +19,11 @@ import static org.apache.beam.sdk.TestUtils.checkCombineFn; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; -import org.apache.beam.sdk.transforms.display.DisplayData; - import com.google.common.collect.Lists; - +import org.apache.beam.sdk.transforms.display.DisplayData; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MeanTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MeanTest.java index 65c876ef0754..1c94e35a957a 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MeanTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MeanTest.java @@ -18,23 +18,19 @@ package org.apache.beam.sdk.transforms; import static org.apache.beam.sdk.TestUtils.checkCombineFn; - import static org.junit.Assert.assertEquals; +import com.google.common.collect.Lists; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.transforms.Mean.CountSum; import org.apache.beam.sdk.transforms.Mean.CountSumCoder; - -import com.google.common.collect.Lists; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** * Tests for Mean. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MinTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MinTest.java index cd03a74c1894..a0eca072e2bf 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MinTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MinTest.java @@ -20,14 +20,11 @@ import static org.apache.beam.sdk.TestUtils.checkCombineFn; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; -import org.apache.beam.sdk.transforms.display.DisplayData; - import com.google.common.collect.Lists; - +import org.apache.beam.sdk.transforms.display.DisplayData; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/NoOpOldDoFn.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/NoOpOldDoFn.java index 8f2bd5e4a43a..c73251000650 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/NoOpOldDoFn.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/NoOpOldDoFn.java @@ -24,7 +24,6 @@ import org.apache.beam.sdk.util.WindowingInternals; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; - import org.joda.time.Instant; /** diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/OldDoFnTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/OldDoFnTest.java index 5946d9a0dff9..e7ae135c5683 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/OldDoFnTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/OldDoFnTest.java @@ -24,6 +24,9 @@ import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertThat; +import com.google.common.collect.ImmutableMap; +import java.io.Serializable; +import java.util.Map; import org.apache.beam.sdk.AggregatorValues; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.Pipeline.PipelineExecutionException; @@ -34,9 +37,6 @@ import org.apache.beam.sdk.transforms.Max.MaxIntegerFn; import org.apache.beam.sdk.transforms.Sum.SumIntegerFn; import org.apache.beam.sdk.transforms.display.DisplayData; - -import com.google.common.collect.ImmutableMap; - import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -44,9 +44,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.Map; - /** * Tests for OldDoFn. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PTransformTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PTransformTest.java index f7074c35714f..bfe8225c9dc9 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PTransformTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PTransformTest.java @@ -22,7 +22,6 @@ import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.values.PCollection; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoLifecycleTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoLifecycleTest.java index c4ba8b7049b5..f69c867bbe22 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoLifecycleTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoLifecycleTest.java @@ -25,21 +25,19 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; +import java.io.Serializable; +import java.util.concurrent.atomic.AtomicBoolean; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.RunnableOnService; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; - import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.concurrent.atomic.AtomicBoolean; - /** * Tests that {@link ParDo} exercises {@link DoFn} methods in the appropriate sequence. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java index 13dec9adfee8..0a4b3cd271f8 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.transforms; +import static com.google.common.base.Preconditions.checkNotNull; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasKey; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasType; @@ -24,7 +25,6 @@ import static org.apache.beam.sdk.util.SerializableUtils.serializeToByteArray; import static org.apache.beam.sdk.util.StringUtils.byteArrayToJsonString; import static org.apache.beam.sdk.util.StringUtils.jsonStringToByteArray; -import static com.google.common.base.Preconditions.checkNotNull; import static org.hamcrest.Matchers.allOf; import static org.hamcrest.Matchers.anyOf; import static org.hamcrest.Matchers.equalTo; @@ -34,6 +34,15 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.CoderException; @@ -56,9 +65,6 @@ import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; - -import com.fasterxml.jackson.annotation.JsonCreator; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Rule; @@ -68,15 +74,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - /** * Tests for ParDo. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PartitionTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PartitionTest.java index 0cc804ef6213..1cbe344adc54 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PartitionTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PartitionTest.java @@ -18,11 +18,13 @@ package org.apache.beam.sdk.transforms; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.PAssert; @@ -32,7 +34,6 @@ import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; - import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -40,10 +41,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.List; - /** * Tests for {@link Partition}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RemoveDuplicatesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RemoveDuplicatesTest.java index 0e919d2c83b7..312cba6b45fb 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RemoveDuplicatesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RemoveDuplicatesTest.java @@ -20,6 +20,10 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.testing.PAssert; @@ -27,17 +31,11 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - /** * Tests for RemovedDuplicates. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SampleTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SampleTest.java index e7f8cd00930e..a0555fa60800 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SampleTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SampleTest.java @@ -17,16 +17,21 @@ */ package org.apache.beam.sdk.transforms; +import static com.google.common.base.Preconditions.checkArgument; import static org.apache.beam.sdk.TestUtils.LINES; import static org.apache.beam.sdk.TestUtils.NO_LINES; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - -import static com.google.common.base.Preconditions.checkArgument; - import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import com.google.common.base.Joiner; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.TreeSet; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; @@ -36,21 +41,11 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.base.Joiner; - import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.Set; -import java.util.TreeSet; - /** * Tests for Sample transform. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SimpleStatsFnsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SimpleStatsFnsTest.java index dc8355b2620a..a782ecc20152 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SimpleStatsFnsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SimpleStatsFnsTest.java @@ -19,15 +19,14 @@ import static org.junit.Assert.assertEquals; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; import org.joda.time.Instant; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - /** * Tests of Min, Max, Mean, and Sum. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SumTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SumTest.java index 4af6db19a58d..b4f723d0d6f8 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SumTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SumTest.java @@ -18,19 +18,16 @@ package org.apache.beam.sdk.transforms; import static org.apache.beam.sdk.TestUtils.checkCombineFn; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; +import com.google.common.collect.Lists; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.BigEndianLongCoder; import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.coders.DoubleCoder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.coders.VarLongCoder; - -import com.google.common.collect.Lists; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/TopTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/TopTest.java index fc0e659f5b9a..b6242524b6a3 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/TopTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/TopTest.java @@ -18,10 +18,14 @@ package org.apache.beam.sdk.transforms; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; +import java.io.Serializable; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.KvCoder; @@ -35,7 +39,6 @@ import org.apache.beam.sdk.transforms.windowing.Window.Bound; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - import org.hamcrest.Matchers; import org.joda.time.Duration; import org.junit.Rule; @@ -45,12 +48,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.Arrays; -import java.util.Collections; -import java.util.Comparator; -import java.util.List; - /** Tests for Top. */ @RunWith(JUnit4.class) public class TopTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ValuesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ValuesTest.java index 238ba7b64842..0bf2e2e9232e 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ValuesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ValuesTest.java @@ -19,6 +19,7 @@ import static org.junit.Assert.assertEquals; +import java.util.Arrays; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.KvCoder; @@ -28,14 +29,11 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; - /** * Tests for Values transform. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java index 170e6ce093de..69b618607057 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java @@ -17,16 +17,26 @@ */ package org.apache.beam.sdk.transforms; -import static org.apache.beam.sdk.values.KV.of; - import static com.google.common.base.Preconditions.checkArgument; - +import static org.apache.beam.sdk.values.KV.of; import static org.hamcrest.Matchers.isA; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import com.google.common.collect.ImmutableList; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.NoSuchElementException; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.Pipeline.PipelineExecutionException; import org.apache.beam.sdk.coders.Coder; @@ -52,9 +62,6 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TypeDescriptor; - -import com.google.common.collect.ImmutableList; - import org.hamcrest.Matchers; import org.joda.time.Duration; import org.joda.time.Instant; @@ -66,18 +73,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.NoSuchElementException; - /** * Tests for {@link View}. See also {@link ParDoTest}, which * provides additional coverage since views can only be diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithKeysTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithKeysTest.java index c23dd3694ad7..f958807fcf77 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithKeysTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithKeysTest.java @@ -19,6 +19,8 @@ import static org.junit.Assert.assertEquals; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.testing.NeedsRunner; @@ -27,15 +29,11 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptor; - import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** * Tests for ExtractKeys transform. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsTest.java index e3814708788e..923b97c52ee3 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsTest.java @@ -19,6 +19,7 @@ import static org.hamcrest.Matchers.isA; +import java.io.Serializable; import org.apache.beam.sdk.Pipeline.PipelineExecutionException; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.PAssert; @@ -26,7 +27,6 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Rule; @@ -36,8 +36,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; - /** * Tests for {@link WithTimestamps}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluator.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluator.java index dc8c1e9fe90c..1783a73101d3 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluator.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluator.java @@ -17,6 +17,9 @@ */ package org.apache.beam.sdk.transforms.display; +import com.google.common.collect.Sets; +import java.util.Objects; +import java.util.Set; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; @@ -29,11 +32,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.POutput; -import com.google.common.collect.Sets; - -import java.util.Objects; -import java.util.Set; - /** * Test utilities to evaluate the {@link DisplayData} in the context of a {@link PipelineRunner}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluatorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluatorTest.java index e2331146ea8d..7630779bb3e1 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluatorTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluatorTest.java @@ -18,11 +18,12 @@ package org.apache.beam.sdk.transforms.display; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.not; import static org.junit.Assert.assertThat; +import java.io.Serializable; +import java.util.Set; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; @@ -30,14 +31,10 @@ import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.POutput; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.Set; - /** * Unit tests for {@link DisplayDataEvaluator}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchers.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchers.java index 025a1f765e5d..e9db5222232b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchers.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchers.java @@ -19,10 +19,9 @@ import static org.hamcrest.Matchers.allOf; -import org.apache.beam.sdk.transforms.display.DisplayData.Item; - import com.google.common.collect.Sets; - +import java.util.Collection; +import org.apache.beam.sdk.transforms.display.DisplayData.Item; import org.hamcrest.CustomTypeSafeMatcher; import org.hamcrest.Description; import org.hamcrest.FeatureMatcher; @@ -32,8 +31,6 @@ import org.joda.time.Duration; import org.joda.time.Instant; -import java.util.Collection; - /** * Hamcrest matcher for making assertions on {@link DisplayData} instances. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchersTest.java index fa44390ab627..3ea6830cbc3f 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchersTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchersTest.java @@ -22,7 +22,6 @@ import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasType; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasValue; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; @@ -30,7 +29,6 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.display.DisplayData.Builder; import org.apache.beam.sdk.values.PCollection; - import org.hamcrest.Description; import org.hamcrest.Matcher; import org.hamcrest.StringDescription; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java index e2f38b44190a..a709bd80af47 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java @@ -24,7 +24,6 @@ import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasType; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasValue; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; - import static org.hamcrest.Matchers.allOf; import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.everyItem; @@ -41,20 +40,23 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableMultimap; +import com.google.common.collect.Multimap; +import com.google.common.testing.EqualsTester; +import java.io.IOException; +import java.io.Serializable; +import java.util.Collection; +import java.util.Map; +import java.util.regex.Pattern; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.display.DisplayData.Builder; import org.apache.beam.sdk.transforms.display.DisplayData.Item; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableMultimap; -import com.google.common.collect.Multimap; -import com.google.common.testing.EqualsTester; - -import com.fasterxml.jackson.databind.JsonNode; -import com.fasterxml.jackson.databind.ObjectMapper; import org.hamcrest.CustomTypeSafeMatcher; import org.hamcrest.FeatureMatcher; import org.hamcrest.Matcher; @@ -69,12 +71,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.io.Serializable; -import java.util.Collection; -import java.util.Map; -import java.util.regex.Pattern; - /** * Tests for {@link DisplayData} class. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGbkResultCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGbkResultCoderTest.java index 0a0a3f132020..18ecd9bab2e6 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGbkResultCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGbkResultCoderTest.java @@ -19,6 +19,7 @@ import static org.junit.Assert.assertFalse; +import com.google.common.collect.ImmutableList; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.DoubleCoder; @@ -28,9 +29,6 @@ import org.apache.beam.sdk.transforms.join.CoGbkResult.CoGbkResultCoder; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; - -import com.google.common.collect.ImmutableList; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGbkResultTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGbkResultTest.java index aa7cb5c3e5a8..50a623772d13 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGbkResultTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGbkResultTest.java @@ -22,18 +22,16 @@ import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertThat; +import java.util.ArrayList; +import java.util.List; import org.apache.beam.sdk.util.common.Reiterable; import org.apache.beam.sdk.util.common.Reiterator; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.ArrayList; -import java.util.List; - /** * Tests the CoGbkResult. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGroupByKeyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGroupByKeyTest.java index c6f82ecb0ca4..e8c8b15ddbfd 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGroupByKeyTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGroupByKeyTest.java @@ -21,6 +21,12 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; +import com.google.common.collect.Iterables; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.KvCoder; @@ -40,21 +46,12 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TupleTag; - -import com.google.common.collect.Iterables; - import org.joda.time.Duration; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Map; - /** * Tests for CoGroupByKeyTest. Implements Serializable for anonymous DoFns. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/UnionCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/UnionCoderTest.java index fb80fb65490d..41ba95289dc1 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/UnionCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/UnionCoderTest.java @@ -19,18 +19,16 @@ import static org.junit.Assert.assertEquals; +import java.util.Arrays; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.DoubleCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.util.CloudObject; import org.apache.beam.sdk.util.Serializer; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; - /** * Tests the UnionCoder. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java index 28f0143d69c1..9317ea21d558 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java @@ -24,7 +24,6 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.UserCodeException; - import org.junit.Before; import org.junit.Rule; import org.junit.Test; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesTest.java index 1a26df22cc34..447b99322440 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesTest.java @@ -17,19 +17,16 @@ */ package org.apache.beam.sdk.transforms.reflect; -import org.apache.beam.sdk.transforms.DoFn; - import com.google.common.reflect.TypeToken; - +import java.lang.reflect.Method; +import java.util.List; +import org.apache.beam.sdk.transforms.DoFn; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.lang.reflect.Method; -import java.util.List; - /** Tests for {@link DoFnSignatures}. */ @RunWith(JUnit4.class) public class DoFnSignaturesTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterAllTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterAllTest.java index b7980132cb5f..b5912296bd84 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterAllTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterAllTest.java @@ -24,7 +24,6 @@ import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger; import org.apache.beam.sdk.util.TriggerTester; import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Test; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterEachTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterEachTest.java index df557eb30132..c413c6ed25c4 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterEachTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterEachTest.java @@ -24,7 +24,6 @@ import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger; import org.apache.beam.sdk.util.TriggerTester; import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Before; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterFirstTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterFirstTest.java index a16669425dd1..415060b6c22b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterFirstTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterFirstTest.java @@ -25,7 +25,6 @@ import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger; import org.apache.beam.sdk.util.TriggerTester; import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Before; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterPaneTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterPaneTest.java index 76ee49c0b402..38d030ec6be1 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterPaneTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterPaneTest.java @@ -23,7 +23,6 @@ import org.apache.beam.sdk.util.TriggerTester; import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Test; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTimeTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTimeTest.java index ea9c2b088e22..13a7acf8ca1e 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTimeTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTimeTest.java @@ -24,7 +24,6 @@ import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger; import org.apache.beam.sdk.util.TriggerTester; import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Test; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTimeTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTimeTest.java index 4c089db19c64..7e6e938f3c8b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTimeTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTimeTest.java @@ -23,7 +23,6 @@ import org.apache.beam.sdk.util.TriggerTester; import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Test; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterWatermarkTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterWatermarkTest.java index 418f746ea452..084027b3e5b1 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterWatermarkTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterWatermarkTest.java @@ -26,7 +26,6 @@ import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger; import org.apache.beam.sdk.util.TriggerTester; import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Before; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/CalendarWindowsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/CalendarWindowsTest.java index 4598a2777c11..54cdd06195ec 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/CalendarWindowsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/CalendarWindowsTest.java @@ -20,12 +20,15 @@ import static org.apache.beam.sdk.testing.WindowFnTestUtils.runWindowFn; import static org.apache.beam.sdk.testing.WindowFnTestUtils.set; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; import org.apache.beam.sdk.transforms.display.DisplayData; - import org.joda.time.DateTime; import org.joda.time.DateTimeConstants; import org.joda.time.DateTimeZone; @@ -34,12 +37,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; - /** * Tests for CalendarWindows WindowFn. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/DefaultTriggerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/DefaultTriggerTest.java index 6ed1c8190df0..673e5554b008 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/DefaultTriggerTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/DefaultTriggerTest.java @@ -23,7 +23,6 @@ import org.apache.beam.sdk.util.TriggerTester; import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Test; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/FixedWindowsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/FixedWindowsTest.java index fc1caac40c09..afa5d60b8a91 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/FixedWindowsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/FixedWindowsTest.java @@ -20,7 +20,6 @@ import static org.apache.beam.sdk.testing.WindowFnTestUtils.runWindowFn; import static org.apache.beam.sdk.testing.WindowFnTestUtils.set; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - import static org.hamcrest.CoreMatchers.containsString; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -28,20 +27,18 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; import org.apache.beam.sdk.testing.WindowFnTestUtils; import org.apache.beam.sdk.transforms.display.DisplayData; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; -import java.util.Set; - /** * Tests for FixedWindows WindowFn. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/IntervalWindowTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/IntervalWindowTest.java index aaface2d658b..fdfbdcb1bbca 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/IntervalWindowTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/IntervalWindowTest.java @@ -20,20 +20,17 @@ import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertThat; +import com.google.common.collect.Lists; +import java.util.List; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.InstantCoder; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.util.CoderUtils; - -import com.google.common.collect.Lists; - import org.joda.time.Instant; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.List; - /** * Tests for {@link Window}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/NeverTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/NeverTest.java index ed64f84ee46d..fb2b4d5c730d 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/NeverTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/NeverTest.java @@ -23,7 +23,6 @@ import org.apache.beam.sdk.util.TriggerTester; import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester; import org.apache.beam.sdk.values.TimestampedValue; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Before; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTriggerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTriggerTest.java index 93971793cfe2..7289d97d7b63 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTriggerTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTriggerTest.java @@ -24,7 +24,6 @@ import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger; import org.apache.beam.sdk.util.TriggerTester; import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Test; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/PaneInfoTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/PaneInfoTest.java index ea4928ea12df..1ce2d8c8d6a2 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/PaneInfoTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/PaneInfoTest.java @@ -23,7 +23,6 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/RepeatedlyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/RepeatedlyTest.java index 3a33182f0f99..6e8930dbd883 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/RepeatedlyTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/RepeatedlyTest.java @@ -27,7 +27,6 @@ import org.apache.beam.sdk.util.TriggerTester; import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Test; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SessionsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SessionsTest.java index a543359e9c76..b13168818cf8 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SessionsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SessionsTest.java @@ -20,30 +20,26 @@ import static org.apache.beam.sdk.testing.WindowFnTestUtils.runWindowFn; import static org.apache.beam.sdk.testing.WindowFnTestUtils.set; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - import static org.hamcrest.Matchers.allOf; import static org.hamcrest.Matchers.containsString; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; +import com.google.common.collect.ImmutableList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; import org.apache.beam.sdk.testing.WindowFnTestUtils; import org.apache.beam.sdk.transforms.display.DisplayData; - -import com.google.common.collect.ImmutableList; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; - /** * Tests for Sessions WindowFn. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SlidingWindowsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SlidingWindowsTest.java index 047a413242b5..54c01a828bc1 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SlidingWindowsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SlidingWindowsTest.java @@ -20,26 +20,23 @@ import static org.apache.beam.sdk.testing.WindowFnTestUtils.runWindowFn; import static org.apache.beam.sdk.testing.WindowFnTestUtils.set; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; import org.apache.beam.sdk.testing.WindowFnTestUtils; import org.apache.beam.sdk.transforms.display.DisplayData; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; -import java.util.Set; - /** * Tests for the SlidingWindows WindowFn. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/StubTrigger.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/StubTrigger.java index 06218cf2c25b..b258a791fd40 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/StubTrigger.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/StubTrigger.java @@ -18,10 +18,8 @@ package org.apache.beam.sdk.transforms.windowing; import com.google.common.collect.Lists; - -import org.joda.time.Instant; - import java.util.List; +import org.joda.time.Instant; /** * No-op {@link OnceTrigger} implementation for testing. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/TriggerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/TriggerTest.java index 43c8bd8c5d76..cfc03b29beca 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/TriggerTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/TriggerTest.java @@ -21,14 +21,13 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import java.util.Arrays; +import java.util.List; import org.joda.time.Instant; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** * Tests for {@link Trigger}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java index c583860d2834..9744fc6c78ba 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java @@ -20,7 +20,6 @@ import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasKey; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; - import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.isOneOf; import static org.hamcrest.Matchers.not; @@ -29,6 +28,7 @@ import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.when; +import java.io.Serializable; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.Coder.NonDeterministicException; @@ -44,7 +44,6 @@ import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TimestampedValue; - import org.hamcrest.Matchers; import org.joda.time.Duration; import org.joda.time.Instant; @@ -56,8 +55,6 @@ import org.junit.runners.JUnit4; import org.mockito.Mockito; -import java.io.Serializable; - /** * Tests for {@link Window}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowingTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowingTest.java index 159e7004f80f..ab208dd57cc3 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowingTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowingTest.java @@ -17,6 +17,10 @@ */ package org.apache.beam.sdk.transforms.windowing; +import java.io.File; +import java.io.FileOutputStream; +import java.io.PrintStream; +import java.io.Serializable; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.io.TextIO; @@ -34,7 +38,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.TimestampedValue; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Rule; @@ -44,11 +47,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.File; -import java.io.FileOutputStream; -import java.io.PrintStream; -import java.io.Serializable; - /** Unit tests for bucketing. */ @RunWith(JUnit4.class) public class WindowingTest implements Serializable { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ApiSurfaceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ApiSurfaceTest.java index b3f874360777..4b76277a027a 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ApiSurfaceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ApiSurfaceTest.java @@ -28,7 +28,10 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; - +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; import org.hamcrest.Description; import org.hamcrest.Matcher; import org.hamcrest.TypeSafeDiagnosingMatcher; @@ -36,11 +39,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Set; - /** * Tests for ApiSurface. These both test the functionality and also that our * public API is conformant to a hard-coded policy. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOffTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOffTest.java index 08f5f5631056..59e0fb7dab28 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOffTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOffTest.java @@ -25,10 +25,8 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import org.apache.beam.sdk.testing.FastNanoClockAndSleeper; - import com.google.api.client.util.BackOff; - +import org.apache.beam.sdk.testing.FastNanoClockAndSleeper; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOffTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOffTest.java index b4a075ccb371..3cfa961865fb 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOffTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOffTest.java @@ -26,7 +26,6 @@ import static org.junit.Assert.assertTrue; import com.google.api.client.util.BackOff; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AvroUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AvroUtilsTest.java index b72ab9ac1334..d8c345c822c0 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AvroUtilsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AvroUtilsTest.java @@ -19,29 +19,27 @@ import static org.junit.Assert.assertEquals; -import org.apache.beam.sdk.coders.AvroCoder; -import org.apache.beam.sdk.coders.DefaultCoder; -import org.apache.beam.sdk.util.AvroUtils.AvroMetadata; - +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Random; import org.apache.avro.Schema; import org.apache.avro.file.CodecFactory; import org.apache.avro.file.DataFileConstants; import org.apache.avro.file.DataFileWriter; import org.apache.avro.io.DatumWriter; import org.apache.avro.reflect.Nullable; +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.util.AvroUtils.AvroMetadata; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Random; - /** * Tests for AvroUtils. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BucketingFunctionTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BucketingFunctionTest.java index ee5a2b3ff4b6..196b90440388 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BucketingFunctionTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BucketingFunctionTest.java @@ -23,7 +23,6 @@ import static org.junit.Assert.assertTrue; import org.apache.beam.sdk.transforms.Combine; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStreamTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStreamTest.java index 09d8992e0db7..36f7028627ee 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStreamTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStreamTest.java @@ -21,18 +21,7 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; -import org.apache.beam.sdk.coders.ByteArrayCoder; -import org.apache.beam.sdk.coders.Coder.Context; - import com.google.common.collect.ImmutableList; - -import org.hamcrest.collection.IsIterableContainingInOrder; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -42,6 +31,14 @@ import java.util.Collections; import java.util.List; import java.util.Random; +import org.apache.beam.sdk.coders.ByteArrayCoder; +import org.apache.beam.sdk.coders.Coder.Context; +import org.hamcrest.collection.IsIterableContainingInOrder; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; /** * Tests for {@link BufferedElementCountingOutputStream}. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CoderUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CoderUtilsTest.java index 0733c4c707ab..4bd2f1916692 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CoderUtilsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CoderUtilsTest.java @@ -18,12 +18,13 @@ package org.apache.beam.sdk.util; import static org.apache.beam.sdk.util.CoderUtils.makeCloudEncoding; - import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyString; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; +import java.io.InputStream; +import java.io.OutputStream; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.Coder; @@ -34,7 +35,6 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.testing.CoderPropertiesTest.ClosingCoder; - import org.hamcrest.CoreMatchers; import org.junit.Assert; import org.junit.Rule; @@ -43,9 +43,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.InputStream; -import java.io.OutputStream; - /** * Tests for CoderUtils. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CombineFnUtilTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CombineFnUtilTest.java index fdb69e4deff7..fe81275ef663 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CombineFnUtilTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CombineFnUtilTest.java @@ -22,14 +22,16 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.withSettings; +import com.google.common.collect.ImmutableList; +import java.io.ByteArrayOutputStream; +import java.io.NotSerializableException; +import java.io.ObjectOutputStream; +import java.util.List; import org.apache.beam.sdk.transforms.CombineWithContext.CombineFnWithContext; import org.apache.beam.sdk.transforms.CombineWithContext.Context; import org.apache.beam.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext; import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.util.state.StateContexts; - -import com.google.common.collect.ImmutableList; - import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -37,11 +39,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.ByteArrayOutputStream; -import java.io.NotSerializableException; -import java.io.ObjectOutputStream; -import java.util.List; - /** * Unit tests for {@link CombineFnUtil}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExecutableTriggerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExecutableTriggerTest.java index fdcd1bd8cea8..1e3a1ff8b9ee 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExecutableTriggerTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExecutableTriggerTest.java @@ -20,17 +20,15 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertSame; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Trigger; - import org.joda.time.Instant; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** * Tests for {@link ExecutableTrigger}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayInputStreamTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayInputStreamTest.java index d717cafd001f..31cf1a81b4ab 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayInputStreamTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayInputStreamTest.java @@ -22,13 +22,12 @@ import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertSame; +import java.io.ByteArrayInputStream; +import java.io.IOException; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.ByteArrayInputStream; -import java.io.IOException; - /** Unit tests for {@link ExposedByteArrayInputStream}. */ @RunWith(JUnit4.class) public class ExposedByteArrayInputStreamTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStreamTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStreamTest.java index 9819a9b1a769..a3a7a1d2d6cf 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStreamTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStreamTest.java @@ -22,13 +22,12 @@ import static org.junit.Assert.assertNotSame; import static org.junit.Assert.assertSame; +import java.io.ByteArrayOutputStream; +import java.io.IOException; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.ByteArrayOutputStream; -import java.io.IOException; - /** Unit tests for {@link ExposedByteArrayOutputStream}. */ @RunWith(JUnit4.class) public class ExposedByteArrayOutputStreamTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FileIOChannelFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FileIOChannelFactoryTest.java index 79e6e5cd6b0c..011b4f5a977c 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FileIOChannelFactoryTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FileIOChannelFactoryTest.java @@ -25,15 +25,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.io.Files; import com.google.common.io.LineReader; - -import org.hamcrest.Matchers; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - import java.io.File; import java.io.FileNotFoundException; import java.io.Reader; @@ -42,6 +33,13 @@ import java.nio.charset.StandardCharsets; import java.nio.file.Path; import java.util.List; +import org.hamcrest.Matchers; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; /** Tests for {@link FileIOChannelFactory}. */ @RunWith(JUnit4.class) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FinishedTriggersSetTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FinishedTriggersSetTest.java index b3b18561d3e4..072d264f231c 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FinishedTriggersSetTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FinishedTriggersSetTest.java @@ -21,12 +21,11 @@ import static org.hamcrest.Matchers.theInstance; import static org.junit.Assert.assertThat; +import java.util.HashSet; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.HashSet; - /** * Tests for {@link FinishedTriggersSet}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GatherAllPanesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GatherAllPanesTest.java index a6522efd36c9..d195623d4634 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GatherAllPanesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GatherAllPanesTest.java @@ -19,6 +19,8 @@ import static org.junit.Assert.fail; +import com.google.common.collect.Iterables; +import java.io.Serializable; import org.apache.beam.sdk.io.CountingInput; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.PAssert; @@ -36,9 +38,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.TypeDescriptor; - -import com.google.common.collect.Iterables; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Test; @@ -46,8 +45,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; - /** * Tests for {@link GatherAllPanes}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsIOChannelFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsIOChannelFactoryTest.java index aae313715c75..6bdb782530d6 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsIOChannelFactoryTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsIOChannelFactoryTest.java @@ -21,7 +21,6 @@ import org.apache.beam.sdk.options.GcsOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; - import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java index 997340a22fa2..681b0aaaa059 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java @@ -29,11 +29,6 @@ import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.when; -import org.apache.beam.sdk.options.GcsOptions; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.testing.FastNanoClockAndSleeper; -import org.apache.beam.sdk.util.gcsfs.GcsPath; - import com.google.api.client.googleapis.batch.BatchRequest; import com.google.api.client.googleapis.json.GoogleJsonError.ErrorInfo; import com.google.api.client.googleapis.json.GoogleJsonResponseException; @@ -58,14 +53,6 @@ import com.google.cloud.hadoop.gcsio.GoogleCloudStorageReadChannel; import com.google.cloud.hadoop.util.ClientRequestHelper; import com.google.common.collect.ImmutableList; - -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; -import org.mockito.Mockito; - import java.io.FileNotFoundException; import java.io.IOException; import java.math.BigInteger; @@ -79,6 +66,16 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +import org.apache.beam.sdk.options.GcsOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.testing.FastNanoClockAndSleeper; +import org.apache.beam.sdk.util.gcsfs.GcsPath; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.Mockito; /** Test case for {@link GcsUtil}. */ @RunWith(JUnit4.class) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IOChannelUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IOChannelUtilsTest.java index 8a7eb0216954..d92d3cd2e10a 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IOChannelUtilsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IOChannelUtilsTest.java @@ -23,7 +23,8 @@ import static org.junit.Assert.fail; import com.google.common.io.Files; - +import java.io.File; +import java.nio.charset.StandardCharsets; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -31,9 +32,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.File; -import java.nio.charset.StandardCharsets; - /** * Tests for IOChannelUtils. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IdentitySideInputWindowFn.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IdentitySideInputWindowFn.java index 705003eb2e6b..2b48b9fc4dd7 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IdentitySideInputWindowFn.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IdentitySideInputWindowFn.java @@ -17,15 +17,14 @@ */ package org.apache.beam.sdk.util; +import java.util.Collection; +import java.util.Collections; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.NonMergingWindowFn; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import java.util.Collection; -import java.util.Collections; - /** * A {@link WindowFn} for use during tests that returns the input window for calls to * {@link #getSideInputWindow(BoundedWindow)}. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/InstanceBuilderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/InstanceBuilderTest.java index 8d197baa1b7b..505b2d325a4d 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/InstanceBuilderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/InstanceBuilderTest.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.util; import org.apache.beam.sdk.values.TupleTag; - import org.hamcrest.Matchers; import org.junit.Assert; import org.junit.Rule; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/KeyedWorkItemCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/KeyedWorkItemCoderTest.java index 2151f8805057..1974d9e705bc 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/KeyedWorkItemCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/KeyedWorkItemCoderTest.java @@ -17,15 +17,13 @@ */ package org.apache.beam.sdk.util; +import com.google.common.collect.ImmutableList; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.TimerInternals.TimerData; import org.apache.beam.sdk.util.state.StateNamespaces; - -import com.google.common.collect.ImmutableList; - import org.joda.time.Instant; import org.junit.Test; import org.junit.runner.RunWith; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/MergingActiveWindowSetTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/MergingActiveWindowSetTest.java index 4750af1c8cad..676a25ab42a0 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/MergingActiveWindowSetTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/MergingActiveWindowSetTest.java @@ -22,16 +22,17 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.Sessions; import org.apache.beam.sdk.util.state.InMemoryStateInternals; import org.apache.beam.sdk.util.state.StateInternals; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.After; @@ -40,10 +41,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - /** * Test NonMergingActiveWindowSet. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/MovingFunctionTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/MovingFunctionTest.java index b95f235c430f..8d57bf4d24b6 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/MovingFunctionTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/MovingFunctionTest.java @@ -23,7 +23,6 @@ import static org.junit.Assert.assertTrue; import org.apache.beam.sdk.transforms.Combine; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/MutationDetectorsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/MutationDetectorsTest.java index 0763912b5667..ebd8297bbf80 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/MutationDetectorsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/MutationDetectorsTest.java @@ -17,26 +17,23 @@ */ package org.apache.beam.sdk.util; +import com.google.common.collect.FluentIterable; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import java.util.Arrays; +import java.util.LinkedList; +import java.util.List; +import java.util.Set; import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.coders.ListCoder; import org.apache.beam.sdk.coders.VarIntCoder; - -import com.google.common.collect.FluentIterable; -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.LinkedList; -import java.util.List; -import java.util.Set; - /** * Tests for {@link MutationDetectors}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PTupleTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PTupleTest.java index fe2014a7d3c4..abead0409c2a 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PTupleTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PTupleTest.java @@ -23,7 +23,6 @@ import static org.junit.Assert.assertTrue; import org.apache.beam.sdk.values.TupleTag; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubClientTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubClientTest.java index 07031240d16c..1a99d38c714f 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubClientTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubClientTest.java @@ -20,12 +20,11 @@ import static org.junit.Assert.assertEquals; +import com.google.common.collect.ImmutableMap; +import java.util.Map; import org.apache.beam.sdk.util.PubsubClient.ProjectPath; import org.apache.beam.sdk.util.PubsubClient.SubscriptionPath; import org.apache.beam.sdk.util.PubsubClient.TopicPath; - -import com.google.common.collect.ImmutableMap; - import org.joda.time.Instant; import org.junit.Rule; import org.junit.Test; @@ -33,8 +32,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Map; - /** * Tests for helper classes and methods in PubsubClient. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubGrpcClientTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubGrpcClientTest.java index 4d0ec29ca282..b36b93474f3b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubGrpcClientTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubGrpcClientTest.java @@ -20,11 +20,6 @@ import static org.junit.Assert.assertEquals; -import org.apache.beam.sdk.util.PubsubClient.IncomingMessage; -import org.apache.beam.sdk.util.PubsubClient.OutgoingMessage; -import org.apache.beam.sdk.util.PubsubClient.SubscriptionPath; -import org.apache.beam.sdk.util.PubsubClient.TopicPath; - import com.google.auth.oauth2.GoogleCredentials; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -38,9 +33,13 @@ import com.google.pubsub.v1.PullResponse; import com.google.pubsub.v1.ReceivedMessage; import com.google.pubsub.v1.SubscriberGrpc; - import io.grpc.ManagedChannel; - +import java.io.IOException; +import java.util.List; +import org.apache.beam.sdk.util.PubsubClient.IncomingMessage; +import org.apache.beam.sdk.util.PubsubClient.OutgoingMessage; +import org.apache.beam.sdk.util.PubsubClient.SubscriptionPath; +import org.apache.beam.sdk.util.PubsubClient.TopicPath; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -48,9 +47,6 @@ import org.junit.runners.JUnit4; import org.mockito.Mockito; -import java.io.IOException; -import java.util.List; - /** * Tests for PubsubGrpcClient. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubJsonClientTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubJsonClientTest.java index 21689358197d..b6d7ccb2c0e3 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubJsonClientTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubJsonClientTest.java @@ -20,11 +20,6 @@ import static org.junit.Assert.assertEquals; -import org.apache.beam.sdk.util.PubsubClient.IncomingMessage; -import org.apache.beam.sdk.util.PubsubClient.OutgoingMessage; -import org.apache.beam.sdk.util.PubsubClient.SubscriptionPath; -import org.apache.beam.sdk.util.PubsubClient.TopicPath; - import com.google.api.services.pubsub.Pubsub; import com.google.api.services.pubsub.model.PublishRequest; import com.google.api.services.pubsub.model.PublishResponse; @@ -34,7 +29,12 @@ import com.google.api.services.pubsub.model.ReceivedMessage; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; - +import java.io.IOException; +import java.util.List; +import org.apache.beam.sdk.util.PubsubClient.IncomingMessage; +import org.apache.beam.sdk.util.PubsubClient.OutgoingMessage; +import org.apache.beam.sdk.util.PubsubClient.SubscriptionPath; +import org.apache.beam.sdk.util.PubsubClient.TopicPath; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -42,9 +42,6 @@ import org.junit.runners.JUnit4; import org.mockito.Mockito; -import java.io.IOException; -import java.util.List; - /** * Tests for PubsubJsonClient. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubTestClientTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubTestClientTest.java index d86f5e7ca89d..b9b1d3f8bc8e 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubTestClientTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubTestClientTest.java @@ -20,25 +20,22 @@ import static org.junit.Assert.assertEquals; +import com.google.api.client.util.Clock; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import java.io.IOException; +import java.util.List; +import java.util.concurrent.atomic.AtomicLong; import org.apache.beam.sdk.util.PubsubClient.IncomingMessage; import org.apache.beam.sdk.util.PubsubClient.OutgoingMessage; import org.apache.beam.sdk.util.PubsubClient.SubscriptionPath; import org.apache.beam.sdk.util.PubsubClient.TopicPath; import org.apache.beam.sdk.util.PubsubTestClient.PubsubTestClientFactory; - -import com.google.api.client.util.Clock; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.util.List; -import java.util.concurrent.atomic.AtomicLong; - /** * Tests for PubsubTestClient. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTest.java index f4ae996143c5..d990ee066e27 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTest.java @@ -19,6 +19,8 @@ import static org.junit.Assert.assertEquals; +import com.google.common.collect.ImmutableList; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; @@ -33,17 +35,12 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.ImmutableList; - import org.joda.time.Duration; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.List; - /** * Tests for {@link Reshuffle}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTriggerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTriggerTest.java index b17ce81cf482..83077f4cc37f 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTriggerTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTriggerTest.java @@ -25,7 +25,6 @@ import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.Trigger; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Test; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/RetryHttpRequestInitializerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/RetryHttpRequestInitializerTest.java index 91d74db619c4..71554b573ac3 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/RetryHttpRequestInitializerTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/RetryHttpRequestInitializerTest.java @@ -46,7 +46,11 @@ import com.google.api.client.util.Sleeper; import com.google.api.services.storage.Storage; import com.google.api.services.storage.Storage.Objects.Get; - +import java.io.IOException; +import java.net.SocketTimeoutException; +import java.security.PrivateKey; +import java.util.Arrays; +import java.util.concurrent.atomic.AtomicLong; import org.hamcrest.Matchers; import org.junit.After; import org.junit.Assert; @@ -59,12 +63,6 @@ import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import java.io.IOException; -import java.net.SocketTimeoutException; -import java.security.PrivateKey; -import java.util.Arrays; -import java.util.concurrent.atomic.AtomicLong; - /** * Tests for RetryHttpRequestInitializer. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SerializableUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SerializableUtilsTest.java index 30406fcb10f7..5435a4588e84 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SerializableUtilsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SerializableUtilsTest.java @@ -19,26 +19,23 @@ import static org.junit.Assert.assertEquals; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.util.List; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.DeterministicStandardCoder; - -import com.google.common.collect.ImmutableList; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.Serializable; -import java.util.List; - /** Tests for {@link SerializableUtils}. */ @RunWith(JUnit4.class) public class SerializableUtilsTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StreamUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StreamUtilsTest.java index 60eeb0daa88d..7a31184dc585 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StreamUtilsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StreamUtilsTest.java @@ -21,16 +21,15 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertSame; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - import java.io.BufferedInputStream; import java.io.ByteArrayInputStream; import java.io.IOException; import java.io.InputStream; import java.util.Arrays; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; /** Unit tests for {@link ExposedByteArrayInputStream}. */ @RunWith(JUnit4.class) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StringUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StringUtilsTest.java index e87bbee58f68..042e9e3d76ec 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StringUtilsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StringUtilsTest.java @@ -24,7 +24,6 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PDone; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StructsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StructsTest.java index 59dbb56ea0e9..91090d1a2d6d 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StructsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StructsTest.java @@ -34,18 +34,17 @@ import static org.apache.beam.sdk.util.Structs.getString; import static org.apache.beam.sdk.util.Structs.getStrings; -import org.hamcrest.Matchers; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import org.hamcrest.Matchers; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; /** * Tests for Structs. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TimerInternalsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TimerInternalsTest.java index bc2930c65c37..e8ffdb34411e 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TimerInternalsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TimerInternalsTest.java @@ -30,7 +30,6 @@ import org.apache.beam.sdk.util.TimerInternals.TimerDataCoder; import org.apache.beam.sdk.util.state.StateNamespace; import org.apache.beam.sdk.util.state.StateNamespaces; - import org.joda.time.Instant; import org.junit.Test; import org.junit.runner.RunWith; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TriggerTester.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TriggerTester.java index 4892bbd23c72..a1f1d21fc3dc 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TriggerTester.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TriggerTester.java @@ -20,9 +20,21 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; - import static org.junit.Assert.assertTrue; +import com.google.common.base.MoreObjects; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.PriorityQueue; +import java.util.Set; +import javax.annotation.Nullable; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; @@ -41,26 +53,9 @@ import org.apache.beam.sdk.util.state.StateTag; import org.apache.beam.sdk.util.state.WatermarkHoldState; import org.apache.beam.sdk.values.TimestampedValue; - -import com.google.common.base.MoreObjects; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; - import org.joda.time.Duration; import org.joda.time.Instant; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.PriorityQueue; -import java.util.Set; - -import javax.annotation.Nullable; - /** * Test utility that runs a {@link Trigger}, using in-memory stub implementation to provide * the {@link StateInternals}. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UnownedInputStreamTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UnownedInputStreamTest.java index dac147c9036c..e3a8d714c176 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UnownedInputStreamTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UnownedInputStreamTest.java @@ -20,6 +20,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import java.io.ByteArrayInputStream; import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -27,8 +28,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.ByteArrayInputStream; - /** Unit tests for {@link UnownedInputStream}. */ @RunWith(JUnit4.class) public class UnownedInputStreamTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UnownedOutputStreamTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UnownedOutputStreamTest.java index 30761e7ef703..e36b9fbbd9be 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UnownedOutputStreamTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UnownedOutputStreamTest.java @@ -19,6 +19,7 @@ import static org.junit.Assert.assertEquals; +import java.io.ByteArrayOutputStream; import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -26,8 +27,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.ByteArrayOutputStream; - /** Unit tests for {@link UnownedOutputStream}. */ @RunWith(JUnit4.class) public class UnownedOutputStreamTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UploadIdResponseInterceptorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UploadIdResponseInterceptorTest.java index 572897732524..8b9f77e66fbf 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UploadIdResponseInterceptorTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UploadIdResponseInterceptorTest.java @@ -17,22 +17,19 @@ */ package org.apache.beam.sdk.util; -import org.apache.beam.sdk.testing.ExpectedLogs; - import com.google.api.client.http.GenericUrl; import com.google.api.client.http.HttpResponse; import com.google.api.client.testing.http.HttpTesting; import com.google.api.client.testing.http.MockHttpTransport; import com.google.api.client.testing.http.MockLowLevelHttpResponse; - +import java.io.IOException; +import org.apache.beam.sdk.testing.ExpectedLogs; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; - /** * A test for {@link UploadIdResponseInterceptor}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UserCodeExceptionTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UserCodeExceptionTest.java index 3be114605928..a6d1cb57ef63 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UserCodeExceptionTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UserCodeExceptionTest.java @@ -24,6 +24,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; +import java.io.IOException; import org.hamcrest.Description; import org.hamcrest.FeatureMatcher; import org.hamcrest.Matcher; @@ -34,8 +35,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; - /** * Tests for {@link UserCodeException} functionality. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/VarIntTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/VarIntTest.java index 82a3689d78a7..02136affbe51 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/VarIntTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/VarIntTest.java @@ -21,17 +21,16 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.EOFException; +import java.io.IOException; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.EOFException; -import java.io.IOException; - /** Unit tests for {@link VarInt}. */ @RunWith(JUnit4.class) public class VarIntTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/WindowedValueTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/WindowedValueTest.java index 90969b784106..0c69a594df5a 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/WindowedValueTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/WindowedValueTest.java @@ -22,6 +22,9 @@ import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertThat; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import java.util.Arrays; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.StringUtf8Coder; @@ -29,18 +32,12 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; - import org.joda.time.Instant; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; - /** Test case for {@link WindowedValue}. */ @RunWith(JUnit4.class) public class WindowedValueTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ZipFilesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ZipFilesTest.java index 31e6b143d3ea..1c038487bd99 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ZipFilesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ZipFilesTest.java @@ -29,14 +29,6 @@ import com.google.common.io.ByteSource; import com.google.common.io.CharSource; import com.google.common.io.Files; - -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - import java.io.File; import java.io.FileOutputStream; import java.io.IOException; @@ -46,6 +38,12 @@ import java.util.Enumeration; import java.util.zip.ZipEntry; import java.util.zip.ZipFile; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; /** * Tests for the {@link ZipFiles} class. These tests make sure that the handling diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/ReflectHelpersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/ReflectHelpersTest.java index 9a5dc92023d0..e1073da48457 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/ReflectHelpersTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/ReflectHelpersTest.java @@ -19,15 +19,13 @@ import static org.junit.Assert.assertEquals; +import java.util.List; +import java.util.Map; import org.apache.beam.sdk.values.TypeDescriptor; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.List; -import java.util.Map; - /** * Tests for {@link ReflectHelpers}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/gcsfs/GcsPathTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/gcsfs/GcsPathTest.java index fdd1dfd6e7aa..5c861845bd3f 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/gcsfs/GcsPathTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/gcsfs/GcsPathTest.java @@ -23,18 +23,17 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; -import org.hamcrest.Matchers; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - import java.net.URI; import java.nio.file.Path; import java.nio.file.Paths; import java.util.Arrays; import java.util.Iterator; import java.util.List; +import org.hamcrest.Matchers; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; /** * Tests of GcsPath. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/CopyOnAccessInMemoryStateInternalsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/CopyOnAccessInMemoryStateInternalsTest.java index b7388ee741d7..ad70bcafe753 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/CopyOnAccessInMemoryStateInternalsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/CopyOnAccessInMemoryStateInternalsTest.java @@ -39,7 +39,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; import org.apache.beam.sdk.transforms.windowing.OutputTimeFns; - import org.joda.time.Instant; import org.junit.Rule; import org.junit.Test; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryStateInternalsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryStateInternalsTest.java index 48d1a30c4cb8..08a6bc1d29d7 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryStateInternalsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryStateInternalsTest.java @@ -21,21 +21,19 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; +import java.util.Arrays; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.OutputTimeFns; - import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; - /** * Tests for {@link InMemoryStateInternals}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/StateNamespacesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/StateNamespacesTest.java index 385ab6c66e8f..f546e561b53a 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/StateNamespacesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/StateNamespacesTest.java @@ -24,7 +24,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; - import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Test; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/StateTagTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/StateTagTest.java index ec7698d0b381..2c8c9ccd00d8 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/StateTagTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/StateTagTest.java @@ -30,7 +30,6 @@ import org.apache.beam.sdk.transforms.Min.MinIntegerFn; import org.apache.beam.sdk.transforms.windowing.OutputTimeFns; import org.apache.beam.sdk.util.CombineFnUtil; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/KVTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/KVTest.java index f87e2ae94e35..202d6f34344d 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/KVTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/KVTest.java @@ -23,13 +23,11 @@ import static org.junit.Assert.assertThat; import com.google.common.collect.ImmutableList; - +import java.util.Comparator; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Comparator; - /** * Tests for KV. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionListTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionListTest.java index af2c14c9bd67..f76bf7e23bf5 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionListTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionListTest.java @@ -21,12 +21,11 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; +import java.util.Collections; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Collections; - /** * Tests for PCollectionLists. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionTupleTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionTupleTest.java index 13218b2979c7..1467ae8a52fb 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionTupleTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionTupleTest.java @@ -21,6 +21,9 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; +import java.io.Serializable; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.RunnableOnService; @@ -30,16 +33,11 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollection.IsBounded; - import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.Arrays; -import java.util.List; - /** Unit tests for {@link PCollectionTuple}. */ @RunWith(JUnit4.class) public final class PCollectionTupleTest implements Serializable { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PDoneTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PDoneTest.java index 5554b313b505..4000e5db0276 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PDoneTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PDoneTest.java @@ -19,6 +19,7 @@ import static org.apache.beam.sdk.TestUtils.LINES; +import java.io.File; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.testing.NeedsRunner; @@ -26,7 +27,6 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.PTransform; - import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; @@ -35,8 +35,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.File; - /** * Tests for PDone. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypeDescriptorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypeDescriptorTest.java index 162c991f8f4e..39472f94ec85 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypeDescriptorTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypeDescriptorTest.java @@ -20,18 +20,16 @@ import static org.junit.Assert.assertEquals; import com.google.common.reflect.TypeToken; - +import java.lang.reflect.Method; +import java.lang.reflect.TypeVariable; +import java.util.List; +import java.util.Set; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.lang.reflect.Method; -import java.lang.reflect.TypeVariable; -import java.util.List; -import java.util.Set; - /** * Tests for TypeDescriptor. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypeDescriptorsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypeDescriptorsTest.java index 59acdf007ddc..1bf0fc9cf20d 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypeDescriptorsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypeDescriptorsTest.java @@ -23,17 +23,15 @@ import static org.apache.beam.sdk.values.TypeDescriptors.lists; import static org.apache.beam.sdk.values.TypeDescriptors.sets; import static org.apache.beam.sdk.values.TypeDescriptors.strings; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; +import java.util.List; +import java.util.Set; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.List; -import java.util.Set; - /** * Tests for {@link TypeDescriptors}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypedPValueTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypedPValueTest.java index 287223f44b15..f33b3a2b692d 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypedPValueTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypedPValueTest.java @@ -28,7 +28,6 @@ import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; diff --git a/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/InnerJoinTest.java b/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/InnerJoinTest.java index 6622fdc67e0e..423ab9c5cdcb 100644 --- a/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/InnerJoinTest.java +++ b/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/InnerJoinTest.java @@ -17,19 +17,17 @@ */ package org.apache.beam.sdk.extensions.joinlibrary; +import java.util.ArrayList; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - import org.junit.Before; import org.junit.Test; -import java.util.ArrayList; -import java.util.List; - /** * This test Inner Join functionality. */ diff --git a/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/OuterLeftJoinTest.java b/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/OuterLeftJoinTest.java index 91b0740f30b3..c32163fd5580 100644 --- a/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/OuterLeftJoinTest.java +++ b/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/OuterLeftJoinTest.java @@ -17,19 +17,17 @@ */ package org.apache.beam.sdk.extensions.joinlibrary; +import java.util.ArrayList; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - import org.junit.Before; import org.junit.Test; -import java.util.ArrayList; -import java.util.List; - /** * This test Outer Left Join functionality. diff --git a/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/OuterRightJoinTest.java b/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/OuterRightJoinTest.java index 7977df77d861..5a45f73e59e7 100644 --- a/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/OuterRightJoinTest.java +++ b/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/OuterRightJoinTest.java @@ -17,19 +17,17 @@ */ package org.apache.beam.sdk.extensions.joinlibrary; +import java.util.ArrayList; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - import org.junit.Before; import org.junit.Test; -import java.util.ArrayList; -import java.util.List; - /** * This test Outer Right Join functionality. diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtils.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtils.java index 48e2258f83fd..7826559e23f8 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtils.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtils.java @@ -26,7 +26,8 @@ import com.google.api.services.bigquery.model.TableSchema; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; - +import java.util.List; +import javax.annotation.Nullable; import org.apache.avro.Schema; import org.apache.avro.Schema.Field; import org.apache.avro.Schema.Type; @@ -34,10 +35,6 @@ import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; -import java.util.List; - -import javax.annotation.Nullable; - /** * A set of utilities for working with Avro files. * diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java index e61dcca993a0..04fb041badd1 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java @@ -21,6 +21,60 @@ import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.api.client.json.JsonFactory; +import com.google.api.client.util.BackOff; +import com.google.api.client.util.BackOffUtils; +import com.google.api.client.util.Sleeper; +import com.google.api.services.bigquery.Bigquery; +import com.google.api.services.bigquery.model.Job; +import com.google.api.services.bigquery.model.JobConfigurationExtract; +import com.google.api.services.bigquery.model.JobConfigurationLoad; +import com.google.api.services.bigquery.model.JobConfigurationQuery; +import com.google.api.services.bigquery.model.JobConfigurationTableCopy; +import com.google.api.services.bigquery.model.JobReference; +import com.google.api.services.bigquery.model.JobStatistics; +import com.google.api.services.bigquery.model.JobStatus; +import com.google.api.services.bigquery.model.TableReference; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import com.google.cloud.hadoop.util.ApiErrorExtractor; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Function; +import com.google.common.base.MoreObjects; +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import com.google.common.io.CountingOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.ObjectInputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.nio.channels.Channels; +import java.nio.channels.WritableByteChannel; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import javax.annotation.Nullable; +import org.apache.avro.generic.GenericRecord; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.Coder; @@ -75,67 +129,10 @@ import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; - -import com.google.api.client.json.JsonFactory; -import com.google.api.client.util.BackOff; -import com.google.api.client.util.BackOffUtils; -import com.google.api.client.util.Sleeper; -import com.google.api.services.bigquery.Bigquery; -import com.google.api.services.bigquery.model.Job; -import com.google.api.services.bigquery.model.JobConfigurationExtract; -import com.google.api.services.bigquery.model.JobConfigurationLoad; -import com.google.api.services.bigquery.model.JobConfigurationQuery; -import com.google.api.services.bigquery.model.JobConfigurationTableCopy; -import com.google.api.services.bigquery.model.JobReference; -import com.google.api.services.bigquery.model.JobStatistics; -import com.google.api.services.bigquery.model.JobStatus; -import com.google.api.services.bigquery.model.TableReference; -import com.google.api.services.bigquery.model.TableRow; -import com.google.api.services.bigquery.model.TableSchema; -import com.google.cloud.hadoop.util.ApiErrorExtractor; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; -import com.google.common.base.MoreObjects; -import com.google.common.base.Strings; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; -import com.google.common.io.CountingOutputStream; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.avro.generic.GenericRecord; import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.io.InputStream; -import java.io.ObjectInputStream; -import java.io.OutputStream; -import java.io.Serializable; -import java.nio.channels.Channels; -import java.nio.channels.WritableByteChannel; -import java.nio.charset.StandardCharsets; -import java.nio.file.Files; -import java.nio.file.Paths; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.NoSuchElementException; -import java.util.Set; -import java.util.UUID; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ThreadLocalRandom; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; -import java.util.regex.Matcher; -import java.util.regex.Pattern; -import javax.annotation.Nullable; - /** * {@link PTransform}s for reading and writing * BigQuery tables. diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServices.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServices.java index 0af6df8a3317..c0951fcdeae5 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServices.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServices.java @@ -17,8 +17,6 @@ */ package org.apache.beam.sdk.io.gcp.bigquery; -import org.apache.beam.sdk.options.BigQueryOptions; - import com.google.api.services.bigquery.model.Dataset; import com.google.api.services.bigquery.model.Job; import com.google.api.services.bigquery.model.JobConfigurationExtract; @@ -30,13 +28,12 @@ import com.google.api.services.bigquery.model.Table; import com.google.api.services.bigquery.model.TableReference; import com.google.api.services.bigquery.model.TableRow; - import java.io.IOException; import java.io.Serializable; import java.util.List; import java.util.NoSuchElementException; - import javax.annotation.Nullable; +import org.apache.beam.sdk.options.BigQueryOptions; /** An interface for real, mock, or fake implementations of Cloud BigQuery services. */ interface BigQueryServices extends Serializable { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java index bd1097f54c0c..6aff3b097710 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java @@ -19,13 +19,6 @@ import static com.google.common.base.Preconditions.checkNotNull; -import org.apache.beam.sdk.options.BigQueryOptions; -import org.apache.beam.sdk.options.GcsOptions; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; -import org.apache.beam.sdk.util.IntervalBoundedExponentialBackOff; -import org.apache.beam.sdk.util.Transport; - import com.google.api.client.googleapis.json.GoogleJsonResponseException; import com.google.api.client.googleapis.services.AbstractGoogleClientRequest; import com.google.api.client.util.BackOff; @@ -51,10 +44,6 @@ import com.google.api.services.bigquery.model.TableRow; import com.google.cloud.hadoop.util.ApiErrorExtractor; import com.google.common.annotations.VisibleForTesting; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.IOException; import java.util.ArrayList; import java.util.LinkedList; @@ -65,8 +54,15 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; - import javax.annotation.Nullable; +import org.apache.beam.sdk.options.BigQueryOptions; +import org.apache.beam.sdk.options.GcsOptions; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; +import org.apache.beam.sdk.util.IntervalBoundedExponentialBackOff; +import org.apache.beam.sdk.util.Transport; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * An implementation of {@link BigQueryServices} that actually communicates with the cloud BigQuery diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableInserter.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableInserter.java index bf038f5e8c50..a64dc9fe9b9a 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableInserter.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableInserter.java @@ -17,10 +17,6 @@ */ package org.apache.beam.sdk.io.gcp.bigquery; -import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition; -import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition; -import org.apache.beam.sdk.options.PipelineOptions; - import com.google.api.client.util.BackOff; import com.google.api.client.util.BackOffUtils; import com.google.api.client.util.ExponentialBackOff; @@ -32,14 +28,14 @@ import com.google.api.services.bigquery.model.TableSchema; import com.google.cloud.hadoop.util.ApiErrorExtractor; import com.google.common.annotations.VisibleForTesting; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.IOException; import java.util.concurrent.TimeUnit; - import javax.annotation.Nullable; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition; +import org.apache.beam.sdk.options.PipelineOptions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Inserts rows into BigQuery. diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableRowIterator.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableRowIterator.java index 3afdffaca595..729da97f7f7e 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableRowIterator.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableRowIterator.java @@ -21,8 +21,6 @@ import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; -import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; - import com.google.api.client.googleapis.services.AbstractGoogleClientRequest; import com.google.api.client.util.BackOff; import com.google.api.client.util.BackOffUtils; @@ -49,11 +47,6 @@ import com.google.common.base.MoreObjects; import com.google.common.collect.ImmutableList; import com.google.common.util.concurrent.Uninterruptibles; - -import org.joda.time.Duration; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.IOException; import java.util.Collection; import java.util.Collections; @@ -64,8 +57,11 @@ import java.util.Objects; import java.util.Random; import java.util.concurrent.TimeUnit; - import javax.annotation.Nullable; +import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; +import org.joda.time.Duration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Iterates over all rows in a table. diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java index bfdf4aaf0a63..3a9ffce487f6 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java @@ -21,6 +21,26 @@ import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; +import com.google.bigtable.v2.MutateRowResponse; +import com.google.bigtable.v2.Mutation; +import com.google.bigtable.v2.Row; +import com.google.bigtable.v2.RowFilter; +import com.google.bigtable.v2.SampleRowKeysResponse; +import com.google.cloud.bigtable.config.BigtableOptions; +import com.google.cloud.bigtable.config.RetryOptions; +import com.google.common.base.MoreObjects; +import com.google.common.collect.ImmutableList; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.protobuf.ByteString; +import io.grpc.Status; +import java.io.IOException; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.concurrent.ConcurrentLinkedQueue; +import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.protobuf.ProtoCoder; @@ -40,32 +60,9 @@ import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; - -import com.google.bigtable.v2.MutateRowResponse; -import com.google.bigtable.v2.Mutation; -import com.google.bigtable.v2.Row; -import com.google.bigtable.v2.RowFilter; -import com.google.bigtable.v2.SampleRowKeysResponse; -import com.google.cloud.bigtable.config.BigtableOptions; -import com.google.cloud.bigtable.config.RetryOptions; -import com.google.common.base.MoreObjects; -import com.google.common.collect.ImmutableList; -import com.google.common.util.concurrent.FutureCallback; -import com.google.common.util.concurrent.Futures; -import com.google.protobuf.ByteString; - -import io.grpc.Status; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.NoSuchElementException; -import java.util.concurrent.ConcurrentLinkedQueue; -import javax.annotation.Nullable; - /** * A bounded source and sink for Google Cloud Bigtable. * diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java index 2a7e3a016a96..ecd38a7468e4 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java @@ -17,20 +17,18 @@ */ package org.apache.beam.sdk.io.gcp.bigtable; -import org.apache.beam.sdk.io.gcp.bigtable.BigtableIO.BigtableSource; -import org.apache.beam.sdk.values.KV; - import com.google.bigtable.v2.MutateRowResponse; import com.google.bigtable.v2.Mutation; import com.google.bigtable.v2.Row; import com.google.bigtable.v2.SampleRowKeysResponse; import com.google.common.util.concurrent.ListenableFuture; import com.google.protobuf.ByteString; - import java.io.IOException; import java.io.Serializable; import java.util.List; import java.util.NoSuchElementException; +import org.apache.beam.sdk.io.gcp.bigtable.BigtableIO.BigtableSource; +import org.apache.beam.sdk.values.KV; /** * An interface for real or fake implementations of Cloud Bigtable. diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java index b9288df11eeb..07a183efc9d9 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java @@ -17,9 +17,6 @@ */ package org.apache.beam.sdk.io.gcp.bigtable; -import org.apache.beam.sdk.io.gcp.bigtable.BigtableIO.BigtableSource; -import org.apache.beam.sdk.values.KV; - import com.google.bigtable.admin.v2.GetTableRequest; import com.google.bigtable.v2.MutateRowRequest; import com.google.bigtable.v2.MutateRowResponse; @@ -40,16 +37,15 @@ import com.google.common.io.Closer; import com.google.common.util.concurrent.ListenableFuture; import com.google.protobuf.ByteString; - import io.grpc.Status.Code; import io.grpc.StatusRuntimeException; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.IOException; import java.util.List; import java.util.NoSuchElementException; +import org.apache.beam.sdk.io.gcp.bigtable.BigtableIO.BigtableSource; +import org.apache.beam.sdk.values.KV; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * An implementation of {@link BigtableService} that actually communicates with the Cloud Bigtable diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java index 852595aa79ab..c7433d37d160 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java @@ -30,27 +30,6 @@ import static com.google.datastore.v1.client.DatastoreHelper.makeUpsert; import static com.google.datastore.v1.client.DatastoreHelper.makeValue; -import org.apache.beam.sdk.annotations.Experimental; -import org.apache.beam.sdk.options.GcpOptions; -import org.apache.beam.sdk.options.PipelineOptions; -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.GroupByKey; -import org.apache.beam.sdk.transforms.MapElements; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.SimpleFunction; -import org.apache.beam.sdk.transforms.Values; -import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.transforms.display.DisplayData.Builder; -import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; -import org.apache.beam.sdk.util.RetryHttpRequestInitializer; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PBegin; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PDone; - import com.google.api.client.auth.oauth2.Credential; import com.google.api.client.util.BackOff; import com.google.api.client.util.BackOffUtils; @@ -76,16 +55,34 @@ import com.google.datastore.v1.client.DatastoreOptions; import com.google.datastore.v1.client.QuerySplitter; import com.google.protobuf.Int32Value; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.IOException; import java.io.Serializable; import java.util.ArrayList; import java.util.List; import java.util.NoSuchElementException; import javax.annotation.Nullable; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.options.GcpOptions; +import org.apache.beam.sdk.options.PipelineOptions; +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.GroupByKey; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.beam.sdk.transforms.Values; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.transforms.display.DisplayData.Builder; +import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; +import org.apache.beam.sdk.util.RetryHttpRequestInitializer; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PDone; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** *

        {@link DatastoreV1} provides an API to Read, Write and Delete {@link PCollection PCollections} diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtilsTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtilsTest.java index 316392fe8f77..59cf1f7c0d2b 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtilsTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtilsTest.java @@ -19,25 +19,22 @@ import static org.junit.Assert.assertEquals; -import org.apache.beam.sdk.coders.AvroCoder; -import org.apache.beam.sdk.coders.DefaultCoder; - import com.google.api.services.bigquery.model.TableFieldSchema; import com.google.api.services.bigquery.model.TableRow; import com.google.api.services.bigquery.model.TableSchema; import com.google.common.collect.Lists; - +import java.util.ArrayList; +import java.util.List; import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; import org.apache.avro.reflect.Nullable; +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.coders.DefaultCoder; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.ArrayList; -import java.util.List; - /** * Tests for {@link BigQueryAvroUtils}. */ diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java index ca60696960e8..a6d7e2f5e957 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java @@ -17,12 +17,10 @@ */ package org.apache.beam.sdk.io.gcp.bigquery; +import static com.google.common.base.Preconditions.checkArgument; import static org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.fromJsonString; import static org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.toJsonString; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - -import static com.google.common.base.Preconditions.checkArgument; - import static org.hamcrest.Matchers.hasItem; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; @@ -34,6 +32,39 @@ import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.when; +import com.google.api.client.util.Data; +import com.google.api.client.util.Strings; +import com.google.api.services.bigquery.model.ErrorProto; +import com.google.api.services.bigquery.model.Job; +import com.google.api.services.bigquery.model.JobConfigurationExtract; +import com.google.api.services.bigquery.model.JobConfigurationLoad; +import com.google.api.services.bigquery.model.JobConfigurationQuery; +import com.google.api.services.bigquery.model.JobConfigurationTableCopy; +import com.google.api.services.bigquery.model.JobReference; +import com.google.api.services.bigquery.model.JobStatistics; +import com.google.api.services.bigquery.model.JobStatistics2; +import com.google.api.services.bigquery.model.JobStatistics4; +import com.google.api.services.bigquery.model.JobStatus; +import com.google.api.services.bigquery.model.Table; +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableReference; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import java.io.File; +import java.io.FileFilter; +import java.io.IOException; +import java.io.Serializable; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Set; +import javax.annotation.Nullable; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.KvCoder; @@ -85,29 +116,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; - -import com.google.api.client.util.Data; -import com.google.api.client.util.Strings; -import com.google.api.services.bigquery.model.ErrorProto; -import com.google.api.services.bigquery.model.Job; -import com.google.api.services.bigquery.model.JobConfigurationExtract; -import com.google.api.services.bigquery.model.JobConfigurationLoad; -import com.google.api.services.bigquery.model.JobConfigurationQuery; -import com.google.api.services.bigquery.model.JobConfigurationTableCopy; -import com.google.api.services.bigquery.model.JobReference; -import com.google.api.services.bigquery.model.JobStatistics; -import com.google.api.services.bigquery.model.JobStatistics2; -import com.google.api.services.bigquery.model.JobStatistics4; -import com.google.api.services.bigquery.model.JobStatus; -import com.google.api.services.bigquery.model.Table; -import com.google.api.services.bigquery.model.TableFieldSchema; -import com.google.api.services.bigquery.model.TableReference; -import com.google.api.services.bigquery.model.TableRow; -import com.google.api.services.bigquery.model.TableSchema; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; - import org.hamcrest.CoreMatchers; import org.hamcrest.Matchers; import org.junit.Assert; @@ -124,19 +132,6 @@ import org.mockito.Mockito; import org.mockito.MockitoAnnotations; -import java.io.File; -import java.io.FileFilter; -import java.io.IOException; -import java.io.Serializable; -import java.nio.file.Files; -import java.nio.file.Paths; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.NoSuchElementException; -import java.util.Set; -import javax.annotation.Nullable; - /** * Tests for BigQueryIO. */ diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java index 686685bd8145..eb5fbe6e05f5 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java @@ -23,15 +23,6 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl.DatasetServiceImpl; -import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl.JobServiceImpl; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.testing.ExpectedLogs; -import org.apache.beam.sdk.testing.FastNanoClockAndSleeper; -import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; -import org.apache.beam.sdk.util.RetryHttpRequestInitializer; -import org.apache.beam.sdk.util.Transport; - import com.google.api.client.googleapis.json.GoogleJsonError; import com.google.api.client.googleapis.json.GoogleJsonError.ErrorInfo; import com.google.api.client.googleapis.json.GoogleJsonErrorContainer; @@ -55,7 +46,19 @@ import com.google.api.services.bigquery.model.TableRow; import com.google.cloud.hadoop.util.ApiErrorExtractor; import com.google.common.collect.ImmutableList; - +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.ArrayList; +import java.util.List; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl.DatasetServiceImpl; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl.JobServiceImpl; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.testing.ExpectedLogs; +import org.apache.beam.sdk.testing.FastNanoClockAndSleeper; +import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; +import org.apache.beam.sdk.util.RetryHttpRequestInitializer; +import org.apache.beam.sdk.util.Transport; import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -65,12 +68,6 @@ import org.mockito.Mock; import org.mockito.MockitoAnnotations; -import java.io.ByteArrayInputStream; -import java.io.IOException; -import java.io.InputStream; -import java.util.ArrayList; -import java.util.List; - /** * Tests for {@link BigQueryServicesImpl}. */ diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableInserterTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableInserterTest.java index dac3911d82a1..fb79c74215d2 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableInserterTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableInserterTest.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.io.gcp.bigquery; import static com.google.common.base.Verify.verifyNotNull; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; import static org.junit.Assert.fail; @@ -28,12 +27,6 @@ import static org.mockito.Mockito.verifyNoMoreInteractions; import static org.mockito.Mockito.when; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.testing.ExpectedLogs; -import org.apache.beam.sdk.util.RetryHttpRequestInitializer; -import org.apache.beam.sdk.util.Transport; - import com.google.api.client.googleapis.json.GoogleJsonError; import com.google.api.client.googleapis.json.GoogleJsonError.ErrorInfo; import com.google.api.client.googleapis.json.GoogleJsonErrorContainer; @@ -51,7 +44,14 @@ import com.google.api.services.bigquery.model.TableReference; import com.google.cloud.hadoop.util.RetryBoundedBackOff; import com.google.common.collect.ImmutableList; - +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.testing.ExpectedLogs; +import org.apache.beam.sdk.util.RetryHttpRequestInitializer; +import org.apache.beam.sdk.util.Transport; import org.junit.After; import org.junit.Before; import org.junit.Rule; @@ -62,10 +62,6 @@ import org.mockito.Mock; import org.mockito.MockitoAnnotations; -import java.io.ByteArrayInputStream; -import java.io.IOException; -import java.io.InputStream; - /** * Tests of {@link BigQueryTableInserter}. */ diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableRowIteratorTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableRowIteratorTest.java index 457b071cac1e..ab848f5b3779 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableRowIteratorTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableRowIteratorTest.java @@ -45,7 +45,10 @@ import com.google.api.services.bigquery.model.TableReference; import com.google.api.services.bigquery.model.TableRow; import com.google.api.services.bigquery.model.TableSchema; - +import java.io.IOException; +import java.util.Arrays; +import java.util.LinkedList; +import java.util.List; import org.junit.After; import org.junit.Before; import org.junit.Rule; @@ -56,11 +59,6 @@ import org.mockito.Mock; import org.mockito.MockitoAnnotations; -import java.io.IOException; -import java.util.Arrays; -import java.util.LinkedList; -import java.util.List; - /** * Tests for {@link BigQueryTableRowIterator}. */ diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilTest.java index 89284df48ee3..e539b33b0ee0 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilTest.java @@ -30,10 +30,6 @@ import static org.mockito.Mockito.verifyNoMoreInteractions; import static org.mockito.Mockito.when; -import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl.DatasetServiceImpl; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.options.PipelineOptionsFactory; - import com.google.api.services.bigquery.Bigquery; import com.google.api.services.bigquery.model.Table; import com.google.api.services.bigquery.model.TableCell; @@ -45,7 +41,14 @@ import com.google.api.services.bigquery.model.TableRow; import com.google.api.services.bigquery.model.TableSchema; import com.google.common.collect.ImmutableList; - +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.LinkedList; +import java.util.List; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl.DatasetServiceImpl; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.hamcrest.Matchers; import org.junit.After; import org.junit.Assert; @@ -60,12 +63,6 @@ import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.LinkedList; -import java.util.List; - /** * Tests for util classes related to BigQuery. */ diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java index adf45a44f38c..61b404ae441d 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.io.gcp.bigtable; +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Verify.verifyNotNull; import static org.apache.beam.sdk.testing.SourceTestUtils.assertSourcesEqualReferenceSource; import static org.apache.beam.sdk.testing.SourceTestUtils.assertSplitAtFractionExhaustive; import static org.apache.beam.sdk.testing.SourceTestUtils.assertSplitAtFractionFails; @@ -25,31 +27,12 @@ import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasKey; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasLabel; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasValue; - -import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Verify.verifyNotNull; import static org.hamcrest.Matchers.allOf; import static org.hamcrest.Matchers.hasSize; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertThat; -import org.apache.beam.sdk.Pipeline.PipelineExecutionException; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.io.BoundedSource.BoundedReader; -import org.apache.beam.sdk.io.gcp.bigtable.BigtableIO.BigtableSource; -import org.apache.beam.sdk.io.range.ByteKey; -import org.apache.beam.sdk.io.range.ByteKeyRange; -import org.apache.beam.sdk.testing.ExpectedLogs; -import org.apache.beam.sdk.testing.PAssert; -import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.transforms.display.DisplayDataEvaluator; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.TypeDescriptor; - import com.google.bigtable.v2.Cell; import com.google.bigtable.v2.Column; import com.google.bigtable.v2.Family; @@ -69,15 +52,6 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.protobuf.ByteString; - -import org.hamcrest.Matchers; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - import java.io.IOException; import java.io.Serializable; import java.util.ArrayList; @@ -90,8 +64,29 @@ import java.util.Set; import java.util.SortedMap; import java.util.TreeMap; - import javax.annotation.Nullable; +import org.apache.beam.sdk.Pipeline.PipelineExecutionException; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.io.BoundedSource.BoundedReader; +import org.apache.beam.sdk.io.gcp.bigtable.BigtableIO.BigtableSource; +import org.apache.beam.sdk.io.range.ByteKey; +import org.apache.beam.sdk.io.range.ByteKeyRange; +import org.apache.beam.sdk.testing.ExpectedLogs; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.transforms.display.DisplayDataEvaluator; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.hamcrest.Matchers; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; /** * Unit tests for {@link BigtableIO}. diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadIT.java index 02d403f9b9ac..a064bd64235b 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadIT.java @@ -17,16 +17,14 @@ */ package org.apache.beam.sdk.io.gcp.bigtable; +import com.google.bigtable.v2.Row; +import com.google.cloud.bigtable.config.BigtableOptions; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.values.PCollection; - -import com.google.bigtable.v2.Row; -import com.google.cloud.bigtable.config.BigtableOptions; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java index ee3a6f97642b..9afcd60f264a 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java @@ -19,14 +19,6 @@ import static org.junit.Assert.assertThat; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.io.CountingInput; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.values.KV; - import com.google.bigtable.admin.v2.ColumnFamily; import com.google.bigtable.admin.v2.CreateTableRequest; import com.google.bigtable.admin.v2.DeleteTableRequest; @@ -44,20 +36,25 @@ import com.google.cloud.bigtable.grpc.scanner.ResultScanner; import com.google.common.collect.ImmutableList; import com.google.protobuf.ByteString; - -import org.hamcrest.Matchers; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - import java.io.IOException; import java.io.Serializable; import java.util.ArrayList; import java.util.Date; import java.util.List; import java.util.Map; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.io.CountingInput; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.KV; +import org.hamcrest.Matchers; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; /** * End-to-end tests of BigtableWrite. diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java index 31b5da43aa1e..ab1df2f3e9f5 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java @@ -17,13 +17,6 @@ */ package org.apache.beam.sdk.io.gcp.datastore; -import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.DATASTORE_BATCH_UPDATE_LIMIT; -import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.DEFAULT_BUNDLE_SIZE_BYTES; -import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.QUERY_BATCH_LIMIT; -import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.getEstimatedSizeBytes; -import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.makeRequest; -import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.isValidKey; -import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static com.google.datastore.v1.PropertyFilter.Operator.EQUAL; import static com.google.datastore.v1.PropertyOrder.Direction.DESCENDING; import static com.google.datastore.v1.client.DatastoreHelper.makeDelete; @@ -32,6 +25,13 @@ import static com.google.datastore.v1.client.DatastoreHelper.makeOrder; import static com.google.datastore.v1.client.DatastoreHelper.makeUpsert; import static com.google.datastore.v1.client.DatastoreHelper.makeValue; +import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.DATASTORE_BATCH_UPDATE_LIMIT; +import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.DEFAULT_BUNDLE_SIZE_BYTES; +import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.QUERY_BATCH_LIMIT; +import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.getEstimatedSizeBytes; +import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.makeRequest; +import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.isValidKey; +import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.lessThanOrEqualTo; @@ -47,6 +47,24 @@ import static org.mockito.Mockito.verifyZeroInteractions; import static org.mockito.Mockito.when; +import com.google.datastore.v1.CommitRequest; +import com.google.datastore.v1.Entity; +import com.google.datastore.v1.EntityResult; +import com.google.datastore.v1.Key; +import com.google.datastore.v1.Mutation; +import com.google.datastore.v1.PartitionId; +import com.google.datastore.v1.Query; +import com.google.datastore.v1.QueryResultBatch; +import com.google.datastore.v1.RunQueryRequest; +import com.google.datastore.v1.RunQueryResponse; +import com.google.datastore.v1.client.Datastore; +import com.google.datastore.v1.client.QuerySplitter; +import com.google.protobuf.Int32Value; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Set; import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.DatastoreWriterFn; import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.DeleteEntity; import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.DeleteEntityFn; @@ -69,21 +87,6 @@ import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.POutput; - -import com.google.datastore.v1.CommitRequest; -import com.google.datastore.v1.Entity; -import com.google.datastore.v1.EntityResult; -import com.google.datastore.v1.Key; -import com.google.datastore.v1.Mutation; -import com.google.datastore.v1.PartitionId; -import com.google.datastore.v1.Query; -import com.google.datastore.v1.QueryResultBatch; -import com.google.datastore.v1.RunQueryRequest; -import com.google.datastore.v1.RunQueryResponse; -import com.google.datastore.v1.client.Datastore; -import com.google.datastore.v1.client.QuerySplitter; -import com.google.protobuf.Int32Value; - import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -96,12 +99,6 @@ import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import java.util.ArrayList; -import java.util.HashSet; -import java.util.LinkedList; -import java.util.List; -import java.util.Set; - /** * Tests for {@link DatastoreV1}. */ diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1ReadIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1ReadIT.java index 8fedc774f661..99988331f775 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1ReadIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1ReadIT.java @@ -23,6 +23,11 @@ import static org.apache.beam.sdk.io.gcp.datastore.V1TestUtil.makeAncestorKey; import static org.apache.beam.sdk.io.gcp.datastore.V1TestUtil.makeEntity; +import com.google.datastore.v1.Entity; +import com.google.datastore.v1.Key; +import com.google.datastore.v1.Query; +import com.google.datastore.v1.client.Datastore; +import java.util.UUID; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.gcp.datastore.V1TestUtil.UpsertMutationBuilder; import org.apache.beam.sdk.io.gcp.datastore.V1TestUtil.V1TestWriter; @@ -31,20 +36,12 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.values.PCollection; - -import com.google.datastore.v1.Entity; -import com.google.datastore.v1.Key; -import com.google.datastore.v1.Query; -import com.google.datastore.v1.client.Datastore; - import org.junit.After; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.UUID; - /** * End-to-end tests for Datastore DatastoreV1.Read. */ diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestOptions.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestOptions.java index 360855fee0f2..9ebba3a4c509 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestOptions.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestOptions.java @@ -18,12 +18,11 @@ package org.apache.beam.sdk.io.gcp.datastore; +import javax.annotation.Nullable; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.testing.TestPipelineOptions; -import javax.annotation.Nullable; - /** * DatastoreV1 Datastore related pipeline options. */ diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java index 1e323ec06a28..a596bb3b7686 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java @@ -25,12 +25,6 @@ import static com.google.datastore.v1.client.DatastoreHelper.makeUpsert; import static com.google.datastore.v1.client.DatastoreHelper.makeValue; -import org.apache.beam.sdk.options.GcpOptions; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; -import org.apache.beam.sdk.util.RetryHttpRequestInitializer; - import com.google.api.client.auth.oauth2.Credential; import com.google.api.client.util.BackOff; import com.google.api.client.util.BackOffUtils; @@ -51,16 +45,19 @@ import com.google.datastore.v1.client.DatastoreFactory; import com.google.datastore.v1.client.DatastoreOptions; import com.google.protobuf.Int32Value; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.IOException; import java.util.ArrayList; import java.util.Iterator; import java.util.List; import java.util.UUID; import javax.annotation.Nullable; +import org.apache.beam.sdk.options.GcpOptions; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; +import org.apache.beam.sdk.util.RetryHttpRequestInitializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; class V1TestUtil { private static final Logger LOG = LoggerFactory.getLogger(V1TestUtil.class); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1WriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1WriteIT.java index b97c05c8c6c4..fa7c1402a5b1 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1WriteIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1WriteIT.java @@ -22,21 +22,19 @@ import static org.apache.beam.sdk.io.gcp.datastore.V1TestUtil.deleteAllEntities; import static org.junit.Assert.assertEquals; +import java.util.UUID; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.CountingInput; import org.apache.beam.sdk.io.gcp.datastore.V1TestUtil.CreateEntityFn; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.ParDo; - import org.junit.After; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.UUID; - /** * End-to-end tests for Datastore DatastoreV1.Write. */ diff --git a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/AvroHDFSFileSource.java b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/AvroHDFSFileSource.java index 9dc926bf66da..26299952daf5 100644 --- a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/AvroHDFSFileSource.java +++ b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/AvroHDFSFileSource.java @@ -17,30 +17,27 @@ */ package org.apache.beam.sdk.io.hdfs; -import org.apache.beam.sdk.coders.AvroCoder; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.KvCoder; -import org.apache.beam.sdk.io.BoundedSource; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.values.KV; - import com.google.common.base.Function; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; - +import java.io.IOException; +import java.util.List; +import javax.annotation.Nullable; import org.apache.avro.Schema; import org.apache.avro.mapred.AvroKey; import org.apache.avro.mapreduce.AvroJob; import org.apache.avro.mapreduce.AvroKeyInputFormat; +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.util.CoderUtils; +import org.apache.beam.sdk.values.KV; import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.mapreduce.InputSplit; import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; -import java.io.IOException; -import java.util.List; -import javax.annotation.Nullable; - /** * A {@code BoundedSource} for reading Avro files resident in a Hadoop filesystem. * diff --git a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/AvroWrapperCoder.java b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/AvroWrapperCoder.java index a831afec8110..c1340c05169d 100644 --- a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/AvroWrapperCoder.java +++ b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/AvroWrapperCoder.java @@ -19,22 +19,20 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.coders.AvroCoder; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.coders.StandardCoder; -import org.apache.beam.sdk.util.CloudObject; -import org.apache.beam.sdk.util.PropertyNames; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.avro.mapred.AvroWrapper; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.util.Collections; import java.util.List; +import org.apache.avro.mapred.AvroWrapper; +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.coders.StandardCoder; +import org.apache.beam.sdk.util.CloudObject; +import org.apache.beam.sdk.util.PropertyNames; /** * A {@code AvroWrapperCoder} is a {@link Coder} for a Java class that implements {@link diff --git a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSink.java b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSink.java index 688447aea656..0b538b355bc1 100644 --- a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSink.java +++ b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSink.java @@ -19,16 +19,18 @@ import static com.google.common.base.Preconditions.checkState; +import com.google.api.client.util.Maps; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import java.io.IOException; +import java.util.Map; +import java.util.Random; +import java.util.Set; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.io.Sink; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.values.KV; - -import com.google.api.client.util.Maps; -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -47,11 +49,6 @@ import org.apache.hadoop.mapreduce.task.JobContextImpl; import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl; -import java.io.IOException; -import java.util.Map; -import java.util.Random; -import java.util.Set; - /** * A {@code Sink} for writing records to a Hadoop filesystem using a Hadoop file-based output * format. diff --git a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSource.java b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSource.java index de68565fabfd..3a4d01fe7746 100644 --- a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSource.java +++ b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSource.java @@ -20,6 +20,19 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; +import com.google.common.base.Function; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import java.io.Externalizable; +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.List; +import java.util.ListIterator; +import java.util.NoSuchElementException; +import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.VoidCoder; @@ -27,11 +40,6 @@ import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.values.KV; - -import com.google.common.base.Function; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; @@ -46,18 +54,6 @@ import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl; -import java.io.Externalizable; -import java.io.IOException; -import java.io.ObjectInput; -import java.io.ObjectOutput; -import java.lang.reflect.InvocationTargetException; -import java.lang.reflect.Method; -import java.util.List; -import java.util.ListIterator; -import java.util.NoSuchElementException; - -import javax.annotation.Nullable; - /** * A {@code BoundedSource} for reading files resident in a Hadoop filesystem (HDFS) using a * Hadoop file-based input format. diff --git a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/WritableCoder.java b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/WritableCoder.java index 4e913ede2d64..f3569eafaf6e 100644 --- a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/WritableCoder.java +++ b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/WritableCoder.java @@ -17,20 +17,20 @@ */ package org.apache.beam.sdk.io.hdfs; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.coders.StandardCoder; -import org.apache.beam.sdk.util.CloudObject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.io.Writable; import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.util.List; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.coders.StandardCoder; +import org.apache.beam.sdk.util.CloudObject; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.Writable; /** * A {@code WritableCoder} is a {@link Coder} for a Java class that implements {@link Writable}. diff --git a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/simpleauth/SimpleAuthAvroHDFSFileSource.java b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/simpleauth/SimpleAuthAvroHDFSFileSource.java index 5dd967302505..d37ced9b84af 100644 --- a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/simpleauth/SimpleAuthAvroHDFSFileSource.java +++ b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/simpleauth/SimpleAuthAvroHDFSFileSource.java @@ -17,18 +17,16 @@ */ package org.apache.beam.sdk.io.hdfs.simpleauth; -import org.apache.beam.sdk.coders.AvroCoder; -import org.apache.beam.sdk.io.hdfs.AvroHDFSFileSource; -import org.apache.beam.sdk.io.hdfs.HDFSFileSource; -import org.apache.beam.sdk.options.PipelineOptions; - import com.google.common.base.Function; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; -import org.apache.hadoop.mapreduce.InputSplit; - import java.util.List; import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.io.hdfs.AvroHDFSFileSource; +import org.apache.beam.sdk.io.hdfs.HDFSFileSource; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.hadoop.mapreduce.InputSplit; /** * Source for Avros on Hadoop/HDFS with Simple Authentication. diff --git a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/simpleauth/SimpleAuthHDFSFileSink.java b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/simpleauth/SimpleAuthHDFSFileSink.java index d0fd8b629085..e2c2c907f1aa 100644 --- a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/simpleauth/SimpleAuthHDFSFileSink.java +++ b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/simpleauth/SimpleAuthHDFSFileSink.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.io.hdfs.simpleauth; +import java.security.PrivilegedExceptionAction; import org.apache.beam.sdk.io.Sink; import org.apache.beam.sdk.io.hdfs.HDFSFileSink; import org.apache.beam.sdk.options.PipelineOptions; @@ -25,8 +26,6 @@ import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; import org.apache.hadoop.security.UserGroupInformation; -import java.security.PrivilegedExceptionAction; - /** * A {@code Sink} for writing records to a Hadoop filesystem using a Hadoop file-based output * format with Simple Authentication. diff --git a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/simpleauth/SimpleAuthHDFSFileSource.java b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/simpleauth/SimpleAuthHDFSFileSource.java index 5b768fc2bc76..6fb340ebd34f 100644 --- a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/simpleauth/SimpleAuthHDFSFileSource.java +++ b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/simpleauth/SimpleAuthHDFSFileSource.java @@ -17,21 +17,19 @@ */ package org.apache.beam.sdk.io.hdfs.simpleauth; +import com.google.common.base.Function; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import java.util.List; +import javax.annotation.Nullable; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.io.hdfs.HDFSFileSource; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.values.KV; - -import com.google.common.base.Function; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; import org.apache.hadoop.mapreduce.InputSplit; import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; -import java.util.List; -import javax.annotation.Nullable; - /** * Source for Hadoop/HDFS with Simple Authentication. * diff --git a/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/AvroWrapperCoderTest.java b/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/AvroWrapperCoderTest.java index 85cbd46178f7..6ebea3a2baf3 100644 --- a/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/AvroWrapperCoderTest.java +++ b/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/AvroWrapperCoderTest.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.io.hdfs; -import org.apache.beam.sdk.coders.AvroCoder; -import org.apache.beam.sdk.testing.CoderProperties; - import org.apache.avro.mapred.AvroKey; import org.apache.avro.mapred.AvroValue; +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.testing.CoderProperties; import org.junit.Test; /** diff --git a/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/HDFSFileSourceTest.java b/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/HDFSFileSourceTest.java index 67df7bcb4bb4..6145952af4ad 100644 --- a/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/HDFSFileSourceTest.java +++ b/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/HDFSFileSourceTest.java @@ -24,13 +24,17 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Random; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.Source; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.SourceTestUtils; import org.apache.beam.sdk.values.KV; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.IntWritable; @@ -42,12 +46,6 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; -import java.io.File; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Random; - /** * Tests for HDFSFileSource. */ diff --git a/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/WritableCoderTest.java b/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/WritableCoderTest.java index ac32c336f998..e78f850c6635 100644 --- a/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/WritableCoderTest.java +++ b/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/WritableCoderTest.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.io.hdfs; import org.apache.beam.sdk.testing.CoderProperties; - import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.NullWritable; import org.junit.Test; diff --git a/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsCheckpointMark.java b/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsCheckpointMark.java index 81c2b826ce8d..ba8fba9d00ae 100644 --- a/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsCheckpointMark.java +++ b/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsCheckpointMark.java @@ -17,18 +17,15 @@ */ package org.apache.beam.sdk.io.jms; +import java.util.ArrayList; +import java.util.List; +import javax.jms.Message; import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.DefaultCoder; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; - import org.joda.time.Instant; -import java.util.ArrayList; -import java.util.List; - -import javax.jms.Message; - /** * Checkpoint for an unbounded JmsIO.Read. Consists of * JMS destination name, and the latest message ID consumed so far. diff --git a/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java b/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java index 557fe13a3a37..f92dbd456b79 100644 --- a/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java +++ b/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java @@ -20,6 +20,22 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; +import com.google.common.annotations.VisibleForTesting; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Enumeration; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import javax.annotation.Nullable; +import javax.jms.Connection; +import javax.jms.ConnectionFactory; +import javax.jms.Destination; +import javax.jms.MessageConsumer; +import javax.jms.MessageProducer; +import javax.jms.Session; +import javax.jms.TextMessage; import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.SerializableCoder; @@ -36,31 +52,11 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.PInput; - -import com.google.common.annotations.VisibleForTesting; - import org.joda.time.Duration; import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Enumeration; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.NoSuchElementException; - -import javax.annotation.Nullable; -import javax.jms.Connection; -import javax.jms.ConnectionFactory; -import javax.jms.Destination; -import javax.jms.MessageConsumer; -import javax.jms.MessageProducer; -import javax.jms.Session; -import javax.jms.TextMessage; - /** * An unbounded source for JMS destinations (queues or topics). * diff --git a/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsRecord.java b/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsRecord.java index aa0c472e631a..65a9189d2cea 100644 --- a/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsRecord.java +++ b/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsRecord.java @@ -20,7 +20,6 @@ import java.io.Serializable; import java.util.Map; import java.util.Objects; - import javax.jms.Destination; /** diff --git a/sdks/java/io/jms/src/test/java/org/apache/beam/sdk/io/jms/JmsIOTest.java b/sdks/java/io/jms/src/test/java/org/apache/beam/sdk/io/jms/JmsIOTest.java index 020794c7cfbd..4c3be6d0bae2 100644 --- a/sdks/java/io/jms/src/test/java/org/apache/beam/sdk/io/jms/JmsIOTest.java +++ b/sdks/java/io/jms/src/test/java/org/apache/beam/sdk/io/jms/JmsIOTest.java @@ -17,6 +17,17 @@ */ package org.apache.beam.sdk.io.jms; +import java.util.ArrayList; +import javax.jms.Connection; +import javax.jms.ConnectionFactory; +import javax.jms.Message; +import javax.jms.MessageConsumer; +import javax.jms.MessageProducer; +import javax.jms.Session; +import javax.jms.TextMessage; +import org.apache.activemq.ActiveMQConnectionFactory; +import org.apache.activemq.broker.BrokerService; +import org.apache.activemq.store.memory.MemoryPersistenceAdapter; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.PAssert; @@ -24,10 +35,6 @@ import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.values.PCollection; - -import org.apache.activemq.ActiveMQConnectionFactory; -import org.apache.activemq.broker.BrokerService; -import org.apache.activemq.store.memory.MemoryPersistenceAdapter; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -36,16 +43,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.ArrayList; - -import javax.jms.Connection; -import javax.jms.ConnectionFactory; -import javax.jms.Message; -import javax.jms.MessageConsumer; -import javax.jms.MessageProducer; -import javax.jms.Session; -import javax.jms.TextMessage; - /** * Tests of {@link JmsIO}. */ diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaCheckpointMark.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaCheckpointMark.java index 4b6b976fa54d..664bb6f766ee 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaCheckpointMark.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaCheckpointMark.java @@ -17,16 +17,14 @@ */ package org.apache.beam.sdk.io.kafka; +import java.io.IOException; +import java.io.Serializable; +import java.util.List; import org.apache.beam.sdk.coders.DefaultCoder; import org.apache.beam.sdk.coders.SerializableCoder; import org.apache.beam.sdk.io.UnboundedSource; - import org.apache.kafka.common.TopicPartition; -import java.io.IOException; -import java.io.Serializable; -import java.util.List; - /** * Checkpoint for an unbounded KafkaIO.Read. Consists of Kafka topic name, partition id, * and the latest offset consumed so far. diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java index 6cc5d6a597d6..885d5d10b4b7 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java @@ -21,6 +21,34 @@ import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Function; +import com.google.common.base.Joiner; +import com.google.common.base.Optional; +import com.google.common.collect.ComparisonChain; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterators; +import com.google.common.collect.Lists; +import com.google.common.io.Closeables; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Random; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.SynchronousQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import javax.annotation.Nullable; import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; @@ -46,18 +74,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.PInput; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; -import com.google.common.base.Joiner; -import com.google.common.base.Optional; -import com.google.common.collect.ComparisonChain; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Iterators; -import com.google.common.collect.Lists; -import com.google.common.io.Closeables; - import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRecord; @@ -79,25 +95,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.NoSuchElementException; -import java.util.Random; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.SynchronousQueue; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import javax.annotation.Nullable; - /** * An unbounded source and a sink for Kafka topics. * Kafka version 0.9 and above are supported. diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaRecord.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaRecord.java index 76e688b17852..fa202e103e92 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaRecord.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaRecord.java @@ -17,10 +17,9 @@ */ package org.apache.beam.sdk.io.kafka; -import org.apache.beam.sdk.values.KV; - import java.io.Serializable; import java.util.Arrays; +import org.apache.beam.sdk.values.KV; /** * KafkaRecord contains key and value of the record as well as metadata for the record (topic name, diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaRecordCoder.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaRecordCoder.java index 8a3e7f51441d..736a752f7741 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaRecordCoder.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaRecordCoder.java @@ -17,6 +17,12 @@ */ package org.apache.beam.sdk.io.kafka; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.List; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.KvCoder; @@ -27,14 +33,6 @@ import org.apache.beam.sdk.util.PropertyNames; import org.apache.beam.sdk.values.KV; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.List; - /** * {@link Coder} for {@link KafkaRecord}. */ diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java index 9a89c3621bcd..772efe1ec294 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java @@ -21,6 +21,21 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import javax.annotation.Nullable; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.Pipeline.PipelineExecutionException; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; @@ -45,10 +60,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; - import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.MockConsumer; @@ -66,20 +77,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import javax.annotation.Nullable; - /** * Tests of {@link KafkaSource}. */ diff --git a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/CombineJava8Test.java b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/CombineJava8Test.java index 132247b2474b..98d99cefd93d 100644 --- a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/CombineJava8Test.java +++ b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/CombineJava8Test.java @@ -21,6 +21,8 @@ import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.not; +import com.google.common.collect.Iterables; +import java.io.Serializable; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; @@ -28,9 +30,6 @@ import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.Iterables; - import org.junit.Assume; import org.junit.Rule; import org.junit.Test; @@ -38,8 +37,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; - /** * Java 8 Tests for {@link Combine}. */ diff --git a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/FilterJava8Test.java b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/FilterJava8Test.java index 3c83be28ca66..afd1c8b48ca5 100644 --- a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/FilterJava8Test.java +++ b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/FilterJava8Test.java @@ -17,13 +17,13 @@ */ package org.apache.beam.sdk.transforms; +import java.io.Serializable; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.RunnableOnService; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.PCollection; - import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -31,8 +31,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; - /** * Java 8 Tests for {@link Filter}. */ diff --git a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsJava8Test.java b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsJava8Test.java index 5ee10d108337..70cc04da9aff 100644 --- a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsJava8Test.java +++ b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsJava8Test.java @@ -17,23 +17,20 @@ */ package org.apache.beam.sdk.transforms; +import com.google.common.collect.ImmutableList; +import java.io.Serializable; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptor; - -import com.google.common.collect.ImmutableList; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.List; - /** * Java 8 Tests for {@link FlatMapElements}. */ diff --git a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/MapElementsJava8Test.java b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/MapElementsJava8Test.java index 339e431be27a..9b556b955686 100644 --- a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/MapElementsJava8Test.java +++ b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/MapElementsJava8Test.java @@ -17,18 +17,16 @@ */ package org.apache.beam.sdk.transforms; +import java.io.Serializable; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptor; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; - /** * Java 8 tests for {@link MapElements}. */ diff --git a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/PartitionJava8Test.java b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/PartitionJava8Test.java index c8283dbf0411..0aeb41ffdc92 100644 --- a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/PartitionJava8Test.java +++ b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/PartitionJava8Test.java @@ -19,20 +19,18 @@ import static org.junit.Assert.assertEquals; +import java.io.Serializable; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.PCollectionList; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; - /** * Java 8 Tests for {@link Filter}. */ diff --git a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/RemoveDuplicatesJava8Test.java b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/RemoveDuplicatesJava8Test.java index aa4c01548c82..99266d487cdd 100644 --- a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/RemoveDuplicatesJava8Test.java +++ b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/RemoveDuplicatesJava8Test.java @@ -23,23 +23,20 @@ import static org.hamcrest.Matchers.not; import static org.junit.Assert.assertThat; +import com.google.common.collect.HashMultimap; +import com.google.common.collect.Multimap; +import java.util.HashSet; +import java.util.Set; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptor; - -import com.google.common.collect.HashMultimap; -import com.google.common.collect.Multimap; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.HashSet; -import java.util.Set; - /** * Java 8 tests for {@link RemoveDuplicates}. */ diff --git a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/WithKeysJava8Test.java b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/WithKeysJava8Test.java index a89e281e6830..a5b9cb1239a0 100644 --- a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/WithKeysJava8Test.java +++ b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/WithKeysJava8Test.java @@ -25,7 +25,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptor; - import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; diff --git a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsJava8Test.java b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsJava8Test.java index 03aa64719fc9..5f1e74bd2ddd 100644 --- a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsJava8Test.java +++ b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsJava8Test.java @@ -17,20 +17,18 @@ */ package org.apache.beam.sdk.transforms; +import java.io.Serializable; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.RunnableOnService; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - import org.joda.time.Instant; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; - /** * Java 8 tests for {@link WithTimestamps}. */ diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/DebuggingWordCount.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/DebuggingWordCount.java index bc55c062fabe..e9f433396e6e 100644 --- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/DebuggingWordCount.java +++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/DebuggingWordCount.java @@ -18,6 +18,9 @@ package ${package}; import ${package}.WordCount; +import java.util.Arrays; +import java.util.List; +import java.util.regex.Pattern; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.options.Default; @@ -30,15 +33,9 @@ import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Arrays; -import java.util.List; -import java.util.regex.Pattern; - - /** * An example that verifies word counts in Shakespeare and includes Dataflow best practices. * diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WindowedWordCount.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WindowedWordCount.java index ffe8b881ca5a..17bf7ca6154b 100644 --- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WindowedWordCount.java +++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WindowedWordCount.java @@ -17,8 +17,14 @@ */ package ${package}; +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableReference; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; import ${package}.common.DataflowExampleUtils; - +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.io.PubsubIO; @@ -33,22 +39,11 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - -import com.google.api.services.bigquery.model.TableFieldSchema; -import com.google.api.services.bigquery.model.TableReference; -import com.google.api.services.bigquery.model.TableRow; -import com.google.api.services.bigquery.model.TableSchema; - import org.joda.time.Duration; import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - - /** * An example that counts words in text, and can run over either unbounded or bounded input * collections. diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleUtils.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleUtils.java index fa29fddcdf99..9e6be788d0e4 100644 --- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleUtils.java +++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleUtils.java @@ -17,12 +17,6 @@ */ package ${package}.common; -import org.apache.beam.runners.dataflow.BlockingDataflowRunner; -import org.apache.beam.runners.dataflow.DataflowPipelineJob; -import org.apache.beam.runners.dataflow.DataflowRunner; -import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; -import org.apache.beam.runners.dataflow.util.MonitoringUtil; - import com.google.api.client.googleapis.json.GoogleJsonResponseException; import com.google.api.client.googleapis.services.AbstractGoogleClientRequest; import com.google.api.services.bigquery.Bigquery; @@ -36,21 +30,24 @@ import com.google.api.services.dataflow.Dataflow; import com.google.api.services.pubsub.Pubsub; import com.google.api.services.pubsub.model.Topic; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.PipelineResult; -import org.apache.beam.sdk.io.TextIO; -import org.apache.beam.sdk.options.BigQueryOptions; -import org.apache.beam.sdk.transforms.IntraBundleParallelization; -import org.apache.beam.sdk.util.Transport; import com.google.common.collect.Lists; import com.google.common.collect.Sets; - import java.io.IOException; import java.util.Collection; import java.util.List; import java.util.Set; - import javax.servlet.http.HttpServletResponse; +import org.apache.beam.runners.dataflow.BlockingDataflowRunner; +import org.apache.beam.runners.dataflow.DataflowPipelineJob; +import org.apache.beam.runners.dataflow.DataflowRunner; +import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; +import org.apache.beam.runners.dataflow.util.MonitoringUtil; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.io.TextIO; +import org.apache.beam.sdk.options.BigQueryOptions; +import org.apache.beam.sdk.transforms.IntraBundleParallelization; +import org.apache.beam.sdk.util.Transport; /** * The utility class that sets up and tears down external resources, starts the Google Cloud Pub/Sub diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExampleBigQueryTableOptions.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExampleBigQueryTableOptions.java index 279f2e098dde..79fa865eb529 100644 --- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExampleBigQueryTableOptions.java +++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExampleBigQueryTableOptions.java @@ -17,14 +17,13 @@ */ package ${package}.common; +import com.google.api.services.bigquery.model.TableSchema; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.DefaultValueFactory; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.PipelineOptions; -import com.google.api.services.bigquery.model.TableSchema; - /** * Options that can be used to configure BigQuery tables in Dataflow examples. * The project defaults to the project being used to run the example. diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/PubsubFileInjector.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/PubsubFileInjector.java index 9b347da6089b..58e082148a4a 100644 --- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/PubsubFileInjector.java +++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/PubsubFileInjector.java @@ -17,6 +17,12 @@ */ package ${package}.common; +import com.google.api.services.pubsub.Pubsub; +import com.google.api.services.pubsub.model.PublishRequest; +import com.google.api.services.pubsub.model.PubsubMessage; +import com.google.common.collect.ImmutableMap; +import java.io.IOException; +import java.util.Arrays; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.TextIO; @@ -28,14 +34,6 @@ import org.apache.beam.sdk.transforms.IntraBundleParallelization; import org.apache.beam.sdk.util.Transport; -import com.google.api.services.pubsub.Pubsub; -import com.google.api.services.pubsub.model.PublishRequest; -import com.google.api.services.pubsub.model.PubsubMessage; -import com.google.common.collect.ImmutableMap; - -import java.io.IOException; -import java.util.Arrays; - /** * A batch Dataflow pipeline for injecting a set of GCS files into * a PubSub topic line by line. Empty lines are skipped. diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/test/java/DebuggingWordCountTest.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/test/java/DebuggingWordCountTest.java index 4d2e324c1be5..dfa1a75d4135 100644 --- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/test/java/DebuggingWordCountTest.java +++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/test/java/DebuggingWordCountTest.java @@ -18,16 +18,14 @@ package ${package}; import com.google.common.io.Files; - +import java.io.File; +import java.nio.charset.StandardCharsets; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.File; -import java.nio.charset.StandardCharsets; - /** * Tests for {@link DebuggingWordCount}. */ diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/test/java/WordCountTest.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/test/java/WordCountTest.java index debfc78623a2..875d3d757a79 100644 --- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/test/java/WordCountTest.java +++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/test/java/WordCountTest.java @@ -20,6 +20,8 @@ import ${package}.WordCount.CountWords; import ${package}.WordCount.ExtractWordsFn; import ${package}.WordCount.FormatAsTextFn; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.testing.PAssert; @@ -29,7 +31,6 @@ import org.apache.beam.sdk.transforms.DoFnTester; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; - import org.hamcrest.CoreMatchers; import org.junit.Assert; import org.junit.Test; @@ -37,8 +38,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; /** * Tests of WordCount. diff --git a/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/src/main/java/StarterPipeline.java b/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/src/main/java/StarterPipeline.java index 6a1c41b0dc87..0b21aa650a8d 100644 --- a/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/src/main/java/StarterPipeline.java +++ b/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/src/main/java/StarterPipeline.java @@ -22,7 +22,6 @@ import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.ParDo; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/src/main/java/it/pkg/StarterPipeline.java b/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/src/main/java/it/pkg/StarterPipeline.java index 7c13350aee52..b332442cadcd 100644 --- a/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/src/main/java/it/pkg/StarterPipeline.java +++ b/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/src/main/java/it/pkg/StarterPipeline.java @@ -22,7 +22,6 @@ import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.ParDo; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/AvroCoderBenchmark.java b/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/AvroCoderBenchmark.java index 39b31ef30908..35d5add1471a 100644 --- a/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/AvroCoderBenchmark.java +++ b/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/AvroCoderBenchmark.java @@ -17,9 +17,10 @@ */ package org.apache.beam.sdk.microbenchmarks.coders; +import java.io.IOException; +import java.util.Arrays; import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.DefaultCoder; - import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.Fork; import org.openjdk.jmh.annotations.Param; @@ -28,9 +29,6 @@ import org.openjdk.jmh.annotations.State; import org.openjdk.jmh.annotations.Warmup; -import java.io.IOException; -import java.util.Arrays; - /** * Benchmarks for {@link AvroCoder}. */ diff --git a/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/ByteArrayCoderBenchmark.java b/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/ByteArrayCoderBenchmark.java index df20a15d4ec2..78909752a5d4 100644 --- a/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/ByteArrayCoderBenchmark.java +++ b/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/ByteArrayCoderBenchmark.java @@ -17,8 +17,9 @@ */ package org.apache.beam.sdk.microbenchmarks.coders; +import java.io.IOException; +import java.util.Arrays; import org.apache.beam.sdk.coders.ByteArrayCoder; - import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.Fork; import org.openjdk.jmh.annotations.Param; @@ -27,9 +28,6 @@ import org.openjdk.jmh.annotations.State; import org.openjdk.jmh.annotations.Warmup; -import java.io.IOException; -import java.util.Arrays; - /** * Benchmarks for {@link ByteArrayCoder}. */ diff --git a/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/CoderBenchmarking.java b/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/CoderBenchmarking.java index 8523cb2a6a1f..c92215d0afc5 100644 --- a/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/CoderBenchmarking.java +++ b/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/CoderBenchmarking.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.microbenchmarks.coders; +import java.io.IOException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.util.CoderUtils; -import java.io.IOException; - /** * Utilities for writing coder benchmarks. */ diff --git a/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/StringUtf8CoderBenchmark.java b/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/StringUtf8CoderBenchmark.java index c0bcb453944b..540c95899c21 100644 --- a/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/StringUtf8CoderBenchmark.java +++ b/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/StringUtf8CoderBenchmark.java @@ -17,8 +17,9 @@ */ package org.apache.beam.sdk.microbenchmarks.coders; +import java.io.IOException; +import java.util.Arrays; import org.apache.beam.sdk.coders.StringUtf8Coder; - import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.Fork; import org.openjdk.jmh.annotations.Param; @@ -27,9 +28,6 @@ import org.openjdk.jmh.annotations.State; import org.openjdk.jmh.annotations.Warmup; -import java.io.IOException; -import java.util.Arrays; - /** * Benchmarks for {@link StringUtf8Coder}. */ From 481a40f38ec32b7999c2fa29299542655f2d1cca Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Wed, 24 Aug 2016 09:49:46 -0700 Subject: [PATCH 061/112] BigQueryIO.Write: raise size limit to 11 TiB BigQuery has changed their total size quota to 12 TiB. https://cloud.google.com/bigquery/quota-policy#import --- .../java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java index 04fb041badd1..01a8a1c5cd4e 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java @@ -1420,8 +1420,8 @@ public static class Bound extends PTransform, PDone> { // Maximum number of files in a single partition. static final int MAX_NUM_FILES = 10000; - // Maximum number of bytes in a single partition. - static final long MAX_SIZE_BYTES = 3 * (1L << 40); + // Maximum number of bytes in a single partition -- 11 TiB just under BQ's 12 TiB limit. + static final long MAX_SIZE_BYTES = 11 * (1L << 40); // The maximum number of retry jobs. static final int MAX_RETRY_JOBS = 3; From ddda21bf96956cc18d8768aefaff9c317710d65c Mon Sep 17 00:00:00 2001 From: Pei He Date: Wed, 24 Aug 2016 14:47:11 -0700 Subject: [PATCH 062/112] Cleanup some javadoc that referring Dataflow --- .../java/org/apache/beam/examples/DebuggingWordCount.java | 2 +- .../main/java/org/apache/beam/examples/WindowedWordCount.java | 3 +-- .../java/org/apache/beam/examples/common/ExampleOptions.java | 2 +- .../src/main/java/org/apache/beam/examples/complete/README.md | 4 ++-- .../apache/beam/examples/complete/StreamingWordExtract.java | 2 +- .../apache/beam/examples/complete/TopWikipediaSessions.java | 2 +- .../org/apache/beam/examples/complete/TrafficMaxLaneFlow.java | 4 ++-- .../java/org/apache/beam/examples/complete/TrafficRoutes.java | 4 ++-- 8 files changed, 11 insertions(+), 12 deletions(-) diff --git a/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java index 4a9aba9da5e8..5a0930cd35f5 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java @@ -37,7 +37,7 @@ /** - * An example that verifies word counts in Shakespeare and includes Dataflow best practices. + * An example that verifies word counts in Shakespeare and includes Beam best practices. * *

        This class, {@link DebuggingWordCount}, is the third in a series of four successively more * detailed 'word count' examples. You may first want to take a look at {@link MinimalWordCount} diff --git a/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java index 6d69f1488526..5f60524209c1 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java @@ -183,8 +183,7 @@ public static interface Options extends WordCount.WordCountOptions, public static void main(String[] args) throws IOException { Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); options.setBigQuerySchema(getSchema()); - // DataflowExampleUtils creates the necessary input sources to simplify execution of this - // Pipeline. + // ExampleUtils creates the necessary input sources to simplify execution of this Pipeline. ExampleUtils exampleUtils = new ExampleUtils(options); exampleUtils.setup(); diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/ExampleOptions.java b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleOptions.java index a7dcc7c17517..8b7ed073f359 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/common/ExampleOptions.java +++ b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleOptions.java @@ -33,7 +33,7 @@ * Options that can be used to configure the Beam examples. */ public interface ExampleOptions extends PipelineOptions { - @Description("Whether to keep jobs running on the Dataflow service after local process exit") + @Description("Whether to keep jobs running after local process exit") @Default.Boolean(false) boolean getKeepJobsRunning(); void setKeepJobsRunning(boolean keepJobsRunning); diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/README.md b/examples/java/src/main/java/org/apache/beam/examples/complete/README.md index 99c93ef4b82a..b98be7a72396 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/README.md +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/README.md @@ -43,14 +43,14 @@ This directory contains end-to-end example pipelines that perform complex data p Windowing to perform time-based aggregations of data.

      • TrafficMaxLaneFlow - — A streaming Cloud Dataflow example using BigQuery output in the + — A streaming Beam Example using BigQuery output in the traffic sensor domain. Demonstrates the Cloud Dataflow streaming runner, sliding windows, Cloud Pub/Sub topic ingestion, the use of the AvroCoder to encode a custom class, and custom Combine transforms.
      • TrafficRoutes - — A streaming Cloud Dataflow example using BigQuery output in the + — A streaming Beam Example using BigQuery output in the traffic sensor domain. Demonstrates the Cloud Dataflow streaming runner, GroupByKey, keyed state, sliding windows, and Cloud Pub/Sub topic ingestion. diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java b/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java index c4ad35bcc818..348bab84b996 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java @@ -37,7 +37,7 @@ import org.apache.beam.sdk.transforms.ParDo; /** - * A streaming Dataflow Example using BigQuery output. + * A streaming Beam Example using BigQuery output. * *

        This pipeline example reads lines of the input text file, splits each line * into individual words, capitalizes those words, and writes the output to diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java index 4c5472bf6f68..1b2064ad068e 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java @@ -181,7 +181,7 @@ public void processElement(ProcessContext c) { /** * Options supported by this class. * - *

        Inherits standard Dataflow configuration options. + *

        Inherits standard Beam configuration options. */ private static interface Options extends PipelineOptions { @Description( diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java index f7ff035dbf50..1b27e650f03e 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java @@ -53,7 +53,7 @@ import org.joda.time.format.DateTimeFormatter; /** - * A Dataflow Example that runs in both batch and streaming modes with traffic sensor data. + * A Beam Example that runs in both batch and streaming modes with traffic sensor data. * You can configure the running mode by setting {@literal --streaming} to true or false. * *

        Concepts: The batch and streaming runners, sliding windows, @@ -329,7 +329,7 @@ public static void main(String[] args) throws IOException { .withValidation() .as(TrafficMaxLaneFlowOptions.class); options.setBigQuerySchema(FormatMaxesFn.getSchema()); - // Using DataflowExampleUtils to set up required resources. + // Using ExampleUtils to set up required resources. ExampleUtils exampleUtils = new ExampleUtils(options); exampleUtils.setup(); diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java index 12eaeaa97257..f3c2d3936ee7 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java @@ -57,7 +57,7 @@ import org.joda.time.format.DateTimeFormatter; /** - * A Dataflow Example that runs in both batch and streaming modes with traffic sensor data. + * A Beam Example that runs in both batch and streaming modes with traffic sensor data. * You can configure the running mode by setting {@literal --streaming} to true or false. * *

        Concepts: The batch and streaming runners, GroupByKey, sliding windows. @@ -340,7 +340,7 @@ public static void main(String[] args) throws IOException { .as(TrafficRoutesOptions.class); options.setBigQuerySchema(FormatStatsFn.getSchema()); - // Using DataflowExampleUtils to set up required resources. + // Using ExampleUtils to set up required resources. ExampleUtils exampleUtils = new ExampleUtils(options); exampleUtils.setup(); From 83ccf082ceb129f536e8a01b755c15ec0f5c2495 Mon Sep 17 00:00:00 2001 From: Pei He Date: Wed, 24 Aug 2016 17:26:50 -0700 Subject: [PATCH 063/112] Move the samples data to gs://apache-beam-samples/ --- .../java/org/apache/beam/examples/DebuggingWordCount.java | 4 ++-- .../main/java/org/apache/beam/examples/MinimalWordCount.java | 2 +- .../java/org/apache/beam/examples/WindowedWordCount.java | 2 +- .../src/main/java/org/apache/beam/examples/WordCount.java | 4 ++-- .../apache/beam/examples/complete/StreamingWordExtract.java | 2 +- .../main/java/org/apache/beam/examples/complete/TfIdf.java | 4 ++-- .../apache/beam/examples/complete/TopWikipediaSessions.java | 5 +++-- .../apache/beam/examples/complete/TrafficMaxLaneFlow.java | 2 +- .../org/apache/beam/examples/complete/TrafficRoutes.java | 2 +- .../apache/beam/examples/cookbook/DatastoreWordCount.java | 2 +- .../java/org/apache/beam/examples/cookbook/DeDupExample.java | 4 ++-- .../org/apache/beam/examples/cookbook/TriggerExample.java | 2 +- .../java/org/apache/beam/examples/MinimalWordCountJava8.java | 2 +- .../apache/beam/examples/complete/game/HourlyTeamScore.java | 2 +- .../org/apache/beam/examples/complete/game/UserScore.java | 2 +- .../org/apache/beam/examples/MinimalWordCountJava8Test.java | 2 +- 16 files changed, 22 insertions(+), 21 deletions(-) diff --git a/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java index 5a0930cd35f5..be3aa419b537 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java @@ -100,8 +100,8 @@ * that changing the default worker log level to TRACE or DEBUG will significantly increase * the amount of logs output. * - *

        The input file defaults to {@code gs://dataflow-samples/shakespeare/kinglear.txt} and can be - * overridden with {@code --inputFile}. + *

        The input file defaults to {@code gs://apache-beam-samples/shakespeare/kinglear.txt} + * and can be overridden with {@code --inputFile}. */ public class DebuggingWordCount { /** A DoFn that filters for a specific key based upon a regular expression. */ diff --git a/examples/java/src/main/java/org/apache/beam/examples/MinimalWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/MinimalWordCount.java index df725e3f1585..f28a20cf9449 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/MinimalWordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/MinimalWordCount.java @@ -87,7 +87,7 @@ public static void main(String[] args) { // Concept #1: Apply a root transform to the pipeline; in this case, TextIO.Read to read a set // of input text files. TextIO.Read returns a PCollection where each element is one line from // the input text (a set of Shakespeare's texts). - p.apply(TextIO.Read.from("gs://dataflow-samples/shakespeare/*")) + p.apply(TextIO.Read.from("gs://apache-beam-samples/shakespeare/*")) // Concept #2: Apply a ParDo transform to our PCollection of text lines. This ParDo invokes a // DoFn (defined in-line) on each element that tokenizes the text line into individual words. // The ParDo returns a PCollection, where each element is an individual word in diff --git a/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java index 5f60524209c1..7af354cee0fa 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java @@ -82,7 +82,7 @@ * *

        Optionally specify the input file path via: * {@code --inputFile=gs://INPUT_PATH}, - * which defaults to {@code gs://dataflow-samples/shakespeare/kinglear.txt}. + * which defaults to {@code gs://apache-beam-samples/shakespeare/kinglear.txt}. * *

        Specify an output BigQuery dataset and optionally, a table for the output. If you don't * specify the table, one will be created for you using the job name. If you don't specify the diff --git a/examples/java/src/main/java/org/apache/beam/examples/WordCount.java b/examples/java/src/main/java/org/apache/beam/examples/WordCount.java index d42d6214973d..0275651bf288 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/WordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/WordCount.java @@ -89,8 +89,8 @@ * --output=gs://YOUR_OUTPUT_PREFIX * } * - *

        The input file defaults to {@code gs://dataflow-samples/shakespeare/kinglear.txt} and can be - * overridden with {@code --inputFile}. + *

        The input file defaults to {@code gs://apache-beam-samples/shakespeare/kinglear.txt} + * and can be overridden with {@code --inputFile}. */ public class WordCount { diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java b/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java index 348bab84b996..869ea69b8ae4 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java @@ -104,7 +104,7 @@ static TableSchema getSchema() { private interface StreamingWordExtractOptions extends ExampleOptions, ExampleBigQueryTableOptions, StreamingOptions { @Description("Path of the file to read from") - @Default.String("gs://dataflow-samples/shakespeare/kinglear.txt") + @Default.String("gs://apache-beam-samples/shakespeare/kinglear.txt") String getInputFile(); void setInputFile(String value); } diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java index a5a939263ee4..87023edac29a 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java @@ -83,7 +83,7 @@ * --output=gs://YOUR_OUTPUT_PREFIX * } * - *

        The default input is {@code gs://dataflow-samples/shakespeare/} and can be overridden with + *

        The default input is {@code gs://apache-beam-samples/shakespeare/} and can be overridden with * {@code --input}. */ public class TfIdf { @@ -94,7 +94,7 @@ public class TfIdf { */ private static interface Options extends PipelineOptions { @Description("Path to the directory or GCS prefix containing files to read from") - @Default.String("gs://dataflow-samples/shakespeare/") + @Default.String("gs://apache-beam-samples/shakespeare/") String getInput(); void setInput(String value); diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java index 1b2064ad068e..d597258d3b92 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java @@ -64,7 +64,7 @@ * --output=gs://YOUR_OUTPUT_PREFIX * } * - *

        The default input is {@code gs://dataflow-samples/wikipedia_edits/*.json} and can be + *

        The default input is {@code gs://apache-beam-samples/wikipedia_edits/*.json} and can be * overridden with {@code --input}. * *

        The input for this example is large enough that it's a good place to enable (experimental) @@ -77,7 +77,8 @@ * This will automatically scale the number of workers up over time until the job completes. */ public class TopWikipediaSessions { - private static final String EXPORTED_WIKI_TABLE = "gs://dataflow-samples/wikipedia_edits/*.json"; + private static final String EXPORTED_WIKI_TABLE = + "gs://apache-beam-samples/wikipedia_edits/*.json"; /** * Extracts user and timestamp from a TableRow representing a Wikipedia edit. diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java index 1b27e650f03e..e4569600c05c 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java @@ -303,7 +303,7 @@ public PCollection apply(PBegin begin) { */ private interface TrafficMaxLaneFlowOptions extends ExampleOptions, ExampleBigQueryTableOptions { @Description("Path of the file to read from") - @Default.String("gs://dataflow-samples/traffic_sensor/" + @Default.String("gs://apache-beam-samples/traffic_sensor/" + "Freeways-5Minaa2010-01-01_to_2010-02-15_test2.csv") String getInputFile(); void setInputFile(String value); diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java index f3c2d3936ee7..95336c644a79 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java @@ -313,7 +313,7 @@ public PCollection apply(PBegin begin) { */ private interface TrafficRoutesOptions extends ExampleOptions, ExampleBigQueryTableOptions { @Description("Path of the file to read from") - @Default.String("gs://dataflow-samples/traffic_sensor/" + @Default.String("gs://apache-beam-samples/traffic_sensor/" + "Freeways-5Minaa2010-01-01_to_2010-02-15_test2.csv") String getInputFile(); void setInputFile(String value); diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java index 9a9e79968670..eb2165f91e19 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java @@ -151,7 +151,7 @@ public void processElement(ProcessContext c) { */ public static interface Options extends PipelineOptions { @Description("Path of the file to read from and store to Datastore") - @Default.String("gs://dataflow-samples/shakespeare/kinglear.txt") + @Default.String("gs://apache-beam-samples/shakespeare/kinglear.txt") String getInput(); void setInput(String value); diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java index d573bcd9d295..57917109addd 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java @@ -47,7 +47,7 @@ * and an output prefix on GCS: * --output=gs://YOUR_OUTPUT_PREFIX * - *

        The input defaults to {@code gs://dataflow-samples/shakespeare/*} and can be + *

        The input defaults to {@code gs://apache-beam-samples/shakespeare/*} and can be * overridden with {@code --input}. */ public class DeDupExample { @@ -59,7 +59,7 @@ public class DeDupExample { */ private static interface Options extends PipelineOptions { @Description("Path to the directory or GCS prefix containing files to read from") - @Default.String("gs://dataflow-samples/shakespeare/*") + @Default.String("gs://apache-beam-samples/shakespeare/*") String getInput(); void setInput(String value); diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java index db5943555545..263054138dd6 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java @@ -420,7 +420,7 @@ public interface TrafficFlowOptions extends ExampleOptions, ExampleBigQueryTableOptions, StreamingOptions { @Description("Input file to read from") - @Default.String("gs://dataflow-samples/traffic_sensor/" + @Default.String("gs://apache-beam-samples/traffic_sensor/" + "Freeways-5Minaa2010-01-01_to_2010-02-15.csv") String getInput(); void setInput(String value); diff --git a/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java b/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java index a49da7bdfbb6..24dd6f9b2c48 100644 --- a/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java +++ b/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java @@ -55,7 +55,7 @@ public static void main(String[] args) { Pipeline p = Pipeline.create(options); - p.apply(TextIO.Read.from("gs://dataflow-samples/shakespeare/*")) + p.apply(TextIO.Read.from("gs://apache-beam-samples/shakespeare/*")) .apply(FlatMapElements.via((String word) -> Arrays.asList(word.split("[^a-zA-Z']+"))) .withOutputType(TypeDescriptors.strings())) .apply(Filter.by((String word) -> !word.isEmpty())) diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java index d408e2132dab..cf1389981fc5 100644 --- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java +++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java @@ -70,7 +70,7 @@ * timestamped after 23:59 PST on 2015-10-18 should not be included in the analysis. * To indicate a time before which data should be filtered out, include the {@code --startMin} arg. * If you're using the default input specified in {@link UserScore}, - * "gs://dataflow-samples/game/gaming_data*.csv", then + * "gs://apache-beam-samples/game/gaming_data*.csv", then * {@code --startMin=2015-11-16-16-10 --stopMin=2015-11-17-16-10} are good values. */ public class HourlyTeamScore extends UserScore { diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java index 65036cee6b92..f05879f93106 100644 --- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java +++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java @@ -182,7 +182,7 @@ public static interface Options extends PipelineOptions { @Description("Path to the data file(s) containing game data.") // The default maps to two large Google Cloud Storage files (each ~12GB) holding two subsequent // day's worth (roughly) of data. - @Default.String("gs://dataflow-samples/game/gaming_data*.csv") + @Default.String("gs://apache-beam-samples/game/gaming_data*.csv") String getInput(); void setInput(String value); diff --git a/examples/java8/src/test/java/org/apache/beam/examples/MinimalWordCountJava8Test.java b/examples/java8/src/test/java/org/apache/beam/examples/MinimalWordCountJava8Test.java index 85841a780147..181921920b00 100644 --- a/examples/java8/src/test/java/org/apache/beam/examples/MinimalWordCountJava8Test.java +++ b/examples/java8/src/test/java/org/apache/beam/examples/MinimalWordCountJava8Test.java @@ -60,7 +60,7 @@ public void testMinimalWordCountJava8() throws Exception { Pipeline p = TestPipeline.create(); p.getOptions().as(GcsOptions.class).setGcsUtil(buildMockGcsUtil()); - p.apply(TextIO.Read.from("gs://dataflow-samples/shakespeare/*")) + p.apply(TextIO.Read.from("gs://apache-beam-samples/shakespeare/*")) .apply(FlatMapElements.via((String word) -> Arrays.asList(word.split("[^a-zA-Z']+"))) .withOutputType(TypeDescriptors.strings())) .apply(Filter.by((String word) -> !word.isEmpty())) From 4e6230cc734ab3dba081e04d135a285b73008270 Mon Sep 17 00:00:00 2001 From: Scott Wegner Date: Wed, 17 Aug 2016 14:38:36 -0700 Subject: [PATCH 064/112] Update DoFn javadocs to remove references to OldDoFn and Dataflow --- .../examples/common/PubsubFileInjector.java | 2 +- .../apache/beam/sdk/util/DoFnRunnerBase.java | 16 +- .../beam/sdk/util/GroupAlsoByWindowsDoFn.java | 2 +- .../apache/beam/sdk/util/ReduceFnRunner.java | 5 +- .../beam/sdk/util/SimpleDoFnRunner.java | 4 +- .../ImmutabilityCheckingBundleFactory.java | 4 +- .../direct/TransformEvaluatorFactory.java | 3 +- .../beam/runners/dataflow/util/DoFnInfo.java | 7 +- .../translation/MultiOutputWordCountTest.java | 2 +- .../spark/translation/SerializationTest.java | 4 +- .../org/apache/beam/sdk/AggregatorValues.java | 4 +- .../beam/sdk/transforms/Aggregator.java | 14 +- .../beam/sdk/transforms/CombineFns.java | 18 +- .../org/apache/beam/sdk/transforms/DoFn.java | 23 +- .../beam/sdk/transforms/DoFnTester.java | 62 ++-- .../beam/sdk/transforms/GroupByKey.java | 7 +- .../beam/sdk/transforms/PTransform.java | 2 +- .../org/apache/beam/sdk/transforms/ParDo.java | 306 +++++++++--------- .../beam/sdk/transforms/SimpleFunction.java | 6 +- .../sdk/transforms/windowing/PaneInfo.java | 10 +- .../beam/sdk/util/BaseExecutionContext.java | 4 +- .../util/ReifyTimestampAndWindowsDoFn.java | 4 +- .../beam/sdk/util/SerializableUtils.java | 2 +- .../beam/sdk/util/SystemDoFnInternal.java | 7 +- .../beam/sdk/util/WindowingInternals.java | 3 +- .../DoFnDelegatingAggregatorTest.java | 2 +- .../beam/sdk/transforms/DoFnTesterTest.java | 3 +- .../beam/sdk/transforms/NoOpOldDoFn.java | 2 +- 28 files changed, 263 insertions(+), 265 deletions(-) diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/PubsubFileInjector.java b/examples/java/src/main/java/org/apache/beam/examples/common/PubsubFileInjector.java index e6a1495e545d..4634159826d3 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/common/PubsubFileInjector.java +++ b/examples/java/src/main/java/org/apache/beam/examples/common/PubsubFileInjector.java @@ -69,7 +69,7 @@ public Bound publish(String outputTopic) { } } - /** A OldDoFn that publishes non-empty lines to Google Cloud PubSub. */ + /** A {@link OldDoFn} that publishes non-empty lines to Google Cloud PubSub. */ public static class Bound extends OldDoFn { private final String outputTopic; private final String timestampLabelKey; diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java index 8a0f6bf868d9..04a0978b6030 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java @@ -58,10 +58,10 @@ */ public abstract class DoFnRunnerBase implements DoFnRunner { - /** The OldDoFn being run. */ + /** The {@link OldDoFn} being run. */ public final OldDoFn fn; - /** The context used for running the OldDoFn. */ + /** The context used for running the {@link OldDoFn}. */ public final DoFnContext context; protected DoFnRunnerBase( @@ -164,8 +164,8 @@ public void finishBundle() { /** * A concrete implementation of {@code OldDoFn.Context} used for running a {@link OldDoFn}. * - * @param the type of the OldDoFn's (main) input elements - * @param the type of the OldDoFn's (main) output elements + * @param the type of the {@link OldDoFn} (main) input elements + * @param the type of the {@link OldDoFn} (main) output elements */ private static class DoFnContext extends OldDoFn.Context { @@ -350,7 +350,7 @@ protected Aggregator createAggreg } /** - * Returns a new {@code OldDoFn.ProcessContext} for the given element. + * Returns a new {@link OldDoFn.ProcessContext} for the given element. */ protected OldDoFn.ProcessContext createProcessContext( WindowedValue elem) { @@ -366,11 +366,11 @@ private boolean isSystemDoFn() { } /** - * A concrete implementation of {@code OldDoFn.ProcessContext} used for + * A concrete implementation of {@link OldDoFn.ProcessContext} used for * running a {@link OldDoFn} over a single element. * - * @param the type of the OldDoFn's (main) input elements - * @param the type of the OldDoFn's (main) output elements + * @param the type of the {@link OldDoFn} (main) input elements + * @param the type of the {@link OldDoFn} (main) output elements */ static class DoFnProcessContext extends OldDoFn.ProcessContext { diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsDoFn.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsDoFn.java index f82e5dfe32e7..f386dfba1220 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsDoFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsDoFn.java @@ -26,7 +26,7 @@ import org.apache.beam.sdk.values.KV; /** - * OldDoFn that merges windows and groups elements in those windows, optionally + * {@link OldDoFn} that merges windows and groups elements in those windows, optionally * combining values. * * @param key type diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java index 61e5b21ebfd9..7c3e4d749a1f 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java @@ -33,7 +33,6 @@ import javax.annotation.Nullable; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Aggregator; -import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.windowing.AfterWatermark; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; @@ -177,8 +176,8 @@ public class ReduceFnRunner { * Store the previously emitted pane (if any) for each window. * *

          - *
        • State: The previous {@link PaneInfo} passed to the user's {@link OldDoFn#processElement}, - * if any. + *
        • State: The previous {@link PaneInfo} passed to the user's {@code DoFn.ProcessElement} + * method, if any. *
        • Style style: DIRECT *
        • Merging: Always keyed by actual window, so does not depend on {@link #activeWindows}. * Cleared when window is merged away. diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/SimpleDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/SimpleDoFnRunner.java index 6c1cf451d61a..1ebe5a874e74 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/SimpleDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/SimpleDoFnRunner.java @@ -28,8 +28,8 @@ /** * Runs a {@link OldDoFn} by constructing the appropriate contexts and passing them in. * - * @param the type of the OldDoFn's (main) input elements - * @param the type of the OldDoFn's (main) output elements + * @param the type of the {@link OldDoFn} (main) input elements + * @param the type of the {@link OldDoFn} (main) output elements */ public class SimpleDoFnRunner extends DoFnRunnerBase{ diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java index d5c0f0c354e6..71bd8b4e5005 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java @@ -25,7 +25,7 @@ import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.transforms.OldDoFn; +import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.util.IllegalMutationException; import org.apache.beam.sdk.util.MutationDetector; import org.apache.beam.sdk.util.MutationDetectors; @@ -40,7 +40,7 @@ * elements added to the bundle will be encoded by the {@link Coder} of the underlying * {@link PCollection}. * - *

          This catches errors during the execution of a {@link OldDoFn} caused by modifying an element + *

          This catches errors during the execution of a {@link DoFn} caused by modifying an element * after it is added to an output {@link PCollection}. */ class ImmutabilityCheckingBundleFactory implements BundleFactory { diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorFactory.java index e9fa06b107f7..ecf2da8f67a8 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorFactory.java @@ -22,7 +22,6 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.transforms.AppliedPTransform; -import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.PTransform; /** @@ -37,7 +36,7 @@ public interface TransformEvaluatorFactory { * Create a new {@link TransformEvaluator} for the application of the {@link PTransform}. * *

          Any work that must be done before input elements are processed (such as calling - * {@link OldDoFn#startBundle(OldDoFn.Context)}) must be done before the + * {@code DoFn.StartBundle}) must be done before the * {@link TransformEvaluator} is made available to the caller. * *

          May return null if the application cannot produce an evaluator (for example, it is a diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DoFnInfo.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DoFnInfo.java index 139db9da0ab9..949c38148993 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DoFnInfo.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DoFnInfo.java @@ -24,10 +24,10 @@ import org.apache.beam.sdk.values.PCollectionView; /** - * Wrapper class holding the necessary information to serialize a OldDoFn. + * Wrapper class holding the necessary information to serialize a {@link OldDoFn}. * - * @param the type of the (main) input elements of the OldDoFn - * @param the type of the (main) output elements of the OldDoFn + * @param the type of the (main) input elements of the {@link OldDoFn} + * @param the type of the (main) output elements of the {@link OldDoFn} */ public class DoFnInfo implements Serializable { private final OldDoFn doFn; @@ -66,3 +66,4 @@ public Coder getInputCoder() { return inputCoder; } } + diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/MultiOutputWordCountTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/MultiOutputWordCountTest.java index 517596aab782..acfa3dfae2b8 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/MultiOutputWordCountTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/MultiOutputWordCountTest.java @@ -100,7 +100,7 @@ public void testRun() throws Exception { } /** - * A OldDoFn that tokenizes lines of text into individual words. + * A {@link DoFn} that tokenizes lines of text into individual words. */ static class ExtractWordsFn extends DoFn { diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SerializationTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SerializationTest.java index 0e9121ca87f9..22a40cd29e2c 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SerializationTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SerializationTest.java @@ -142,7 +142,7 @@ public void testRun() throws Exception { } /** - * A OldDoFn that tokenizes lines of text into individual words. + * A {@link DoFn} that tokenizes lines of text into individual words. */ static class ExtractWordsFn extends DoFn { private static final Pattern WORD_BOUNDARY = Pattern.compile("[^a-zA-Z']+"); @@ -170,7 +170,7 @@ public void processElement(ProcessContext c) { } /** - * A OldDoFn that converts a Word and Count into a printable string. + * A {@link DoFn} that converts a Word and Count into a printable string. */ private static class FormatCountsFn extends DoFn, StringHolder> { @ProcessElement diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/AggregatorValues.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/AggregatorValues.java index 6297085319e2..1fd034a02cc0 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/AggregatorValues.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/AggregatorValues.java @@ -21,11 +21,11 @@ import java.util.Map; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.Combine.CombineFn; -import org.apache.beam.sdk.transforms.OldDoFn; +import org.apache.beam.sdk.transforms.DoFn; /** * A collection of values associated with an {@link Aggregator}. Aggregators declared in a - * {@link OldDoFn} are emitted on a per-{@code OldDoFn}-application basis. + * {@link DoFn} are emitted on a per-{@link DoFn}-application basis. * * @param the output type of the aggregator */ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Aggregator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Aggregator.java index db4ab33ff1b0..67d399fbc820 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Aggregator.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Aggregator.java @@ -25,8 +25,8 @@ * to be combined across all bundles. * *

          Aggregators are created by calling - * {@link OldDoFn#createAggregator OldDoFn.createAggregatorForDoFn}, - * typically from the {@link OldDoFn} constructor. Elements can be added to the + * {@link DoFn#createAggregator DoFn.createAggregatorForDoFn}, + * typically from the {@link DoFn} constructor. Elements can be added to the * {@code Aggregator} by calling {@link Aggregator#addValue}. * *

          Aggregators are visible in the monitoring UI, when the pipeline is run @@ -37,14 +37,14 @@ * *

          Example: *

           {@code
          - * class MyDoFn extends OldDoFn {
          + * class MyDoFn extends DoFn {
            *   private Aggregator myAggregator;
            *
            *   public MyDoFn() {
            *     myAggregator = createAggregatorForDoFn("myAggregator", new Sum.SumIntegerFn());
            *   }
            *
          - *   @Override
          + *   @ProcessElement
            *   public void processElement(ProcessContext c) {
            *     myAggregator.addValue(1);
            *   }
          @@ -79,8 +79,8 @@ interface AggregatorFactory {
               /**
                * Create an aggregator with the given {@code name} and {@link CombineFn}.
                *
          -     *  

          This method is called to create an aggregator for a {@link OldDoFn}. It receives the - * class of the {@link OldDoFn} being executed and the context of the step it is being + *

          This method is called to create an aggregator for a {@link DoFn}. It receives the + * class of the {@link DoFn} being executed and the context of the step it is being * executed in. */ Aggregator createAggregatorForDoFn( @@ -90,7 +90,7 @@ Aggregator createAggregatorForDoFn( // TODO: Consider the following additional API conveniences: // - In addition to createAggregatorForDoFn(), consider adding getAggregator() to - // avoid the need to store the aggregator locally in a OldDoFn, i.e., create + // avoid the need to store the aggregator locally in a DoFn, i.e., create // if not already present. // - Add a shortcut for the most common aggregator: // c.createAggregatorForDoFn("name", new Sum.SumIntegerFn()). diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFns.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFns.java index 9fa8ded6b240..6f059932f5f4 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFns.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFns.java @@ -67,7 +67,7 @@ public class CombineFns { *

          The same {@link TupleTag} cannot be used in a composition multiple times. * *

          Example: - *

          { @code
          +   * 
          
              * PCollection> latencies = ...;
              *
              * TupleTag maxLatencyTag = new TupleTag();
          @@ -75,7 +75,7 @@ public class CombineFns {
              *
              * SimpleFunction identityFn =
              *     new SimpleFunction() {
          -   *       @Override
          +   *      {@literal @}Override
              *       public Integer apply(Integer input) {
              *           return input;
              *       }};
          @@ -87,8 +87,8 @@ public class CombineFns {
              *
              * PCollection finalResultCollection = maxAndMean
              *     .apply(ParDo.of(
          -   *         new OldDoFn, T>() {
          -   *           @Override
          +   *         new DoFn, T>() {
          +   *          {@literal @}ProcessElement
              *           public void processElement(ProcessContext c) throws Exception {
              *             KV e = c.element();
              *             Integer maxLatency = e.getValue().get(maxLatencyTag);
          @@ -97,7 +97,7 @@ public class CombineFns {
              *             c.output(...some T...);
              *           }
              *         }));
          -   * } 
          + *
          */ public static ComposeKeyedCombineFnBuilder composeKeyed() { return new ComposeKeyedCombineFnBuilder(); @@ -110,7 +110,7 @@ public static ComposeKeyedCombineFnBuilder composeKeyed() { *

          The same {@link TupleTag} cannot be used in a composition multiple times. * *

          Example: - *

          { @code
          +   * 
          
              * PCollection globalLatencies = ...;
              *
              * TupleTag maxLatencyTag = new TupleTag();
          @@ -130,8 +130,8 @@ public static ComposeKeyedCombineFnBuilder composeKeyed() {
              *
              * PCollection finalResultCollection = maxAndMean
              *     .apply(ParDo.of(
          -   *         new OldDoFn() {
          -   *           @Override
          +   *         new DoFn() {
          +   *          {@literal @}ProcessElement
              *           public void processElement(ProcessContext c) throws Exception {
              *             CoCombineResult e = c.element();
              *             Integer maxLatency = e.get(maxLatencyTag);
          @@ -140,7 +140,7 @@ public static ComposeKeyedCombineFnBuilder composeKeyed() {
              *             c.output(...some T...);
              *           }
              *         }));
          -   * } 
          + *
          */ public static ComposeCombineFnBuilder compose() { return new ComposeCombineFnBuilder(); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java index 9f898261c7bb..59c832323686 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java @@ -63,8 +63,6 @@ * that satisfies the requirements described there. See the {@link ProcessElement} * for details. * - *

          This functionality is experimental and likely to change. - * *

          Example usage: * *

           {@code
          @@ -123,7 +121,7 @@ public abstract class Context {
                *
                * 

          If invoked from {@link ProcessElement}), the timestamp * must not be older than the input element's timestamp minus - * {@link OldDoFn#getAllowedTimestampSkew}. The output element will + * {@link DoFn#getAllowedTimestampSkew}. The output element will * be in the same windows as the input element. * *

          If invoked from {@link StartBundle} or {@link FinishBundle}, @@ -172,7 +170,7 @@ public abstract class Context { * *

          If invoked from {@link ProcessElement}), the timestamp * must not be older than the input element's timestamp minus - * {@link OldDoFn#getAllowedTimestampSkew}. The output element will + * {@link DoFn#getAllowedTimestampSkew}. The output element will * be in the same windows as the input element. * *

          If invoked from {@link StartBundle} or {@link FinishBundle}, @@ -190,7 +188,7 @@ public abstract void sideOutputWithTimestamp( } /** - * Information accessible when running {@link OldDoFn#processElement}. + * Information accessible when running a {@link DoFn.ProcessElement} method. */ public abstract class ProcessContext extends Context { @@ -359,9 +357,14 @@ public OutputReceiver outputReceiver() { * Annotation for the method to use to prepare an instance for processing a batch of elements. * The method annotated with this must satisfy the following constraints: *

            - *
          • It must have at least one argument. + *
          • It must have exactly one argument. *
          • Its first (and only) argument must be a {@link DoFn.Context}. *
          + * + *

          A simple method declaration would look like: + * + * public void setup(DoFn.Context c) { .. } + * */ @Documented @Retention(RetentionPolicy.RUNTIME) @@ -414,13 +417,13 @@ public OutputReceiver outputReceiver() { /** * Returns an {@link Aggregator} with aggregation logic specified by the * {@link CombineFn} argument. The name provided must be unique across - * {@link Aggregator}s created within the OldDoFn. Aggregators can only be created + * {@link Aggregator}s created within the {@link DoFn}. Aggregators can only be created * during pipeline construction. * * @param name the name of the aggregator * @param combiner the {@link CombineFn} to use in the aggregator * @return an aggregator for the provided name and combiner in the scope of - * this OldDoFn + * this {@link DoFn} * @throws NullPointerException if the name or combiner is null * @throws IllegalArgumentException if the given name collides with another * aggregator in this scope @@ -447,13 +450,13 @@ public OutputReceiver outputReceiver() { /** * Returns an {@link Aggregator} with the aggregation logic specified by the * {@link SerializableFunction} argument. The name provided must be unique - * across {@link Aggregator}s created within the OldDoFn. Aggregators can only be + * across {@link Aggregator}s created within the {@link DoFn}. Aggregators can only be * created during pipeline construction. * * @param name the name of the aggregator * @param combiner the {@link SerializableFunction} to use in the aggregator * @return an aggregator for the provided name and combiner in the scope of - * this OldDoFn + * this {@link DoFn} * @throws NullPointerException if the name or combiner is null * @throws IllegalArgumentException if the given name collides with another * aggregator in this scope diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java index 82c12938fa68..6801768c1096 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java @@ -50,12 +50,12 @@ import org.joda.time.Instant; /** - * A harness for unit-testing a {@link OldDoFn}. + * A harness for unit-testing a {@link DoFn}. * *

          For example: * *

           {@code
          - * OldDoFn fn = ...;
          + * DoFn fn = ...;
            *
            * DoFnTester fnTester = DoFnTester.of(fn);
            *
          @@ -72,17 +72,17 @@
            * Assert.assertThat(fnTester.processBundle(i1, i2, ...), Matchers.hasItems(...));
            * } 
          * - * @param the type of the {@code OldDoFn}'s (main) input elements - * @param the type of the {@code OldDoFn}'s (main) output elements + * @param the type of the {@link DoFn}'s (main) input elements + * @param the type of the {@link DoFn}'s (main) output elements */ public class DoFnTester { /** * Returns a {@code DoFnTester} supporting unit-testing of the given - * {@link OldDoFn}. + * {@link DoFn}. */ @SuppressWarnings("unchecked") - public static DoFnTester of(OldDoFn fn) { - return new DoFnTester(fn); + public static DoFnTester of(DoFn fn) { + return new DoFnTester<>(DoFnAdapters.toOldDoFn(fn)); } /** @@ -90,19 +90,19 @@ public static DoFnTester of(OldDoFn DoFnTester - of(DoFn fn) { - return new DoFnTester(DoFnAdapters.toOldDoFn(fn)); + public static DoFnTester + of(OldDoFn fn) { + return new DoFnTester<>(fn); } /** * Registers the tuple of values of the side input {@link PCollectionView}s to - * pass to the {@link OldDoFn} under test. + * pass to the {@link DoFn} under test. * *

          Resets the state of this {@link DoFnTester}. * *

          If this isn't called, {@code DoFnTester} assumes the - * {@link OldDoFn} takes no side inputs. + * {@link DoFn} takes no side inputs. */ public void setSideInputs(Map, Map> sideInputs) { this.sideInputs = sideInputs; @@ -110,7 +110,7 @@ public void setSideInputs(Map, Map> sideInp } /** - * Registers the values of a side input {@link PCollectionView} to pass to the {@link OldDoFn} + * Registers the values of a side input {@link PCollectionView} to pass to the {@link DoFn} * under test. * *

          The provided value is the final value of the side input in the specified window, not @@ -129,7 +129,7 @@ public void setSideInput(PCollectionView sideInput, BoundedWindow window, } /** - * Whether or not a {@link DoFnTester} should clone the {@link OldDoFn} under test. + * Whether or not a {@link DoFnTester} should clone the {@link DoFn} under test. */ public enum CloningBehavior { CLONE, @@ -137,14 +137,14 @@ public enum CloningBehavior { } /** - * Instruct this {@link DoFnTester} whether or not to clone the {@link OldDoFn} under test. + * Instruct this {@link DoFnTester} whether or not to clone the {@link DoFn} under test. */ public void setCloningBehavior(CloningBehavior newValue) { this.cloningBehavior = newValue; } /** - * Indicates whether this {@link DoFnTester} will clone the {@link OldDoFn} under test. + * Indicates whether this {@link DoFnTester} will clone the {@link DoFn} under test. */ public CloningBehavior getCloningBehavior() { return cloningBehavior; @@ -166,7 +166,7 @@ public List processBundle(Iterable inputElements) th } /** - * A convenience method for testing {@link OldDoFn DoFns} with bundles of elements. + * A convenience method for testing {@link DoFn DoFns} with bundles of elements. * Logic proceeds as follows: * *

            @@ -182,9 +182,9 @@ public final List processBundle(InputT... inputElements) throws Excepti } /** - * Calls {@link OldDoFn#startBundle} on the {@code OldDoFn} under test. + * Calls the {@link DoFn.StartBundle} method on the {@link DoFn} under test. * - *

            If needed, first creates a fresh instance of the OldDoFn under test. + *

            If needed, first creates a fresh instance of the {@link DoFn} under test. */ public void startBundle() throws Exception { resetState(); @@ -210,14 +210,14 @@ private static void unwrapUserCodeException(UserCodeException e) throws Exceptio } /** - * Calls {@link OldDoFn#processElement} on the {@code OldDoFn} under test, in a - * context where {@link OldDoFn.ProcessContext#element} returns the + * Calls the {@link DoFn.ProcessElement} method on the {@link DoFn} under test, in a + * context where {@link DoFn.ProcessContext#element} returns the * given element. * *

            Will call {@link #startBundle} automatically, if it hasn't * already been called. * - * @throws IllegalStateException if the {@code OldDoFn} under test has already + * @throws IllegalStateException if the {@code DoFn} under test has already * been finished */ public void processElement(InputT element) throws Exception { @@ -235,12 +235,12 @@ public void processElement(InputT element) throws Exception { } /** - * Calls {@link OldDoFn#finishBundle} of the {@code OldDoFn} under test. + * Calls the {@link DoFn.FinishBundle} method of the {@link DoFn} under test. * *

            Will call {@link #startBundle} automatically, if it hasn't * already been called. * - * @throws IllegalStateException if the {@code OldDoFn} under test has already + * @throws IllegalStateException if the {@link DoFn} under test has already * been finished */ public void finishBundle() throws Exception { @@ -674,7 +674,7 @@ protected Aggregator createAggreg ///////////////////////////////////////////////////////////////////////////// - /** The possible states of processing a OldDoFn. */ + /** The possible states of processing a {@link DoFn}. */ enum State { UNSTARTED, STARTED, @@ -683,23 +683,23 @@ enum State { private final PipelineOptions options = PipelineOptionsFactory.create(); - /** The original OldDoFn under test. */ + /** The original {@link OldDoFn} under test. */ private final OldDoFn origFn; /** - * Whether to clone the original {@link OldDoFn} or just use it as-is. + * Whether to clone the original {@link DoFn} or just use it as-is. * - *

            Worker-side {@link OldDoFn DoFns} may not be serializable, and are not required to be. + *

            Worker-side {@link DoFn DoFns} may not be serializable, and are not required to be. */ private CloningBehavior cloningBehavior = CloningBehavior.CLONE; - /** The side input values to provide to the OldDoFn under test. */ + /** The side input values to provide to the {@link DoFn} under test. */ private Map, Map> sideInputs = new HashMap<>(); private Map accumulators; - /** The output tags used by the OldDoFn under test. */ + /** The output tags used by the {@link DoFn} under test. */ private TupleTag mainOutputTag = new TupleTag<>(); /** The original OldDoFn under test, if started. */ @@ -708,7 +708,7 @@ enum State { /** The ListOutputManager to examine the outputs. */ private Map, List>> outputs; - /** The state of processing of the OldDoFn under test. */ + /** The state of processing of the {@link DoFn} under test. */ private State state; private DoFnTester(OldDoFn origFn) { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupByKey.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupByKey.java index ed7f411aec11..3a3da65e077d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupByKey.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupByKey.java @@ -63,18 +63,19 @@ * {@code Coder} of the values of the input. * *

            Example of use: - *

             {@code
            + * 
            
              * PCollection> urlDocPairs = ...;
              * PCollection>> urlToDocs =
              *     urlDocPairs.apply(GroupByKey.create());
              * PCollection results =
            - *     urlToDocs.apply(ParDo.of(new OldDoFn>, R>() {
            + *     urlToDocs.apply(ParDo.of(new DoFn>, R>() {
            + *      {@literal @}ProcessElement
              *       public void processElement(ProcessContext c) {
              *         String url = c.element().getKey();
              *         Iterable docsWithThatUrl = c.element().getValue();
              *         ... process all docs having that url ...
              *       }}));
            - * } 
            + *
            * *

            {@code GroupByKey} is a key primitive in data-parallel * processing, since it is the main way to efficiently bring diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java index 19abef90cea1..4a58141ab503 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java @@ -146,7 +146,7 @@ * implementing {@code Serializable}. * *

            {@code PTransform} is marked {@code Serializable} solely - * because it is common for an anonymous {@code OldDoFn}, + * because it is common for an anonymous {@link DoFn}, * instance to be created within an * {@code apply()} method of a composite {@code PTransform}. * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java index 5efbe9f19024..f9cb557c1e5d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java @@ -22,7 +22,6 @@ import java.util.Arrays; import java.util.List; import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.runners.PipelineRunner; @@ -50,13 +49,12 @@ *

            The {@link ParDo} processing style is similar to what happens inside * the "Mapper" or "Reducer" class of a MapReduce-style algorithm. * - *

            {@link OldDoFn DoFns}

            + *

            {@link DoFn DoFns}

            * *

            The function to use to process each element is specified by a - * {@link OldDoFn OldDoFn<InputT, OutputT>}, primarily via its - * {@link OldDoFn#processElement processElement} method. The {@link OldDoFn} may also - * override the default implementations of {@link OldDoFn#startBundle startBundle} - * and {@link OldDoFn#finishBundle finishBundle}. + * {@link DoFn DoFn<InputT, OutputT>}, primarily via its + * {@link DoFn.ProcessElement ProcessElement} method. The {@link DoFn} may also + * provide a {@link DoFn.StartBundle StartBundle} and {@link DoFn.FinishBundle finishBundle} method. * *

            Conceptually, when a {@link ParDo} transform is executed, the * elements of the input {@link PCollection} are first divided up @@ -66,39 +64,38 @@ * *

              *
            1. If required, a fresh instance of the argument {@link DoFn} is created - * on a worker, and {@link DoFn#setup()} is called on this instance. This may be through - * deserialization or other means. A {@link PipelineRunner} may reuse {@link DoFn} instances for - * multiple bundles. A {@link DoFn} that has terminated abnormally (by throwing an + * on a worker, and the {@link DoFn.Setup} method is called on this instance. This may be + * through deserialization or other means. A {@link PipelineRunner} may reuse {@link DoFn} + * instances for multiple bundles. A {@link DoFn} that has terminated abnormally (by throwing an * {@link Exception}) will never be reused.
            2. - *
            3. The {@link OldDoFn OldDoFn's} {@link OldDoFn#startBundle} method is called to - * initialize it. If this method is not overridden, the call may be optimized - * away.
            4. - *
            5. The {@link OldDoFn OldDoFn's} {@link OldDoFn#processElement} method + *
            6. The {@link DoFn DoFn's} {@link DoFn.StartBundle} method, if provided, is called to + * initialize it.
            7. + *
            8. The {@link DoFn DoFn's} {@link DoFn.ProcessElement} method * is called on each of the input elements in the bundle.
            9. - *
            10. The {@link OldDoFn OldDoFn's} {@link OldDoFn#finishBundle} method is called - * to complete its work. After {@link OldDoFn#finishBundle} is called, the - * framework will not again invoke {@link OldDoFn#processElement} or - * {@link OldDoFn#finishBundle} - * until a new call to {@link OldDoFn#startBundle} has occurred. - * If this method is not overridden, this call may be optimized away.
            11. - *
            12. If any of {@link DoFn#setup}, {@link DoFn#startBundle}, {@link DoFn#processElement} or - * {@link DoFn#finishBundle} throw an exception, {@link DoFn#teardown} will be called on the - * {@link DoFn} instance.
            13. - *
            14. If a runner will no longer use a {@link DoFn}, {@link DoFn#teardown()} will be called on - * the discarded instance.
            15. + *
            16. The {@link DoFn DoFn's} {@link DoFn.FinishBundle} method, if provided, is called + * to complete its work. After {@link DoFn.FinishBundle} is called, the + * framework will not again invoke {@link DoFn.ProcessElement} or + * {@link DoFn.FinishBundle} + * until a new call to {@link DoFn.StartBundle} has occurred.
            17. + *
            18. If any of {@link DoFn.Setup}, {@link DoFn.StartBundle}, {@link DoFn.ProcessElement} or + * {@link DoFn.FinishBundle} methods throw an exception, the {@link DoFn.Teardown} method, if + * provided, will be called on the {@link DoFn} instance.
            19. + *
            20. If a runner will no longer use a {@link DoFn}, the {@link DoFn.Teardown} method, if + * provided, will be called on the discarded instance.
            21. *
            * - * Each of the calls to any of the {@link OldDoFn OldDoFn's} processing + * Each of the calls to any of the {@link DoFn DoFn's} processing * methods can produce zero or more output elements. All of the - * of output elements from all of the {@link OldDoFn} instances + * of output elements from all of the {@link DoFn} instances * are included in the output {@link PCollection}. * *

            For example: * - *

             {@code
            + * 
            
              * PCollection lines = ...;
              * PCollection words =
            - *     lines.apply(ParDo.of(new OldDoFn() {
            + *     lines.apply(ParDo.of(new DoFn() {
            + *        {@literal @}ProcessElement
              *         public void processElement(ProcessContext c) {
              *           String line = c.element();
              *           for (String word : line.split("[^a-zA-Z']+")) {
            @@ -106,13 +103,14 @@
              *           }
              *         }}));
              * PCollection wordLengths =
            - *     words.apply(ParDo.of(new OldDoFn() {
            + *     words.apply(ParDo.of(new DoFn() {
            + *        {@literal @}ProcessElement
              *         public void processElement(ProcessContext c) {
              *           String word = c.element();
              *           Integer length = word.length();
              *           c.output(length);
              *         }}));
            - * } 
            + *
            * *

            Each output element has the same timestamp and is in the same windows * as its corresponding input element, and the output {@code PCollection} @@ -131,9 +129,9 @@ * *

             {@code
              * PCollection words =
            - *     lines.apply("ExtractWords", ParDo.of(new OldDoFn() { ... }));
            + *     lines.apply("ExtractWords", ParDo.of(new DoFn() { ... }));
              * PCollection wordLengths =
            - *     words.apply("ComputeWordLengths", ParDo.of(new OldDoFn() { ... }));
            + *     words.apply("ComputeWordLengths", ParDo.of(new DoFn() { ... }));
              * } 
            * *

            Side Inputs

            @@ -145,17 +143,18 @@ * {@link PCollection PCollections} computed by earlier pipeline operations, * passed in to the {@link ParDo} transform using * {@link #withSideInputs}, and their contents accessible to each of - * the {@link OldDoFn} operations via {@link OldDoFn.ProcessContext#sideInput sideInput}. + * the {@link DoFn} operations via {@link DoFn.ProcessContext#sideInput sideInput}. * For example: * - *
             {@code
            + * 
            
              * PCollection words = ...;
              * PCollection maxWordLengthCutOff = ...; // Singleton PCollection
              * final PCollectionView maxWordLengthCutOffView =
              *     maxWordLengthCutOff.apply(View.asSingleton());
              * PCollection wordsBelowCutOff =
              *     words.apply(ParDo.withSideInputs(maxWordLengthCutOffView)
            - *                      .of(new OldDoFn() {
            + *                      .of(new DoFn() {
            + *        {@literal @}ProcessElement
              *         public void processElement(ProcessContext c) {
              *           String word = c.element();
              *           int lengthCutOff = c.sideInput(maxWordLengthCutOffView);
            @@ -163,7 +162,7 @@
              *             c.output(word);
              *           }
              *         }}));
            - * } 
            + *
            * *

            Side Outputs

            * @@ -174,13 +173,13 @@ * and bundled in a {@link PCollectionTuple}. The {@link TupleTag TupleTags} * to be used for the output {@link PCollectionTuple} are specified by * invoking {@link #withOutputTags}. Unconsumed side outputs do not - * necessarily need to be explicitly specified, even if the {@link OldDoFn} - * generates them. Within the {@link OldDoFn}, an element is added to the + * necessarily need to be explicitly specified, even if the {@link DoFn} + * generates them. Within the {@link DoFn}, an element is added to the * main output {@link PCollection} as normal, using - * {@link OldDoFn.Context#output}, while an element is added to a side output - * {@link PCollection} using {@link OldDoFn.Context#sideOutput}. For example: + * {@link DoFn.Context#output}, while an element is added to a side output + * {@link PCollection} using {@link DoFn.Context#sideOutput}. For example: * - *
             {@code
            + * 
            
              * PCollection words = ...;
              * // Select words whose length is below a cut off,
              * // plus the lengths of words that are above the cut off.
            @@ -201,10 +200,11 @@
              *         .withOutputTags(wordsBelowCutOffTag,
              *                         TupleTagList.of(wordLengthsAboveCutOffTag)
              *                                     .and(markedWordsTag))
            - *         .of(new OldDoFn() {
            + *         .of(new DoFn() {
              *             // Create a tag for the unconsumed side output.
              *             final TupleTag specialWordsTag =
              *                 new TupleTag(){};
            + *            {@literal @}ProcessElement
              *             public void processElement(ProcessContext c) {
              *               String word = c.element();
              *               if (word.length() <= wordLengthCutOff) {
            @@ -230,14 +230,13 @@
              *     results.get(wordLengthsAboveCutOffTag);
              * PCollection markedWords =
              *     results.get(markedWordsTag);
            - * } 
            + *
            * *

            Properties May Be Specified In Any Order

            * *

            Several properties can be specified for a {@link ParDo} - * {@link PTransform}, including name, side inputs, side output tags, - * and {@link OldDoFn} to invoke. Only the {@link OldDoFn} is required; the - * name is encouraged but not required, and side inputs and side + * {@link PTransform}, including side inputs, side output tags, + * and {@link DoFn} to invoke. Only the {@link DoFn} is required; side inputs and side * output tags are only specified when they're needed. These * properties can be specified in any order, as long as they're * specified before the {@link ParDo} {@link PTransform} is applied. @@ -250,23 +249,23 @@ * {@link ParDo.Bound} nested classes, each of which offer * property setter instance methods to enable setting additional * properties. {@link ParDo.Bound} is used for {@link ParDo} - * transforms whose {@link OldDoFn} is specified and whose input and + * transforms whose {@link DoFn} is specified and whose input and * output static types have been bound. {@link ParDo.Unbound ParDo.Unbound} is used * for {@link ParDo} transforms that have not yet had their - * {@link OldDoFn} specified. Only {@link ParDo.Bound} instances can be + * {@link DoFn} specified. Only {@link ParDo.Bound} instances can be * applied. * *

            Another benefit of this approach is that it reduces the number * of type parameters that need to be specified manually. In * particular, the input and output types of the {@link ParDo} * {@link PTransform} are inferred automatically from the type - * parameters of the {@link OldDoFn} argument passed to {@link ParDo#of}. + * parameters of the {@link DoFn} argument passed to {@link ParDo#of}. * *

            Output Coders

            * *

            By default, the {@link Coder Coder<OutputT>} for the * elements of the main output {@link PCollection PCollection<OutputT>} is - * inferred from the concrete type of the {@link OldDoFn OldDoFn<InputT, OutputT>}. + * inferred from the concrete type of the {@link DoFn DoFn<InputT, OutputT>}. * *

            By default, the {@link Coder Coder<SideOutputT>} for the elements of * a side output {@link PCollection PCollection<SideOutputT>} is inferred @@ -286,74 +285,74 @@ * This style of {@code TupleTag} instantiation is used in the example of * multiple side outputs, above. * - *

            Serializability of {@link OldDoFn DoFns}

            + *

            Serializability of {@link DoFn DoFns}

            * - *

            A {@link OldDoFn} passed to a {@link ParDo} transform must be - * {@link Serializable}. This allows the {@link OldDoFn} instance + *

            A {@link DoFn} passed to a {@link ParDo} transform must be + * {@link Serializable}. This allows the {@link DoFn} instance * created in this "main program" to be sent (in serialized form) to * remote worker machines and reconstituted for bundles of elements - * of the input {@link PCollection} being processed. A {@link OldDoFn} + * of the input {@link PCollection} being processed. A {@link DoFn} * can have instance variable state, and non-transient instance * variable state will be serialized in the main program and then * deserialized on remote worker machines for some number of bundles * of elements to process. * - *

            {@link OldDoFn DoFns} expressed as anonymous inner classes can be + *

            {@link DoFn DoFns} expressed as anonymous inner classes can be * convenient, but due to a quirk in Java's rules for serializability, * non-static inner or nested classes (including anonymous inner * classes) automatically capture their enclosing class's instance in * their serialized state. This can lead to including much more than - * intended in the serialized state of a {@link OldDoFn}, or even things + * intended in the serialized state of a {@link DoFn}, or even things * that aren't {@link Serializable}. * *

            There are two ways to avoid unintended serialized state in a - * {@link OldDoFn}: + * {@link DoFn}: * *

              * - *
            • Define the {@link OldDoFn} as a named, static class. + *
            • Define the {@link DoFn} as a named, static class. * - *
            • Define the {@link OldDoFn} as an anonymous inner class inside of + *
            • Define the {@link DoFn} as an anonymous inner class inside of * a static method. * *
            * *

            Both of these approaches ensure that there is no implicit enclosing - * instance serialized along with the {@link OldDoFn} instance. + * instance serialized along with the {@link DoFn} instance. * *

            Prior to Java 8, any local variables of the enclosing * method referenced from within an anonymous inner class need to be - * marked as {@code final}. If defining the {@link OldDoFn} as a named + * marked as {@code final}. If defining the {@link DoFn} as a named * static class, such variables would be passed as explicit * constructor arguments and stored in explicit instance variables. * *

            There are three main ways to initialize the state of a - * {@link OldDoFn} instance processing a bundle: + * {@link DoFn} instance processing a bundle: * *

              * *
            • Define instance variable state (including implicit instance * variables holding final variables captured by an anonymous inner - * class), initialized by the {@link OldDoFn}'s constructor (which is + * class), initialized by the {@link DoFn}'s constructor (which is * implicit for an anonymous inner class). This state will be - * automatically serialized and then deserialized in the {@code OldDoFn} + * automatically serialized and then deserialized in the {@link DoFn} * instances created for bundles. This method is good for state - * known when the original {@code OldDoFn} is created in the main + * known when the original {@link DoFn} is created in the main * program, if it's not overly large. This is not suitable for any - * state which must only be used for a single bundle, as {@link OldDoFn OldDoFn's} + * state which must only be used for a single bundle, as {@link DoFn DoFn's} * may be used to process multiple bundles. * *
            • Compute the state as a singleton {@link PCollection} and pass it - * in as a side input to the {@link OldDoFn}. This is good if the state + * in as a side input to the {@link DoFn}. This is good if the state * needs to be computed by the pipeline, or if the state is very large * and so is best read from file(s) rather than sent as part of the - * {@code OldDoFn}'s serialized state. + * {@link DoFn DoFn's} serialized state. * - *
            • Initialize the state in each {@link OldDoFn} instance, in - * {@link OldDoFn#startBundle}. This is good if the initialization + *
            • Initialize the state in each {@link DoFn} instance, in a + * {@link DoFn.StartBundle} method. This is good if the initialization * doesn't depend on any information known only by the main program or * computed by earlier pipeline operations, but is the same for all - * instances of this {@link OldDoFn} for all program executions, say + * instances of this {@link DoFn} for all program executions, say * setting up empty caches or initializing constant data. * *
            @@ -363,16 +362,16 @@ *

            {@link ParDo} operations are intended to be able to run in * parallel across multiple worker machines. This precludes easy * sharing and updating mutable state across those machines. There is - * no support in the Google Cloud Dataflow system for communicating + * no support in the Beam model for communicating * and synchronizing updates to shared state across worker machines, * so programs should not access any mutable static variable state in - * their {@link OldDoFn}, without understanding that the Java processes + * their {@link DoFn}, without understanding that the Java processes * for the main program and workers will each have its own independent * copy of such state, and there won't be any automatic copying of * that state across Java processes. All information should be - * communicated to {@link OldDoFn} instances via main and side inputs and + * communicated to {@link DoFn} instances via main and side inputs and * serialized state, and all output should be communicated from a - * {@link OldDoFn} instance via main and side outputs, in the absence of + * {@link DoFn} instance via main and side outputs, in the absence of * external communication mechanisms written by user code. * *

            Fault Tolerance

            @@ -380,29 +379,28 @@ *

            In a distributed system, things can fail: machines can crash, * machines can be unable to communicate across the network, etc. * While individual failures are rare, the larger the job, the greater - * the chance that something, somewhere, will fail. The Google Cloud - * Dataflow service strives to mask such failures automatically, - * principally by retrying failed {@link OldDoFn} bundle. This means - * that a {@code OldDoFn} instance might process a bundle partially, then - * crash for some reason, then be rerun (often on a different worker - * machine) on that same bundle and on the same elements as before. - * Sometimes two or more {@link OldDoFn} instances will be running on the + * the chance that something, somewhere, will fail. Beam runners may strive + * to mask such failures by retrying failed {@link DoFn} bundle. This means + * that a {@link DoFn} instance might process a bundle partially, then + * crash for some reason, then be rerun (often in a new JVM) on that + * same bundle and on the same elements as before. + * Sometimes two or more {@link DoFn} instances will be running on the * same bundle simultaneously, with the system taking the results of * the first instance to complete successfully. Consequently, the - * code in a {@link OldDoFn} needs to be written such that these + * code in a {@link DoFn} needs to be written such that these * duplicate (sequential or concurrent) executions do not cause - * problems. If the outputs of a {@link OldDoFn} are a pure function of + * problems. If the outputs of a {@link DoFn} are a pure function of * its inputs, then this requirement is satisfied. However, if a - * {@link OldDoFn OldDoFn's} execution has external side-effects, such as performing - * updates to external HTTP services, then the {@link OldDoFn OldDoFn's} code + * {@link DoFn DoFn's} execution has external side-effects, such as performing + * updates to external HTTP services, then the {@link DoFn DoFn's} code * needs to take care to ensure that those updates are idempotent and * that concurrent updates are acceptable. This property can be * difficult to achieve, so it is advisable to strive to keep - * {@link OldDoFn DoFns} as pure functions as much as possible. + * {@link DoFn DoFns} as pure functions as much as possible. * *

            Optimization

            * - *

            The Google Cloud Dataflow service automatically optimizes a + *

            Beam runners may choose to apply optimizations to a * pipeline before it is executed. A key optimization, fusion, * relates to {@link ParDo} operations. If one {@link ParDo} operation produces a * {@link PCollection} that is then consumed as the main input of another @@ -419,18 +417,16 @@ * written to disk, saving all the I/O and space expense of * constructing it. * - *

            The Google Cloud Dataflow service applies fusion as much as - * possible, greatly reducing the cost of executing pipelines. As a - * result, it is essentially "free" to write {@link ParDo} operations in a + *

            When Beam runners apply fusion optimization, it is essentially "free" + * to write {@link ParDo} operations in a * very modular, composable style, each {@link ParDo} operation doing one * clear task, and stringing together sequences of {@link ParDo} operations to * get the desired overall effect. Such programs can be easier to * understand, easier to unit-test, easier to extend and evolve, and * easier to reuse in new programs. The predefined library of - * PTransforms that come with Google Cloud Dataflow makes heavy use of - * this modular, composable style, trusting to the Google Cloud - * Dataflow service's optimizer to "flatten out" all the compositions - * into highly optimized stages. + * PTransforms that come with Beam makes heavy use of + * this modular, composable style, trusting to the runner to + * "flatten out" all the compositions into highly optimized stages. * * @see the web * documentation for ParDo @@ -443,15 +439,15 @@ public class ParDo { * *

            Side inputs are {@link PCollectionView PCollectionViews}, whose contents are * computed during pipeline execution and then made accessible to - * {@link OldDoFn} code via {@link OldDoFn.ProcessContext#sideInput sideInput}. Each - * invocation of the {@link OldDoFn} receives the same values for these + * {@link DoFn} code via {@link DoFn.ProcessContext#sideInput sideInput}. Each + * invocation of the {@link DoFn} receives the same values for these * side inputs. * *

            See the discussion of Side Inputs above for more explanation. * *

            The resulting {@link PTransform} is incomplete, and its * input/output types are not yet bound. Use - * {@link ParDo.Unbound#of} to specify the {@link OldDoFn} to + * {@link ParDo.Unbound#of} to specify the {@link DoFn} to * invoke, which will also bind the input/output types of this * {@link PTransform}. */ @@ -464,13 +460,13 @@ public static Unbound withSideInputs(PCollectionView... sideInputs) { * *

            Side inputs are {@link PCollectionView}s, whose contents are * computed during pipeline execution and then made accessible to - * {@code OldDoFn} code via {@link OldDoFn.ProcessContext#sideInput sideInput}. + * {@link DoFn} code via {@link DoFn.ProcessContext#sideInput sideInput}. * *

            See the discussion of Side Inputs above for more explanation. * *

            The resulting {@link PTransform} is incomplete, and its * input/output types are not yet bound. Use - * {@link ParDo.Unbound#of} to specify the {@link OldDoFn} to + * {@link ParDo.Unbound#of} to specify the {@link DoFn} to * invoke, which will also bind the input/output types of this * {@link PTransform}. */ @@ -486,11 +482,11 @@ public static Unbound withSideInputs( * *

            {@link TupleTag TupleTags} are used to name (with its static element * type {@code T}) each main and side output {@code PCollection}. - * This {@link PTransform PTransform's} {@link OldDoFn} emits elements to the main + * This {@link PTransform PTransform's} {@link DoFn} emits elements to the main * output {@link PCollection} as normal, using - * {@link OldDoFn.Context#output}. The {@link OldDoFn} emits elements to + * {@link DoFn.Context#output}. The {@link DoFn} emits elements to * a side output {@code PCollection} using - * {@link OldDoFn.Context#sideOutput}, passing that side output's tag + * {@link DoFn.Context#sideOutput}, passing that side output's tag * as an argument. The result of invoking this {@link PTransform} * will be a {@link PCollectionTuple}, and any of the the main and * side output {@code PCollection}s can be retrieved from it via @@ -501,7 +497,7 @@ public static Unbound withSideInputs( * *

            The resulting {@link PTransform} is incomplete, and its input * type is not yet bound. Use {@link ParDo.UnboundMulti#of} - * to specify the {@link OldDoFn} to invoke, which will also bind the + * to specify the {@link DoFn} to invoke, which will also bind the * input type of this {@link PTransform}. */ public static UnboundMulti withOutputTags( @@ -510,6 +506,20 @@ public static UnboundMulti withOutputTags( return new Unbound().withOutputTags(mainOutputTag, sideOutputTags); } + /** + * Creates a {@link ParDo} {@link PTransform} that will invoke the + * given {@link DoFn} function. + * + *

            The resulting {@link PTransform PTransform's} types have been bound, with the + * input being a {@code PCollection} and the output a + * {@code PCollection}, inferred from the types of the argument + * {@code DoFn}. It is ready to be applied, or further + * properties can be set on it first. + */ + public static Bound of(DoFn fn) { + return of(adapt(fn), fn.getClass()); + } + /** * Creates a {@link ParDo} {@link PTransform} that will invoke the * given {@link OldDoFn} function. @@ -537,29 +547,11 @@ private static Bound of( return DoFnAdapters.toOldDoFn(fn); } - /** - * Creates a {@link ParDo} {@link PTransform} that will invoke the - * given {@link DoFn} function. - * - *

            The resulting {@link PTransform PTransform's} types have been bound, with the - * input being a {@code PCollection} and the output a - * {@code PCollection}, inferred from the types of the argument - * {@code OldDoFn}. It is ready to be applied, or further - * properties can be set on it first. - * - *

            {@link DoFn} is an experimental alternative to - * {@link OldDoFn} which simplifies accessing the window of the element. - */ - @Experimental - public static Bound of(DoFn fn) { - return of(adapt(fn), fn.getClass()); - } - /** * An incomplete {@link ParDo} transform, with unbound input/output types. * *

            Before being applied, {@link ParDo.Unbound#of} must be - * invoked to specify the {@link OldDoFn} to invoke, which will also + * invoked to specify the {@link DoFn} to invoke, which will also * bind the input/output types of this {@link PTransform}. */ public static class Unbound { @@ -619,6 +611,18 @@ public UnboundMulti withOutputTags(TupleTag mainOutp name, sideInputs, mainOutputTag, sideOutputTags); } + /** + * Returns a new {@link ParDo} {@link PTransform} that's like this + * transform but which will invoke the given {@link DoFn} + * function, and which has its input and output types bound. Does + * not modify this transform. The resulting {@link PTransform} is + * sufficiently specified to be applied, but more properties can + * still be specified. + */ + public Bound of(DoFn fn) { + return of(adapt(fn), fn.getClass()); + } + /** * Returns a new {@link ParDo} {@link PTransform} that's like this * transform but that will invoke the given {@link OldDoFn} @@ -638,24 +642,11 @@ private Bound of( OldDoFn fn, Class fnClass) { return new Bound<>(name, sideInputs, fn, fnClass); } - - - /** - * Returns a new {@link ParDo} {@link PTransform} that's like this - * transform but which will invoke the given {@link DoFn} - * function, and which has its input and output types bound. Does - * not modify this transform. The resulting {@link PTransform} is - * sufficiently specified to be applied, but more properties can - * still be specified. - */ - public Bound of(DoFn fn) { - return of(adapt(fn), fn.getClass()); - } } /** * A {@link PTransform} that, when applied to a {@code PCollection}, - * invokes a user-specified {@code OldDoFn} on all its elements, + * invokes a user-specified {@code DoFn} on all its elements, * with all its outputs collected into an output * {@code PCollection}. * @@ -756,9 +747,9 @@ protected String getKindString() { /** * {@inheritDoc} * - *

            {@link ParDo} registers its internal {@link OldDoFn} as a subcomponent for display data. - * {@link OldDoFn} implementations can register display data by overriding - * {@link OldDoFn#populateDisplayData}. + *

            {@link ParDo} registers its internal {@link DoFn} as a subcomponent for display data. + * {@link DoFn} implementations can register display data by overriding + * {@link DoFn#populateDisplayData}. */ @Override public void populateDisplayData(Builder builder) { @@ -780,7 +771,7 @@ public List> getSideInputs() { * input type. * *

            Before being applied, {@link ParDo.UnboundMulti#of} must be - * invoked to specify the {@link OldDoFn} to invoke, which will also + * invoked to specify the {@link DoFn} to invoke, which will also * bind the input type of this {@link PTransform}. * * @param the type of the main output {@code PCollection} elements @@ -836,38 +827,41 @@ public UnboundMulti withSideInputs( /** * Returns a new multi-output {@link ParDo} {@link PTransform} - * that's like this transform but that will invoke the given - * {@link OldDoFn} function, and that has its input type bound. + * that's like this transform but which will invoke the given + * {@link DoFn} function, and which has its input type bound. * Does not modify this transform. The resulting * {@link PTransform} is sufficiently specified to be applied, but * more properties can still be specified. */ - public BoundMulti of(OldDoFn fn) { - return of(fn, fn.getClass()); - } - - public BoundMulti of(OldDoFn fn, Class fnClass) { - return new BoundMulti<>( - name, sideInputs, mainOutputTag, sideOutputTags, fn, fnClass); + public BoundMulti of(DoFn fn) { + return of(adapt(fn), fn.getClass()); } /** * Returns a new multi-output {@link ParDo} {@link PTransform} - * that's like this transform but which will invoke the given - * {@link DoFn} function, and which has its input type bound. + * that's like this transform but that will invoke the given + * {@link OldDoFn} function, and that has its input type bound. * Does not modify this transform. The resulting * {@link PTransform} is sufficiently specified to be applied, but * more properties can still be specified. + * + * @deprecated please port your {@link OldDoFn} to a {@link DoFn} */ - public BoundMulti of(DoFn fn) { - return of(adapt(fn), fn.getClass()); + @Deprecated + public BoundMulti of(OldDoFn fn) { + return of(fn, fn.getClass()); + } + + private BoundMulti of(OldDoFn fn, Class fnClass) { + return new BoundMulti<>( + name, sideInputs, mainOutputTag, sideOutputTags, fn, fnClass); } } /** * A {@link PTransform} that, when applied to a * {@code PCollection}, invokes a user-specified - * {@code OldDoFn} on all its elements, which can emit elements + * {@code DoFn} on all its elements, which can emit elements * to any of the {@link PTransform}'s main and side output * {@code PCollection}s, which are bundled into a result * {@code PCollectionTuple}. @@ -939,7 +933,7 @@ public PCollectionTuple apply(PCollection input) { input.isBounded()); // The fn will likely be an instance of an anonymous subclass - // such as OldDoFn { }, thus will have a high-fidelity + // such as DoFn { }, thus will have a high-fidelity // TypeDescriptor for the output type. outputs.get(mainOutputTag).setTypeDescriptorInternal(fn.getOutputTypeDescriptor()); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SimpleFunction.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SimpleFunction.java index bf075f8dcbd2..86046598fb4c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SimpleFunction.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SimpleFunction.java @@ -37,7 +37,7 @@ SimpleFunction fromSerializableFunctionWithOutputType( /** * Returns a {@link TypeDescriptor} capturing what is known statically - * about the input type of this {@code OldDoFn} instance's most-derived + * about the input type of this {@link SimpleFunction} instance's most-derived * class. * *

            See {@link #getOutputTypeDescriptor} for more discussion. @@ -48,10 +48,10 @@ public TypeDescriptor getInputTypeDescriptor() { /** * Returns a {@link TypeDescriptor} capturing what is known statically - * about the output type of this {@code OldDoFn} instance's + * about the output type of this {@link SimpleFunction} instance's * most-derived class. * - *

            In the normal case of a concrete {@code OldDoFn} subclass with + *

            In the normal case of a concrete {@link SimpleFunction} subclass with * no generic type parameters of its own (including anonymous inner * classes), this will be a complete non-generic type, which is good * for choosing a default output {@code Coder} for the output diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PaneInfo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PaneInfo.java index 0c87e2271ec9..727a4925cb31 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PaneInfo.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PaneInfo.java @@ -29,15 +29,15 @@ import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.GroupByKey; -import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.util.VarInt; /** * Provides information about the pane an element belongs to. Every pane is implicitly associated * with a window. Panes are observable only via the - * {@link OldDoFn.ProcessContext#pane} method of the context - * passed to a {@link OldDoFn#processElement} overridden method. + * {@link DoFn.ProcessContext#pane} method of the context + * passed to a {@link DoFn.ProcessElement} method. * *

            Note: This does not uniquely identify a pane, and should not be used for comparisons. */ @@ -72,8 +72,8 @@ public final class PaneInfo { * definitions: *

              *
            1. We'll call a pipeline 'simple' if it does not use - * {@link OldDoFn.Context#outputWithTimestamp} in - * any {@code OldDoFn}, and it uses the same + * {@link DoFn.Context#outputWithTimestamp} in + * any {@link DoFn}, and it uses the same * {@link org.apache.beam.sdk.transforms.windowing.Window.Bound#withAllowedLateness} * argument value on all windows (or uses the default of {@link org.joda.time.Duration#ZERO}). *
            2. We'll call an element 'locally late', from the point of view of a computation on a diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BaseExecutionContext.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BaseExecutionContext.java index dead76eb3d00..9ee55ad3a4f0 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BaseExecutionContext.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BaseExecutionContext.java @@ -106,7 +106,7 @@ public Collection getAllStepContexts() { /** * Hook for subclasses to implement that will be called whenever - * {@link OldDoFn.Context#output} + * {@code DoFn.Context#output} * is called. */ @Override @@ -114,7 +114,7 @@ public void noteOutput(WindowedValue output) {} /** * Hook for subclasses to implement that will be called whenever - * {@link OldDoFn.Context#sideOutput} + * {@code DoFn.Context#sideOutput} * is called. */ @Override diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReifyTimestampAndWindowsDoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReifyTimestampAndWindowsDoFn.java index 2808ca90568e..8f3f540e66ae 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReifyTimestampAndWindowsDoFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReifyTimestampAndWindowsDoFn.java @@ -21,8 +21,8 @@ import org.apache.beam.sdk.values.KV; /** - * OldDoFn that makes timestamps and window assignments explicit in the value part of each key/value - * pair. + * {@link OldDoFn} that makes timestamps and window assignments explicit in the value part of each + * key/value pair. * * @param the type of the keys of the input and output {@code PCollection}s * @param the type of the values of the input {@code PCollection} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SerializableUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SerializableUtils.java index 354aa5d91182..6b3218ecf199 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SerializableUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SerializableUtils.java @@ -102,7 +102,7 @@ public static T clone(T value) { */ public static CloudObject ensureSerializable(Coder coder) { // Make sure that Coders are java serializable as well since - // they are regularly captured within OldDoFn's. + // they are regularly captured within DoFn's. Coder copy = (Coder) ensureSerializable((Serializable) coder); CloudObject cloudObject = copy.asCloudObject(); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SystemDoFnInternal.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SystemDoFnInternal.java index e9904b2d7660..004496b8319b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SystemDoFnInternal.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SystemDoFnInternal.java @@ -22,15 +22,14 @@ import java.lang.annotation.Retention; import java.lang.annotation.RetentionPolicy; import java.lang.annotation.Target; -import org.apache.beam.sdk.transforms.OldDoFn; /** - * Annotation to mark {@link OldDoFn DoFns} as an internal component of the Dataflow SDK. + * Annotation to mark {@code DoFns} as an internal component of the Beam SDK. * *

              Currently, the only effect of this is to mark any aggregators reported by an annotated - * {@code OldDoFn} as a system counter (as opposed to a user counter). + * {@code DoFn} as a system counter (as opposed to a user counter). * - *

              This is internal to the Dataflow SDK. + *

              This is internal to the Beam SDK. */ @Documented @Retention(RetentionPolicy.RUNTIME) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowingInternals.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowingInternals.java index 54158d24e520..016276cb4bf3 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowingInternals.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowingInternals.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.util.Collection; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.state.StateInternals; @@ -28,7 +29,7 @@ import org.joda.time.Instant; /** - * Interface that may be required by some (internal) {@code OldDoFn}s to implement windowing. It + * Interface that may be required by some (internal) {@link DoFn}s to implement windowing. It * should not be necessary for general user code to interact with this at all. * *

              This interface should be provided by runner implementors to support windowing on their runner. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnDelegatingAggregatorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnDelegatingAggregatorTest.java index 25b909aabbaf..c072fd7c47db 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnDelegatingAggregatorTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnDelegatingAggregatorTest.java @@ -35,7 +35,7 @@ import org.mockito.MockitoAnnotations; /** - * Tests for OldDoFn.DelegatingAggregator. + * Tests for {@link OldDoFn.DelegatingAggregator}. */ @RunWith(JUnit4.class) public class DoFnDelegatingAggregatorTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java index 2f1519cc4276..2649be5aadf3 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java @@ -276,7 +276,8 @@ public void processElement(ProcessContext c) throws Exception { } /** - * A OldDoFn that adds values to an aggregator and converts input to String in processElement. + * An {@link OldDoFn} that adds values to an aggregator and converts input to String in + * {@link OldDoFn#processElement). */ private static class CounterDoFn extends OldDoFn { Aggregator agg = createAggregator("ctr", new Sum.SumLongFn()); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/NoOpOldDoFn.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/NoOpOldDoFn.java index c73251000650..302b66aa7c4e 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/NoOpOldDoFn.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/NoOpOldDoFn.java @@ -28,7 +28,7 @@ /** * A {@link OldDoFn} that does nothing with provided elements. Used for testing - * methods provided by the OldDoFn abstract class. + * methods provided by the {@link OldDoFn} abstract class. * * @param unused. * @param unused. From b04776d4d5235340148233f5f9e3f26f4c01076b Mon Sep 17 00:00:00 2001 From: Pei He Date: Wed, 10 Aug 2016 15:21:41 -0700 Subject: [PATCH 065/112] [BEAM-545] Promote JobName to PipelineOptions --- .../common/ExampleBigQueryTableOptions.java | 3 +- .../beam/examples/common/ExampleOptions.java | 43 ----------------- ...mplePubsubTopicAndSubscriptionOptions.java | 2 +- .../common/ExamplePubsubTopicOptions.java | 2 +- .../runners/flink/FlinkPipelineOptions.java | 34 ------------- .../options/DataflowPipelineOptions.java | 48 ------------------- .../options/DataflowPipelineOptionsTest.java | 42 +++++++++++++--- .../beam/sdk/options/PipelineOptions.java | 42 ++++++++++++++++ 8 files changed, 80 insertions(+), 136 deletions(-) diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/ExampleBigQueryTableOptions.java b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleBigQueryTableOptions.java index 2eef525faed0..5d815c72cc09 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/common/ExampleBigQueryTableOptions.java +++ b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleBigQueryTableOptions.java @@ -49,8 +49,7 @@ public interface ExampleBigQueryTableOptions extends GcpOptions { static class BigQueryTableFactory implements DefaultValueFactory { @Override public String create(PipelineOptions options) { - return options.as(ExampleOptions.class).getNormalizedUniqueName() - .replace('-', '_'); + return options.getJobName().replace('-', '_'); } } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/ExampleOptions.java b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleOptions.java index 8b7ed073f359..85643e41f352 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/common/ExampleOptions.java +++ b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleOptions.java @@ -17,17 +17,9 @@ */ package org.apache.beam.examples.common; -import com.google.common.base.MoreObjects; -import java.util.concurrent.ThreadLocalRandom; -import org.apache.beam.sdk.options.ApplicationNameOptions; import org.apache.beam.sdk.options.Default; -import org.apache.beam.sdk.options.DefaultValueFactory; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.PipelineOptions; -import org.joda.time.DateTimeUtils; -import org.joda.time.DateTimeZone; -import org.joda.time.format.DateTimeFormat; -import org.joda.time.format.DateTimeFormatter; /** * Options that can be used to configure the Beam examples. @@ -42,39 +34,4 @@ public interface ExampleOptions extends PipelineOptions { @Default.Integer(1) int getInjectorNumWorkers(); void setInjectorNumWorkers(int numWorkers); - - @Description("A normalized unique name that is used to name anything related to the pipeline." - + "It defaults to ApplicationName-UserName-Date-RandomInteger") - @Default.InstanceFactory(NormalizedUniqueNameFactory.class) - String getNormalizedUniqueName(); - void setNormalizedUniqueName(String numWorkers); - - /** - * Returns a normalized unique name constructed from {@link ApplicationNameOptions#getAppName()}, - * the local system user name (if available), the current time, and a random integer. - * - *

              The normalization makes sure that the name matches the pattern of - * [a-z]([-a-z0-9]*[a-z0-9])?. - */ - public static class NormalizedUniqueNameFactory implements DefaultValueFactory { - private static final DateTimeFormatter FORMATTER = - DateTimeFormat.forPattern("MMddHHmmss").withZone(DateTimeZone.UTC); - - @Override - public String create(PipelineOptions options) { - String appName = options.as(ApplicationNameOptions.class).getAppName(); - String normalizedAppName = appName == null || appName.length() == 0 ? "BeamApp" - : appName.toLowerCase() - .replaceAll("[^a-z0-9]", "0") - .replaceAll("^[^a-z]", "a"); - String userName = MoreObjects.firstNonNull(System.getProperty("user.name"), ""); - String normalizedUserName = userName.toLowerCase() - .replaceAll("[^a-z0-9]", "0"); - String datePart = FORMATTER.print(DateTimeUtils.currentTimeMillis()); - - String randomPart = Integer.toHexString(ThreadLocalRandom.current().nextInt()); - return String.format("%s-%s-%s-%s", - normalizedAppName, normalizedUserName, datePart, randomPart); - } - } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicAndSubscriptionOptions.java b/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicAndSubscriptionOptions.java index 36893a3a6990..7f954a1c1850 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicAndSubscriptionOptions.java +++ b/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicAndSubscriptionOptions.java @@ -39,7 +39,7 @@ static class PubsubSubscriptionFactory implements DefaultValueFactory { @Override public String create(PipelineOptions options) { return "projects/" + options.as(GcpOptions.class).getProject() - + "/subscriptions/" + options.as(ExampleOptions.class).getNormalizedUniqueName(); + + "/subscriptions/" + options.getJobName(); } } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicOptions.java b/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicOptions.java index 1c9270b8f065..71879b724276 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicOptions.java +++ b/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicOptions.java @@ -39,7 +39,7 @@ static class PubsubTopicFactory implements DefaultValueFactory { @Override public String create(PipelineOptions options) { return "projects/" + options.as(GcpOptions.class).getProject() - + "/topics/" + options.as(ExampleOptions.class).getNormalizedUniqueName(); + + "/topics/" + options.getJobName(); } } } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java index 6d1a8d040f1a..6561fa5ef996 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java @@ -22,14 +22,9 @@ import java.util.List; import org.apache.beam.sdk.options.ApplicationNameOptions; import org.apache.beam.sdk.options.Default; -import org.apache.beam.sdk.options.DefaultValueFactory; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.StreamingOptions; -import org.joda.time.DateTimeUtils; -import org.joda.time.DateTimeZone; -import org.joda.time.format.DateTimeFormat; -import org.joda.time.format.DateTimeFormatter; /** * Options which can be used to configure a Flink PipelineRunner. @@ -49,15 +44,6 @@ public interface FlinkPipelineOptions extends PipelineOptions, ApplicationNameOp List getFilesToStage(); void setFilesToStage(List value); - /** - * The job name is used to identify jobs running on a Flink cluster. - */ - @Description("Flink job name, to uniquely identify active jobs. " - + "Defaults to using the ApplicationName-UserName-Date.") - @Default.InstanceFactory(JobNameFactory.class) - String getJobName(); - void setJobName(String value); - /** * The url of the Flink JobManager on which to execute pipelines. This can either be * the the address of a cluster JobManager, in the form "host:port" or one of the special @@ -93,24 +79,4 @@ public interface FlinkPipelineOptions extends PipelineOptions, ApplicationNameOp @Default.Long(-1L) Long getExecutionRetryDelay(); void setExecutionRetryDelay(Long delay); - - - class JobNameFactory implements DefaultValueFactory { - private static final DateTimeFormatter FORMATTER = - DateTimeFormat.forPattern("MMddHHmmss").withZone(DateTimeZone.UTC); - - @Override - public String create(PipelineOptions options) { - String appName = options.as(ApplicationNameOptions.class).getAppName(); - String normalizedAppName = appName == null || appName.length() == 0 ? "FlinkRunner" - : appName.toLowerCase() - .replaceAll("[^a-z0-9]", "0") - .replaceAll("^[^a-z]", "a"); - String userName = System.getProperty("user.name", ""); - String normalizedUserName = userName.toLowerCase() - .replaceAll("[^a-z0-9]", "0"); - String datePart = FORMATTER.print(DateTimeUtils.currentTimeMillis()); - return normalizedAppName + "-" + normalizedUserName + "-" + datePart; - } - } } diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java index 8ef43c5efaf6..9f58f93c9ecc 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java @@ -20,7 +20,6 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Strings.isNullOrEmpty; -import com.google.common.base.MoreObjects; import java.io.IOException; import org.apache.beam.runners.dataflow.DataflowRunner; import org.apache.beam.sdk.options.ApplicationNameOptions; @@ -35,10 +34,6 @@ import org.apache.beam.sdk.options.StreamingOptions; import org.apache.beam.sdk.options.Validation; import org.apache.beam.sdk.util.IOChannelUtils; -import org.joda.time.DateTimeUtils; -import org.joda.time.DateTimeZone; -import org.joda.time.format.DateTimeFormat; -import org.joda.time.format.DateTimeFormatter; /** * Options that can be used to configure the {@link DataflowRunner}. @@ -75,21 +70,6 @@ public interface DataflowPipelineOptions String getStagingLocation(); void setStagingLocation(String value); - /** - * The Dataflow job name is used as an idempotence key within the Dataflow service. - * If there is an existing job that is currently active, another active job with the same - * name will not be able to be created. Defaults to using the ApplicationName-UserName-Date. - */ - @Description("The Dataflow job name is used as an idempotence key within the Dataflow service. " - + "For each running job in the same GCP project, jobs with the same name cannot be created " - + "unless the new job is an explicit update of the previous one. Defaults to using " - + "ApplicationName-UserName-Date. The job name must match the regular expression " - + "'[a-z]([-a-z0-9]{0,38}[a-z0-9])?'. The runner will automatically truncate the name of the " - + "job and convert to lower case.") - @Default.InstanceFactory(JobNameFactory.class) - String getJobName(); - void setJobName(String value); - /** * Whether to update the currently running pipeline with the same name as this one. */ @@ -99,34 +79,6 @@ public interface DataflowPipelineOptions boolean isUpdate(); void setUpdate(boolean value); - /** - * Returns a normalized job name constructed from {@link ApplicationNameOptions#getAppName()}, the - * local system user name (if available), and the current time. The normalization makes sure that - * the job name matches the required pattern of [a-z]([-a-z0-9]*[a-z0-9])? and length limit of 40 - * characters. - * - *

              This job name factory is only able to generate one unique name per second per application - * and user combination. - */ - public static class JobNameFactory implements DefaultValueFactory { - private static final DateTimeFormatter FORMATTER = - DateTimeFormat.forPattern("MMddHHmmss").withZone(DateTimeZone.UTC); - - @Override - public String create(PipelineOptions options) { - String appName = options.as(ApplicationNameOptions.class).getAppName(); - String normalizedAppName = appName == null || appName.length() == 0 ? "dataflow" - : appName.toLowerCase() - .replaceAll("[^a-z0-9]", "0") - .replaceAll("^[^a-z]", "a"); - String userName = MoreObjects.firstNonNull(System.getProperty("user.name"), ""); - String normalizedUserName = userName.toLowerCase() - .replaceAll("[^a-z0-9]", "0"); - String datePart = FORMATTER.print(DateTimeUtils.currentTimeMillis()); - return normalizedAppName + "-" + normalizedUserName + "-" + datePart; - } - } - /** * Returns a default staging location under {@link GcpOptions#getGcpTempLocation}. */ diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptionsTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptionsTest.java index 61d39923ae5d..202d04b1e9a8 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptionsTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptionsTest.java @@ -53,7 +53,13 @@ public void testUserNameIsNotSet() { System.getProperties().remove("user.name"); DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); options.setAppName("TestApplication"); - assertEquals("testapplication--1208190706", options.getJobName()); + String[] nameComponents = options.getJobName().split("-"); + assertEquals(4, nameComponents.length); + assertEquals("testapplication", nameComponents[0]); + assertEquals("", nameComponents[1]); + assertEquals("1208190706", nameComponents[2]); + // Verify the last component is a hex integer (unsigned). + Long.parseLong(nameComponents[3], 16); assertTrue(options.getJobName().length() <= 40); } @@ -63,9 +69,13 @@ public void testAppNameAndUserNameAreLong() { System.getProperties().put("user.name", "abcdeabcdeabcdeabcdeabcdeabcde"); DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); options.setAppName("1234567890123456789012345678901234567890"); - assertEquals( - "a234567890123456789012345678901234567890-abcdeabcdeabcdeabcdeabcdeabcde-1208190706", - options.getJobName()); + String[] nameComponents = options.getJobName().split("-"); + assertEquals(4, nameComponents.length); + assertEquals("a234567890123456789012345678901234567890", nameComponents[0]); + assertEquals("abcdeabcdeabcdeabcdeabcdeabcde", nameComponents[1]); + assertEquals("1208190706", nameComponents[2]); + // Verify the last component is a hex integer (unsigned). + Long.parseLong(nameComponents[3], 16); } @Test @@ -74,7 +84,13 @@ public void testAppNameIsLong() { System.getProperties().put("user.name", "abcde"); DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); options.setAppName("1234567890123456789012345678901234567890"); - assertEquals("a234567890123456789012345678901234567890-abcde-1208190706", options.getJobName()); + String[] nameComponents = options.getJobName().split("-"); + assertEquals(4, nameComponents.length); + assertEquals("a234567890123456789012345678901234567890", nameComponents[0]); + assertEquals("abcde", nameComponents[1]); + assertEquals("1208190706", nameComponents[2]); + // Verify the last component is a hex integer (unsigned). + Long.parseLong(nameComponents[3], 16); } @Test @@ -83,7 +99,13 @@ public void testUserNameIsLong() { System.getProperties().put("user.name", "abcdeabcdeabcdeabcdeabcdeabcde"); DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); options.setAppName("1234567890"); - assertEquals("a234567890-abcdeabcdeabcdeabcdeabcdeabcde-1208190706", options.getJobName()); + String[] nameComponents = options.getJobName().split("-"); + assertEquals(4, nameComponents.length); + assertEquals("a234567890", nameComponents[0]); + assertEquals("abcdeabcdeabcdeabcdeabcdeabcde", nameComponents[1]); + assertEquals("1208190706", nameComponents[2]); + // Verify the last component is a hex integer (unsigned). + Long.parseLong(nameComponents[3], 16); } @Test @@ -92,7 +114,13 @@ public void testUtf8UserNameAndApplicationNameIsNormalized() { System.getProperties().put("user.name", "ði ıntəˈnæʃənəl "); DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); options.setAppName("fəˈnɛtık əsoʊsiˈeıʃn"); - assertEquals("f00n0t0k00so0si0e00n-0i00nt00n000n0l0-1208190706", options.getJobName()); + String[] nameComponents = options.getJobName().split("-"); + assertEquals(4, nameComponents.length); + assertEquals("f00n0t0k00so0si0e00n", nameComponents[0]); + assertEquals("0i00nt00n000n0l0", nameComponents[1]); + assertEquals("1208190706", nameComponents[2]); + // Verify the last component is a hex integer (unsigned). + Long.parseLong(nameComponents[3], 16); } @Test diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java index db54d0ac76b6..701ae70eafde 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java @@ -22,8 +22,10 @@ import com.fasterxml.jackson.databind.annotation.JsonDeserialize; import com.fasterxml.jackson.databind.annotation.JsonSerialize; import com.google.auto.service.AutoService; +import com.google.common.base.MoreObjects; import java.lang.reflect.Proxy; import java.util.ServiceLoader; +import java.util.concurrent.ThreadLocalRandom; import javax.annotation.concurrent.ThreadSafe; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.options.GoogleApiDebugOptions.GoogleApiTracer; @@ -33,6 +35,10 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFn.Context; import org.apache.beam.sdk.transforms.display.HasDisplayData; +import org.joda.time.DateTimeUtils; +import org.joda.time.DateTimeZone; +import org.joda.time.format.DateTimeFormat; +import org.joda.time.format.DateTimeFormatter; /** * PipelineOptions are used to configure Pipelines. You can extend {@link PipelineOptions} @@ -258,6 +264,13 @@ public static enum CheckEnabled { String getTempLocation(); void setTempLocation(String value); + @Description("Name of the pipeline execution." + + "It must match the regular expression '[a-z]([-a-z0-9]{0,38}[a-z0-9])?'." + + "It defaults to ApplicationName-UserName-Date-RandomInteger") + @Default.InstanceFactory(JobNameFactory.class) + String getJobName(); + void setJobName(String numWorkers); + /** * A {@link DefaultValueFactory} that obtains the class of the {@code DirectRunner} if it exists * on the classpath, and throws an exception otherwise. @@ -284,4 +297,33 @@ public Class create(PipelineOptions options) { } } } + + /** + * Returns a normalized job name constructed from {@link ApplicationNameOptions#getAppName()}, + * the local system user name (if available), the current time, and a random integer. + * + *

              The normalization makes sure that the name matches the pattern of + * [a-z]([-a-z0-9]*[a-z0-9])?. + */ + static class JobNameFactory implements DefaultValueFactory { + private static final DateTimeFormatter FORMATTER = + DateTimeFormat.forPattern("MMddHHmmss").withZone(DateTimeZone.UTC); + + @Override + public String create(PipelineOptions options) { + String appName = options.as(ApplicationNameOptions.class).getAppName(); + String normalizedAppName = appName == null || appName.length() == 0 ? "BeamApp" + : appName.toLowerCase() + .replaceAll("[^a-z0-9]", "0") + .replaceAll("^[^a-z]", "a"); + String userName = MoreObjects.firstNonNull(System.getProperty("user.name"), ""); + String normalizedUserName = userName.toLowerCase() + .replaceAll("[^a-z0-9]", "0"); + String datePart = FORMATTER.print(DateTimeUtils.currentTimeMillis()); + + String randomPart = Integer.toHexString(ThreadLocalRandom.current().nextInt()); + return String.format("%s-%s-%s-%s", + normalizedAppName, normalizedUserName, datePart, randomPart); + } + } } From 81352b4264999a4a26da8be0bb93bf3a218354d4 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Thu, 25 Aug 2016 14:58:56 -0700 Subject: [PATCH 066/112] Make WriteTest more resilient to Randomness In the worst case scenario for random key assignment in Write.ApplyShardingKey, the chance of the number of records per output shard was too high. This makes the test significantly less likely to flake. --- .../src/test/java/org/apache/beam/sdk/io/WriteTest.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java index 28651884d298..997566ae619b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java @@ -192,11 +192,12 @@ public void testShardedWriteBalanced() { inputs.add(String.format("elt%04d", i)); } + int numShards = 10; runShardedWrite( inputs, new WindowAndReshuffle<>( Window.into(Sessions.withGapDuration(Duration.millis(1)))), - Optional.of(10)); + Optional.of(numShards)); // Check that both the min and max number of results per shard are close to the expected. int min = Integer.MAX_VALUE; @@ -205,7 +206,9 @@ public void testShardedWriteBalanced() { min = Math.min(min, i); max = Math.max(max, i); } - assertThat((double) min, Matchers.greaterThanOrEqualTo(max * 0.9)); + double expected = numElements / (double) numShards; + assertThat((double) min, Matchers.greaterThanOrEqualTo(expected * 0.6)); + assertThat((double) max, Matchers.lessThanOrEqualTo(expected * 1.4)); } /** From 89cf4613465647e2711983674879afd5f67c519d Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Tue, 23 Aug 2016 18:54:23 -0700 Subject: [PATCH 067/112] checkstyle: prohibit API client repackaged Guava Apparently the IllegalImport check only blocks packages, so we had to move to Regexp to get individual classes. As a bonus, this enforcement let us remove two bogus dependencies. Smaller JARs for the win! --- runners/spark/pom.xml | 4 ---- .../streaming/StreamingTransformTranslator.java | 6 +++--- .../build-tools/src/main/resources/beam/checkstyle.xml | 10 +++++++++- .../beam/sdk/io/gcp/bigquery/BigQueryIOTest.java | 2 +- sdks/java/io/hdfs/pom.xml | 5 ----- .../java/org/apache/beam/sdk/io/hdfs/HDFSFileSink.java | 2 +- 6 files changed, 14 insertions(+), 15 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index a6311b5dcf1f..a5e99a06e80b 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -154,10 +154,6 @@ auto-service true - - com.google.http-client - google-http-client - com.fasterxml.jackson.core jackson-core diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java index 43160f550066..5f35ebb20186 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java @@ -17,9 +17,9 @@ */ package org.apache.beam.runners.spark.translation.streaming; -import com.google.api.client.util.Lists; -import com.google.api.client.util.Maps; -import com.google.api.client.util.Sets; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; import com.google.common.reflect.TypeToken; import java.lang.reflect.ParameterizedType; import java.lang.reflect.Type; diff --git a/sdks/java/build-tools/src/main/resources/beam/checkstyle.xml b/sdks/java/build-tools/src/main/resources/beam/checkstyle.xml index 47ddc5b2e37a..c7d9b2c2a813 100644 --- a/sdks/java/build-tools/src/main/resources/beam/checkstyle.xml +++ b/sdks/java/build-tools/src/main/resources/beam/checkstyle.xml @@ -100,7 +100,15 @@ page at http://checkstyle.sourceforge.net/config.html --> - + + + + + + diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java index a6d7e2f5e957..7a7575bfdd24 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java @@ -33,7 +33,6 @@ import static org.mockito.Mockito.when; import com.google.api.client.util.Data; -import com.google.api.client.util.Strings; import com.google.api.services.bigquery.model.ErrorProto; import com.google.api.services.bigquery.model.Job; import com.google.api.services.bigquery.model.JobConfigurationExtract; @@ -50,6 +49,7 @@ import com.google.api.services.bigquery.model.TableReference; import com.google.api.services.bigquery.model.TableRow; import com.google.api.services.bigquery.model.TableSchema; +import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; diff --git a/sdks/java/io/hdfs/pom.xml b/sdks/java/io/hdfs/pom.xml index 07b9eb6b4bfe..0ec542ccf9d8 100644 --- a/sdks/java/io/hdfs/pom.xml +++ b/sdks/java/io/hdfs/pom.xml @@ -81,11 +81,6 @@ annotations - - com.google.http-client - google-http-client - - org.apache.avro avro diff --git a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSink.java b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSink.java index 0b538b355bc1..6d30307f2e07 100644 --- a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSink.java +++ b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSink.java @@ -19,8 +19,8 @@ import static com.google.common.base.Preconditions.checkState; -import com.google.api.client.util.Maps; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import com.google.common.collect.Sets; import java.io.IOException; import java.util.Map; From e233e5f64d8bfeb5b4da7d96515e939c4bfd8b0e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Thu, 25 Aug 2016 14:32:20 +0200 Subject: [PATCH 068/112] [BEAM-294] Rename dataflow references to beam --- runners/spark/pom.xml | 2 +- .../java/org/apache/beam/runners/spark/SparkRunner.java | 2 +- .../org/apache/beam/runners/spark/TestSparkRunner.java | 2 +- .../beam/runners/spark/aggregators/NamedAggregators.java | 2 +- .../org/apache/beam/runners/spark/examples/WordCount.java | 2 +- .../runners/spark/io/hadoop/ShardNameTemplateHelper.java | 6 +++--- .../beam/runners/spark/translation/DoFnFunction.java | 2 +- .../spark/translation/SparkPipelineTranslator.java | 2 +- .../runners/spark/translation/SparkRuntimeContext.java | 2 +- .../runners/spark/translation/TransformTranslator.java | 4 ++-- .../streaming/StreamingTransformTranslator.java | 8 ++++---- .../apache/beam/runners/spark/util/BroadcastHelper.java | 4 ++-- .../spark/translation/TransformTranslatorTest.java | 2 +- .../spark/translation/streaming/KafkaStreamingTest.java | 2 +- 14 files changed, 21 insertions(+), 21 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index a5e99a06e80b..b924cb8e3ea8 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -330,7 +330,7 @@ - com.google.common diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java index 2ce1ff697e42..fa85a2e25e26 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java @@ -50,7 +50,7 @@ /** * The SparkRunner translate operations defined on a pipeline to a representation * executable by Spark, and then submitting the job to Spark to be executed. If we wanted to run - * a dataflow pipeline with the default options of a single threaded spark instance in local mode, + * a Beam pipeline with the default options of a single threaded spark instance in local mode, * we would do the following: * * {@code diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java index 50ed5f3f5ce8..376b80ff953b 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java @@ -28,7 +28,7 @@ /** * The SparkRunner translate operations defined on a pipeline to a representation executable - * by Spark, and then submitting the job to Spark to be executed. If we wanted to run a dataflow + * by Spark, and then submitting the job to Spark to be executed. If we wanted to run a Beam * pipeline with the default options of a single threaded spark instance in local mode, we would do * the following: * diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java index c15e276c43db..e2cd9632e37b 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java @@ -32,7 +32,7 @@ /** * This class wraps a map of named aggregators. Spark expects that all accumulators be declared - * before a job is launched. Dataflow allows aggregators to be used and incremented on the fly. + * before a job is launched. Beam allows aggregators to be used and incremented on the fly. * We create a map of named aggregators and instantiate in the the spark context before the job * is launched. We can then add aggregators on the fly in Spark. */ diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/examples/WordCount.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/examples/WordCount.java index 06770305f2c1..1af84add67ec 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/examples/WordCount.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/examples/WordCount.java @@ -110,7 +110,7 @@ public PCollection> apply(PCollection lines) { */ public static interface WordCountOptions extends PipelineOptions { @Description("Path of the file to read from") - @Default.String("gs://dataflow-samples/shakespeare/kinglear.txt") + @Default.String("gs://beam-samples/shakespeare/kinglear.txt") String getInputFile(); void setInputFile(String value); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateHelper.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateHelper.java index 7f8e2978c439..4a7058bfd521 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateHelper.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateHelper.java @@ -36,9 +36,9 @@ public final class ShardNameTemplateHelper { private static final Logger LOG = LoggerFactory.getLogger(ShardNameTemplateHelper.class); - public static final String OUTPUT_FILE_PREFIX = "spark.dataflow.fileoutputformat.prefix"; - public static final String OUTPUT_FILE_TEMPLATE = "spark.dataflow.fileoutputformat.template"; - public static final String OUTPUT_FILE_SUFFIX = "spark.dataflow.fileoutputformat.suffix"; + public static final String OUTPUT_FILE_PREFIX = "spark.beam.fileoutputformat.prefix"; + public static final String OUTPUT_FILE_TEMPLATE = "spark.beam.fileoutputformat.template"; + public static final String OUTPUT_FILE_SUFFIX = "spark.beam.fileoutputformat.suffix"; private ShardNameTemplateHelper() { } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java index 800d614e5e96..454b7607e9df 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java @@ -31,7 +31,7 @@ import org.slf4j.LoggerFactory; /** - * Dataflow's Do functions correspond to Spark's FlatMap functions. + * Beam's Do functions correspond to Spark's FlatMap functions. * * @param Input element type. * @param Output element type. diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineTranslator.java index 997940bb99de..1f7ccf1e005b 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineTranslator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineTranslator.java @@ -20,7 +20,7 @@ import org.apache.beam.sdk.transforms.PTransform; /** - * Translator to support translation between Dataflow transformations and Spark transformations. + * Translator to support translation between Beam transformations and Spark transformations. */ public interface SparkPipelineTranslator { diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java index 4bc0c00973f8..2634c65b2448 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java @@ -57,7 +57,7 @@ public class SparkRuntimeContext implements Serializable { private final String serializedPipelineOptions; /** - * Map fo names to dataflow aggregators. + * Map fo names to Beam aggregators. */ private final Map> aggregators = new HashMap<>(); private transient CoderRegistry coderRegistry; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java index 08e3fda7a36b..eaceb852ce34 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java @@ -94,7 +94,7 @@ import scala.Tuple2; /** - * Supports translation between a DataFlow transform, and Spark's operations on RDDs. + * Supports translation between a Beam transform, and Spark's operations on RDDs. */ public final class TransformTranslator { @@ -895,7 +895,7 @@ private static Map, BroadcastHelper> getSideInputs( } /** - * Translator matches Dataflow transformation with the appropriate evaluator. + * Translator matches Beam transformation with the appropriate evaluator. */ public static class Translator implements SparkPipelineTranslator { diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java index 5f35ebb20186..43dcef665fb2 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java @@ -68,7 +68,7 @@ /** - * Supports translation between a DataFlow transform, and Spark's operations on DStreams. + * Supports translation between a Beam transform, and Spark's operations on DStreams. */ public final class StreamingTransformTranslator { @@ -349,13 +349,13 @@ public void evaluate(Window.Bound transform, EvaluationContext context) { (TransformEvaluator) EVALUATORS.get(clazz); if (transform == null) { if (UNSUPPORTED_EVALUATORS.contains(clazz)) { - throw new UnsupportedOperationException("Dataflow transformation " + clazz + throw new UnsupportedOperationException("Beam transformation " + clazz .getCanonicalName() + " is currently unsupported by the Spark streaming pipeline"); } // DStream transformations will transform an RDD into another RDD // Actions will create output - // In Dataflow it depends on the PTransform's Input and Output class + // In Beam it depends on the PTransform's Input and Output class Class pTOutputClazz = getPTransformOutputClazz(clazz); if (PDone.class.equals(pTOutputClazz)) { return foreachRDD(rddTranslator); @@ -373,7 +373,7 @@ public void evaluate(Window.Bound transform, EvaluationContext context) { } /** - * Translator matches Dataflow transformation with the appropriate Spark streaming evaluator. + * Translator matches Beam transformation with the appropriate Spark streaming evaluator. * rddTranslator uses Spark evaluators in transform/foreachRDD to evaluate the transformation */ public static class Translator implements SparkPipelineTranslator { diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/BroadcastHelper.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/BroadcastHelper.java index 29c2dd995b6e..5f0c79576e59 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/BroadcastHelper.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/BroadcastHelper.java @@ -34,12 +34,12 @@ public abstract class BroadcastHelper implements Serializable { /** - * If the property {@code dataflow.spark.directBroadcast} is set to + * If the property {@code beam.spark.directBroadcast} is set to * {@code true} then Spark serialization (Kryo) will be used to broadcast values * in View objects. By default this property is not set, and values are coded using * the appropriate {@link Coder}. */ - public static final String DIRECT_BROADCAST = "dataflow.spark.directBroadcast"; + public static final String DIRECT_BROADCAST = "beam.spark.directBroadcast"; private static final Logger LOG = LoggerFactory.getLogger(BroadcastHelper.class); diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java index f61ad1c9a9fd..f72eba7ea6ca 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java @@ -56,7 +56,7 @@ public class TransformTranslatorTest { /** * Builds a simple pipeline with TextIO.Read and TextIO.Write, runs the pipeline - * in DirectRunner and on SparkRunner, with the mapped dataflow-to-spark + * in DirectRunner and on SparkRunner, with the mapped beam-to-spark * transforms. Finally it makes sure that the results are the same for both runs. */ @Test diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/KafkaStreamingTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/KafkaStreamingTest.java index 27d6f5ea6068..ac77922bc57c 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/KafkaStreamingTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/KafkaStreamingTest.java @@ -56,7 +56,7 @@ public class KafkaStreamingTest { new EmbeddedKafkaCluster.EmbeddedZookeeper(); private static final EmbeddedKafkaCluster EMBEDDED_KAFKA_CLUSTER = new EmbeddedKafkaCluster(EMBEDDED_ZOOKEEPER.getConnection(), new Properties()); - private static final String TOPIC = "kafka_dataflow_test_topic"; + private static final String TOPIC = "kafka_beam_test_topic"; private static final Map KAFKA_MESSAGES = ImmutableMap.of( "k1", "v1", "k2", "v2", "k3", "v3", "k4", "v4" ); From e871f1731d849fa7eda0a508b57e8b94514eb236 Mon Sep 17 00:00:00 2001 From: Ian Zhou Date: Mon, 15 Aug 2016 15:39:56 -0700 Subject: [PATCH 069/112] Modified BigtableIO to use DoFn setup/tearDown methods instead of startBundle/finishBundle --- .../beam/sdk/io/gcp/bigtable/BigtableIO.java | 17 +++++++++++++---- .../sdk/io/gcp/bigtable/BigtableService.java | 9 ++++++++- .../io/gcp/bigtable/BigtableServiceImpl.java | 8 ++++++++ .../sdk/io/gcp/bigtable/BigtableIOTest.java | 3 +++ 4 files changed, 32 insertions(+), 5 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java index 3a9ffce487f6..67dde507313d 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java @@ -515,9 +515,13 @@ public BigtableWriterFn(String tableId, BigtableService bigtableService) { this.failures = new ConcurrentLinkedQueue<>(); } - @StartBundle - public void startBundle(Context c) throws Exception { + @Setup + public void setup() throws Exception { bigtableWriter = bigtableService.openForWriting(tableId); + } + + @StartBundle + public void startBundle(Context c) { recordsWritten = 0; } @@ -531,12 +535,17 @@ public void processElement(ProcessContext c) throws Exception { @FinishBundle public void finishBundle(Context c) throws Exception { - bigtableWriter.close(); - bigtableWriter = null; + bigtableWriter.flush(); checkForFailures(); logger.info("Wrote {} records", recordsWritten); } + @Teardown + public void tearDown() throws Exception { + bigtableWriter.close(); + bigtableWriter = null; + } + @Override public void populateDisplayData(DisplayData.Builder builder) { Write.this.populateDisplayData(builder); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java index ecd38a7468e4..c656bbbf3fd9 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java @@ -50,10 +50,17 @@ ListenableFuture writeRecord(KV writeRecord( return Futures.immediateFuture(MutateRowResponse.getDefaultInstance()); } + @Override + public void flush() {} + @Override public void close() {} } From 226dea2f04de2c000733f1182bdd3d18d516d4e4 Mon Sep 17 00:00:00 2001 From: staslev Date: Fri, 26 Aug 2016 10:26:38 +0300 Subject: [PATCH 070/112] Added support for reporting aggregator values to Spark sinks --- runners/spark/pom.xml | 6 + .../runners/spark/SparkPipelineOptions.java | 7 +- .../spark/aggregators/NamedAggregators.java | 38 +++- .../aggregators/metrics/AggregatorMetric.java | 44 +++++ .../metrics/AggregatorMetricSource.java | 49 +++++ .../metrics/WithNamedAggregatorsSupport.java | 169 ++++++++++++++++++ .../aggregators/metrics/package-info.java | 22 +++ .../aggregators/metrics/sink/CsvSink.java | 39 ++++ .../metrics/sink/GraphiteSink.java | 39 ++++ .../metrics/sink/package-info.java | 23 +++ .../beam/runners/spark/io/ConsoleIO.java | 2 +- .../runners/spark/io/hadoop/HadoopIO.java | 4 +- .../translation/SparkRuntimeContext.java | 29 ++- .../runners/spark/util/BroadcastHelper.java | 4 +- .../spark/InMemoryMetricsSinkRule.java | 32 ++++ .../runners/spark/SimpleWordCountTest.java | 12 ++ .../metrics/sink/InMemoryMetrics.java | 79 ++++++++ .../src/test/resources/metrics.properties | 29 +++ 18 files changed, 611 insertions(+), 16 deletions(-) create mode 100644 runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/AggregatorMetric.java create mode 100644 runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/AggregatorMetricSource.java create mode 100644 runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/WithNamedAggregatorsSupport.java create mode 100644 runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/package-info.java create mode 100644 runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/sink/CsvSink.java create mode 100644 runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/sink/GraphiteSink.java create mode 100644 runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/sink/package-info.java create mode 100644 runners/spark/src/test/java/org/apache/beam/runners/spark/InMemoryMetricsSinkRule.java create mode 100644 runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/InMemoryMetrics.java create mode 100644 runners/spark/src/test/resources/metrics.properties diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index b924cb8e3ea8..b928b4452967 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -37,6 +37,7 @@ 1.6.2 2.2.0 0.8.2.1 + 3.1.2 @@ -231,6 +232,11 @@ + + io.dropwizard.metrics + metrics-core + ${dropwizard.metrics.version} + diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java index 080ff19a0c25..be4f7f06a9bc 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java @@ -38,10 +38,15 @@ public interface SparkPipelineOptions extends PipelineOptions, StreamingOptions, + "execution is stopped") @Default.Long(-1) Long getTimeout(); - void setTimeout(Long batchInterval); + void setTimeout(Long timeoutMillis); @Description("Batch interval for Spark streaming in milliseconds.") @Default.Long(1000) Long getBatchIntervalMillis(); void setBatchIntervalMillis(Long batchInterval); + + @Description("Enable/disable sending aggregator values to Spark's metric sinks") + @Default.Boolean(true) + Boolean getEnableSparkSinks(); + void setEnableSparkSinks(Boolean enableSparkSinks); } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java index e2cd9632e37b..4e96466706dc 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java @@ -18,13 +18,18 @@ package org.apache.beam.runners.spark.aggregators; +import com.google.common.base.Function; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; + import java.io.IOException; import java.io.ObjectInputStream; import java.io.ObjectOutputStream; import java.io.Serializable; import java.util.Map; import java.util.TreeMap; + import org.apache.beam.runners.spark.translation.SparkRuntimeContext; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; @@ -69,6 +74,22 @@ public T getValue(String name, Class typeClass) { return typeClass.cast(mNamedAggregators.get(name).render()); } + /** + * @return a map of all the aggregator names and their rendered values + */ + public Map renderAll() { + return + ImmutableMap.copyOf( + Maps.transformValues(mNamedAggregators, + new Function, Object>() { + + @Override + public Object apply(State state) { + return state.render(); + } + })); + } + /** * Merges another NamedAggregators instance with this instance. * @@ -116,6 +137,7 @@ public String toString() { * @param Output data type */ public interface State extends Serializable { + /** * @param element new element to update state */ @@ -133,16 +155,16 @@ public interface State extends Serializable { /** * => combineFunction in data flow. */ - public static class CombineFunctionState - implements State { + public static class CombineFunctionState + implements State { - private Combine.CombineFn combineFn; + private Combine.CombineFn combineFn; private Coder inCoder; private SparkRuntimeContext ctxt; private transient InterT state; public CombineFunctionState( - Combine.CombineFn combineFn, + Combine.CombineFn combineFn, Coder inCoder, SparkRuntimeContext ctxt) { this.combineFn = combineFn; @@ -157,7 +179,7 @@ public void update(InputT element) { } @Override - public State merge(State other) { + public State merge(State other) { this.state = combineFn.mergeAccumulators(ImmutableList.of(current(), other.current())); return this; } @@ -168,12 +190,12 @@ public InterT current() { } @Override - public OutpuT render() { + public OutputT render() { return combineFn.extractOutput(state); } @Override - public Combine.CombineFn getCombineFn() { + public Combine.CombineFn getCombineFn() { return combineFn; } @@ -192,7 +214,7 @@ private void writeObject(ObjectOutputStream oos) throws IOException { @SuppressWarnings("unchecked") private void readObject(ObjectInputStream ois) throws IOException, ClassNotFoundException { ctxt = (SparkRuntimeContext) ois.readObject(); - combineFn = (Combine.CombineFn) ois.readObject(); + combineFn = (Combine.CombineFn) ois.readObject(); inCoder = (Coder) ois.readObject(); try { state = combineFn.getAccumulatorCoder(ctxt.getCoderRegistry(), inCoder) diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/AggregatorMetric.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/AggregatorMetric.java new file mode 100644 index 000000000000..c07a0697e8f1 --- /dev/null +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/AggregatorMetric.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.runners.spark.aggregators.metrics; + +import com.codahale.metrics.Metric; + +import org.apache.beam.runners.spark.aggregators.NamedAggregators; + +/** + * An adapter between the {@link NamedAggregators} and codahale's {@link Metric} + * interface. + */ +public class AggregatorMetric implements Metric { + + private final NamedAggregators namedAggregators; + + private AggregatorMetric(final NamedAggregators namedAggregators) { + this.namedAggregators = namedAggregators; + } + + public static AggregatorMetric of(final NamedAggregators namedAggregators) { + return new AggregatorMetric(namedAggregators); + } + + NamedAggregators getNamedAggregators() { + return namedAggregators; + } +} diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/AggregatorMetricSource.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/AggregatorMetricSource.java new file mode 100644 index 000000000000..0658e049a999 --- /dev/null +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/AggregatorMetricSource.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.runners.spark.aggregators.metrics; + +import com.codahale.metrics.MetricRegistry; + +import org.apache.beam.runners.spark.aggregators.NamedAggregators; +import org.apache.spark.metrics.source.Source; + +/** + * A Spark {@link Source} that is tailored to expose an {@link AggregatorMetric}, + * wrapping an underlying {@link NamedAggregators} instance. + */ +public class AggregatorMetricSource implements Source { + + private static final String SOURCE_NAME = "NamedAggregators"; + + private final MetricRegistry metricRegistry = new MetricRegistry(); + + public AggregatorMetricSource(final NamedAggregators aggregators) { + metricRegistry.register(SOURCE_NAME, AggregatorMetric.of(aggregators)); + } + + @Override + public String sourceName() { + return SOURCE_NAME; + } + + @Override + public MetricRegistry metricRegistry() { + return metricRegistry; + } +} diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/WithNamedAggregatorsSupport.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/WithNamedAggregatorsSupport.java new file mode 100644 index 000000000000..88e2211cf9e1 --- /dev/null +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/WithNamedAggregatorsSupport.java @@ -0,0 +1,169 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.runners.spark.aggregators.metrics; + +import com.codahale.metrics.Counter; +import com.codahale.metrics.Gauge; +import com.codahale.metrics.Histogram; +import com.codahale.metrics.Meter; +import com.codahale.metrics.Metric; +import com.codahale.metrics.MetricFilter; +import com.codahale.metrics.MetricRegistry; +import com.codahale.metrics.Timer; + +import com.google.common.base.Function; +import com.google.common.base.Optional; +import com.google.common.base.Predicate; +import com.google.common.base.Predicates; +import com.google.common.collect.FluentIterable; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSortedMap; +import com.google.common.collect.Maps; +import com.google.common.collect.Ordering; + +import java.util.Map; +import java.util.SortedMap; + +import org.apache.beam.runners.spark.aggregators.NamedAggregators; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A {@link MetricRegistry} decorator-like* that supports {@link AggregatorMetric} by exposing + * the underlying * {@link org.apache.beam.runners.spark.aggregators.NamedAggregators}' + * aggregators as {@link Gauge}s. + *

              + * *{@link MetricRegistry} is not an interface, so this is not a by-the-book decorator. + * That said, it delegates all metric related getters to the "decorated" instance. + *

              + */ +public class WithNamedAggregatorsSupport extends MetricRegistry { + + private static final Logger LOG = LoggerFactory.getLogger(WithNamedAggregatorsSupport.class); + + private MetricRegistry internalMetricRegistry; + + private WithNamedAggregatorsSupport(final MetricRegistry internalMetricRegistry) { + this.internalMetricRegistry = internalMetricRegistry; + } + + public static WithNamedAggregatorsSupport forRegistry(final MetricRegistry metricRegistry) { + return new WithNamedAggregatorsSupport(metricRegistry); + } + + @Override + public SortedMap getTimers(final MetricFilter filter) { + return internalMetricRegistry.getTimers(filter); + } + + @Override + public SortedMap getMeters(final MetricFilter filter) { + return internalMetricRegistry.getMeters(filter); + } + + @Override + public SortedMap getHistograms(final MetricFilter filter) { + return internalMetricRegistry.getHistograms(filter); + } + + @Override + public SortedMap getCounters(final MetricFilter filter) { + return internalMetricRegistry.getCounters(filter); + } + + @Override + public SortedMap getGauges(final MetricFilter filter) { + return + new ImmutableSortedMap.Builder( + Ordering.from(String.CASE_INSENSITIVE_ORDER)) + .putAll(internalMetricRegistry.getGauges(filter)) + .putAll(extractGauges(internalMetricRegistry, filter)) + .build(); + } + + private Map extractGauges(final MetricRegistry metricRegistry, + final MetricFilter filter) { + + // find the AggregatorMetric metrics from within all currently registered metrics + final Optional> gauges = + FluentIterable + .from(metricRegistry.getMetrics().entrySet()) + .firstMatch(isAggregatorMetric()) + .transform(toGauges()); + + return + gauges.isPresent() + ? Maps.filterEntries(gauges.get(), matches(filter)) + : ImmutableMap.of(); + } + + private Function, Map> toGauges() { + return new Function, Map>() { + @Override + public Map apply(final Map.Entry entry) { + final NamedAggregators agg = ((AggregatorMetric) entry.getValue()).getNamedAggregators(); + final Map gaugeMap = Maps.transformEntries(agg.renderAll(), toGauge()); + return Maps.filterValues(gaugeMap, Predicates.notNull()); + } + }; + } + + private Maps.EntryTransformer toGauge() { + return new Maps.EntryTransformer() { + + @Override + public Gauge transformEntry(final String name, final Object rawValue) { + return new Gauge() { + + @Override + public Double getValue() { + // at the moment the metric's type is assumed to be + // compatible with Double. While far from perfect, it seems reasonable at + // this point in time + try { + return Double.parseDouble(rawValue.toString()); + } catch (final Exception e) { + LOG.warn("Failed reporting metric with name [{}], of type [{}], since it could not be" + + " converted to double", name, rawValue.getClass().getSimpleName(), e); + return null; + } + } + }; + } + }; + } + + private Predicate> matches(final MetricFilter filter) { + return new Predicate>() { + @Override + public boolean apply(final Map.Entry entry) { + return filter.matches(entry.getKey(), entry.getValue()); + } + }; + } + + private Predicate> isAggregatorMetric() { + return new Predicate>() { + @Override + public boolean apply(final Map.Entry metricEntry) { + return (metricEntry.getValue() instanceof AggregatorMetric); + } + }; + } +} diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/package-info.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/package-info.java new file mode 100644 index 000000000000..f19f63536001 --- /dev/null +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/package-info.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Defines classes for integrating with Spark's metrics mechanism (Sinks, Sources, etc.). + */ +package org.apache.beam.runners.spark.aggregators.metrics; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/sink/CsvSink.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/sink/CsvSink.java new file mode 100644 index 000000000000..af1601aea8be --- /dev/null +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/sink/CsvSink.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.runners.spark.aggregators.metrics.sink; + +import com.codahale.metrics.MetricRegistry; + +import java.util.Properties; + +import org.apache.beam.runners.spark.aggregators.metrics.AggregatorMetric; +import org.apache.beam.runners.spark.aggregators.metrics.WithNamedAggregatorsSupport; +import org.apache.spark.metrics.sink.Sink; + +/** + * A Spark {@link Sink} that is tailored to report {@link AggregatorMetric} metrics + * to a CSV file. + */ +public class CsvSink extends org.apache.spark.metrics.sink.CsvSink { + public CsvSink(final Properties properties, + final MetricRegistry metricRegistry, + final org.apache.spark.SecurityManager securityMgr) { + super(properties, WithNamedAggregatorsSupport.forRegistry(metricRegistry), securityMgr); + } +} diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/sink/GraphiteSink.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/sink/GraphiteSink.java new file mode 100644 index 000000000000..7a45ef7f4882 --- /dev/null +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/sink/GraphiteSink.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.runners.spark.aggregators.metrics.sink; + +import com.codahale.metrics.MetricRegistry; + +import java.util.Properties; + +import org.apache.beam.runners.spark.aggregators.metrics.AggregatorMetric; +import org.apache.beam.runners.spark.aggregators.metrics.WithNamedAggregatorsSupport; +import org.apache.spark.metrics.sink.Sink; + +/** + * A Spark {@link Sink} that is tailored to report {@link AggregatorMetric} metrics + * to Graphite. + */ +public class GraphiteSink extends org.apache.spark.metrics.sink.GraphiteSink { + public GraphiteSink(final Properties properties, + final MetricRegistry metricRegistry, + final org.apache.spark.SecurityManager securityMgr) { + super(properties, WithNamedAggregatorsSupport.forRegistry(metricRegistry), securityMgr); + } +} diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/sink/package-info.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/sink/package-info.java new file mode 100644 index 000000000000..2e6dd0d3a9db --- /dev/null +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/sink/package-info.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Spark sinks that support + * the {@link org.apache.beam.runners.spark.aggregators.metrics.AggregatorMetric}. + */ +package org.apache.beam.runners.spark.aggregators.metrics.sink; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/ConsoleIO.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/ConsoleIO.java index eefea77de5d2..b1c567c757d7 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/ConsoleIO.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/ConsoleIO.java @@ -47,7 +47,7 @@ public static Unbound from(int num) { /** * {@link PTransform} writing {@link PCollection} on the console. - * @param + * @param the type of the elements in the {@link PCollection} */ public static class Unbound extends PTransform, PDone> { diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/HadoopIO.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/HadoopIO.java index 7b106105ee19..70bec78e51f4 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/HadoopIO.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/HadoopIO.java @@ -55,8 +55,8 @@ public static Bound from(String filepattern, /** * A {@link PTransform} reading bounded collection of data from HDFS. - * @param - * @param + * @param the type of the keys + * @param the type of the values */ public static class Bound extends PTransform>> { diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java index 2634c65b2448..4e4cd1a170ce 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java @@ -20,14 +20,19 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; + import com.google.common.collect.ImmutableList; + import java.io.IOException; import java.io.Serializable; import java.util.Collection; import java.util.HashMap; import java.util.Map; + +import org.apache.beam.runners.spark.SparkPipelineOptions; import org.apache.beam.runners.spark.aggregators.AggAccumParam; import org.apache.beam.runners.spark.aggregators.NamedAggregators; +import org.apache.beam.runners.spark.aggregators.metrics.AggregatorMetricSource; import org.apache.beam.sdk.AggregatorValues; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.CannotProvideCoderException; @@ -41,7 +46,9 @@ import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.spark.Accumulator; +import org.apache.spark.SparkEnv$; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.metrics.MetricsSystem; /** @@ -63,8 +70,9 @@ public class SparkRuntimeContext implements Serializable { private transient CoderRegistry coderRegistry; SparkRuntimeContext(JavaSparkContext jsc, Pipeline pipeline) { - this.accum = jsc.accumulator(new NamedAggregators(), new AggAccumParam()); - this.serializedPipelineOptions = serializePipelineOptions(pipeline.getOptions()); + final SparkPipelineOptions opts = pipeline.getOptions().as(SparkPipelineOptions.class); + accum = registerMetrics(jsc, opts); + serializedPipelineOptions = serializePipelineOptions(opts); } private static String serializePipelineOptions(PipelineOptions pipelineOptions) { @@ -83,6 +91,23 @@ private static PipelineOptions deserializePipelineOptions(String serializedPipel } } + private Accumulator registerMetrics(final JavaSparkContext jsc, + final SparkPipelineOptions opts) { + final NamedAggregators initialValue = new NamedAggregators(); + final Accumulator accum = jsc.accumulator(initialValue, new AggAccumParam()); + + if (opts.getEnableSparkSinks()) { + final MetricsSystem metricsSystem = SparkEnv$.MODULE$.get().metricsSystem(); + final AggregatorMetricSource aggregatorMetricSource = + new AggregatorMetricSource(initialValue); + // in case the context was not cleared + metricsSystem.removeSource(aggregatorMetricSource); + metricsSystem.registerSource(aggregatorMetricSource); + } + + return accum; + } + /** * Retrieves corresponding value of an aggregator. * diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/BroadcastHelper.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/BroadcastHelper.java index 5f0c79576e59..5c13b8005894 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/BroadcastHelper.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/BroadcastHelper.java @@ -58,7 +58,7 @@ public static BroadcastHelper create(T value, Coder coder) { * A {@link BroadcastHelper} that relies on the underlying * Spark serialization (Kryo) to broadcast values. This is appropriate when * broadcasting very large values, since no copy of the object is made. - * @param + * @param the type of the value stored in the broadcast variable */ static class DirectBroadcastHelper extends BroadcastHelper { private Broadcast bcast; @@ -86,7 +86,7 @@ public void broadcast(JavaSparkContext jsc) { * A {@link BroadcastHelper} that uses a * {@link Coder} to encode values as byte arrays * before broadcasting. - * @param + * @param the type of the value stored in the broadcast variable */ static class CodedBroadcastHelper extends BroadcastHelper { private Broadcast bcast; diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/InMemoryMetricsSinkRule.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/InMemoryMetricsSinkRule.java new file mode 100644 index 000000000000..506dbbdeed7d --- /dev/null +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/InMemoryMetricsSinkRule.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.runners.spark; + +import org.apache.beam.runners.spark.aggregators.metrics.sink.InMemoryMetrics; +import org.junit.rules.ExternalResource; + +/** + * A rule that cleans the {@link InMemoryMetrics} after the tests has finished. + */ +class InMemoryMetricsSinkRule extends ExternalResource { + @Override + protected void before() throws Throwable { + InMemoryMetrics.clearAll(); + } +} diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/SimpleWordCountTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/SimpleWordCountTest.java index f644765682f1..8b7762fd7336 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/SimpleWordCountTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/SimpleWordCountTest.java @@ -19,6 +19,8 @@ package org.apache.beam.runners.spark; import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.nullValue; import static org.junit.Assert.assertThat; import com.google.common.collect.ImmutableSet; @@ -27,6 +29,7 @@ import java.util.Arrays; import java.util.List; import java.util.Set; +import org.apache.beam.runners.spark.aggregators.metrics.sink.InMemoryMetrics; import org.apache.beam.runners.spark.examples.WordCount; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.StringUtf8Coder; @@ -39,12 +42,17 @@ import org.apache.commons.io.FileUtils; import org.junit.Rule; import org.junit.Test; +import org.junit.rules.ExternalResource; import org.junit.rules.TemporaryFolder; /** * Simple word count test. */ public class SimpleWordCountTest { + + @Rule + public ExternalResource inMemoryMetricsSink = new InMemoryMetricsSinkRule(); + private static final String[] WORDS_ARRAY = { "hi there", "hi", "hi sue bob", "hi sue", "", "bob hi"}; @@ -54,6 +62,8 @@ public class SimpleWordCountTest { @Test public void testInMem() throws Exception { + assertThat(InMemoryMetrics.valueOf("emptyLines"), is(nullValue())); + SparkPipelineOptions options = PipelineOptionsFactory.as(SparkPipelineOptions.class); options.setRunner(SparkRunner.class); Pipeline p = Pipeline.create(options); @@ -66,6 +76,8 @@ public void testInMem() throws Exception { EvaluationResult res = (EvaluationResult) p.run(); res.close(); + + assertThat(InMemoryMetrics.valueOf("emptyLines"), is(1d)); } @Rule diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/InMemoryMetrics.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/InMemoryMetrics.java new file mode 100644 index 000000000000..35e67173cba8 --- /dev/null +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/InMemoryMetrics.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.runners.spark.aggregators.metrics.sink; + +import com.codahale.metrics.MetricFilter; +import com.codahale.metrics.MetricRegistry; + +import java.util.Properties; + +import org.apache.beam.runners.spark.aggregators.metrics.WithNamedAggregatorsSupport; +import org.apache.spark.metrics.sink.Sink; + +/** + * An in-memory {@link Sink} implementation for tests. + */ +public class InMemoryMetrics implements Sink { + + private static WithNamedAggregatorsSupport extendedMetricsRegistry; + private static MetricRegistry internalMetricRegistry; + + public InMemoryMetrics(final Properties properties, + final MetricRegistry metricRegistry, + final org.apache.spark.SecurityManager securityMgr) { + extendedMetricsRegistry = WithNamedAggregatorsSupport.forRegistry(metricRegistry); + internalMetricRegistry = metricRegistry; + } + + @SuppressWarnings("unchecked") + public static T valueOf(final String name) { + final T retVal; + + if (extendedMetricsRegistry != null + && extendedMetricsRegistry.getGauges().containsKey(name)) { + retVal = (T) extendedMetricsRegistry.getGauges().get(name).getValue(); + } else { + retVal = null; + } + + return retVal; + } + + public static void clearAll() { + if (internalMetricRegistry != null) { + internalMetricRegistry.removeMatching(MetricFilter.ALL); + } + } + + @Override + public void start() { + + } + + @Override + public void stop() { + + } + + @Override + public void report() { + + } + +} diff --git a/runners/spark/src/test/resources/metrics.properties b/runners/spark/src/test/resources/metrics.properties new file mode 100644 index 000000000000..4aa01d2dfd4f --- /dev/null +++ b/runners/spark/src/test/resources/metrics.properties @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +*.sink.memory.class=org.apache.beam.runners.spark.aggregators.metrics.sink.InMemoryMetrics + +#*.sink.csv.class=org.apache.beam.runners.spark.aggregators.metrics.sink.CsvSink +#*.sink.csv.directory=/tmp/spark-metrics +#*.sink.csv.period=1 +#*.sink.graphite.unit=SECONDS + +#*.sink.graphite.class=org.apache.beam.runners.spark.aggregators.metrics.sink.GraphiteSink +#*.sink.graphite.host=YOUR_HOST +#*.sink.graphite.port=2003 +#*.sink.graphite.prefix=spark +#*.sink.graphite.period=1 +#*.sink.graphite.unit=SECONDS From ba16e946aa716836c2bc98470bead4dcf2fdf549 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Tue, 23 Aug 2016 20:21:30 -0700 Subject: [PATCH 071/112] travis.yml: disable updating snapshots Will still update releases with the --update-snapshots version (I know, bad name) --- .travis.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.travis.yml b/.travis.yml index 656aba0fdf4b..233128266f1c 100644 --- a/.travis.yml +++ b/.travis.yml @@ -52,7 +52,7 @@ install: - rm -rf "$HOME/.m2/repository/org/apache/beam" script: - - travis_retry mvn --settings .travis/settings.xml --batch-mode --update-snapshots $MAVEN_OVERRIDE verify + - travis_retry mvn --settings .travis/settings.xml --batch-mode --update-snapshots --no-snapshot-updates $MAVEN_OVERRIDE verify - travis_retry .travis/test_wordcount.sh cache: From d9d0f8d28f7ef31f91998f60ce4e8bfab1cee913 Mon Sep 17 00:00:00 2001 From: Vikas Kedigehalli Date: Tue, 23 Aug 2016 16:44:08 -0700 Subject: [PATCH 072/112] Query latest timestamp --- .../sdk/io/gcp/datastore/DatastoreV1.java | 41 +++++++++++-- .../sdk/io/gcp/datastore/DatastoreV1Test.java | 60 ++++++++++++++++--- 2 files changed, 88 insertions(+), 13 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java index c7433d37d160..8456e0287def 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java @@ -239,6 +239,7 @@ static int getEstimatedNumSplits(Datastore datastore, Query query, @Nullable Str int numSplits; try { long estimatedSizeBytes = getEstimatedSizeBytes(datastore, query, namespace); + LOG.info("Estimated size bytes for the query is: {}", estimatedSizeBytes); numSplits = (int) Math.min(NUM_QUERY_SPLITS_MAX, Math.round(((double) estimatedSizeBytes) / DEFAULT_BUNDLE_SIZE_BYTES)); } catch (Exception e) { @@ -249,6 +250,33 @@ static int getEstimatedNumSplits(Datastore datastore, Query query, @Nullable Str return Math.max(numSplits, NUM_QUERY_SPLITS_MIN); } + /** + * Datastore system tables with statistics are periodically updated. This method fetches + * the latest timestamp (in microseconds) of statistics update using the {@code __Stat_Total__} + * table. + */ + private static long queryLatestStatisticsTimestamp(Datastore datastore, + @Nullable String namespace) throws DatastoreException { + Query.Builder query = Query.newBuilder(); + if (namespace == null) { + query.addKindBuilder().setName("__Stat_Total__"); + } else { + query.addKindBuilder().setName("__Stat_Ns_Total__"); + } + query.addOrder(makeOrder("timestamp", DESCENDING)); + query.setLimit(Int32Value.newBuilder().setValue(1)); + RunQueryRequest request = makeRequest(query.build(), namespace); + + RunQueryResponse response = datastore.runQuery(request); + QueryResultBatch batch = response.getBatch(); + if (batch.getEntityResultsCount() == 0) { + throw new NoSuchElementException( + "Datastore total statistics unavailable"); + } + Entity entity = batch.getEntityResults(0).getEntity(); + return entity.getProperties().get("timestamp").getTimestampValue().getSeconds() * 1000000; + } + /** * Get the estimated size of the data returned by the given query. * @@ -261,17 +289,17 @@ static int getEstimatedNumSplits(Datastore datastore, Query query, @Nullable Str static long getEstimatedSizeBytes(Datastore datastore, Query query, @Nullable String namespace) throws DatastoreException { String ourKind = query.getKind(0).getName(); + long latestTimestamp = queryLatestStatisticsTimestamp(datastore, namespace); + LOG.info("Latest stats timestamp : {}", latestTimestamp); + Query.Builder queryBuilder = Query.newBuilder(); if (namespace == null) { queryBuilder.addKindBuilder().setName("__Stat_Kind__"); } else { - queryBuilder.addKindBuilder().setName("__Ns_Stat_Kind__"); + queryBuilder.addKindBuilder().setName("__Stat_Ns_Kind__"); } queryBuilder.setFilter(makeFilter("kind_name", EQUAL, makeValue(ourKind).build())); - - // Get the latest statistics - queryBuilder.addOrder(makeOrder("timestamp", DESCENDING)); - queryBuilder.setLimit(Int32Value.newBuilder().setValue(1)); + queryBuilder.setFilter(makeFilter("timestamp", EQUAL, makeValue(latestTimestamp).build())); RunQueryRequest request = makeRequest(queryBuilder.build(), namespace); @@ -547,6 +575,7 @@ public void processElement(ProcessContext c) throws Exception { estimatedNumSplits = numSplits; } + LOG.info("Splitting the query into {} splits", estimatedNumSplits); List querySplits; try { querySplits = splitQuery(query, options.getNamespace(), datastore, querySplitter, @@ -866,7 +895,7 @@ public void processElement(ProcessContext c) throws Exception { @FinishBundle public void finishBundle(Context c) throws Exception { - if (mutations.size() > 0) { + if (!mutations.isEmpty()) { flushBatch(); } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java index ab1df2f3e9f5..138671d4a007 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java @@ -61,6 +61,7 @@ import com.google.datastore.v1.client.QuerySplitter; import com.google.protobuf.Int32Value; import java.util.ArrayList; +import java.util.Date; import java.util.HashSet; import java.util.LinkedList; import java.util.List; @@ -561,14 +562,23 @@ private void datastoreWriterFnTest(int numMutations) throws Exception { @Test public void testEstimatedSizeBytes() throws Exception { long entityBytes = 100L; + // In seconds + long timestamp = 1234L; + + RunQueryRequest latestTimestampRequest = makeRequest(makeLatestTimestampQuery(NAMESPACE), + NAMESPACE); + RunQueryResponse latestTimestampResponse = makeLatestTimestampResponse(timestamp); // Per Kind statistics request and response - RunQueryRequest statRequest = makeRequest(makeStatKindQuery(NAMESPACE), NAMESPACE); + RunQueryRequest statRequest = makeRequest(makeStatKindQuery(NAMESPACE, timestamp), NAMESPACE); RunQueryResponse statResponse = makeStatKindResponse(entityBytes); + when(mockDatastore.runQuery(latestTimestampRequest)) + .thenReturn(latestTimestampResponse); when(mockDatastore.runQuery(statRequest)) .thenReturn(statResponse); assertEquals(entityBytes, getEstimatedSizeBytes(mockDatastore, QUERY, NAMESPACE)); + verify(mockDatastore, times(1)).runQuery(latestTimestampRequest); verify(mockDatastore, times(1)).runQuery(statRequest); } @@ -609,11 +619,19 @@ public void testSplitQueryFnWithoutNumSplits() throws Exception { int numSplits = 0; int expectedNumSplits = 20; long entityBytes = expectedNumSplits * DEFAULT_BUNDLE_SIZE_BYTES; + // In seconds + long timestamp = 1234L; + + RunQueryRequest latestTimestampRequest = makeRequest(makeLatestTimestampQuery(NAMESPACE), + NAMESPACE); + RunQueryResponse latestTimestampResponse = makeLatestTimestampResponse(timestamp); // Per Kind statistics request and response - RunQueryRequest statRequest = makeRequest(makeStatKindQuery(NAMESPACE), NAMESPACE); + RunQueryRequest statRequest = makeRequest(makeStatKindQuery(NAMESPACE, timestamp), NAMESPACE); RunQueryResponse statResponse = makeStatKindResponse(entityBytes); + when(mockDatastore.runQuery(latestTimestampRequest)) + .thenReturn(latestTimestampResponse); when(mockDatastore.runQuery(statRequest)) .thenReturn(statResponse); when(mockQuerySplitter.getSplits( @@ -629,6 +647,7 @@ public void testSplitQueryFnWithoutNumSplits() throws Exception { verifyUniqueKeys(queries); verify(mockQuerySplitter, times(1)).getSplits( eq(QUERY), any(PartitionId.class), eq(expectedNumSplits), any(Datastore.class)); + verify(mockDatastore, times(1)).runQuery(latestTimestampRequest); verify(mockDatastore, times(1)).runQuery(statRequest); } @@ -752,7 +771,7 @@ public RunQueryResponse answer(InvocationOnMock invocationOnMock) throws Throwab /** Builds a per-kind statistics response with the given entity size. */ private static RunQueryResponse makeStatKindResponse(long entitySizeInBytes) { - RunQueryResponse.Builder timestampResponse = RunQueryResponse.newBuilder(); + RunQueryResponse.Builder statKindResponse = RunQueryResponse.newBuilder(); Entity.Builder entity = Entity.newBuilder(); entity.setKey(makeKey("dummyKind", "dummyId")); entity.getMutableProperties().put("entity_bytes", makeValue(entitySizeInBytes).build()); @@ -760,24 +779,51 @@ private static RunQueryResponse makeStatKindResponse(long entitySizeInBytes) { entityResult.setEntity(entity); QueryResultBatch.Builder batch = QueryResultBatch.newBuilder(); batch.addEntityResults(entityResult); + statKindResponse.setBatch(batch); + return statKindResponse.build(); + } + + /** Builds a response of the given timestamp. */ + private static RunQueryResponse makeLatestTimestampResponse(long timestamp) { + RunQueryResponse.Builder timestampResponse = RunQueryResponse.newBuilder(); + Entity.Builder entity = Entity.newBuilder(); + entity.setKey(makeKey("dummyKind", "dummyId")); + entity.getMutableProperties().put("timestamp", makeValue(new Date(timestamp * 1000)).build()); + EntityResult.Builder entityResult = EntityResult.newBuilder(); + entityResult.setEntity(entity); + QueryResultBatch.Builder batch = QueryResultBatch.newBuilder(); + batch.addEntityResults(entityResult); timestampResponse.setBatch(batch); return timestampResponse.build(); } /** Builds a per-kind statistics query for the given timestamp and namespace. */ - private static Query makeStatKindQuery(String namespace) { + private static Query makeStatKindQuery(String namespace, long timestamp) { Query.Builder statQuery = Query.newBuilder(); if (namespace == null) { statQuery.addKindBuilder().setName("__Stat_Kind__"); } else { - statQuery.addKindBuilder().setName("__Ns_Stat_Kind__"); + statQuery.addKindBuilder().setName("__Stat_Ns_Kind__"); } statQuery.setFilter(makeFilter("kind_name", EQUAL, makeValue(KIND)).build()); - statQuery.addOrder(makeOrder("timestamp", DESCENDING)); - statQuery.setLimit(Int32Value.newBuilder().setValue(1)); + statQuery.setFilter(makeFilter("timestamp", EQUAL, makeValue(timestamp * 1000000L)).build()); return statQuery.build(); } + /** Builds a latest timestamp statistics query. */ + private static Query makeLatestTimestampQuery(String namespace) { + Query.Builder timestampQuery = Query.newBuilder(); + if (namespace == null) { + timestampQuery.addKindBuilder().setName("__Stat_Total__"); + } else { + timestampQuery.addKindBuilder().setName("__Stat_Ns_Total__"); + } + timestampQuery.addOrder(makeOrder("timestamp", DESCENDING)); + timestampQuery.setLimit(Int32Value.newBuilder().setValue(1)); + return timestampQuery.build(); + } + + /** Generate dummy query splits. */ private List splitQuery(Query query, int numSplits) { List queries = new LinkedList<>(); From 310ea7497a151d1a9567f3e9a3b18e54ddcdc7f0 Mon Sep 17 00:00:00 2001 From: gaurav gupta Date: Thu, 25 Aug 2016 14:00:06 -0700 Subject: [PATCH 073/112] [BEAM-589] Fixing IO.Read transformation --- .../org/apache/beam/examples/complete/TfIdf.java | 6 +++--- .../runners/core/UnboundedReadFromBoundedSource.java | 6 +++--- .../apache/beam/runners/flink/examples/TFIDF.java | 6 +++--- .../apache/beam/runners/dataflow/DataflowRunner.java | 6 +++--- .../DataflowUnboundedReadFromBoundedSource.java | 6 +++--- .../beam/runners/dataflow/DataflowRunnerTest.java | 4 ++-- .../apache/beam/runners/spark/io/CreateStream.java | 7 +++---- .../org/apache/beam/runners/spark/io/KafkaIO.java | 6 +++--- .../beam/runners/spark/io/hadoop/HadoopIO.java | 6 +++--- .../src/main/java/org/apache/beam/sdk/io/AvroIO.java | 6 +++--- .../beam/sdk/io/BoundedReadFromUnboundedSource.java | 6 +++--- .../main/java/org/apache/beam/sdk/io/PubsubIO.java | 6 +++--- .../src/main/java/org/apache/beam/sdk/io/Read.java | 10 +++++----- .../src/main/java/org/apache/beam/sdk/io/TextIO.java | 6 +++--- .../java/org/apache/beam/sdk/transforms/Create.java | 12 ++++++------ .../test/java/org/apache/beam/sdk/io/AvroIOTest.java | 2 +- .../java/org/apache/beam/sdk/io/PubsubIOTest.java | 2 +- .../test/java/org/apache/beam/sdk/io/TextIOTest.java | 2 +- .../apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java | 8 ++++---- .../beam/sdk/io/gcp/bigquery/BigQueryIOTest.java | 4 ++-- .../main/java/org/apache/beam/sdk/io/jms/JmsIO.java | 3 +-- .../java/org/apache/beam/sdk/io/kafka/KafkaIO.java | 3 +-- 22 files changed, 60 insertions(+), 63 deletions(-) diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java index 87023edac29a..6684553c41eb 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java @@ -51,11 +51,11 @@ import org.apache.beam.sdk.util.GcsUtil; import org.apache.beam.sdk.util.gcsfs.GcsPath; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PDone; -import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.TupleTag; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -152,7 +152,7 @@ public static Set listInputDocuments(Options options) * from the documents tagged with which document they are from. */ public static class ReadDocuments - extends PTransform>> { + extends PTransform>> { private Iterable uris; public ReadDocuments(Iterable uris) { @@ -165,7 +165,7 @@ public Coder getDefaultOutputCoder() { } @Override - public PCollection> apply(PInput input) { + public PCollection> apply(PBegin input) { Pipeline pipeline = input.getPipeline(); // Create one TextIO.Read transform for each document diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSource.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSource.java index 73688d45592e..91a17158fa0c 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSource.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSource.java @@ -51,8 +51,8 @@ import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.PropertyNames; +import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.TimestampedValue; import org.joda.time.Instant; import org.slf4j.Logger; @@ -74,7 +74,7 @@ *

              This transform is intended to be used by a runner during pipeline translation to convert * a Read.Bounded into a Read.Unbounded. */ -public class UnboundedReadFromBoundedSource extends PTransform> { +public class UnboundedReadFromBoundedSource extends PTransform> { private static final Logger LOG = LoggerFactory.getLogger(UnboundedReadFromBoundedSource.class); @@ -88,7 +88,7 @@ public UnboundedReadFromBoundedSource(BoundedSource source) { } @Override - public PCollection apply(PInput input) { + public PCollection apply(PBegin input) { return input.getPipeline().apply( Read.from(new BoundedToUnboundedSourceAdapter<>(source))); } diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/TFIDF.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/TFIDF.java index 0ca94a13463b..a92d3397cdfd 100644 --- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/TFIDF.java +++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/TFIDF.java @@ -53,11 +53,11 @@ import org.apache.beam.sdk.util.GcsUtil; import org.apache.beam.sdk.util.gcsfs.GcsPath; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PDone; -import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.TupleTag; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -154,7 +154,7 @@ public static Set listInputDocuments(Options options) * from the documents tagged with which document they are from. */ public static class ReadDocuments - extends PTransform>> { + extends PTransform>> { private static final long serialVersionUID = 0; private Iterable uris; @@ -169,7 +169,7 @@ public Coder getDefaultOutputCoder() { } @Override - public PCollection> apply(PInput input) { + public PCollection> apply(PBegin input) { Pipeline pipeline = input.getPipeline(); // Create one TextIO.Read transform for each document diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index a0e24b14a575..0ce4b58aa6ac 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -400,7 +400,7 @@ public OutputT apply( return windowed; } else if (Flatten.FlattenPCollectionList.class.equals(transform.getClass()) && ((PCollectionList) input).size() == 0) { - return (OutputT) Pipeline.applyTransform(input, Create.of()); + return (OutputT) Pipeline.applyTransform((PBegin) input, Create.of()); } else if (overrides.containsKey(transform.getClass())) { // It is the responsibility of whoever constructs overrides to ensure this is type safe. @SuppressWarnings("unchecked") @@ -2318,7 +2318,7 @@ public void processElement(ProcessContext c) { * Specialized implementation for {@link org.apache.beam.sdk.io.Read.Bounded Read.Bounded} for the * Dataflow runner in streaming mode. */ - private static class StreamingBoundedRead extends PTransform> { + private static class StreamingBoundedRead extends PTransform> { private final BoundedSource source; /** Builds an instance of this class from the overridden transform. */ @@ -2333,7 +2333,7 @@ protected Coder getDefaultOutputCoder() { } @Override - public final PCollection apply(PInput input) { + public final PCollection apply(PBegin input) { source.validate(); return Pipeline.applyTransform(input, new DataflowUnboundedReadFromBoundedSource<>(source)) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowUnboundedReadFromBoundedSource.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowUnboundedReadFromBoundedSource.java index 85f5e734a253..866da13ccab6 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowUnboundedReadFromBoundedSource.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowUnboundedReadFromBoundedSource.java @@ -51,8 +51,8 @@ import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.PropertyNames; +import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.TimestampedValue; import org.joda.time.Instant; import org.slf4j.Logger; @@ -78,7 +78,7 @@ * time dependency. It should be replaced in the dataflow worker as an execution time dependency. */ @Deprecated -public class DataflowUnboundedReadFromBoundedSource extends PTransform> { +public class DataflowUnboundedReadFromBoundedSource extends PTransform> { private static final Logger LOG = LoggerFactory.getLogger(DataflowUnboundedReadFromBoundedSource.class); @@ -93,7 +93,7 @@ public DataflowUnboundedReadFromBoundedSource(BoundedSource source) { } @Override - public PCollection apply(PInput input) { + public PCollection apply(PBegin input) { return input.getPipeline().apply( Read.from(new BoundedToUnboundedSourceAdapter<>(source))); } diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java index 208e84c96678..58a01aa475f0 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java @@ -101,8 +101,8 @@ import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.gcsfs.GcsPath; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; import org.hamcrest.Description; @@ -970,7 +970,7 @@ private static PipelineOptions makeOptions(boolean streaming) { return options; } - private void testUnsupportedSource(PTransform source, String name, boolean streaming) + private void testUnsupportedSource(PTransform source, String name, boolean streaming) throws Exception { String mode = streaming ? "streaming" : "batch"; thrown.expect(UnsupportedOperationException.class); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/CreateStream.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/CreateStream.java index b3beae6c6631..a08c54ed133e 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/CreateStream.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/CreateStream.java @@ -21,9 +21,8 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.WindowingStrategy; +import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PInput; - /** * Create an input stream from Queue. @@ -49,7 +48,7 @@ public static QueuedValues fromQueue(Iterable> queuedValues) /** * {@link PTransform} for queueing values. */ - public static final class QueuedValues extends PTransform> { + public static final class QueuedValues extends PTransform> { private final Iterable> queuedValues; @@ -64,7 +63,7 @@ public Iterable> getQueuedValues() { } @Override - public PCollection apply(PInput input) { + public PCollection apply(PBegin input) { // Spark streaming micro batches are bounded by default return PCollection.createPrimitiveOutputInternal(input.getPipeline(), WindowingStrategy.globalDefault(), PCollection.IsBounded.UNBOUNDED); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/KafkaIO.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/KafkaIO.java index f57c114e69fd..8cf20830bc75 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/KafkaIO.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/KafkaIO.java @@ -25,8 +25,8 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PInput; /** * Read stream from Kafka. @@ -68,7 +68,7 @@ public static Unbound from(Class> keyDecoder, /** * A {@link PTransform} reading from Kafka topics and providing {@link PCollection}. */ - public static class Unbound extends PTransform>> { + public static class Unbound extends PTransform>> { private final Class> keyDecoderClass; private final Class> valueDecoderClass; @@ -120,7 +120,7 @@ public Map getKafkaParams() { } @Override - public PCollection> apply(PInput input) { + public PCollection> apply(PBegin input) { // Spark streaming micro batches are bounded by default return PCollection.createPrimitiveOutputInternal(input.getPipeline(), WindowingStrategy.globalDefault(), PCollection.IsBounded.UNBOUNDED); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/HadoopIO.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/HadoopIO.java index 70bec78e51f4..042c316c1919 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/HadoopIO.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/HadoopIO.java @@ -26,9 +26,9 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; -import org.apache.beam.sdk.values.PInput; import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; @@ -58,7 +58,7 @@ public static Bound from(String filepattern, * @param the type of the keys * @param the type of the values */ - public static class Bound extends PTransform>> { + public static class Bound extends PTransform>> { private final String filepattern; private final Class> formatClass; @@ -94,7 +94,7 @@ public Class getKeyClass() { } @Override - public PCollection> apply(PInput input) { + public PCollection> apply(PBegin input) { return PCollection.createPrimitiveOutputInternal(input.getPipeline(), WindowingStrategy.globalDefault(), PCollection.IsBounded.BOUNDED); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java index e7c302bed1d7..267265db1dae 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java @@ -40,9 +40,9 @@ import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.util.MimeTypes; +import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; -import org.apache.beam.sdk.values.PInput; /** * {@link PTransform}s for reading and writing Avro files. @@ -184,7 +184,7 @@ public static Bound withoutValidation() { * @param the type of each of the elements of the resulting * PCollection */ - public static class Bound extends PTransform> { + public static class Bound extends PTransform> { /** The filepattern to read from. */ @Nullable final String filepattern; @@ -270,7 +270,7 @@ public Bound withoutValidation() { } @Override - public PCollection apply(PInput input) { + public PCollection apply(PBegin input) { if (filepattern == null) { throw new IllegalStateException( "need to set the filepattern of an AvroIO.Read transform"); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java index ede65a96cc99..28d77468bbc0 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java @@ -36,8 +36,8 @@ import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.util.IntervalBoundedExponentialBackOff; import org.apache.beam.sdk.util.ValueWithRecordId; +import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PInput; import org.joda.time.Duration; import org.joda.time.Instant; @@ -48,7 +48,7 @@ * *

              Created by {@link Read}. */ -class BoundedReadFromUnboundedSource extends PTransform> { +class BoundedReadFromUnboundedSource extends PTransform> { private final UnboundedSource source; private final long maxNumRecords; private final Duration maxReadTime; @@ -82,7 +82,7 @@ public BoundedReadFromUnboundedSource withMaxReadTime(Duration maxReadTime) { } @Override - public PCollection apply(PInput input) { + public PCollection apply(PBegin input) { PCollection> read = Pipeline.applyTransform(input, Read.from(new UnboundedToBoundedSourceAdapter<>(source, maxNumRecords, maxReadTime))); if (source.requiresDeduping()) { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java index b137f1551010..d11345780295 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java @@ -46,9 +46,9 @@ import org.apache.beam.sdk.util.PubsubClient.SubscriptionPath; import org.apache.beam.sdk.util.PubsubClient.TopicPath; import org.apache.beam.sdk.util.PubsubJsonClient; +import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; -import org.apache.beam.sdk.values.PInput; import org.joda.time.Duration; import org.joda.time.Instant; import org.slf4j.Logger; @@ -481,7 +481,7 @@ public static Bound maxReadTime(Duration maxReadTime) { * A {@link PTransform} that reads from a Cloud Pub/Sub source and returns * a unbounded {@link PCollection} containing the items from the stream. */ - public static class Bound extends PTransform> { + public static class Bound extends PTransform> { /** The Cloud Pub/Sub topic to read from. */ @Nullable private final PubsubTopic topic; @@ -610,7 +610,7 @@ public Bound maxReadTime(Duration maxReadTime) { } @Override - public PCollection apply(PInput input) { + public PCollection apply(PBegin input) { if (topic == null && subscription == null) { throw new IllegalStateException("Need to set either the topic or the subscription for " + "a PubsubIO.Read transform"); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java index f99877d84d29..29c4e47e64ec 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java @@ -25,9 +25,9 @@ import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.WindowingStrategy; +import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; -import org.apache.beam.sdk.values.PInput; import org.joda.time.Duration; /** @@ -87,7 +87,7 @@ public Unbounded from(UnboundedSource source) { /** * {@link PTransform} that reads from a {@link BoundedSource}. */ - public static class Bounded extends PTransform> { + public static class Bounded extends PTransform> { private final BoundedSource source; private Bounded(@Nullable String name, BoundedSource source) { @@ -101,7 +101,7 @@ protected Coder getDefaultOutputCoder() { } @Override - public final PCollection apply(PInput input) { + public final PCollection apply(PBegin input) { source.validate(); return PCollection.createPrimitiveOutputInternal(input.getPipeline(), @@ -134,7 +134,7 @@ public void populateDisplayData(DisplayData.Builder builder) { /** * {@link PTransform} that reads from a {@link UnboundedSource}. */ - public static class Unbounded extends PTransform> { + public static class Unbounded extends PTransform> { private final UnboundedSource source; private Unbounded(@Nullable String name, UnboundedSource source) { @@ -169,7 +169,7 @@ protected Coder getDefaultOutputCoder() { } @Override - public final PCollection apply(PInput input) { + public final PCollection apply(PBegin input) { source.validate(); return PCollection.createPrimitiveOutputInternal( diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java index ed9a62790afe..242470b58792 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java @@ -43,9 +43,9 @@ import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.util.MimeTypes; +import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; -import org.apache.beam.sdk.values.PInput; /** * {@link PTransform}s for reading and writing text files. @@ -189,7 +189,7 @@ public static Bound withCompressionType(TextIO.CompressionType compressi * may use {@link #withCoder(Coder)} to supply a {@code Coder} to produce a * {@code PCollection} instead. */ - public static class Bound extends PTransform> { + public static class Bound extends PTransform> { /** The filepattern to read from. */ @Nullable private final String filepattern; @@ -269,7 +269,7 @@ public Bound withCompressionType(TextIO.CompressionType compressionType) { } @Override - public PCollection apply(PInput input) { + public PCollection apply(PBegin input) { if (filepattern == null) { throw new IllegalStateException("need to set the filepattern of a TextIO.Read transform"); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java index e261db2b30f9..7cd47117565d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java @@ -46,8 +46,8 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TimestampedValue.TimestampedValueCoder; import org.apache.beam.sdk.values.TypeDescriptor; @@ -218,7 +218,7 @@ public static TimestampedValues timestamped( /** * A {@code PTransform} that creates a {@code PCollection} from a set of in-memory objects. */ - public static class Values extends PTransform> { + public static class Values extends PTransform> { /** * Returns a {@link Create.Values} PTransform like this one that uses the given * {@code Coder} to decode each of the objects into a @@ -240,7 +240,7 @@ public Iterable getElements() { } @Override - public PCollection apply(PInput input) { + public PCollection apply(PBegin input) { try { Coder coder = getDefaultOutputCoder(input); try { @@ -257,7 +257,7 @@ public PCollection apply(PInput input) { } @Override - public Coder getDefaultOutputCoder(PInput input) throws CannotProvideCoderException { + public Coder getDefaultOutputCoder(PBegin input) throws CannotProvideCoderException { if (coder.isPresent()) { return coder.get(); } else { @@ -421,7 +421,7 @@ protected boolean advanceImpl() throws IOException { * A {@code PTransform} that creates a {@code PCollection} whose elements have * associated timestamps. */ - public static class TimestampedValues extends PTransform>{ + public static class TimestampedValues extends PTransform>{ /** * Returns a {@link Create.TimestampedValues} PTransform like this one that uses the given * {@code Coder} to decode each of the objects into a @@ -440,7 +440,7 @@ public TimestampedValues withCoder(Coder coder) { } @Override - public PCollection apply(PInput input) { + public PCollection apply(PBegin input) { try { Iterable rawElements = Iterables.transform( diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java index a8a7746f214b..81f05d7cfbc9 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java @@ -291,7 +291,7 @@ public void testPrimitiveReadDisplayData() { .withSchema(Schema.create(Schema.Type.STRING)) .withoutValidation(); - Set displayData = evaluator.displayDataForPrimitiveTransforms(read); + Set displayData = evaluator.displayDataForPrimitiveSourceTransforms(read); assertThat("AvroIO.Read should include the file pattern in its primitive transform", displayData, hasItem(hasDisplayItem("filePattern"))); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubIOTest.java index 4067055b25c7..086b72644581 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubIOTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubIOTest.java @@ -111,7 +111,7 @@ public void testPrimitiveReadDisplayData() { PubsubIO.Read.subscription("projects/project/subscriptions/subscription") .maxNumRecords(1); - Set displayData = evaluator.displayDataForPrimitiveTransforms(read); + Set displayData = evaluator.displayDataForPrimitiveSourceTransforms(read); assertThat("PubsubIO.Read should include the subscription in its primitive display data", displayData, hasItem(hasDisplayItem("subscription"))); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java index 358a30f3d2b8..8f9476606302 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java @@ -210,7 +210,7 @@ public void testPrimitiveReadDisplayData() { .from("foobar") .withoutValidation(); - Set displayData = evaluator.displayDataForPrimitiveTransforms(read); + Set displayData = evaluator.displayDataForPrimitiveSourceTransforms(read); assertThat("TextIO.Read should include the file prefix in its primitive display data", displayData, hasItem(hasDisplayItem(hasValue(startsWith("foobar"))))); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java index 01a8a1c5cd4e..304dc820ac9d 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java @@ -122,11 +122,11 @@ import org.apache.beam.sdk.util.Transport; import org.apache.beam.sdk.util.gcsfs.GcsPath; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PDone; -import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; import org.joda.time.Instant; @@ -377,7 +377,7 @@ public static Bound withoutValidation() { * A {@link PTransform} that reads from a BigQuery table and returns a bounded * {@link PCollection} of {@link TableRow TableRows}. */ - public static class Bound extends PTransform> { + public static class Bound extends PTransform> { @Nullable final String jsonTableRef; @Nullable final String query; @@ -480,7 +480,7 @@ Bound withTestServices(BigQueryServices testServices) { } @Override - public void validate(PInput input) { + public void validate(PBegin input) { // Even if existence validation is disabled, we need to make sure that the BigQueryIO // read is properly specified. BigQueryOptions bqOptions = input.getPipeline().getOptions().as(BigQueryOptions.class); @@ -524,7 +524,7 @@ public void validate(PInput input) { } @Override - public PCollection apply(PInput input) { + public PCollection apply(PBegin input) { String uuid = randomUUIDString(); final String jobIdToken = "beam_job_" + uuid; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java index 7a7575bfdd24..57eb4ffb2394 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java @@ -671,7 +671,7 @@ public void testTableSourcePrimitiveDisplayData() throws IOException, Interrupte .withJobService(mockJobService)) .withoutValidation(); - Set displayData = evaluator.displayDataForPrimitiveTransforms(read); + Set displayData = evaluator.displayDataForPrimitiveSourceTransforms(read); assertThat("BigQueryIO.Read should include the table spec in its primitive display data", displayData, hasItem(hasDisplayItem("table"))); } @@ -688,7 +688,7 @@ public void testQuerySourcePrimitiveDisplayData() throws IOException, Interrupte .withJobService(mockJobService)) .withoutValidation(); - Set displayData = evaluator.displayDataForPrimitiveTransforms(read); + Set displayData = evaluator.displayDataForPrimitiveSourceTransforms(read); assertThat("BigQueryIO.Read should include the query in its primitive display data", displayData, hasItem(hasDisplayItem("query"))); } diff --git a/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java b/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java index f92dbd456b79..29d0c5fac1d8 100644 --- a/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java +++ b/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java @@ -51,7 +51,6 @@ import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; -import org.apache.beam.sdk.values.PInput; import org.joda.time.Duration; import org.joda.time.Instant; import org.slf4j.Logger; @@ -140,7 +139,7 @@ public PCollection apply(PBegin input) { // handles unbounded source to bounded conversion if maxNumRecords is set. Unbounded unbounded = org.apache.beam.sdk.io.Read.from(createSource()); - PTransform> transform = unbounded; + PTransform> transform = unbounded; if (maxNumRecords != Long.MAX_VALUE) { transform = unbounded.withMaxNumRecords(maxNumRecords); diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java index 885d5d10b4b7..f6394220ceba 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java @@ -73,7 +73,6 @@ import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; -import org.apache.beam.sdk.values.PInput; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRecord; @@ -450,7 +449,7 @@ public PCollection> apply(PBegin input) { Unbounded> unbounded = org.apache.beam.sdk.io.Read.from(makeSource()); - PTransform>> transform = unbounded; + PTransform>> transform = unbounded; if (maxNumRecords < Long.MAX_VALUE) { transform = unbounded.withMaxNumRecords(maxNumRecords); From bed22de6880339465637a339b2d8b8527b6cacc6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Pastuszka=20Przemys=C5=82aw?= Date: Mon, 18 Jul 2016 20:03:37 +0200 Subject: [PATCH 074/112] kinesis: a connector for Amazon Kinesis --- sdks/java/io/kinesis/pom.xml | 179 +++++++++ .../sdk/io/kinesis/CheckpointGenerator.java | 30 ++ .../beam/sdk/io/kinesis/CustomOptional.java | 85 ++++ .../kinesis/DynamicCheckpointGenerator.java | 58 +++ .../io/kinesis/GetKinesisRecordsResult.java | 54 +++ .../sdk/io/kinesis/KinesisClientProvider.java | 31 ++ .../apache/beam/sdk/io/kinesis/KinesisIO.java | 190 +++++++++ .../beam/sdk/io/kinesis/KinesisReader.java | 145 +++++++ .../io/kinesis/KinesisReaderCheckpoint.java | 98 +++++ .../beam/sdk/io/kinesis/KinesisRecord.java | 121 ++++++ .../sdk/io/kinesis/KinesisRecordCoder.java | 75 ++++ .../beam/sdk/io/kinesis/KinesisSource.java | 114 ++++++ .../beam/sdk/io/kinesis/RecordFilter.java | 41 ++ .../beam/sdk/io/kinesis/RoundRobin.java | 54 +++ .../beam/sdk/io/kinesis/ShardCheckpoint.java | 175 ++++++++ .../sdk/io/kinesis/ShardRecordsIterator.java | 98 +++++ .../io/kinesis/SimplifiedKinesisClient.java | 158 ++++++++ .../beam/sdk/io/kinesis/StartingPoint.java | 85 ++++ .../io/kinesis/StaticCheckpointGenerator.java | 42 ++ .../io/kinesis/TransientKinesisException.java | 29 ++ .../beam/sdk/io/kinesis/package-info.java | 22 + .../sdk/io/kinesis/AmazonKinesisMock.java | 375 ++++++++++++++++++ .../sdk/io/kinesis/CustomOptionalTest.java | 31 ++ .../DynamicCheckpointGeneratorTest.java | 56 +++ .../sdk/io/kinesis/KinesisMockReadTest.java | 92 +++++ .../kinesis/KinesisReaderCheckpointTest.java | 67 ++++ .../beam/sdk/io/kinesis/KinesisReaderIT.java | 119 ++++++ .../sdk/io/kinesis/KinesisReaderTest.java | 119 ++++++ .../io/kinesis/KinesisRecordCoderTest.java | 46 +++ .../sdk/io/kinesis/KinesisTestOptions.java | 47 +++ .../beam/sdk/io/kinesis/KinesisUploader.java | 84 ++++ .../beam/sdk/io/kinesis/RecordFilterTest.java | 66 +++ .../beam/sdk/io/kinesis/RoundRobinTest.java | 57 +++ .../sdk/io/kinesis/ShardCheckpointTest.java | 148 +++++++ .../io/kinesis/ShardRecordsIteratorTest.java | 150 +++++++ .../kinesis/SimplifiedKinesisClientTest.java | 223 +++++++++++ .../beam/sdk/io/kinesis/package-info.java | 22 + sdks/java/io/pom.xml | 1 + 38 files changed, 3587 insertions(+) create mode 100644 sdks/java/io/kinesis/pom.xml create mode 100644 sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/CheckpointGenerator.java create mode 100644 sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/CustomOptional.java create mode 100644 sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGenerator.java create mode 100644 sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/GetKinesisRecordsResult.java create mode 100644 sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisClientProvider.java create mode 100644 sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisIO.java create mode 100644 sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReader.java create mode 100644 sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpoint.java create mode 100644 sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecord.java create mode 100644 sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoder.java create mode 100644 sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisSource.java create mode 100644 sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/RecordFilter.java create mode 100644 sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/RoundRobin.java create mode 100644 sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardCheckpoint.java create mode 100644 sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIterator.java create mode 100644 sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClient.java create mode 100644 sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/StartingPoint.java create mode 100644 sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/StaticCheckpointGenerator.java create mode 100644 sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/TransientKinesisException.java create mode 100644 sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/package-info.java create mode 100644 sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/AmazonKinesisMock.java create mode 100644 sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/CustomOptionalTest.java create mode 100644 sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGeneratorTest.java create mode 100644 sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisMockReadTest.java create mode 100644 sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpointTest.java create mode 100644 sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderIT.java create mode 100644 sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderTest.java create mode 100644 sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoderTest.java create mode 100644 sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisTestOptions.java create mode 100644 sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisUploader.java create mode 100644 sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RecordFilterTest.java create mode 100644 sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RoundRobinTest.java create mode 100644 sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardCheckpointTest.java create mode 100644 sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIteratorTest.java create mode 100644 sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClientTest.java create mode 100644 sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/package-info.java diff --git a/sdks/java/io/kinesis/pom.xml b/sdks/java/io/kinesis/pom.xml new file mode 100644 index 000000000000..aec1786eee91 --- /dev/null +++ b/sdks/java/io/kinesis/pom.xml @@ -0,0 +1,179 @@ + + + + 4.0.0 + + + org.apache.beam + beam-sdks-java-io-parent + 0.3.0-incubating-SNAPSHOT + ../pom.xml + + + beam-sdks-java-io-kinesis + Apache Beam :: SDKs :: Java :: IO :: Kinesis + Library to read Kinesis streams. + + + + + org.apache.maven.plugins + maven-compiler-plugin + + + org.apache.maven.plugins + maven-source-plugin + + + org.apache.maven.plugins + maven-surefire-plugin + + + false + + + + + org.apache.maven.plugins + maven-jar-plugin + + + org.apache.maven.plugins + maven-checkstyle-plugin + + + org.apache.maven.plugins + maven-javadoc-plugin + + -Xdoclint:missing + + + + + org.apache.maven.plugins + maven-failsafe-plugin + + false + true + + + + + integration-test + verify + + + + ${integrationTestPipelineOptions} + + + + + + + + + + 1.11.18 + + + + + org.apache.beam + beam-sdks-java-core + + + + com.amazonaws + aws-java-sdk-kinesis + ${aws.version} + + + + com.amazonaws + amazon-kinesis-client + 1.6.1 + + + + org.slf4j + slf4j-api + + + + joda-time + joda-time + + + + com.google.guava + guava + + + + commons-lang + commons-lang + 2.6 + + + + com.amazonaws + aws-java-sdk-core + ${aws.version} + + + + com.google.code.findbugs + annotations + + + + + junit + junit + test + + + + org.mockito + mockito-all + test + + + + org.assertj + assertj-core + 2.5.0 + test + + + + org.hamcrest + hamcrest-all + test + + + + org.apache.beam + beam-runners-direct-java + ${project.version} + test + + + + diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/CheckpointGenerator.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/CheckpointGenerator.java new file mode 100644 index 000000000000..919d85aacb4c --- /dev/null +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/CheckpointGenerator.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.kinesis; + + +import java.io.Serializable; + +/** + * Used to generate checkpoint object on demand. + * How exactly the checkpoint is generated is up to implementing class. + */ +interface CheckpointGenerator extends Serializable { + KinesisReaderCheckpoint generate(SimplifiedKinesisClient client) + throws TransientKinesisException; +} diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/CustomOptional.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/CustomOptional.java new file mode 100644 index 000000000000..804d6ccb97a0 --- /dev/null +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/CustomOptional.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.kinesis; + +import java.util.NoSuchElementException; + +/*** + * Similar to Guava {@code Optional}, but throws {@link NoSuchElementException} for missing element. + */ +abstract class CustomOptional { + public static CustomOptional absent() { + return Absent.INSTANCE; + } + + public static CustomOptional of(T v) { + return new Present<>(v); + } + + public abstract boolean isPresent(); + + public abstract T get(); + + private static class Present extends CustomOptional { + private final T value; + + private Present(T value) { + this.value = value; + } + + @Override + public boolean isPresent() { + return true; + } + + @Override + public T get() { + return value; + } + + + @Override + public boolean equals(Object o) { + Present present = (Present) o; + + return value != null ? value.equals(present.value) : present.value == null; + } + + @Override + public int hashCode() { + return value != null ? value.hashCode() : 0; + } + } + + private static class Absent extends CustomOptional { + public static final Absent INSTANCE = new Absent(); + + private Absent() { + } + + @Override + public boolean isPresent() { + return false; + } + + @Override + public T get() { + throw new NoSuchElementException(); + } + } +} diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGenerator.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGenerator.java new file mode 100644 index 000000000000..d86960f0d030 --- /dev/null +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGenerator.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.kinesis; + +import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.collect.Lists.transform; + + +import com.google.common.base.Function; + +import com.amazonaws.services.kinesis.model.Shard; + +/** + * Creates {@link KinesisReaderCheckpoint}, which spans over all shards in given stream. + * List of shards is obtained dynamically on call to {@link #generate(SimplifiedKinesisClient)}. + */ +class DynamicCheckpointGenerator implements CheckpointGenerator { + private final String streamName; + private final StartingPoint startingPoint; + + public DynamicCheckpointGenerator(String streamName, StartingPoint startingPoint) { + this.streamName = checkNotNull(streamName, "streamName"); + this.startingPoint = checkNotNull(startingPoint, "startingPoint"); + } + + @Override + public KinesisReaderCheckpoint generate(SimplifiedKinesisClient kinesis) + throws TransientKinesisException { + return new KinesisReaderCheckpoint( + transform(kinesis.listShards(streamName), new Function() { + @Override + public ShardCheckpoint apply(Shard shard) { + return new ShardCheckpoint(streamName, shard.getShardId(), startingPoint); + } + }) + ); + } + + @Override + public String toString() { + return String.format("Checkpoint generator for %s: %s", streamName, startingPoint); + } +} diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/GetKinesisRecordsResult.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/GetKinesisRecordsResult.java new file mode 100644 index 000000000000..f48b9d53ade8 --- /dev/null +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/GetKinesisRecordsResult.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.kinesis; + +import static com.google.common.collect.Lists.transform; +import com.google.common.base.Function; + +import com.amazonaws.services.kinesis.clientlibrary.types.UserRecord; +import java.util.List; +import javax.annotation.Nullable; + +/*** + * Represents the output of 'get' operation on Kinesis stream. + */ +class GetKinesisRecordsResult { + private final List records; + private final String nextShardIterator; + + public GetKinesisRecordsResult(List records, String nextShardIterator, + final String streamName, final String shardId) { + this.records = transform(records, new Function() { + @Nullable + @Override + public KinesisRecord apply(@Nullable UserRecord input) { + assert input != null; // to make FindBugs happy + return new KinesisRecord(input, streamName, shardId); + } + }); + this.nextShardIterator = nextShardIterator; + } + + public List getRecords() { + return records; + } + + public String getNextShardIterator() { + return nextShardIterator; + } +} diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisClientProvider.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisClientProvider.java new file mode 100644 index 000000000000..36c8953f601b --- /dev/null +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisClientProvider.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.kinesis; + +import com.amazonaws.services.kinesis.AmazonKinesis; +import java.io.Serializable; + +/** + * Provides instances of {@link AmazonKinesis} interface. + * + * Please note, that any instance of {@link KinesisClientProvider} must be + * {@link Serializable} to ensure it can be sent to worker machines. + */ +interface KinesisClientProvider extends Serializable { + AmazonKinesis get(); +} diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisIO.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisIO.java new file mode 100644 index 000000000000..b3cb464e1cfb --- /dev/null +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisIO.java @@ -0,0 +1,190 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.kinesis; + + +import org.apache.beam.sdk.transforms.PTransform; +import static com.google.common.base.Preconditions.checkNotNull; + +import com.amazonaws.auth.AWSCredentialsProvider; +import com.amazonaws.auth.BasicAWSCredentials; +import com.amazonaws.internal.StaticCredentialsProvider; +import com.amazonaws.regions.Regions; +import com.amazonaws.services.kinesis.AmazonKinesis; +import com.amazonaws.services.kinesis.AmazonKinesisClient; +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream; +import org.joda.time.Instant; + +/** + * {@link PTransform}s for reading from + * Kinesis streams. + * + *

              Usage

              + * + *

              Main class you're going to operate is called {@link KinesisIO}. + * It follows the usage conventions laid out by other *IO classes like + * BigQueryIO or PubsubIOLet's see how you can set up a simple Pipeline, which reads from Kinesis: + * + *

              {@code}
              + * p.
              + *   apply(KinesisIO.Read.
              + *     from("streamName", InitialPositionInStream.LATEST).
              + *     using("AWS_KEY", _"AWS_SECRET", STREAM_REGION).
              + *     apply( ... ) // other transformations
              + *
              + *

              + * + *

              + * As you can see you need to provide 3 things: + *

                + *
              • name of the stream you're going to read
              • + *
              • position in the stream where reading should start. There are two options:
              • + *
                  + *
                • {@link InitialPositionInStream#LATEST} - reading will begin from end of the stream
                • + *
                • {@link InitialPositionInStream#TRIM_HORIZON} - reading will begin at + * the very beginning of the stream
                • + *
                + *
              • data used to initialize {@link AmazonKinesis} client
              • + *
                  + *
                • credentials (aws key, aws secret)
                • + *
                • region where the stream is located
                • + *
                + *
              + *

              + * + *

              In case when you want to set up {@link AmazonKinesis} client by your own + * (for example if you're using more sophisticated authorization methods like Amazon STS, etc.) + * you can do it by implementing {@link KinesisClientProvider} class: + * + *

              {@code}
              + * public class MyCustomKinesisClientProvider implements KinesisClientProvider {
              + *   @Override
              + *   public AmazonKinesis get() {
              + *     // set up your client here
              + *   }
              + * }
              + * 
              + * + * Usage is pretty straightforward: + * + *
              {@code}
              + * p.
              + *   apply(KinesisIO.Read.
              + *    from("streamName", InitialPositionInStream.LATEST).
              + *    using(MyCustomKinesisClientProvider()).
              + *    apply( ... ) // other transformations
              + * 
              + *

              + * + *

              There’s also possibility to start reading using arbitrary point in time - + * in this case you need to provide {@link Instant} object: + * + *

              {@code}
              + * p.
              + *   apply(KinesisIO.Read.
              + *     from("streamName", instant).
              + *     using(MyCustomKinesisClientProvider()).
              + *     apply( ... ) // other transformations
              + * 
              + *

              + * + */ +public final class KinesisIO { + /*** + * A {@link PTransform} that reads from a Kinesis stream. + */ + public static final class Read { + + private final String streamName; + private final StartingPoint initialPosition; + + private Read(String streamName, StartingPoint initialPosition) { + this.streamName = checkNotNull(streamName, "streamName"); + this.initialPosition = checkNotNull(initialPosition, "initialPosition"); + } + + /*** + * Specify reading from streamName at some initial position. + */ + public static Read from(String streamName, InitialPositionInStream initialPosition) { + return new Read(streamName, new StartingPoint( + checkNotNull(initialPosition, "initialPosition"))); + } + + /*** + * Specify reading from streamName beginning at given {@link Instant}. + * This {@link Instant} must be in the past, i.e. before {@link Instant#now()}. + */ + public static Read from(String streamName, Instant initialTimestamp) { + return new Read(streamName, new StartingPoint( + checkNotNull(initialTimestamp, "initialTimestamp"))); + } + + /*** + * Allows to specify custom {@link KinesisClientProvider}. + * {@link KinesisClientProvider} provides {@link AmazonKinesis} instances which are later + * used for communication with Kinesis. + * You should use this method if {@link Read#using(String, String, Regions)} does not + * suite your needs. + */ + public org.apache.beam.sdk.io.Read.Unbounded using + (KinesisClientProvider kinesisClientProvider) { + return org.apache.beam.sdk.io.Read.from( + new KinesisSource(kinesisClientProvider, streamName, + initialPosition)); + } + + /*** + * Specify credential details and region to be used to read from Kinesis. + * If you need more sophisticated credential protocol, then you should look at + * {@link Read#using(KinesisClientProvider)}. + */ + public org.apache.beam.sdk.io.Read.Unbounded using(String awsAccessKey, + String awsSecretKey, + Regions region) { + return using(new BasicKinesisProvider(awsAccessKey, awsSecretKey, region)); + } + + private static final class BasicKinesisProvider implements KinesisClientProvider { + + private final String accessKey; + private final String secretKey; + private final Regions region; + + private BasicKinesisProvider(String accessKey, String secretKey, Regions region) { + this.accessKey = checkNotNull(accessKey, "accessKey"); + this.secretKey = checkNotNull(secretKey, "secretKey"); + this.region = checkNotNull(region, "region"); + } + + + private AWSCredentialsProvider getCredentialsProvider() { + return new StaticCredentialsProvider(new BasicAWSCredentials( + accessKey, + secretKey + )); + + } + + @Override + public AmazonKinesis get() { + return new AmazonKinesisClient(getCredentialsProvider()).withRegion(region); + } + } + } +} diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReader.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReader.java new file mode 100644 index 000000000000..38a0050fa72a --- /dev/null +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReader.java @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.kinesis; + + +import org.apache.beam.sdk.io.UnboundedSource; +import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.collect.Lists.newArrayList; + +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import java.io.IOException; +import java.util.List; +import java.util.NoSuchElementException; + + +/*** + * Reads data from multiple kinesis shards in a single thread. + * It uses simple round robin algorithm when fetching data from shards. + */ +class KinesisReader extends UnboundedSource.UnboundedReader { + private static final Logger LOG = LoggerFactory.getLogger(KinesisReader.class); + + private final SimplifiedKinesisClient kinesis; + private final UnboundedSource source; + private final CheckpointGenerator initialCheckpointGenerator; + private RoundRobin shardIterators; + private CustomOptional currentRecord = CustomOptional.absent(); + + public KinesisReader(SimplifiedKinesisClient kinesis, + CheckpointGenerator initialCheckpointGenerator, + UnboundedSource source) { + this.kinesis = checkNotNull(kinesis, "kinesis"); + this.initialCheckpointGenerator = + checkNotNull(initialCheckpointGenerator, "initialCheckpointGenerator"); + this.source = source; + } + + /*** + * Generates initial checkpoint and instantiates iterators for shards. + */ + @Override + public boolean start() throws IOException { + LOG.info("Starting reader using {}", initialCheckpointGenerator); + + try { + KinesisReaderCheckpoint initialCheckpoint = + initialCheckpointGenerator.generate(kinesis); + List iterators = newArrayList(); + for (ShardCheckpoint checkpoint : initialCheckpoint) { + iterators.add(checkpoint.getShardRecordsIterator(kinesis)); + } + shardIterators = new RoundRobin<>(iterators); + } catch (TransientKinesisException e) { + throw new IOException(e); + } + + return advance(); + } + + /*** + * Moves to the next record in one of the shards. + * If current shard iterator can be move forward (i.e. there's a record present) then we do it. + * If not, we iterate over shards in a round-robin manner. + */ + @Override + public boolean advance() throws IOException { + try { + for (int i = 0; i < shardIterators.size(); ++i) { + currentRecord = shardIterators.getCurrent().next(); + if (currentRecord.isPresent()) { + return true; + } else { + shardIterators.moveForward(); + } + } + } catch (TransientKinesisException e) { + LOG.warn("Transient exception occurred", e); + } + return false; + } + + @Override + public byte[] getCurrentRecordId() throws NoSuchElementException { + return currentRecord.get().getUniqueId(); + } + + @Override + public KinesisRecord getCurrent() throws NoSuchElementException { + return currentRecord.get(); + } + + /*** + * When {@link KinesisReader} was advanced to the current record. + * We cannot use approximate arrival timestamp given for each record by Kinesis as it + * is not guaranteed to be accurate - this could lead to mark some records as "late" + * even if they were not. + */ + @Override + public Instant getCurrentTimestamp() throws NoSuchElementException { + return currentRecord.get().getReadTime(); + } + + @Override + public void close() throws IOException { + } + + /*** + * Current time. + * We cannot give better approximation of the watermark with current semantics of + * {@link KinesisReader#getCurrentTimestamp()}, because we don't know when the next + * {@link KinesisReader#advance()} will be called. + */ + @Override + public Instant getWatermark() { + return Instant.now(); + } + + @Override + public UnboundedSource.CheckpointMark getCheckpointMark() { + return KinesisReaderCheckpoint.asCurrentStateOf(shardIterators); + } + + @Override + public UnboundedSource getCurrentSource() { + return source; + } + +} diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpoint.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpoint.java new file mode 100644 index 000000000000..6ceb74295d86 --- /dev/null +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpoint.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.kinesis; + +import org.apache.beam.sdk.io.UnboundedSource; +import static com.google.common.collect.Iterables.transform; +import static com.google.common.collect.Lists.newArrayList; +import static com.google.common.collect.Lists.partition; + + +import com.google.common.base.Function; +import com.google.common.collect.ImmutableList; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Iterator; +import java.util.List; +import javax.annotation.Nullable; + +/*** + * Checkpoint representing a total progress in a set of shards in single stream. + * The set of shards covered by {@link KinesisReaderCheckpoint} may or may not be equal to set of + * all shards present in the stream. + * This class is immutable. + */ +class KinesisReaderCheckpoint implements Iterable, UnboundedSource + .CheckpointMark, Serializable { + private final List shardCheckpoints; + + public KinesisReaderCheckpoint(Iterable shardCheckpoints) { + this.shardCheckpoints = ImmutableList.copyOf(shardCheckpoints); + } + + public static KinesisReaderCheckpoint asCurrentStateOf(Iterable + iterators) { + return new KinesisReaderCheckpoint(transform(iterators, + new Function() { + + @Nullable + @Override + public ShardCheckpoint apply(@Nullable + ShardRecordsIterator shardRecordsIterator) { + assert shardRecordsIterator != null; + return shardRecordsIterator.getCheckpoint(); + } + })); + } + + /*** + * Splits given multi-shard checkpoint into partitions of approximately equal size. + * + * @param desiredNumSplits - upper limit for number of partitions to generate. + * @return list of checkpoints covering consecutive partitions of current checkpoint. + */ + public List splitInto(int desiredNumSplits) { + int partitionSize = divideAndRoundUp(shardCheckpoints.size(), desiredNumSplits); + + List checkpoints = newArrayList(); + for (List shardPartition : partition(shardCheckpoints, partitionSize)) { + checkpoints.add(new KinesisReaderCheckpoint(shardPartition)); + } + return checkpoints; + } + + private int divideAndRoundUp(int nominator, int denominator) { + return (nominator + denominator - 1) / denominator; + } + + @Override + public void finalizeCheckpoint() throws IOException { + + } + + @Override + public String toString() { + return shardCheckpoints.toString(); + } + + @Override + public Iterator iterator() { + return shardCheckpoints.iterator(); + } +} diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecord.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecord.java new file mode 100644 index 000000000000..cdb495ce0078 --- /dev/null +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecord.java @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.kinesis; + +import com.google.common.base.Charsets; + +import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber; +import com.amazonaws.services.kinesis.clientlibrary.types.UserRecord; +import static org.apache.commons.lang.builder.HashCodeBuilder.reflectionHashCode; +import org.apache.commons.lang.builder.EqualsBuilder; +import org.joda.time.Instant; +import java.io.Serializable; +import java.nio.ByteBuffer; + +/** + * {@link UserRecord} enhanced with utility methods. + */ +public class KinesisRecord implements Serializable { + private Instant readTime; + private String streamName; + private String shardId; + private long subSequenceNumber; + private String sequenceNumber; + private Instant approximateArrivalTimestamp; + private ByteBuffer data; + private String partitionKey; + + public KinesisRecord(UserRecord record, String streamName, String shardId) { + this(record.getData(), record.getSequenceNumber(), record.getSubSequenceNumber(), + record.getPartitionKey(), + new Instant(record.getApproximateArrivalTimestamp()), + Instant.now(), + streamName, shardId); + } + + public KinesisRecord(ByteBuffer data, String sequenceNumber, long subSequenceNumber, + String partitionKey, Instant approximateArrivalTimestamp, + Instant readTime, + String streamName, String shardId) { + this.data = data; + this.sequenceNumber = sequenceNumber; + this.subSequenceNumber = subSequenceNumber; + this.partitionKey = partitionKey; + this.approximateArrivalTimestamp = approximateArrivalTimestamp; + this.readTime = readTime; + this.streamName = streamName; + this.shardId = shardId; + } + + public ExtendedSequenceNumber getExtendedSequenceNumber() { + return new ExtendedSequenceNumber(getSequenceNumber(), getSubSequenceNumber()); + } + + /*** + * @return unique id of the record based on its position in the stream + */ + public byte[] getUniqueId() { + return getExtendedSequenceNumber().toString().getBytes(Charsets.UTF_8); + } + + public Instant getReadTime() { + return readTime; + } + + public String getStreamName() { + return streamName; + } + + public String getShardId() { + return shardId; + } + + public byte[] getDataAsBytes() { + return getData().array(); + } + + @Override + public boolean equals(Object obj) { + return EqualsBuilder.reflectionEquals(this, obj); + } + + @Override + public int hashCode() { + return reflectionHashCode(this); + } + + public long getSubSequenceNumber() { + return subSequenceNumber; + } + + public String getSequenceNumber() { + return sequenceNumber; + } + + public Instant getApproximateArrivalTimestamp() { + return approximateArrivalTimestamp; + } + + public ByteBuffer getData() { + return data; + } + + public String getPartitionKey() { + return partitionKey; + } +} diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoder.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoder.java new file mode 100644 index 000000000000..c383a4ff8a91 --- /dev/null +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoder.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.kinesis; + +import org.apache.beam.sdk.coders.AtomicCoder; +import org.apache.beam.sdk.coders.ByteArrayCoder; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.InstantCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.coders.VarLongCoder; + +import org.joda.time.Instant; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.ByteBuffer; + +/*** + * A {@link Coder} for {@link KinesisRecord}. + */ +class KinesisRecordCoder extends AtomicCoder { + private static final StringUtf8Coder STRING_CODER = StringUtf8Coder.of(); + private static final ByteArrayCoder BYTE_ARRAY_CODER = ByteArrayCoder.of(); + private static final InstantCoder INSTANT_CODER = InstantCoder.of(); + private static final VarLongCoder VAR_LONG_CODER = VarLongCoder.of(); + + public static KinesisRecordCoder of() { + return new KinesisRecordCoder(); + } + + @Override + public void encode(KinesisRecord value, OutputStream outStream, Context context) throws + IOException { + Context nested = context.nested(); + BYTE_ARRAY_CODER.encode(value.getData().array(), outStream, nested); + STRING_CODER.encode(value.getSequenceNumber(), outStream, nested); + STRING_CODER.encode(value.getPartitionKey(), outStream, nested); + INSTANT_CODER.encode(value.getApproximateArrivalTimestamp(), outStream, nested); + VAR_LONG_CODER.encode(value.getSubSequenceNumber(), outStream, nested); + INSTANT_CODER.encode(value.getReadTime(), outStream, nested); + STRING_CODER.encode(value.getStreamName(), outStream, nested); + STRING_CODER.encode(value.getShardId(), outStream, nested); + } + + @Override + public KinesisRecord decode(InputStream inStream, Context context) throws IOException { + Context nested = context.nested(); + ByteBuffer data = ByteBuffer.wrap(BYTE_ARRAY_CODER.decode(inStream, nested)); + String sequenceNumber = STRING_CODER.decode(inStream, nested); + String partitionKey = STRING_CODER.decode(inStream, nested); + Instant approximateArrivalTimestamp = INSTANT_CODER.decode(inStream, nested); + long subSequenceNumber = VAR_LONG_CODER.decode(inStream, nested); + Instant readTimestamp = INSTANT_CODER.decode(inStream, nested); + String streamName = STRING_CODER.decode(inStream, nested); + String shardId = STRING_CODER.decode(inStream, nested); + return new KinesisRecord(data, sequenceNumber, subSequenceNumber, partitionKey, + approximateArrivalTimestamp, readTimestamp, streamName, shardId + ); + } +} diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisSource.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisSource.java new file mode 100644 index 000000000000..38c9fa4072ef --- /dev/null +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisSource.java @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.kinesis; + +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.SerializableCoder; +import org.apache.beam.sdk.io.UnboundedSource; +import org.apache.beam.sdk.options.PipelineOptions; + + +import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.collect.Lists.newArrayList; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import java.util.List; + + +/*** + * Represents source for single stream in Kinesis. + */ +class KinesisSource extends UnboundedSource { + private static final Logger LOG = LoggerFactory.getLogger(KinesisSource.class); + + private final KinesisClientProvider kinesis; + private CheckpointGenerator initialCheckpointGenerator; + + public KinesisSource(KinesisClientProvider kinesis, String streamName, + StartingPoint startingPoint) { + this(kinesis, new DynamicCheckpointGenerator(streamName, startingPoint)); + } + + private KinesisSource(KinesisClientProvider kinesisClientProvider, + CheckpointGenerator initialCheckpoint) { + this.kinesis = kinesisClientProvider; + this.initialCheckpointGenerator = initialCheckpoint; + validate(); + } + + /*** + * Generate splits for reading from the stream. + * Basically, it'll try to evenly split set of shards in the stream into + * {@code desiredNumSplits} partitions. Each partition is then a split. + */ + @Override + public List generateInitialSplits(int desiredNumSplits, + PipelineOptions options) throws Exception { + KinesisReaderCheckpoint checkpoint = + initialCheckpointGenerator.generate(SimplifiedKinesisClient.from(kinesis)); + + List sources = newArrayList(); + + for (KinesisReaderCheckpoint partition : checkpoint.splitInto(desiredNumSplits)) { + sources.add(new KinesisSource( + kinesis, + new StaticCheckpointGenerator(partition))); + } + return sources; + } + + /*** + * Creates reader based on given {@link KinesisReaderCheckpoint}. + * If {@link KinesisReaderCheckpoint} is not given, then we use + * {@code initialCheckpointGenerator} to generate new checkpoint. + */ + @Override + public UnboundedReader createReader(PipelineOptions options, + KinesisReaderCheckpoint checkpointMark) { + + CheckpointGenerator checkpointGenerator = initialCheckpointGenerator; + + if (checkpointMark != null) { + checkpointGenerator = new StaticCheckpointGenerator(checkpointMark); + } + + LOG.info("Creating new reader using {}", checkpointGenerator); + + return new KinesisReader( + SimplifiedKinesisClient.from(kinesis), + checkpointGenerator, + this); + } + + @Override + public Coder getCheckpointMarkCoder() { + return SerializableCoder.of(KinesisReaderCheckpoint.class); + } + + @Override + public void validate() { + checkNotNull(kinesis); + checkNotNull(initialCheckpointGenerator); + } + + @Override + public Coder getDefaultOutputCoder() { + return KinesisRecordCoder.of(); + } +} diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/RecordFilter.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/RecordFilter.java new file mode 100644 index 000000000000..4c7f39ae8c61 --- /dev/null +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/RecordFilter.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.kinesis; + +import static com.google.common.collect.Lists.newArrayList; + +import java.util.List; + + +/** + * Filters out records, which were already processed and checkpointed. + *

              + * We need this step, because we can get iterators from Kinesis only with "sequenceNumber" accuracy, + * not with "subSequenceNumber" accuracy. + */ +class RecordFilter { + public List apply(List records, ShardCheckpoint checkpoint) { + List filteredRecords = newArrayList(); + for (KinesisRecord record : records) { + if (checkpoint.isBeforeOrAt(record)) { + filteredRecords.add(record); + } + } + return filteredRecords; + } +} diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/RoundRobin.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/RoundRobin.java new file mode 100644 index 000000000000..7257aa1492bb --- /dev/null +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/RoundRobin.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.kinesis; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.collect.Queues.newArrayDeque; + + +import java.util.Deque; +import java.util.Iterator; + +/*** + * Very simple implementation of round robin algorithm. + */ +class RoundRobin implements Iterable { + private final Deque deque; + + public RoundRobin(Iterable collection) { + this.deque = newArrayDeque(collection); + checkArgument(!deque.isEmpty(), "Tried to initialize RoundRobin with empty collection"); + } + + public T getCurrent() { + return deque.getFirst(); + } + + public void moveForward() { + deque.addLast(deque.removeFirst()); + } + + public int size() { + return deque.size(); + } + + @Override + public Iterator iterator() { + return deque.iterator(); + } +} diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardCheckpoint.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardCheckpoint.java new file mode 100644 index 000000000000..1d8628bc8851 --- /dev/null +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardCheckpoint.java @@ -0,0 +1,175 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.kinesis; + + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; + +import static com.amazonaws.services.kinesis.model.ShardIteratorType.AFTER_SEQUENCE_NUMBER; +import static com.amazonaws.services.kinesis.model.ShardIteratorType.AT_SEQUENCE_NUMBER; +import static com.amazonaws.services.kinesis.model.ShardIteratorType.AT_TIMESTAMP; +import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber; +import com.amazonaws.services.kinesis.model.Record; +import com.amazonaws.services.kinesis.model.ShardIteratorType; +import org.joda.time.Instant; +import java.io.Serializable; + + +/*** + * Checkpoint mark for single shard in the stream. + * Current position in the shard is determined by either: + *

                + *
              • {@link #shardIteratorType} if it is equal to {@link ShardIteratorType#LATEST} or + * {@link ShardIteratorType#TRIM_HORIZON}
              • + *
              • combination of + * {@link #sequenceNumber} and {@link #subSequenceNumber} if + * {@link ShardIteratorType#AFTER_SEQUENCE_NUMBER} or + * {@link ShardIteratorType#AT_SEQUENCE_NUMBER}
              • + *
              + * This class is immutable. + */ +class ShardCheckpoint implements Serializable { + private final String streamName; + private final String shardId; + private final String sequenceNumber; + private final ShardIteratorType shardIteratorType; + private final Long subSequenceNumber; + private final Instant timestamp; + + public ShardCheckpoint(String streamName, String shardId, StartingPoint + startingPoint) { + this(streamName, shardId, + ShardIteratorType.fromValue(startingPoint.getPositionName()), + startingPoint.getTimestamp()); + } + + public ShardCheckpoint(String streamName, String shardId, ShardIteratorType + shardIteratorType, Instant timestamp) { + this(streamName, shardId, shardIteratorType, null, null, timestamp); + } + + public ShardCheckpoint(String streamName, String shardId, ShardIteratorType + shardIteratorType, String sequenceNumber, Long subSequenceNumber) { + this(streamName, shardId, shardIteratorType, sequenceNumber, subSequenceNumber, null); + } + + private ShardCheckpoint(String streamName, String shardId, ShardIteratorType shardIteratorType, + String sequenceNumber, Long subSequenceNumber, Instant timestamp) { + this.shardIteratorType = checkNotNull(shardIteratorType, "shardIteratorType"); + this.streamName = checkNotNull(streamName, "streamName"); + this.shardId = checkNotNull(shardId, "shardId"); + if (shardIteratorType == AT_SEQUENCE_NUMBER || shardIteratorType == AFTER_SEQUENCE_NUMBER) { + checkNotNull(sequenceNumber, + "You must provide sequence number for AT_SEQUENCE_NUMBER" + + " or AFTER_SEQUENCE_NUMBER"); + } else { + checkArgument(sequenceNumber == null, + "Sequence number must be null for LATEST, TRIM_HORIZON or AT_TIMESTAMP"); + } + if (shardIteratorType == AT_TIMESTAMP) { + checkNotNull(timestamp, + "You must provide timestamp for AT_SEQUENCE_NUMBER" + + " or AFTER_SEQUENCE_NUMBER"); + } else { + checkArgument(timestamp == null, + "Timestamp must be null for an iterator type other than AT_TIMESTAMP"); + } + + this.subSequenceNumber = subSequenceNumber; + this.sequenceNumber = sequenceNumber; + this.timestamp = timestamp; + } + + /*** + * Used to compare {@link ShardCheckpoint} object to {@link KinesisRecord}. Depending + * on the the underlying shardIteratorType, it will either compare the timestamp or the + * {@link ExtendedSequenceNumber}. + * + * @param other + * @return if current checkpoint mark points before or at given {@link ExtendedSequenceNumber} + */ + public boolean isBeforeOrAt(KinesisRecord other) { + if (shardIteratorType == AT_TIMESTAMP) { + return timestamp.compareTo(other.getApproximateArrivalTimestamp()) <= 0; + } + int result = extendedSequenceNumber().compareTo(other.getExtendedSequenceNumber()); + if (result == 0) { + return shardIteratorType == AT_SEQUENCE_NUMBER; + } + return result < 0; + } + + private ExtendedSequenceNumber extendedSequenceNumber() { + String fullSequenceNumber = sequenceNumber; + if (fullSequenceNumber == null) { + fullSequenceNumber = shardIteratorType.toString(); + } + return new ExtendedSequenceNumber(fullSequenceNumber, subSequenceNumber); + } + + @Override + public String toString() { + return String.format("Checkpoint %s for stream %s, shard %s: %s", shardIteratorType, + streamName, shardId, + sequenceNumber); + } + + public ShardRecordsIterator getShardRecordsIterator(SimplifiedKinesisClient kinesis) + throws TransientKinesisException { + return new ShardRecordsIterator(this, kinesis); + } + + public String getShardIterator(SimplifiedKinesisClient kinesisClient) + throws TransientKinesisException { + if (checkpointIsInTheMiddleOfAUserRecord()) { + return kinesisClient.getShardIterator(streamName, + shardId, AT_SEQUENCE_NUMBER, + sequenceNumber, null); + } + return kinesisClient.getShardIterator(streamName, + shardId, shardIteratorType, + sequenceNumber, timestamp); + } + + private boolean checkpointIsInTheMiddleOfAUserRecord() { + return shardIteratorType == AFTER_SEQUENCE_NUMBER && subSequenceNumber != null; + } + + /*** + * Used to advance checkpoint mark to position after given {@link Record}. + * + * @param record + * @return new checkpoint object pointing directly after given {@link Record} + */ + public ShardCheckpoint moveAfter(KinesisRecord record) { + return new ShardCheckpoint( + streamName, shardId, + AFTER_SEQUENCE_NUMBER, + record.getSequenceNumber(), + record.getSubSequenceNumber()); + } + + public String getStreamName() { + return streamName; + } + + public String getShardId() { + return shardId; + } +} diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIterator.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIterator.java new file mode 100644 index 000000000000..7dfe158ca41d --- /dev/null +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIterator.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.kinesis; + +import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.collect.Queues.newArrayDeque; + +import com.amazonaws.services.kinesis.model.ExpiredIteratorException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import java.util.Deque; + +/*** + * Iterates over records in a single shard. + * Under the hood records are retrieved from Kinesis in batches and stored in the in-memory queue. + * Then the caller of {@link ShardRecordsIterator#next()} can read from queue one by one. + */ +class ShardRecordsIterator { + private static final Logger LOG = LoggerFactory.getLogger(ShardRecordsIterator.class); + + private final SimplifiedKinesisClient kinesis; + private final RecordFilter filter; + private ShardCheckpoint checkpoint; + private String shardIterator; + private Deque data = newArrayDeque(); + + public ShardRecordsIterator(final ShardCheckpoint initialCheckpoint, + SimplifiedKinesisClient simplifiedKinesisClient) throws + TransientKinesisException { + this(initialCheckpoint, simplifiedKinesisClient, new RecordFilter()); + } + + public ShardRecordsIterator(final ShardCheckpoint initialCheckpoint, + SimplifiedKinesisClient simplifiedKinesisClient, + RecordFilter filter) throws + TransientKinesisException { + + this.checkpoint = checkNotNull(initialCheckpoint, "initialCheckpoint"); + this.filter = checkNotNull(filter, "filter"); + this.kinesis = checkNotNull(simplifiedKinesisClient, "simplifiedKinesisClient"); + shardIterator = checkpoint.getShardIterator(kinesis); + } + + /*** + * Returns record if there's any present. + * Returns absent() if there are no new records at this time in the shard. + */ + public CustomOptional next() throws TransientKinesisException { + readMoreIfNecessary(); + + if (data.isEmpty()) { + return CustomOptional.absent(); + } else { + KinesisRecord record = data.removeFirst(); + checkpoint = checkpoint.moveAfter(record); + return CustomOptional.of(record); + } + } + + private void readMoreIfNecessary() throws TransientKinesisException { + if (data.isEmpty()) { + GetKinesisRecordsResult response; + try { + response = kinesis.getRecords(shardIterator, checkpoint.getStreamName(), + checkpoint.getShardId()); + } catch (ExpiredIteratorException e) { + LOG.info("Refreshing expired iterator", e); + shardIterator = checkpoint.getShardIterator(kinesis); + response = kinesis.getRecords(shardIterator, checkpoint.getStreamName(), + checkpoint.getShardId()); + } + LOG.debug("Fetched {} new records", response.getRecords().size()); + shardIterator = response.getNextShardIterator(); + data.addAll(filter.apply(response.getRecords(), checkpoint)); + } + } + + public ShardCheckpoint getCheckpoint() { + return checkpoint; + } + + +} diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClient.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClient.java new file mode 100644 index 000000000000..f9a1ea2af810 --- /dev/null +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClient.java @@ -0,0 +1,158 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.kinesis; + + +import com.google.common.collect.Lists; + +import com.amazonaws.AmazonServiceException; +import com.amazonaws.services.kinesis.AmazonKinesis; +import com.amazonaws.services.kinesis.clientlibrary.types.UserRecord; +import com.amazonaws.services.kinesis.model.ExpiredIteratorException; +import com.amazonaws.services.kinesis.model.GetRecordsRequest; +import com.amazonaws.services.kinesis.model.GetRecordsResult; +import com.amazonaws.services.kinesis.model.GetShardIteratorRequest; +import com.amazonaws.services.kinesis.model.LimitExceededException; +import com.amazonaws.services.kinesis.model.ProvisionedThroughputExceededException; +import com.amazonaws.services.kinesis.model.Shard; +import com.amazonaws.services.kinesis.model.ShardIteratorType; +import com.amazonaws.services.kinesis.model.StreamDescription; +import org.joda.time.Instant; +import java.util.Date; +import java.util.List; +import java.util.concurrent.Callable; + +/*** + * Wraps {@link AmazonKinesis} class providing much simpler interface and + * proper error handling. + */ +class SimplifiedKinesisClient { + private final AmazonKinesis kinesis; + + public SimplifiedKinesisClient(AmazonKinesis kinesis) { + this.kinesis = kinesis; + } + + public static SimplifiedKinesisClient from(KinesisClientProvider provider) { + return new SimplifiedKinesisClient(provider.get()); + } + + public String getShardIterator(final String streamName, final String shardId, + final ShardIteratorType shardIteratorType, + final String startingSequenceNumber, final Instant timestamp) + throws TransientKinesisException { + final Date date = timestamp != null ? timestamp.toDate() : null; + return wrapExceptions(new Callable() { + @Override + public String call() throws Exception { + return kinesis.getShardIterator(new GetShardIteratorRequest() + .withStreamName(streamName) + .withShardId(shardId) + .withShardIteratorType(shardIteratorType) + .withStartingSequenceNumber(startingSequenceNumber) + .withTimestamp(date) + ).getShardIterator(); + } + }); + } + + public List listShards(final String streamName) throws TransientKinesisException { + return wrapExceptions(new Callable>() { + @Override + public List call() throws Exception { + List shards = Lists.newArrayList(); + String lastShardId = null; + + StreamDescription description; + do { + description = kinesis.describeStream(streamName, lastShardId) + .getStreamDescription(); + + shards.addAll(description.getShards()); + lastShardId = shards.get(shards.size() - 1).getShardId(); + } while (description.getHasMoreShards()); + + return shards; + } + }); + } + + /*** + * Gets records from Kinesis and deaggregates them if needed. + * + * @return list of deaggregated records + * @throws TransientKinesisException - in case of recoverable situation + */ + public GetKinesisRecordsResult getRecords(String shardIterator, String streamName, + String shardId) throws TransientKinesisException { + return getRecords(shardIterator, streamName, shardId, null); + } + + /*** + * Gets records from Kinesis and deaggregates them if needed. + * + * @return list of deaggregated records + * @throws TransientKinesisException - in case of recoverable situation + */ + public GetKinesisRecordsResult getRecords(final String shardIterator, final String streamName, + final String shardId, final Integer limit) + throws + TransientKinesisException { + return wrapExceptions(new Callable() { + @Override + public GetKinesisRecordsResult call() throws Exception { + GetRecordsResult response = kinesis.getRecords(new GetRecordsRequest() + .withShardIterator(shardIterator) + .withLimit(limit)); + return new GetKinesisRecordsResult( + UserRecord.deaggregate(response.getRecords()), + response.getNextShardIterator(), + streamName, shardId); + } + }); + } + + /*** + * Wraps Amazon specific exceptions into more friendly format. + * + * @throws TransientKinesisException - in case of recoverable situation, i.e. + * the request rate is too high, Kinesis remote service + * failed, network issue, etc. + * @throws ExpiredIteratorException - if iterator needs to be refreshed + * @throws RuntimeException - in all other cases + */ + private T wrapExceptions(Callable callable) throws TransientKinesisException { + try { + return callable.call(); + } catch (ExpiredIteratorException e) { + throw e; + } catch (LimitExceededException | ProvisionedThroughputExceededException e) { + throw new TransientKinesisException( + "Too many requests to Kinesis. Wait some time and retry.", e); + } catch (AmazonServiceException e) { + if (e.getErrorType() == AmazonServiceException.ErrorType.Service) { + throw new TransientKinesisException( + "Kinesis backend failed. Wait some time and retry.", e); + } + throw new RuntimeException("Kinesis client side failure", e); + } catch (Exception e) { + throw new RuntimeException("Unknown kinesis failure, when trying to reach kinesis", e); + } + } + +} diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/StartingPoint.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/StartingPoint.java new file mode 100644 index 000000000000..8140269c170c --- /dev/null +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/StartingPoint.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.kinesis; + + +import static com.google.common.base.Preconditions.checkNotNull; + +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream; +import com.amazonaws.services.kinesis.model.ShardIteratorType; +import org.joda.time.Instant; +import java.io.Serializable; +import java.util.Objects; + +/*** + * Denotes a point at which the reader should start reading from a Kinesis stream. + * It can be expressed either as an {@link InitialPositionInStream} enum constant or a timestamp, + * in which case the reader will start reading at the specified point in time. + */ +class StartingPoint implements Serializable { + private final InitialPositionInStream position; + private final Instant timestamp; + + public StartingPoint(InitialPositionInStream position) { + this.position = checkNotNull(position, "position"); + this.timestamp = null; + } + + public StartingPoint(Instant timestamp) { + this.timestamp = checkNotNull(timestamp, "timestamp"); + this.position = null; + } + + public InitialPositionInStream getPosition() { + return position; + } + + public String getPositionName() { + return position != null ? position.name() : ShardIteratorType.AT_TIMESTAMP.name(); + } + + public Instant getTimestamp() { + return timestamp != null ? timestamp : null; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + StartingPoint that = (StartingPoint) o; + return position == that.position && Objects.equals(timestamp, that.timestamp); + } + + @Override + public int hashCode() { + return Objects.hash(position, timestamp); + } + + @Override + public String toString() { + if (timestamp == null) { + return position.toString(); + } else { + return "Starting at timestamp " + timestamp; + } + } +} diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/StaticCheckpointGenerator.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/StaticCheckpointGenerator.java new file mode 100644 index 000000000000..22dc9734f441 --- /dev/null +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/StaticCheckpointGenerator.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.kinesis; + +import static com.google.common.base.Preconditions.checkNotNull; + +/** + * Always returns the same instance of checkpoint. + */ +class StaticCheckpointGenerator implements CheckpointGenerator { + private final KinesisReaderCheckpoint checkpoint; + + public StaticCheckpointGenerator(KinesisReaderCheckpoint checkpoint) { + checkNotNull(checkpoint, "checkpoint"); + this.checkpoint = checkpoint; + } + + @Override + public KinesisReaderCheckpoint generate(SimplifiedKinesisClient client) { + return checkpoint; + } + + @Override + public String toString() { + return checkpoint.toString(); + } +} diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/TransientKinesisException.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/TransientKinesisException.java new file mode 100644 index 000000000000..a1a974bf3464 --- /dev/null +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/TransientKinesisException.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.kinesis; + +import com.amazonaws.AmazonServiceException; + +/** + * Created by p.pastuszka on 21.06.2016. + */ +class TransientKinesisException extends Exception { + public TransientKinesisException(String s, AmazonServiceException e) { + super(s, e); + } +} diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/package-info.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/package-info.java new file mode 100644 index 000000000000..5e37ef124988 --- /dev/null +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/package-info.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Tests for KinesisIO. + */ +package org.apache.beam.sdk.io.kinesis; diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/AmazonKinesisMock.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/AmazonKinesisMock.java new file mode 100644 index 000000000000..7ca8e0b4bd8d --- /dev/null +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/AmazonKinesisMock.java @@ -0,0 +1,375 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.kinesis; + +import static com.google.common.collect.Lists.newArrayList; +import static com.google.common.collect.Lists.transform; +import com.google.common.base.Function; + +import com.amazonaws.AmazonWebServiceRequest; +import com.amazonaws.ResponseMetadata; +import com.amazonaws.regions.Region; +import com.amazonaws.services.kinesis.AmazonKinesis; +import com.amazonaws.services.kinesis.model.AddTagsToStreamRequest; +import com.amazonaws.services.kinesis.model.AddTagsToStreamResult; +import com.amazonaws.services.kinesis.model.CreateStreamRequest; +import com.amazonaws.services.kinesis.model.CreateStreamResult; +import com.amazonaws.services.kinesis.model.DecreaseStreamRetentionPeriodRequest; +import com.amazonaws.services.kinesis.model.DecreaseStreamRetentionPeriodResult; +import com.amazonaws.services.kinesis.model.DeleteStreamRequest; +import com.amazonaws.services.kinesis.model.DeleteStreamResult; +import com.amazonaws.services.kinesis.model.DescribeStreamRequest; +import com.amazonaws.services.kinesis.model.DescribeStreamResult; +import com.amazonaws.services.kinesis.model.DisableEnhancedMonitoringRequest; +import com.amazonaws.services.kinesis.model.DisableEnhancedMonitoringResult; +import com.amazonaws.services.kinesis.model.EnableEnhancedMonitoringRequest; +import com.amazonaws.services.kinesis.model.EnableEnhancedMonitoringResult; +import com.amazonaws.services.kinesis.model.GetRecordsRequest; +import com.amazonaws.services.kinesis.model.GetRecordsResult; +import com.amazonaws.services.kinesis.model.GetShardIteratorRequest; +import com.amazonaws.services.kinesis.model.GetShardIteratorResult; +import com.amazonaws.services.kinesis.model.IncreaseStreamRetentionPeriodRequest; +import com.amazonaws.services.kinesis.model.IncreaseStreamRetentionPeriodResult; +import com.amazonaws.services.kinesis.model.ListStreamsRequest; +import com.amazonaws.services.kinesis.model.ListStreamsResult; +import com.amazonaws.services.kinesis.model.ListTagsForStreamRequest; +import com.amazonaws.services.kinesis.model.ListTagsForStreamResult; +import com.amazonaws.services.kinesis.model.MergeShardsRequest; +import com.amazonaws.services.kinesis.model.MergeShardsResult; +import com.amazonaws.services.kinesis.model.PutRecordRequest; +import com.amazonaws.services.kinesis.model.PutRecordResult; +import com.amazonaws.services.kinesis.model.PutRecordsRequest; +import com.amazonaws.services.kinesis.model.PutRecordsResult; +import com.amazonaws.services.kinesis.model.Record; +import com.amazonaws.services.kinesis.model.RemoveTagsFromStreamRequest; +import com.amazonaws.services.kinesis.model.RemoveTagsFromStreamResult; +import com.amazonaws.services.kinesis.model.Shard; +import com.amazonaws.services.kinesis.model.ShardIteratorType; +import com.amazonaws.services.kinesis.model.SplitShardRequest; +import com.amazonaws.services.kinesis.model.SplitShardResult; +import com.amazonaws.services.kinesis.model.StreamDescription; +import static org.apache.commons.lang.builder.HashCodeBuilder.reflectionHashCode; +import org.apache.commons.lang.builder.EqualsBuilder; +import org.joda.time.Instant; +import static java.lang.Integer.parseInt; +import static java.lang.Math.min; +import java.io.Serializable; +import java.nio.ByteBuffer; +import java.util.List; +import javax.annotation.Nullable; + +/** + * Created by p.pastuszka on 21.07.2016. + */ +class AmazonKinesisMock implements AmazonKinesis { + + static class TestData implements Serializable { + private final String data; + private final Instant arrivalTimestamp; + private final String sequenceNumber; + + public TestData(KinesisRecord record) { + this(new String(record.getData().array()), + record.getApproximateArrivalTimestamp(), + record.getSequenceNumber()); + } + + public TestData(String data, Instant arrivalTimestamp, String sequenceNumber) { + this.data = data; + this.arrivalTimestamp = arrivalTimestamp; + this.sequenceNumber = sequenceNumber; + } + + public Record convertToRecord() { + return new Record(). + withApproximateArrivalTimestamp(arrivalTimestamp.toDate()). + withData(ByteBuffer.wrap(data.getBytes())). + withSequenceNumber(sequenceNumber). + withPartitionKey(""); + } + + @Override + public boolean equals(Object obj) { + return EqualsBuilder.reflectionEquals(this, obj); + } + + @Override + public int hashCode() { + return reflectionHashCode(this); + } + } + + static class Provider implements KinesisClientProvider { + + private final List> shardedData; + private final int numberOfRecordsPerGet; + + public Provider(List> shardedData, int numberOfRecordsPerGet) { + this.shardedData = shardedData; + this.numberOfRecordsPerGet = numberOfRecordsPerGet; + } + + @Override + public AmazonKinesis get() { + return new AmazonKinesisMock(transform(shardedData, + new Function, List>() { + @Override + public List apply(@Nullable List testDatas) { + return transform(testDatas, new Function() { + @Override + public Record apply(@Nullable TestData testData) { + return testData.convertToRecord(); + } + }); + } + }), numberOfRecordsPerGet); + } + } + + private final List> shardedData; + private final int numberOfRecordsPerGet; + + public AmazonKinesisMock(List> shardedData, int numberOfRecordsPerGet) { + this.shardedData = shardedData; + this.numberOfRecordsPerGet = numberOfRecordsPerGet; + } + + @Override + public GetRecordsResult getRecords(GetRecordsRequest getRecordsRequest) { + String[] shardIteratorParts = getRecordsRequest.getShardIterator().split(":"); + int shardId = parseInt(shardIteratorParts[0]); + int startingRecord = parseInt(shardIteratorParts[1]); + List shardData = shardedData.get(shardId); + + int toIndex = min(startingRecord + numberOfRecordsPerGet, shardData.size()); + int fromIndex = min(startingRecord, toIndex); + return new GetRecordsResult(). + withRecords(shardData.subList(fromIndex, toIndex)). + withNextShardIterator(String.format("%s:%s", shardId, toIndex)); + } + + @Override + public GetShardIteratorResult getShardIterator( + GetShardIteratorRequest getShardIteratorRequest) { + ShardIteratorType shardIteratorType = ShardIteratorType.fromValue( + getShardIteratorRequest.getShardIteratorType()); + + String shardIterator; + if (shardIteratorType == ShardIteratorType.TRIM_HORIZON) { + shardIterator = String.format("%s:%s", getShardIteratorRequest.getShardId(), 0); + } else { + throw new RuntimeException("Not implemented"); + } + + return new GetShardIteratorResult().withShardIterator(shardIterator); + } + + @Override + public DescribeStreamResult describeStream(String streamName, String exclusiveStartShardId) { + int nextShardId = 0; + if (exclusiveStartShardId != null) { + nextShardId = parseInt(exclusiveStartShardId) + 1; + } + boolean hasMoreShards = nextShardId + 1 < shardedData.size(); + + List shards = newArrayList(); + if (nextShardId < shardedData.size()) { + shards.add(new Shard().withShardId(Integer.toString(nextShardId))); + } + + return new DescribeStreamResult().withStreamDescription( + new StreamDescription().withHasMoreShards(hasMoreShards).withShards(shards) + ); + } + + @Override + public void setEndpoint(String endpoint) { + + } + + @Override + public void setRegion(Region region) { + + } + + @Override + public AddTagsToStreamResult addTagsToStream(AddTagsToStreamRequest addTagsToStreamRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public CreateStreamResult createStream(CreateStreamRequest createStreamRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public CreateStreamResult createStream(String streamName, Integer shardCount) { + throw new RuntimeException("Not implemented"); + } + + @Override + public DecreaseStreamRetentionPeriodResult decreaseStreamRetentionPeriod( + DecreaseStreamRetentionPeriodRequest decreaseStreamRetentionPeriodRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public DeleteStreamResult deleteStream(DeleteStreamRequest deleteStreamRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public DeleteStreamResult deleteStream(String streamName) { + throw new RuntimeException("Not implemented"); + } + + @Override + public DescribeStreamResult describeStream(DescribeStreamRequest describeStreamRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public DescribeStreamResult describeStream(String streamName) { + + throw new RuntimeException("Not implemented"); + } + + @Override + public DescribeStreamResult describeStream(String streamName, + Integer limit, String exclusiveStartShardId) { + throw new RuntimeException("Not implemented"); + } + + @Override + public DisableEnhancedMonitoringResult disableEnhancedMonitoring( + DisableEnhancedMonitoringRequest disableEnhancedMonitoringRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public EnableEnhancedMonitoringResult enableEnhancedMonitoring( + EnableEnhancedMonitoringRequest enableEnhancedMonitoringRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public GetShardIteratorResult getShardIterator(String streamName, + String shardId, + String shardIteratorType) { + throw new RuntimeException("Not implemented"); + } + + @Override + public GetShardIteratorResult getShardIterator(String streamName, + String shardId, + String shardIteratorType, + String startingSequenceNumber) { + throw new RuntimeException("Not implemented"); + } + + @Override + public IncreaseStreamRetentionPeriodResult increaseStreamRetentionPeriod( + IncreaseStreamRetentionPeriodRequest increaseStreamRetentionPeriodRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public ListStreamsResult listStreams(ListStreamsRequest listStreamsRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public ListStreamsResult listStreams() { + throw new RuntimeException("Not implemented"); + } + + @Override + public ListStreamsResult listStreams(String exclusiveStartStreamName) { + throw new RuntimeException("Not implemented"); + } + + @Override + public ListStreamsResult listStreams(Integer limit, String exclusiveStartStreamName) { + throw new RuntimeException("Not implemented"); + } + + @Override + public ListTagsForStreamResult listTagsForStream( + ListTagsForStreamRequest listTagsForStreamRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public MergeShardsResult mergeShards(MergeShardsRequest mergeShardsRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public MergeShardsResult mergeShards(String streamName, + String shardToMerge, String adjacentShardToMerge) { + throw new RuntimeException("Not implemented"); + } + + @Override + public PutRecordResult putRecord(PutRecordRequest putRecordRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public PutRecordResult putRecord(String streamName, ByteBuffer data, String partitionKey) { + throw new RuntimeException("Not implemented"); + } + + @Override + public PutRecordResult putRecord(String streamName, ByteBuffer data, + String partitionKey, String sequenceNumberForOrdering) { + throw new RuntimeException("Not implemented"); + } + + @Override + public PutRecordsResult putRecords(PutRecordsRequest putRecordsRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public RemoveTagsFromStreamResult removeTagsFromStream( + RemoveTagsFromStreamRequest removeTagsFromStreamRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public SplitShardResult splitShard(SplitShardRequest splitShardRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public SplitShardResult splitShard(String streamName, + String shardToSplit, String newStartingHashKey) { + throw new RuntimeException("Not implemented"); + } + + @Override + public void shutdown() { + + } + + @Override + public ResponseMetadata getCachedResponseMetadata(AmazonWebServiceRequest request) { + throw new RuntimeException("Not implemented"); + } +} diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/CustomOptionalTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/CustomOptionalTest.java new file mode 100644 index 000000000000..152fd6dfde43 --- /dev/null +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/CustomOptionalTest.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.kinesis; + +import org.junit.Test; +import java.util.NoSuchElementException; + +/** + * Created by ppastuszka on 12.12.15. + */ +public class CustomOptionalTest { + @Test(expected = NoSuchElementException.class) + public void absentThrowsNoSuchElementExceptionOnGet() { + CustomOptional.absent().get(); + } +} diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGeneratorTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGeneratorTest.java new file mode 100644 index 000000000000..a9e5a69a2d51 --- /dev/null +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGeneratorTest.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.kinesis; + +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream; +import com.amazonaws.services.kinesis.model.Shard; +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.BDDMockito.given; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; +import static java.util.Arrays.asList; + + +/*** + */ +@RunWith(MockitoJUnitRunner.class) +public class DynamicCheckpointGeneratorTest { + + @Mock + private SimplifiedKinesisClient kinesisClient; + @Mock + private Shard shard1, shard2, shard3; + + @Test + public void shouldMapAllShardsToCheckpoints() throws Exception { + given(shard1.getShardId()).willReturn("shard-01"); + given(shard2.getShardId()).willReturn("shard-02"); + given(shard3.getShardId()).willReturn("shard-03"); + given(kinesisClient.listShards("stream")).willReturn(asList(shard1, shard2, shard3)); + + StartingPoint startingPoint = new StartingPoint(InitialPositionInStream.LATEST); + DynamicCheckpointGenerator underTest = new DynamicCheckpointGenerator("stream", + startingPoint); + + KinesisReaderCheckpoint checkpoint = underTest.generate(kinesisClient); + + assertThat(checkpoint).hasSize(3); + } +} diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisMockReadTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisMockReadTest.java new file mode 100644 index 000000000000..61a858f8dea5 --- /dev/null +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisMockReadTest.java @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.kinesis; + +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; +import static com.google.common.collect.Lists.newArrayList; + +import com.google.common.collect.Iterables; + +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream; +import org.joda.time.DateTime; +import org.junit.Test; +import java.util.List; + +/** + * Created by p.pastuszka on 22.07.2016. + */ +public class KinesisMockReadTest { + @Test + public void readsDataFromMockKinesis() { + int noOfShards = 3; + int noOfEventsPerShard = 100; + List> testData = + provideTestData(noOfShards, noOfEventsPerShard); + + final Pipeline p = TestPipeline.create(); + PCollection result = p. + apply( + KinesisIO.Read. + from("stream", InitialPositionInStream.TRIM_HORIZON). + using(new AmazonKinesisMock.Provider(testData, 10)). + withMaxNumRecords(noOfShards * noOfEventsPerShard)). + apply(ParDo.of(new KinesisRecordToTestData())); + PAssert.that(result).containsInAnyOrder(Iterables.concat(testData)); + p.run(); + } + + private static class KinesisRecordToTestData extends + DoFn { + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + c.output(new AmazonKinesisMock.TestData(c.element())); + } + } + + private List> provideTestData( + int noOfShards, + int noOfEventsPerShard) { + + int seqNumber = 0; + + List> shardedData = newArrayList(); + for (int i = 0; i < noOfShards; ++i) { + List shardData = newArrayList(); + shardedData.add(shardData); + + DateTime arrival = DateTime.now(); + for (int j = 0; j < noOfEventsPerShard; ++j) { + arrival = arrival.plusSeconds(1); + + seqNumber++; + shardData.add(new AmazonKinesisMock.TestData( + Integer.toString(seqNumber), + arrival.toInstant(), + Integer.toString(seqNumber)) + ); + } + } + + return shardedData; + } +} diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpointTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpointTest.java new file mode 100644 index 000000000000..205f050bb8ac --- /dev/null +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpointTest.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.kinesis; + + +import com.google.common.collect.Iterables; + +import static org.assertj.core.api.Assertions.assertThat; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; +import static java.util.Arrays.asList; +import java.util.Iterator; +import java.util.List; + +/*** + * + */ +@RunWith(MockitoJUnitRunner.class) +public class KinesisReaderCheckpointTest { + @Mock + private ShardCheckpoint a, b, c; + + private KinesisReaderCheckpoint checkpoint; + + @Before + public void setUp() { + checkpoint = new KinesisReaderCheckpoint(asList(a, b, c)); + } + + @Test + public void splitsCheckpointAccordingly() { + verifySplitInto(1); + verifySplitInto(2); + verifySplitInto(3); + verifySplitInto(4); + } + + @Test(expected = UnsupportedOperationException.class) + public void isImmutable() { + Iterator iterator = checkpoint.iterator(); + iterator.remove(); + } + + private void verifySplitInto(int size) { + List split = checkpoint.splitInto(size); + assertThat(Iterables.concat(split)).containsOnly(a, b, c); + assertThat(split).hasSize(Math.min(size, 3)); + } +} diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderIT.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderIT.java new file mode 100644 index 000000000000..fbc7c66aa3b9 --- /dev/null +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderIT.java @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.kinesis; + +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; +import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.collect.Lists.newArrayList; + +import com.amazonaws.regions.Regions; +import static org.assertj.core.api.Assertions.assertThat; +import org.apache.commons.lang.RandomStringUtils; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Ignore; +import org.junit.Test; +import static java.util.concurrent.Executors.newSingleThreadExecutor; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; + +/** + * Integration test, that reads from the real Kinesis. + * You need to provide all {@link KinesisTestOptions} in order to run this. + */ +public class KinesisReaderIT { + private static final long PIPELINE_STARTUP_TIME = TimeUnit.SECONDS.toMillis(10); + private ExecutorService singleThreadExecutor = newSingleThreadExecutor(); + + + @Ignore + @Test + public void readsDataFromRealKinesisStream() + throws IOException, InterruptedException, ExecutionException { + KinesisTestOptions options = readKinesisOptions(); + List testData = prepareTestData(1000); + + Future future = startTestPipeline(testData, options); + KinesisUploader.uploadAll(testData, options); + future.get(); + } + + private List prepareTestData(int count) { + List data = newArrayList(); + for (int i = 0; i < count; ++i) { + data.add(RandomStringUtils.randomAlphabetic(32)); + } + return data; + } + + private Future startTestPipeline(List testData, KinesisTestOptions options) + throws InterruptedException { + final Pipeline p = TestPipeline.create(); + PCollection result = p. + apply(KinesisIO.Read. + from(options.getAwsKinesisStream(), Instant.now()). + using(options.getAwsAccessKey(), options.getAwsSecretKey(), + Regions.fromName(options.getAwsKinesisRegion())). + withMaxReadTime(Duration.standardMinutes(3)) + ). + apply(ParDo.of(new RecordDataToString())); + PAssert.that(result).containsInAnyOrder(testData); + + Future future = singleThreadExecutor.submit(new Callable() { + @Override + public Void call() throws Exception { + PipelineResult result = p.run(); + PipelineResult.State state = result.getState(); + while (state != PipelineResult.State.DONE && state != PipelineResult.State.FAILED) { + Thread.sleep(1000); + state = result.getState(); + } + assertThat(state).isEqualTo(PipelineResult.State.DONE); + return null; + } + }); + Thread.sleep(PIPELINE_STARTUP_TIME); + return future; + } + + private KinesisTestOptions readKinesisOptions() { + PipelineOptionsFactory.register(KinesisTestOptions.class); + return TestPipeline.testingPipelineOptions().as(KinesisTestOptions.class); + } + + private static class RecordDataToString extends DoFn { + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + checkNotNull(c.element(), "Null record given"); + c.output(new String(c.element().getData().array(), StandardCharsets.UTF_8)); + } + } +} diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderTest.java new file mode 100644 index 000000000000..793fb579c26b --- /dev/null +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderTest.java @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.kinesis; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.when; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; +import static java.util.Arrays.asList; +import java.io.IOException; +import java.util.NoSuchElementException; + +/** + * Created by ppastuszka on 12.12.15. + */ +@RunWith(MockitoJUnitRunner.class) +public class KinesisReaderTest { + @Mock + private SimplifiedKinesisClient kinesis; + @Mock + private CheckpointGenerator generator; + @Mock + private ShardCheckpoint firstCheckpoint, secondCheckpoint; + @Mock + private ShardRecordsIterator firstIterator, secondIterator; + @Mock + private KinesisRecord a, b, c, d; + + private KinesisReader reader; + + @Before + public void setUp() throws IOException, TransientKinesisException { + when(generator.generate(kinesis)).thenReturn(new KinesisReaderCheckpoint( + asList(firstCheckpoint, secondCheckpoint) + )); + when(firstCheckpoint.getShardRecordsIterator(kinesis)).thenReturn(firstIterator); + when(secondCheckpoint.getShardRecordsIterator(kinesis)).thenReturn(secondIterator); + when(firstIterator.next()).thenReturn(CustomOptional.absent()); + when(secondIterator.next()).thenReturn(CustomOptional.absent()); + + reader = new KinesisReader(kinesis, generator, null); + } + + @Test + public void startReturnsFalseIfNoDataAtTheBeginning() throws IOException { + assertThat(reader.start()).isFalse(); + } + + @Test(expected = NoSuchElementException.class) + public void throwsNoSuchElementExceptionIfNoData() throws IOException { + reader.start(); + reader.getCurrent(); + } + + @Test + public void startReturnsTrueIfSomeDataAvailable() throws IOException, + TransientKinesisException { + when(firstIterator.next()). + thenReturn(CustomOptional.of(a)). + thenReturn(CustomOptional.absent()); + + assertThat(reader.start()).isTrue(); + } + + @Test + public void advanceReturnsFalseIfThereIsTransientExceptionInKinesis() + throws IOException, TransientKinesisException { + reader.start(); + + when(firstIterator.next()).thenThrow(TransientKinesisException.class); + + assertThat(reader.advance()).isFalse(); + } + + @Test + public void readsThroughAllDataAvailable() throws IOException, TransientKinesisException { + when(firstIterator.next()). + thenReturn(CustomOptional.absent()). + thenReturn(CustomOptional.of(a)). + thenReturn(CustomOptional.absent()). + thenReturn(CustomOptional.of(b)). + thenReturn(CustomOptional.absent()); + + when(secondIterator.next()). + thenReturn(CustomOptional.of(c)). + thenReturn(CustomOptional.absent()). + thenReturn(CustomOptional.of(d)). + thenReturn(CustomOptional.absent()); + + assertThat(reader.start()).isTrue(); + assertThat(reader.getCurrent()).isEqualTo(c); + assertThat(reader.advance()).isTrue(); + assertThat(reader.getCurrent()).isEqualTo(a); + assertThat(reader.advance()).isTrue(); + assertThat(reader.getCurrent()).isEqualTo(d); + assertThat(reader.advance()).isTrue(); + assertThat(reader.getCurrent()).isEqualTo(b); + assertThat(reader.advance()).isFalse(); + } + +} diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoderTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoderTest.java new file mode 100644 index 000000000000..b09b7eb9b378 --- /dev/null +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoderTest.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.kinesis; + +import org.apache.beam.sdk.testing.CoderProperties; + +import org.joda.time.Instant; +import org.junit.Test; +import java.nio.ByteBuffer; + +/** + * Created by p.pastuszka on 20.07.2016. + */ +public class KinesisRecordCoderTest { + @Test + public void encodingAndDecodingWorks() throws Exception { + KinesisRecord record = new KinesisRecord( + ByteBuffer.wrap("data".getBytes()), + "sequence", + 128L, + "partition", + Instant.now(), + Instant.now(), + "stream", + "shard" + ); + CoderProperties.coderDecodeEncodeEqual( + new KinesisRecordCoder(), record + ); + } +} diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisTestOptions.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisTestOptions.java new file mode 100644 index 000000000000..65a7605ea46a --- /dev/null +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisTestOptions.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.kinesis; + +import org.apache.beam.sdk.options.Default; +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.testing.TestPipelineOptions; + +/*** + * Options for Kinesis integration tests. + */ +public interface KinesisTestOptions extends TestPipelineOptions { + @Description("AWS region where Kinesis stream resided") + @Default.String("aws-kinesis-region") + String getAwsKinesisRegion(); + void setAwsKinesisRegion(String value); + + @Description("Kinesis stream name") + @Default.String("aws-kinesis-stream") + String getAwsKinesisStream(); + void setAwsKinesisStream(String value); + + @Description("AWS secret key") + @Default.String("aws-secret-key") + String getAwsSecretKey(); + void setAwsSecretKey(String value); + + @Description("AWS access key") + @Default.String("aws-access-key") + String getAwsAccessKey(); + void setAwsAccessKey(String value); +} diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisUploader.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisUploader.java new file mode 100644 index 000000000000..0dcede985671 --- /dev/null +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisUploader.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.kinesis; + +import static com.google.common.collect.Lists.newArrayList; +import com.google.common.base.Charsets; +import com.google.common.collect.Lists; + +import com.amazonaws.auth.BasicAWSCredentials; +import com.amazonaws.internal.StaticCredentialsProvider; +import com.amazonaws.regions.Regions; +import com.amazonaws.services.kinesis.AmazonKinesis; +import com.amazonaws.services.kinesis.AmazonKinesisClient; +import com.amazonaws.services.kinesis.model.PutRecordsRequest; +import com.amazonaws.services.kinesis.model.PutRecordsRequestEntry; +import com.amazonaws.services.kinesis.model.PutRecordsResult; +import com.amazonaws.services.kinesis.model.PutRecordsResultEntry; +import java.nio.ByteBuffer; +import java.util.List; + +/*** + * Sends records to Kinesis in reliable way. + */ +public class KinesisUploader { + + public static final int MAX_NUMBER_OF_RECORDS_IN_BATCH = 499; + + public static void uploadAll(List data, KinesisTestOptions options) { + AmazonKinesis client = new AmazonKinesisClient( + new StaticCredentialsProvider( + new BasicAWSCredentials( + options.getAwsAccessKey(), options.getAwsSecretKey())) + ).withRegion(Regions.fromName(options.getAwsKinesisRegion())); + + List> partitions = Lists.partition(data, MAX_NUMBER_OF_RECORDS_IN_BATCH); + + + for (List partition : partitions) { + List allRecords = newArrayList(); + for (String row : partition) { + allRecords.add(new PutRecordsRequestEntry(). + withData(ByteBuffer.wrap(row.getBytes(Charsets.UTF_8))). + withPartitionKey(Integer.toString(row.hashCode())) + + ); + } + + PutRecordsResult result; + do { + result = client.putRecords( + new PutRecordsRequest(). + withStreamName(options.getAwsKinesisStream()). + withRecords(allRecords)); + List failedRecords = newArrayList(); + int i = 0; + for (PutRecordsResultEntry row : result.getRecords()) { + if (row.getErrorCode() != null) { + failedRecords.add(allRecords.get(i)); + } + ++i; + } + allRecords = failedRecords; + } + + while (result.getFailedRecordCount() > 0); + } + } + +} diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RecordFilterTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RecordFilterTest.java new file mode 100644 index 000000000000..360106d00c4b --- /dev/null +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RecordFilterTest.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.kinesis; + +import com.google.common.collect.Lists; + +import static org.mockito.BDDMockito.given; +import org.assertj.core.api.Assertions; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; +import java.util.Collections; +import java.util.List; + + +/*** + */ +@RunWith(MockitoJUnitRunner.class) +public class RecordFilterTest { + @Mock + private ShardCheckpoint checkpoint; + @Mock + private KinesisRecord record1, record2, record3, record4, record5; + + @Test + public void shouldFilterOutRecordsBeforeOrAtCheckpoint() { + given(checkpoint.isBeforeOrAt(record1)).willReturn(false); + given(checkpoint.isBeforeOrAt(record2)).willReturn(true); + given(checkpoint.isBeforeOrAt(record3)).willReturn(true); + given(checkpoint.isBeforeOrAt(record4)).willReturn(false); + given(checkpoint.isBeforeOrAt(record5)).willReturn(true); + List records = Lists.newArrayList(record1, record2, + record3, record4, record5); + RecordFilter underTest = new RecordFilter(); + + List retainedRecords = underTest.apply(records, checkpoint); + + Assertions.assertThat(retainedRecords).containsOnly(record2, record3, record5); + } + + @Test + public void shouldNotFailOnEmptyList() { + List records = Collections.emptyList(); + RecordFilter underTest = new RecordFilter(); + + List retainedRecords = underTest.apply(records, checkpoint); + + Assertions.assertThat(retainedRecords).isEmpty(); + } +} diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RoundRobinTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RoundRobinTest.java new file mode 100644 index 000000000000..a508ddf5ce0f --- /dev/null +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RoundRobinTest.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.kinesis; + +import static com.google.common.collect.Lists.newArrayList; + +import static org.assertj.core.api.Assertions.assertThat; +import org.junit.Test; +import java.util.Collections; +import java.util.List; + +/** + * Created by ppastuszka on 12.12.15. + */ +public class RoundRobinTest { + @Test(expected = IllegalArgumentException.class) + public void doesNotAllowCreationWithEmptyCollection() { + new RoundRobin<>(Collections.emptyList()); + } + + @Test + public void goesThroughElementsInCycle() { + List input = newArrayList("a", "b", "c"); + + RoundRobin roundRobin = new RoundRobin<>(newArrayList(input)); + + input.addAll(input); // duplicate the input + for (String element : input) { + assertThat(roundRobin.getCurrent()).isEqualTo(element); + assertThat(roundRobin.getCurrent()).isEqualTo(element); + roundRobin.moveForward(); + } + } + + @Test + public void usualIteratorGoesThroughElementsOnce() { + List input = newArrayList("a", "b", "c"); + + RoundRobin roundRobin = new RoundRobin<>(input); + assertThat(roundRobin).hasSize(3).containsOnly(input.toArray(new String[0])); + } +} diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardCheckpointTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardCheckpointTest.java new file mode 100644 index 000000000000..2227cef63828 --- /dev/null +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardCheckpointTest.java @@ -0,0 +1,148 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.kinesis; + +import static com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream.LATEST; +import static com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream.TRIM_HORIZON; +import static com.amazonaws.services.kinesis.model.ShardIteratorType.AFTER_SEQUENCE_NUMBER; +import static com.amazonaws.services.kinesis.model.ShardIteratorType.AT_SEQUENCE_NUMBER; +import static com.amazonaws.services.kinesis.model.ShardIteratorType.AT_TIMESTAMP; +import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber; +import com.amazonaws.services.kinesis.model.ShardIteratorType; +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.BDDMockito.given; +import static org.mockito.Matchers.anyString; +import static org.mockito.Matchers.eq; +import static org.mockito.Matchers.isNull; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import org.joda.time.DateTime; +import org.joda.time.Instant; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; +import java.io.IOException; + +/** + * + */ +@RunWith(MockitoJUnitRunner.class) +public class ShardCheckpointTest { + private static final String AT_SEQUENCE_SHARD_IT = "AT_SEQUENCE_SHARD_IT"; + private static final String AFTER_SEQUENCE_SHARD_IT = "AFTER_SEQUENCE_SHARD_IT"; + private static final String STREAM_NAME = "STREAM"; + private static final String SHARD_ID = "SHARD_ID"; + @Mock + private SimplifiedKinesisClient client; + + @Before + public void setUp() throws IOException, TransientKinesisException { + when(client.getShardIterator( + eq(STREAM_NAME), eq(SHARD_ID), eq(AT_SEQUENCE_NUMBER), + anyString(), isNull(Instant.class))). + thenReturn(AT_SEQUENCE_SHARD_IT); + when(client.getShardIterator( + eq(STREAM_NAME), eq(SHARD_ID), eq(AFTER_SEQUENCE_NUMBER), + anyString(), isNull(Instant.class))). + thenReturn(AFTER_SEQUENCE_SHARD_IT); + } + + @Test + public void testProvidingShardIterator() throws IOException, TransientKinesisException { + assertThat(checkpoint(AT_SEQUENCE_NUMBER, "100", null).getShardIterator(client)) + .isEqualTo(AT_SEQUENCE_SHARD_IT); + assertThat(checkpoint(AFTER_SEQUENCE_NUMBER, "100", null).getShardIterator(client)) + .isEqualTo(AFTER_SEQUENCE_SHARD_IT); + assertThat(checkpoint(AT_SEQUENCE_NUMBER, "100", 10L).getShardIterator(client)).isEqualTo + (AT_SEQUENCE_SHARD_IT); + assertThat(checkpoint(AFTER_SEQUENCE_NUMBER, "100", 10L).getShardIterator(client)) + .isEqualTo(AT_SEQUENCE_SHARD_IT); + } + + @Test + public void testComparisonWithExtendedSequenceNumber() { + assertThat(new ShardCheckpoint("", "", new StartingPoint(LATEST)).isBeforeOrAt( + recordWith(new ExtendedSequenceNumber("100", 0L)) + )).isTrue(); + + assertThat(new ShardCheckpoint("", "", new StartingPoint(TRIM_HORIZON)).isBeforeOrAt( + recordWith(new ExtendedSequenceNumber("100", 0L)) + )).isTrue(); + + assertThat(checkpoint(AFTER_SEQUENCE_NUMBER, "10", 1L).isBeforeOrAt( + recordWith(new ExtendedSequenceNumber("100", 0L)) + )).isTrue(); + + assertThat(checkpoint(AT_SEQUENCE_NUMBER, "100", 0L).isBeforeOrAt( + recordWith(new ExtendedSequenceNumber("100", 0L)) + )).isTrue(); + + assertThat(checkpoint(AFTER_SEQUENCE_NUMBER, "100", 0L).isBeforeOrAt( + recordWith(new ExtendedSequenceNumber("100", 0L)) + )).isFalse(); + + assertThat(checkpoint(AT_SEQUENCE_NUMBER, "100", 1L).isBeforeOrAt( + recordWith(new ExtendedSequenceNumber("100", 0L)) + )).isFalse(); + + assertThat(checkpoint(AFTER_SEQUENCE_NUMBER, "100", 0L).isBeforeOrAt( + recordWith(new ExtendedSequenceNumber("99", 1L)) + )).isFalse(); + } + + @Test + public void testComparisonWithTimestamp() { + DateTime referenceTimestamp = DateTime.now(); + + assertThat(checkpoint(AT_TIMESTAMP, referenceTimestamp.toInstant()) + .isBeforeOrAt(recordWith(referenceTimestamp.minusMillis(10).toInstant())) + ).isFalse(); + + assertThat(checkpoint(AT_TIMESTAMP, referenceTimestamp.toInstant()) + .isBeforeOrAt(recordWith(referenceTimestamp.toInstant())) + ).isTrue(); + + assertThat(checkpoint(AT_TIMESTAMP, referenceTimestamp.toInstant()) + .isBeforeOrAt(recordWith(referenceTimestamp.plusMillis(10).toInstant())) + ).isTrue(); + } + + private KinesisRecord recordWith(ExtendedSequenceNumber extendedSequenceNumber) { + KinesisRecord record = mock(KinesisRecord.class); + given(record.getExtendedSequenceNumber()).willReturn(extendedSequenceNumber); + return record; + } + + private ShardCheckpoint checkpoint(ShardIteratorType iteratorType, String sequenceNumber, + Long subSequenceNumber) { + return new ShardCheckpoint(STREAM_NAME, SHARD_ID, iteratorType, sequenceNumber, + subSequenceNumber); + } + + private KinesisRecord recordWith(Instant approximateArrivalTimestamp) { + KinesisRecord record = mock(KinesisRecord.class); + given(record.getApproximateArrivalTimestamp()).willReturn(approximateArrivalTimestamp); + return record; + } + + private ShardCheckpoint checkpoint(ShardIteratorType iteratorType, Instant timestamp) { + return new ShardCheckpoint(STREAM_NAME, SHARD_ID, iteratorType, timestamp); + } +} diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIteratorTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIteratorTest.java new file mode 100644 index 000000000000..e2a3ccc0057e --- /dev/null +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIteratorTest.java @@ -0,0 +1,150 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.kinesis; + +import com.amazonaws.services.kinesis.model.ExpiredIteratorException; +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyListOf; +import static org.mockito.Mockito.when; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.runners.MockitoJUnitRunner; +import org.mockito.stubbing.Answer; +import static java.util.Arrays.asList; +import static java.util.Collections.singletonList; +import java.io.IOException; +import java.util.Collections; + +/** + * Created by ppastuszka on 12.12.15. + */ +@RunWith(MockitoJUnitRunner.class) +public class ShardRecordsIteratorTest { + private static final String INITIAL_ITERATOR = "INITIAL_ITERATOR"; + private static final String SECOND_ITERATOR = "SECOND_ITERATOR"; + private static final String SECOND_REFRESHED_ITERATOR = "SECOND_REFRESHED_ITERATOR"; + private static final String THIRD_ITERATOR = "THIRD_ITERATOR"; + private static final String STREAM_NAME = "STREAM_NAME"; + private static final String SHARD_ID = "SHARD_ID"; + + @Mock + private SimplifiedKinesisClient kinesisClient; + @Mock + private ShardCheckpoint firstCheckpoint, aCheckpoint, bCheckpoint, cCheckpoint, dCheckpoint; + @Mock + private GetKinesisRecordsResult firstResult, secondResult, thirdResult; + @Mock + private KinesisRecord a, b, c, d; + @Mock + private RecordFilter recordFilter; + + private ShardRecordsIterator iterator; + + @Before + public void setUp() throws IOException, TransientKinesisException { + when(firstCheckpoint.getShardIterator(kinesisClient)).thenReturn(INITIAL_ITERATOR); + when(firstCheckpoint.getStreamName()).thenReturn(STREAM_NAME); + when(firstCheckpoint.getShardId()).thenReturn(SHARD_ID); + + when(firstCheckpoint.moveAfter(a)).thenReturn(aCheckpoint); + when(aCheckpoint.moveAfter(b)).thenReturn(bCheckpoint); + when(aCheckpoint.getStreamName()).thenReturn(STREAM_NAME); + when(aCheckpoint.getShardId()).thenReturn(SHARD_ID); + when(bCheckpoint.moveAfter(c)).thenReturn(cCheckpoint); + when(bCheckpoint.getStreamName()).thenReturn(STREAM_NAME); + when(bCheckpoint.getShardId()).thenReturn(SHARD_ID); + when(cCheckpoint.moveAfter(d)).thenReturn(dCheckpoint); + when(cCheckpoint.getStreamName()).thenReturn(STREAM_NAME); + when(cCheckpoint.getShardId()).thenReturn(SHARD_ID); + when(dCheckpoint.getStreamName()).thenReturn(STREAM_NAME); + when(dCheckpoint.getShardId()).thenReturn(SHARD_ID); + + when(kinesisClient.getRecords(INITIAL_ITERATOR, STREAM_NAME, SHARD_ID)) + .thenReturn(firstResult); + when(kinesisClient.getRecords(SECOND_ITERATOR, STREAM_NAME, SHARD_ID)) + .thenReturn(secondResult); + when(kinesisClient.getRecords(THIRD_ITERATOR, STREAM_NAME, SHARD_ID)) + .thenReturn(thirdResult); + + when(firstResult.getNextShardIterator()).thenReturn(SECOND_ITERATOR); + when(secondResult.getNextShardIterator()).thenReturn(THIRD_ITERATOR); + when(thirdResult.getNextShardIterator()).thenReturn(THIRD_ITERATOR); + + when(firstResult.getRecords()).thenReturn(Collections.emptyList()); + when(secondResult.getRecords()).thenReturn(Collections.emptyList()); + when(thirdResult.getRecords()).thenReturn(Collections.emptyList()); + + when(recordFilter.apply(anyListOf(KinesisRecord.class), any(ShardCheckpoint + .class))).thenAnswer(new IdentityAnswer()); + + iterator = new ShardRecordsIterator(firstCheckpoint, kinesisClient, recordFilter); + } + + @Test + public void returnsAbsentIfNoRecordsPresent() throws IOException, TransientKinesisException { + assertThat(iterator.next()).isEqualTo(CustomOptional.absent()); + assertThat(iterator.next()).isEqualTo(CustomOptional.absent()); + assertThat(iterator.next()).isEqualTo(CustomOptional.absent()); + } + + @Test + public void goesThroughAvailableRecords() throws IOException, TransientKinesisException { + when(firstResult.getRecords()).thenReturn(asList(a, b, c)); + when(secondResult.getRecords()).thenReturn(singletonList(d)); + + assertThat(iterator.getCheckpoint()).isEqualTo(firstCheckpoint); + assertThat(iterator.next()).isEqualTo(CustomOptional.of(a)); + assertThat(iterator.getCheckpoint()).isEqualTo(aCheckpoint); + assertThat(iterator.next()).isEqualTo(CustomOptional.of(b)); + assertThat(iterator.getCheckpoint()).isEqualTo(bCheckpoint); + assertThat(iterator.next()).isEqualTo(CustomOptional.of(c)); + assertThat(iterator.getCheckpoint()).isEqualTo(cCheckpoint); + assertThat(iterator.next()).isEqualTo(CustomOptional.of(d)); + assertThat(iterator.getCheckpoint()).isEqualTo(dCheckpoint); + assertThat(iterator.next()).isEqualTo(CustomOptional.absent()); + assertThat(iterator.getCheckpoint()).isEqualTo(dCheckpoint); + } + + @Test + public void refreshesExpiredIterator() throws IOException, TransientKinesisException { + when(firstResult.getRecords()).thenReturn(singletonList(a)); + when(secondResult.getRecords()).thenReturn(singletonList(b)); + + when(kinesisClient.getRecords(SECOND_ITERATOR, STREAM_NAME, SHARD_ID)) + .thenThrow(ExpiredIteratorException.class); + when(aCheckpoint.getShardIterator(kinesisClient)) + .thenReturn(SECOND_REFRESHED_ITERATOR); + when(kinesisClient.getRecords(SECOND_REFRESHED_ITERATOR, STREAM_NAME, SHARD_ID)) + .thenReturn(secondResult); + + assertThat(iterator.next()).isEqualTo(CustomOptional.of(a)); + assertThat(iterator.next()).isEqualTo(CustomOptional.of(b)); + assertThat(iterator.next()).isEqualTo(CustomOptional.absent()); + } + + private static class IdentityAnswer implements Answer { + @Override + public Object answer(InvocationOnMock invocation) throws Throwable { + return invocation.getArguments()[0]; + } + } +} diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClientTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClientTest.java new file mode 100644 index 000000000000..44d29d6cf8fd --- /dev/null +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClientTest.java @@ -0,0 +1,223 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.kinesis; + +import com.amazonaws.AmazonServiceException; +import com.amazonaws.AmazonServiceException.ErrorType; +import com.amazonaws.services.kinesis.AmazonKinesis; +import com.amazonaws.services.kinesis.model.DescribeStreamResult; +import com.amazonaws.services.kinesis.model.ExpiredIteratorException; +import com.amazonaws.services.kinesis.model.GetShardIteratorRequest; +import com.amazonaws.services.kinesis.model.GetShardIteratorResult; +import com.amazonaws.services.kinesis.model.LimitExceededException; +import com.amazonaws.services.kinesis.model.ProvisionedThroughputExceededException; +import com.amazonaws.services.kinesis.model.Shard; +import com.amazonaws.services.kinesis.model.ShardIteratorType; +import com.amazonaws.services.kinesis.model.StreamDescription; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.failBecauseExceptionWasNotThrown; +import static org.mockito.BDDMockito.given; +import static org.mockito.Mockito.reset; +import org.joda.time.Instant; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.InjectMocks; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; +import java.util.List; + +/*** + */ +@RunWith(MockitoJUnitRunner.class) +public class SimplifiedKinesisClientTest { + private static final String STREAM = "stream"; + private static final String SHARD_1 = "shard-01"; + private static final String SHARD_2 = "shard-02"; + private static final String SHARD_3 = "shard-03"; + private static final String SHARD_ITERATOR = "iterator"; + private static final String SEQUENCE_NUMBER = "abc123"; + + @Mock + private AmazonKinesis kinesis; + @InjectMocks + private SimplifiedKinesisClient underTest; + + @Test + public void shouldReturnIteratorStartingWithSequenceNumber() throws Exception { + given(kinesis.getShardIterator(new GetShardIteratorRequest() + .withStreamName(STREAM) + .withShardId(SHARD_1) + .withShardIteratorType(ShardIteratorType.AT_SEQUENCE_NUMBER) + .withStartingSequenceNumber(SEQUENCE_NUMBER) + )).willReturn(new GetShardIteratorResult() + .withShardIterator(SHARD_ITERATOR)); + + String stream = underTest.getShardIterator(STREAM, SHARD_1, + ShardIteratorType.AT_SEQUENCE_NUMBER, SEQUENCE_NUMBER, null); + + assertThat(stream).isEqualTo(SHARD_ITERATOR); + } + + @Test + public void shouldReturnIteratorStartingWithTimestamp() throws Exception { + Instant timestamp = Instant.now(); + given(kinesis.getShardIterator(new GetShardIteratorRequest() + .withStreamName(STREAM) + .withShardId(SHARD_1) + .withShardIteratorType(ShardIteratorType.AT_SEQUENCE_NUMBER) + .withTimestamp(timestamp.toDate()) + )).willReturn(new GetShardIteratorResult() + .withShardIterator(SHARD_ITERATOR)); + + String stream = underTest.getShardIterator(STREAM, SHARD_1, + ShardIteratorType.AT_SEQUENCE_NUMBER, null, timestamp); + + assertThat(stream).isEqualTo(SHARD_ITERATOR); + } + + @Test + public void shouldHandleExpiredIterationExceptionForGetShardIterator() { + shouldHandleGetShardIteratorError(new ExpiredIteratorException(""), + ExpiredIteratorException.class); + } + + @Test + public void shouldHandleLimitExceededExceptionForGetShardIterator() { + shouldHandleGetShardIteratorError(new LimitExceededException(""), + TransientKinesisException.class); + } + + @Test + public void shouldHandleProvisionedThroughputExceededExceptionForGetShardIterator() { + shouldHandleGetShardIteratorError(new ProvisionedThroughputExceededException(""), + TransientKinesisException.class); + } + + @Test + public void shouldHandleServiceErrorForGetShardIterator() { + shouldHandleGetShardIteratorError(newAmazonServiceException(ErrorType.Service), + TransientKinesisException.class); + } + + @Test + public void shouldHandleClientErrorForGetShardIterator() { + shouldHandleGetShardIteratorError(newAmazonServiceException(ErrorType.Client), + RuntimeException.class); + } + + @Test + public void shouldHandleUnexpectedExceptionForGetShardIterator() { + shouldHandleGetShardIteratorError(new NullPointerException(), + RuntimeException.class); + } + + private void shouldHandleGetShardIteratorError( + Exception thrownException, + Class expectedExceptionClass) { + GetShardIteratorRequest request = new GetShardIteratorRequest() + .withStreamName(STREAM) + .withShardId(SHARD_1) + .withShardIteratorType(ShardIteratorType.LATEST); + + given(kinesis.getShardIterator(request)).willThrow(thrownException); + + try { + underTest.getShardIterator(STREAM, SHARD_1, ShardIteratorType.LATEST, null, null); + failBecauseExceptionWasNotThrown(expectedExceptionClass); + } catch (Exception e) { + assertThat(e).isExactlyInstanceOf(expectedExceptionClass); + } finally { + reset(kinesis); + } + } + + @Test + public void shouldListAllShards() throws Exception { + Shard shard1 = new Shard().withShardId(SHARD_1); + Shard shard2 = new Shard().withShardId(SHARD_2); + Shard shard3 = new Shard().withShardId(SHARD_3); + given(kinesis.describeStream(STREAM, null)).willReturn(new DescribeStreamResult() + .withStreamDescription(new StreamDescription() + .withShards(shard1, shard2) + .withHasMoreShards(true))); + given(kinesis.describeStream(STREAM, SHARD_2)).willReturn(new DescribeStreamResult() + .withStreamDescription(new StreamDescription() + .withShards(shard3) + .withHasMoreShards(false))); + + List shards = underTest.listShards(STREAM); + + assertThat(shards).containsOnly(shard1, shard2, shard3); + } + + @Test + public void shouldHandleExpiredIterationExceptionForShardListing() { + shouldHandleShardListingError(new ExpiredIteratorException(""), + ExpiredIteratorException.class); + } + + @Test + public void shouldHandleLimitExceededExceptionForShardListing() { + shouldHandleShardListingError(new LimitExceededException(""), + TransientKinesisException.class); + } + + @Test + public void shouldHandleProvisionedThroughputExceededExceptionForShardListing() { + shouldHandleShardListingError(new ProvisionedThroughputExceededException(""), + TransientKinesisException.class); + } + + @Test + public void shouldHandleServiceErrorForShardListing() { + shouldHandleShardListingError(newAmazonServiceException(ErrorType.Service), + TransientKinesisException.class); + } + + @Test + public void shouldHandleClientErrorForShardListing() { + shouldHandleShardListingError(newAmazonServiceException(ErrorType.Client), + RuntimeException.class); + } + + @Test + public void shouldHandleUnexpectedExceptionForShardListing() { + shouldHandleShardListingError(new NullPointerException(), + RuntimeException.class); + } + + private void shouldHandleShardListingError( + Exception thrownException, + Class expectedExceptionClass) { + given(kinesis.describeStream(STREAM, null)).willThrow(thrownException); + try { + underTest.listShards(STREAM); + failBecauseExceptionWasNotThrown(expectedExceptionClass); + } catch (Exception e) { + assertThat(e).isExactlyInstanceOf(expectedExceptionClass); + } finally { + reset(kinesis); + } + } + + private AmazonServiceException newAmazonServiceException(ErrorType errorType) { + AmazonServiceException exception = new AmazonServiceException(""); + exception.setErrorType(errorType); + return exception; + } +} diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/package-info.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/package-info.java new file mode 100644 index 000000000000..44dbf4a92cd2 --- /dev/null +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/package-info.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Transforms for reading and writing from Amazon Kinesis. + */ +package org.apache.beam.sdk.io.kinesis; diff --git a/sdks/java/io/pom.xml b/sdks/java/io/pom.xml index 419849956cea..6cbd615a1d6e 100644 --- a/sdks/java/io/pom.xml +++ b/sdks/java/io/pom.xml @@ -37,6 +37,7 @@ hdfs jms kafka + kinesis From bed50565cc5cd8892d1a4577cb5d6505f7bb0367 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Fri, 26 Aug 2016 15:17:05 -0700 Subject: [PATCH 075/112] Organize imports in Kinesis --- .../kinesis/DynamicCheckpointGenerator.java | 4 +-- .../io/kinesis/GetKinesisRecordsResult.java | 2 +- .../apache/beam/sdk/io/kinesis/KinesisIO.java | 2 +- .../beam/sdk/io/kinesis/KinesisReader.java | 8 +++--- .../io/kinesis/KinesisReaderCheckpoint.java | 4 +-- .../beam/sdk/io/kinesis/KinesisRecord.java | 8 +++--- .../sdk/io/kinesis/KinesisRecordCoder.java | 9 +++--- .../beam/sdk/io/kinesis/KinesisSource.java | 10 +++---- .../beam/sdk/io/kinesis/RoundRobin.java | 1 - .../beam/sdk/io/kinesis/ShardCheckpoint.java | 8 +++--- .../sdk/io/kinesis/ShardRecordsIterator.java | 2 +- .../io/kinesis/SimplifiedKinesisClient.java | 5 ++-- .../beam/sdk/io/kinesis/StartingPoint.java | 2 +- .../sdk/io/kinesis/AmazonKinesisMock.java | 12 ++++---- .../sdk/io/kinesis/CustomOptionalTest.java | 2 +- .../DynamicCheckpointGeneratorTest.java | 7 +++-- .../sdk/io/kinesis/KinesisMockReadTest.java | 11 ++++---- .../kinesis/KinesisReaderCheckpointTest.java | 10 +++---- .../beam/sdk/io/kinesis/KinesisReaderIT.java | 28 +++++++++---------- .../sdk/io/kinesis/KinesisReaderTest.java | 7 +++-- .../io/kinesis/KinesisRecordCoderTest.java | 3 +- .../beam/sdk/io/kinesis/KinesisUploader.java | 4 +-- .../beam/sdk/io/kinesis/RecordFilterTest.java | 8 +++--- .../beam/sdk/io/kinesis/RoundRobinTest.java | 4 +-- .../sdk/io/kinesis/ShardCheckpointTest.java | 7 +++-- .../io/kinesis/ShardRecordsIteratorTest.java | 11 ++++---- .../kinesis/SimplifiedKinesisClientTest.java | 11 ++++---- 27 files changed, 92 insertions(+), 98 deletions(-) diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGenerator.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGenerator.java index d86960f0d030..2ec293cfeac2 100644 --- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGenerator.java +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGenerator.java @@ -20,10 +20,8 @@ import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.collect.Lists.transform; - -import com.google.common.base.Function; - import com.amazonaws.services.kinesis.model.Shard; +import com.google.common.base.Function; /** * Creates {@link KinesisReaderCheckpoint}, which spans over all shards in given stream. diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/GetKinesisRecordsResult.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/GetKinesisRecordsResult.java index f48b9d53ade8..c0f00de610c7 100644 --- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/GetKinesisRecordsResult.java +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/GetKinesisRecordsResult.java @@ -18,9 +18,9 @@ package org.apache.beam.sdk.io.kinesis; import static com.google.common.collect.Lists.transform; -import com.google.common.base.Function; import com.amazonaws.services.kinesis.clientlibrary.types.UserRecord; +import com.google.common.base.Function; import java.util.List; import javax.annotation.Nullable; diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisIO.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisIO.java index b3cb464e1cfb..811051c5fb0a 100644 --- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisIO.java +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisIO.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.io.kinesis; -import org.apache.beam.sdk.transforms.PTransform; import static com.google.common.base.Preconditions.checkNotNull; import com.amazonaws.auth.AWSCredentialsProvider; @@ -28,6 +27,7 @@ import com.amazonaws.services.kinesis.AmazonKinesis; import com.amazonaws.services.kinesis.AmazonKinesisClient; import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream; +import org.apache.beam.sdk.transforms.PTransform; import org.joda.time.Instant; /** diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReader.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReader.java index 38a0050fa72a..219a7059207d 100644 --- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReader.java +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReader.java @@ -18,16 +18,16 @@ package org.apache.beam.sdk.io.kinesis; -import org.apache.beam.sdk.io.UnboundedSource; import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.collect.Lists.newArrayList; -import org.joda.time.Instant; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.io.IOException; import java.util.List; import java.util.NoSuchElementException; +import org.apache.beam.sdk.io.UnboundedSource; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /*** diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpoint.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpoint.java index 6ceb74295d86..663ba44b3340 100644 --- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpoint.java +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpoint.java @@ -17,20 +17,18 @@ */ package org.apache.beam.sdk.io.kinesis; -import org.apache.beam.sdk.io.UnboundedSource; import static com.google.common.collect.Iterables.transform; import static com.google.common.collect.Lists.newArrayList; import static com.google.common.collect.Lists.partition; - import com.google.common.base.Function; import com.google.common.collect.ImmutableList; - import java.io.IOException; import java.io.Serializable; import java.util.Iterator; import java.util.List; import javax.annotation.Nullable; +import org.apache.beam.sdk.io.UnboundedSource; /*** * Checkpoint representing a total progress in a set of shards in single stream. diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecord.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecord.java index cdb495ce0078..fe2a33dec798 100644 --- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecord.java +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecord.java @@ -17,15 +17,15 @@ */ package org.apache.beam.sdk.io.kinesis; -import com.google.common.base.Charsets; +import static org.apache.commons.lang.builder.HashCodeBuilder.reflectionHashCode; import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber; import com.amazonaws.services.kinesis.clientlibrary.types.UserRecord; -import static org.apache.commons.lang.builder.HashCodeBuilder.reflectionHashCode; -import org.apache.commons.lang.builder.EqualsBuilder; -import org.joda.time.Instant; +import com.google.common.base.Charsets; import java.io.Serializable; import java.nio.ByteBuffer; +import org.apache.commons.lang.builder.EqualsBuilder; +import org.joda.time.Instant; /** * {@link UserRecord} enhanced with utility methods. diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoder.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoder.java index c383a4ff8a91..5b13e319691a 100644 --- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoder.java +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoder.java @@ -17,18 +17,17 @@ */ package org.apache.beam.sdk.io.kinesis; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.ByteBuffer; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.InstantCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarLongCoder; - import org.joda.time.Instant; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.nio.ByteBuffer; /*** * A {@link Coder} for {@link KinesisRecord}. diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisSource.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisSource.java index 38c9fa4072ef..62cba0849856 100644 --- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisSource.java +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisSource.java @@ -17,18 +17,16 @@ */ package org.apache.beam.sdk.io.kinesis; +import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.collect.Lists.newArrayList; + +import java.util.List; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.SerializableCoder; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.options.PipelineOptions; - - -import static com.google.common.base.Preconditions.checkNotNull; -import static com.google.common.collect.Lists.newArrayList; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.List; /*** diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/RoundRobin.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/RoundRobin.java index 7257aa1492bb..7adae4b6ffd9 100644 --- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/RoundRobin.java +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/RoundRobin.java @@ -20,7 +20,6 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.collect.Queues.newArrayDeque; - import java.util.Deque; import java.util.Iterator; diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardCheckpoint.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardCheckpoint.java index 1d8628bc8851..9920aca0300a 100644 --- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardCheckpoint.java +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardCheckpoint.java @@ -18,17 +18,17 @@ package org.apache.beam.sdk.io.kinesis; -import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Preconditions.checkNotNull; - import static com.amazonaws.services.kinesis.model.ShardIteratorType.AFTER_SEQUENCE_NUMBER; import static com.amazonaws.services.kinesis.model.ShardIteratorType.AT_SEQUENCE_NUMBER; import static com.amazonaws.services.kinesis.model.ShardIteratorType.AT_TIMESTAMP; +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; + import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber; import com.amazonaws.services.kinesis.model.Record; import com.amazonaws.services.kinesis.model.ShardIteratorType; -import org.joda.time.Instant; import java.io.Serializable; +import org.joda.time.Instant; /*** diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIterator.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIterator.java index 7dfe158ca41d..d17996a19218 100644 --- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIterator.java +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIterator.java @@ -21,9 +21,9 @@ import static com.google.common.collect.Queues.newArrayDeque; import com.amazonaws.services.kinesis.model.ExpiredIteratorException; +import java.util.Deque; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Deque; /*** * Iterates over records in a single shard. diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClient.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClient.java index f9a1ea2af810..96267d14f0be 100644 --- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClient.java +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClient.java @@ -18,8 +18,6 @@ package org.apache.beam.sdk.io.kinesis; -import com.google.common.collect.Lists; - import com.amazonaws.AmazonServiceException; import com.amazonaws.services.kinesis.AmazonKinesis; import com.amazonaws.services.kinesis.clientlibrary.types.UserRecord; @@ -32,10 +30,11 @@ import com.amazonaws.services.kinesis.model.Shard; import com.amazonaws.services.kinesis.model.ShardIteratorType; import com.amazonaws.services.kinesis.model.StreamDescription; -import org.joda.time.Instant; +import com.google.common.collect.Lists; import java.util.Date; import java.util.List; import java.util.concurrent.Callable; +import org.joda.time.Instant; /*** * Wraps {@link AmazonKinesis} class providing much simpler interface and diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/StartingPoint.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/StartingPoint.java index 8140269c170c..b7ee91746fa3 100644 --- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/StartingPoint.java +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/StartingPoint.java @@ -22,9 +22,9 @@ import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream; import com.amazonaws.services.kinesis.model.ShardIteratorType; -import org.joda.time.Instant; import java.io.Serializable; import java.util.Objects; +import org.joda.time.Instant; /*** * Denotes a point at which the reader should start reading from a Kinesis stream. diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/AmazonKinesisMock.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/AmazonKinesisMock.java index 7ca8e0b4bd8d..b007fa411db4 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/AmazonKinesisMock.java +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/AmazonKinesisMock.java @@ -19,7 +19,9 @@ import static com.google.common.collect.Lists.newArrayList; import static com.google.common.collect.Lists.transform; -import com.google.common.base.Function; +import static java.lang.Integer.parseInt; +import static java.lang.Math.min; +import static org.apache.commons.lang.builder.HashCodeBuilder.reflectionHashCode; import com.amazonaws.AmazonWebServiceRequest; import com.amazonaws.ResponseMetadata; @@ -63,15 +65,13 @@ import com.amazonaws.services.kinesis.model.SplitShardRequest; import com.amazonaws.services.kinesis.model.SplitShardResult; import com.amazonaws.services.kinesis.model.StreamDescription; -import static org.apache.commons.lang.builder.HashCodeBuilder.reflectionHashCode; -import org.apache.commons.lang.builder.EqualsBuilder; -import org.joda.time.Instant; -import static java.lang.Integer.parseInt; -import static java.lang.Math.min; +import com.google.common.base.Function; import java.io.Serializable; import java.nio.ByteBuffer; import java.util.List; import javax.annotation.Nullable; +import org.apache.commons.lang.builder.EqualsBuilder; +import org.joda.time.Instant; /** * Created by p.pastuszka on 21.07.2016. diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/CustomOptionalTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/CustomOptionalTest.java index 152fd6dfde43..cb0d0e248f88 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/CustomOptionalTest.java +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/CustomOptionalTest.java @@ -17,8 +17,8 @@ */ package org.apache.beam.sdk.io.kinesis; -import org.junit.Test; import java.util.NoSuchElementException; +import org.junit.Test; /** * Created by ppastuszka on 12.12.15. diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGeneratorTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGeneratorTest.java index a9e5a69a2d51..c92ac9a4b293 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGeneratorTest.java +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGeneratorTest.java @@ -17,15 +17,16 @@ */ package org.apache.beam.sdk.io.kinesis; -import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream; -import com.amazonaws.services.kinesis.model.Shard; +import static java.util.Arrays.asList; import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.BDDMockito.given; + +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream; +import com.amazonaws.services.kinesis.model.Shard; import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mock; import org.mockito.runners.MockitoJUnitRunner; -import static java.util.Arrays.asList; /*** diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisMockReadTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisMockReadTest.java index 61a858f8dea5..304220be4697 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisMockReadTest.java +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisMockReadTest.java @@ -17,20 +17,19 @@ */ package org.apache.beam.sdk.io.kinesis; +import static com.google.common.collect.Lists.newArrayList; + +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream; +import com.google.common.collect.Iterables; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; -import static com.google.common.collect.Lists.newArrayList; - -import com.google.common.collect.Iterables; - -import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream; import org.joda.time.DateTime; import org.junit.Test; -import java.util.List; /** * Created by p.pastuszka on 22.07.2016. diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpointTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpointTest.java index 205f050bb8ac..8c8da641804f 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpointTest.java +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpointTest.java @@ -18,17 +18,17 @@ package org.apache.beam.sdk.io.kinesis; -import com.google.common.collect.Iterables; - +import static java.util.Arrays.asList; import static org.assertj.core.api.Assertions.assertThat; + +import com.google.common.collect.Iterables; +import java.util.Iterator; +import java.util.List; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mock; import org.mockito.runners.MockitoJUnitRunner; -import static java.util.Arrays.asList; -import java.util.Iterator; -import java.util.List; /*** * diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderIT.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderIT.java index fbc7c66aa3b9..73a24555fa9c 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderIT.java +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderIT.java @@ -17,6 +17,20 @@ */ package org.apache.beam.sdk.io.kinesis; +import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.collect.Lists.newArrayList; +import static java.util.concurrent.Executors.newSingleThreadExecutor; +import static org.assertj.core.api.Assertions.assertThat; + +import com.amazonaws.regions.Regions; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.options.PipelineOptionsFactory; @@ -25,25 +39,11 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; -import static com.google.common.base.Preconditions.checkNotNull; -import static com.google.common.collect.Lists.newArrayList; - -import com.amazonaws.regions.Regions; -import static org.assertj.core.api.Assertions.assertThat; import org.apache.commons.lang.RandomStringUtils; import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Ignore; import org.junit.Test; -import static java.util.concurrent.Executors.newSingleThreadExecutor; -import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.util.List; -import java.util.concurrent.Callable; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; /** * Integration test, that reads from the real Kinesis. diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderTest.java index 793fb579c26b..29a24821bdf3 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderTest.java +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderTest.java @@ -17,16 +17,17 @@ */ package org.apache.beam.sdk.io.kinesis; +import static java.util.Arrays.asList; import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.Mockito.when; + +import java.io.IOException; +import java.util.NoSuchElementException; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mock; import org.mockito.runners.MockitoJUnitRunner; -import static java.util.Arrays.asList; -import java.io.IOException; -import java.util.NoSuchElementException; /** * Created by ppastuszka on 12.12.15. diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoderTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoderTest.java index b09b7eb9b378..d301f256a483 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoderTest.java +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoderTest.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.io.kinesis; +import java.nio.ByteBuffer; import org.apache.beam.sdk.testing.CoderProperties; - import org.joda.time.Instant; import org.junit.Test; -import java.nio.ByteBuffer; /** * Created by p.pastuszka on 20.07.2016. diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisUploader.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisUploader.java index 0dcede985671..c98242b4a0a2 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisUploader.java +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisUploader.java @@ -18,8 +18,6 @@ package org.apache.beam.sdk.io.kinesis; import static com.google.common.collect.Lists.newArrayList; -import com.google.common.base.Charsets; -import com.google.common.collect.Lists; import com.amazonaws.auth.BasicAWSCredentials; import com.amazonaws.internal.StaticCredentialsProvider; @@ -30,6 +28,8 @@ import com.amazonaws.services.kinesis.model.PutRecordsRequestEntry; import com.amazonaws.services.kinesis.model.PutRecordsResult; import com.amazonaws.services.kinesis.model.PutRecordsResultEntry; +import com.google.common.base.Charsets; +import com.google.common.collect.Lists; import java.nio.ByteBuffer; import java.util.List; diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RecordFilterTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RecordFilterTest.java index 360106d00c4b..f979c0108cdc 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RecordFilterTest.java +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RecordFilterTest.java @@ -17,16 +17,16 @@ */ package org.apache.beam.sdk.io.kinesis; -import com.google.common.collect.Lists; - import static org.mockito.BDDMockito.given; + +import com.google.common.collect.Lists; +import java.util.Collections; +import java.util.List; import org.assertj.core.api.Assertions; import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mock; import org.mockito.runners.MockitoJUnitRunner; -import java.util.Collections; -import java.util.List; /*** diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RoundRobinTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RoundRobinTest.java index a508ddf5ce0f..aedc89eac8b8 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RoundRobinTest.java +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RoundRobinTest.java @@ -18,11 +18,11 @@ package org.apache.beam.sdk.io.kinesis; import static com.google.common.collect.Lists.newArrayList; - import static org.assertj.core.api.Assertions.assertThat; -import org.junit.Test; + import java.util.Collections; import java.util.List; +import org.junit.Test; /** * Created by ppastuszka on 12.12.15. diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardCheckpointTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardCheckpointTest.java index 2227cef63828..39ab36f9255a 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardCheckpointTest.java +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardCheckpointTest.java @@ -22,8 +22,6 @@ import static com.amazonaws.services.kinesis.model.ShardIteratorType.AFTER_SEQUENCE_NUMBER; import static com.amazonaws.services.kinesis.model.ShardIteratorType.AT_SEQUENCE_NUMBER; import static com.amazonaws.services.kinesis.model.ShardIteratorType.AT_TIMESTAMP; -import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber; -import com.amazonaws.services.kinesis.model.ShardIteratorType; import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.BDDMockito.given; import static org.mockito.Matchers.anyString; @@ -31,6 +29,10 @@ import static org.mockito.Matchers.isNull; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; + +import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber; +import com.amazonaws.services.kinesis.model.ShardIteratorType; +import java.io.IOException; import org.joda.time.DateTime; import org.joda.time.Instant; import org.junit.Before; @@ -38,7 +40,6 @@ import org.junit.runner.RunWith; import org.mockito.Mock; import org.mockito.runners.MockitoJUnitRunner; -import java.io.IOException; /** * diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIteratorTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIteratorTest.java index e2a3ccc0057e..585b8843c489 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIteratorTest.java +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIteratorTest.java @@ -17,11 +17,16 @@ */ package org.apache.beam.sdk.io.kinesis; -import com.amazonaws.services.kinesis.model.ExpiredIteratorException; +import static java.util.Arrays.asList; +import static java.util.Collections.singletonList; import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyListOf; import static org.mockito.Mockito.when; + +import com.amazonaws.services.kinesis.model.ExpiredIteratorException; +import java.io.IOException; +import java.util.Collections; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -29,10 +34,6 @@ import org.mockito.invocation.InvocationOnMock; import org.mockito.runners.MockitoJUnitRunner; import org.mockito.stubbing.Answer; -import static java.util.Arrays.asList; -import static java.util.Collections.singletonList; -import java.io.IOException; -import java.util.Collections; /** * Created by ppastuszka on 12.12.15. diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClientTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClientTest.java index 44d29d6cf8fd..96434fd4c838 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClientTest.java +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClientTest.java @@ -17,6 +17,11 @@ */ package org.apache.beam.sdk.io.kinesis; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.failBecauseExceptionWasNotThrown; +import static org.mockito.BDDMockito.given; +import static org.mockito.Mockito.reset; + import com.amazonaws.AmazonServiceException; import com.amazonaws.AmazonServiceException.ErrorType; import com.amazonaws.services.kinesis.AmazonKinesis; @@ -29,17 +34,13 @@ import com.amazonaws.services.kinesis.model.Shard; import com.amazonaws.services.kinesis.model.ShardIteratorType; import com.amazonaws.services.kinesis.model.StreamDescription; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.failBecauseExceptionWasNotThrown; -import static org.mockito.BDDMockito.given; -import static org.mockito.Mockito.reset; +import java.util.List; import org.joda.time.Instant; import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.InjectMocks; import org.mockito.Mock; import org.mockito.runners.MockitoJUnitRunner; -import java.util.List; /*** */ From a5005fbe7c0a43bdd2d825c2fc77556956bf0bfa Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Fri, 26 Aug 2016 15:20:44 -0700 Subject: [PATCH 076/112] Fix javadoc in Kinesis --- .../apache/beam/sdk/io/kinesis/TransientKinesisException.java | 2 +- .../java/org/apache/beam/sdk/io/kinesis/AmazonKinesisMock.java | 2 +- .../java/org/apache/beam/sdk/io/kinesis/CustomOptionalTest.java | 2 +- .../org/apache/beam/sdk/io/kinesis/KinesisMockReadTest.java | 2 +- .../java/org/apache/beam/sdk/io/kinesis/KinesisReaderTest.java | 2 +- .../org/apache/beam/sdk/io/kinesis/KinesisRecordCoderTest.java | 2 +- .../java/org/apache/beam/sdk/io/kinesis/RoundRobinTest.java | 2 +- .../apache/beam/sdk/io/kinesis/ShardRecordsIteratorTest.java | 2 +- 8 files changed, 8 insertions(+), 8 deletions(-) diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/TransientKinesisException.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/TransientKinesisException.java index a1a974bf3464..57ad8a89103e 100644 --- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/TransientKinesisException.java +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/TransientKinesisException.java @@ -20,7 +20,7 @@ import com.amazonaws.AmazonServiceException; /** - * Created by p.pastuszka on 21.06.2016. + * A transient exception thrown by Kinesis. */ class TransientKinesisException extends Exception { public TransientKinesisException(String s, AmazonServiceException e) { diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/AmazonKinesisMock.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/AmazonKinesisMock.java index b007fa411db4..046c9d9126d2 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/AmazonKinesisMock.java +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/AmazonKinesisMock.java @@ -74,7 +74,7 @@ import org.joda.time.Instant; /** - * Created by p.pastuszka on 21.07.2016. + * Mock implemenation of {@link AmazonKinesis} for testing. */ class AmazonKinesisMock implements AmazonKinesis { diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/CustomOptionalTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/CustomOptionalTest.java index cb0d0e248f88..20e837250614 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/CustomOptionalTest.java +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/CustomOptionalTest.java @@ -21,7 +21,7 @@ import org.junit.Test; /** - * Created by ppastuszka on 12.12.15. + * Tests {@link CustomOptional}. */ public class CustomOptionalTest { @Test(expected = NoSuchElementException.class) diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisMockReadTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisMockReadTest.java index 304220be4697..f0ab46c3ae70 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisMockReadTest.java +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisMockReadTest.java @@ -32,7 +32,7 @@ import org.junit.Test; /** - * Created by p.pastuszka on 22.07.2016. + * Tests {@link AmazonKinesisMock}. */ public class KinesisMockReadTest { @Test diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderTest.java index 29a24821bdf3..3111029d74e5 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderTest.java +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderTest.java @@ -30,7 +30,7 @@ import org.mockito.runners.MockitoJUnitRunner; /** - * Created by ppastuszka on 12.12.15. + * Tests {@link KinesisReader}. */ @RunWith(MockitoJUnitRunner.class) public class KinesisReaderTest { diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoderTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoderTest.java index d301f256a483..8771c86c82a5 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoderTest.java +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoderTest.java @@ -23,7 +23,7 @@ import org.junit.Test; /** - * Created by p.pastuszka on 20.07.2016. + * Tests {@link KinesisRecordCoder}. */ public class KinesisRecordCoderTest { @Test diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RoundRobinTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RoundRobinTest.java index aedc89eac8b8..f032eeab377d 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RoundRobinTest.java +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RoundRobinTest.java @@ -25,7 +25,7 @@ import org.junit.Test; /** - * Created by ppastuszka on 12.12.15. + * Tests {@link RoundRobin}. */ public class RoundRobinTest { @Test(expected = IllegalArgumentException.class) diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIteratorTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIteratorTest.java index 585b8843c489..49e806dc12e8 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIteratorTest.java +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIteratorTest.java @@ -36,7 +36,7 @@ import org.mockito.stubbing.Answer; /** - * Created by ppastuszka on 12.12.15. + * Tests {@link ShardRecordsIterator}. */ @RunWith(MockitoJUnitRunner.class) public class ShardRecordsIteratorTest { From ef828de6d1ee71c6b0e29c6468d5509805794e7b Mon Sep 17 00:00:00 2001 From: Mark Liu Date: Fri, 26 Aug 2016 14:23:54 -0700 Subject: [PATCH 077/112] [BEAM-592] Fix SparkRunner Dependency Problem in WordCount --- examples/java/pom.xml | 6 ++++++ examples/java8/pom.xml | 6 ++++++ 2 files changed, 12 insertions(+) diff --git a/examples/java/pom.xml b/examples/java/pom.xml index 096bc4e32fef..47f036613ac6 100644 --- a/examples/java/pom.xml +++ b/examples/java/pom.xml @@ -97,6 +97,12 @@ ${spark.version} runtime true + + + org.slf4j + jul-to-slf4j + + diff --git a/examples/java8/pom.xml b/examples/java8/pom.xml index f3f2c3c83f17..44cf918ba2e6 100644 --- a/examples/java8/pom.xml +++ b/examples/java8/pom.xml @@ -98,6 +98,12 @@ ${spark.version} runtime true + + + org.slf4j + jul-to-slf4j + + From ee71688924a15bcd211e950d5822b6750b20eeb8 Mon Sep 17 00:00:00 2001 From: Daniel Halperin Date: Sat, 27 Aug 2016 09:42:08 -0700 Subject: [PATCH 078/112] DataflowRunner: get PBegin from PInput Fixes an invalid cast that breaks some PCollectionList-related tests. --- .../java/org/apache/beam/runners/dataflow/DataflowRunner.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index 0ce4b58aa6ac..e5b66147c382 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -400,7 +400,7 @@ public OutputT apply( return windowed; } else if (Flatten.FlattenPCollectionList.class.equals(transform.getClass()) && ((PCollectionList) input).size() == 0) { - return (OutputT) Pipeline.applyTransform((PBegin) input, Create.of()); + return (OutputT) Pipeline.applyTransform(input.getPipeline().begin(), Create.of()); } else if (overrides.containsKey(transform.getClass())) { // It is the responsibility of whoever constructs overrides to ensure this is type safe. @SuppressWarnings("unchecked") From 017da7bac3e844ef7391aabbcbaf86c9c99af968 Mon Sep 17 00:00:00 2001 From: Abbass MAROUNI Date: Mon, 29 Aug 2016 13:28:46 +0200 Subject: [PATCH 079/112] [BEAM-313] Provide a context for SparkRunner --- .../runners/spark/SparkPipelineOptions.java | 13 ++ .../beam/runners/spark/SparkRunner.java | 16 +- .../spark/ProvidedSparkContextTest.java | 138 ++++++++++++++++++ 3 files changed, 164 insertions(+), 3 deletions(-) create mode 100644 runners/spark/src/test/java/org/apache/beam/runners/spark/ProvidedSparkContextTest.java diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java index be4f7f06a9bc..db6b75cb4a03 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java @@ -18,11 +18,14 @@ package org.apache.beam.runners.spark; +import com.fasterxml.jackson.annotation.JsonIgnore; + import org.apache.beam.sdk.options.ApplicationNameOptions; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.StreamingOptions; +import org.apache.spark.api.java.JavaSparkContext; /** * Spark runner pipeline options. @@ -49,4 +52,14 @@ public interface SparkPipelineOptions extends PipelineOptions, StreamingOptions, @Default.Boolean(true) Boolean getEnableSparkSinks(); void setEnableSparkSinks(Boolean enableSparkSinks); + + @Description("If the spark runner will be initialized with a provided Spark Context") + @Default.Boolean(false) + boolean getUsesProvidedSparkContext(); + void setUsesProvidedSparkContext(boolean value); + + @Description("Provided Java Spark Context") + @JsonIgnore + JavaSparkContext getProvidedSparkContext(); + void setProvidedSparkContext(JavaSparkContext jsc); } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java index fa85a2e25e26..9f1a83996fb7 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java @@ -143,9 +143,19 @@ private SparkRunner(SparkPipelineOptions options) { public EvaluationResult run(Pipeline pipeline) { try { LOG.info("Executing pipeline using the SparkRunner."); - JavaSparkContext jsc = SparkContextFactory.getSparkContext(mOptions.getSparkMaster(), - mOptions.getAppName()); - + JavaSparkContext jsc; + if (mOptions.getUsesProvidedSparkContext()) { + LOG.info("Using a provided Spark Context"); + jsc = mOptions.getProvidedSparkContext(); + if (jsc == null || jsc.sc().isStopped()){ + LOG.error("The provided Spark context " + + jsc + " was not created or was stopped"); + throw new RuntimeException("The provided Spark context was not created or was stopped"); + } + } else { + LOG.info("Creating a new Spark Context"); + jsc = SparkContextFactory.getSparkContext(mOptions.getSparkMaster(), mOptions.getAppName()); + } if (mOptions.isStreaming()) { SparkPipelineTranslator translator = new StreamingTransformTranslator.Translator(new TransformTranslator.Translator()); diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/ProvidedSparkContextTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/ProvidedSparkContextTest.java new file mode 100644 index 000000000000..cbc5976b9542 --- /dev/null +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/ProvidedSparkContextTest.java @@ -0,0 +1,138 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.runners.spark; + +import static org.junit.Assert.fail; + +import com.google.common.collect.ImmutableSet; +import java.util.Arrays; +import java.util.List; +import java.util.Set; +import org.apache.beam.runners.spark.examples.WordCount; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.values.PCollection; +import org.apache.spark.api.java.JavaSparkContext; +import org.junit.Test; + +/** + * Provided Spark Context tests. + */ +public class ProvidedSparkContextTest { + private static final String[] WORDS_ARRAY = { + "hi there", "hi", "hi sue bob", + "hi sue", "", "bob hi"}; + private static final List WORDS = Arrays.asList(WORDS_ARRAY); + private static final Set EXPECTED_COUNT_SET = + ImmutableSet.of("hi: 5", "there: 1", "sue: 2", "bob: 2"); + private static final String PROVIDED_CONTEXT_EXCEPTION = + "The provided Spark context was not created or was stopped"; + + /** + * Provide a context and call pipeline run. + * @throws Exception + */ + @Test + public void testWithProvidedContext() throws Exception { + JavaSparkContext jsc = new JavaSparkContext("local[*]", "Existing_Context"); + + SparkPipelineOptions options = PipelineOptionsFactory.as(SparkPipelineOptions.class); + options.setRunner(SparkRunner.class); + options.setUsesProvidedSparkContext(true); + options.setProvidedSparkContext(jsc); + + Pipeline p = Pipeline.create(options); + PCollection inputWords = p.apply(Create.of(WORDS).withCoder(StringUtf8Coder + .of())); + PCollection output = inputWords.apply(new WordCount.CountWords()) + .apply(MapElements.via(new WordCount.FormatAsTextFn())); + + PAssert.that(output).containsInAnyOrder(EXPECTED_COUNT_SET); + + // Run test from pipeline + p.run(); + + jsc.stop(); + } + + /** + * Provide a context and call pipeline run. + * @throws Exception + */ + @Test + public void testWithNullContext() throws Exception { + JavaSparkContext jsc = null; + + SparkPipelineOptions options = PipelineOptionsFactory.as(SparkPipelineOptions.class); + options.setRunner(SparkRunner.class); + options.setUsesProvidedSparkContext(true); + options.setProvidedSparkContext(jsc); + + Pipeline p = Pipeline.create(options); + PCollection inputWords = p.apply(Create.of(WORDS).withCoder(StringUtf8Coder + .of())); + PCollection output = inputWords.apply(new WordCount.CountWords()) + .apply(MapElements.via(new WordCount.FormatAsTextFn())); + + PAssert.that(output).containsInAnyOrder(EXPECTED_COUNT_SET); + + try { + p.run(); + fail("Should throw an exception when The provided Spark context is null"); + } catch (RuntimeException e){ + assert(e.getMessage().contains(PROVIDED_CONTEXT_EXCEPTION)); + } + } + + /** + * A SparkRunner with a stopped provided Spark context cannot run pipelines. + * @throws Exception + */ + @Test + public void testWithStoppedProvidedContext() throws Exception { + JavaSparkContext jsc = new JavaSparkContext("local[*]", "Existing_Context"); + // Stop the provided Spark context directly + jsc.stop(); + + SparkPipelineOptions options = PipelineOptionsFactory.as(SparkPipelineOptions.class); + options.setRunner(SparkRunner.class); + options.setUsesProvidedSparkContext(true); + options.setProvidedSparkContext(jsc); + + Pipeline p = Pipeline.create(options); + PCollection inputWords = p.apply(Create.of(WORDS).withCoder(StringUtf8Coder + .of())); + PCollection output = inputWords.apply(new WordCount.CountWords()) + .apply(MapElements.via(new WordCount.FormatAsTextFn())); + + PAssert.that(output).containsInAnyOrder(EXPECTED_COUNT_SET); + + try { + p.run(); + fail("Should throw an exception when The provided Spark context is stopped"); + } catch (RuntimeException e){ + assert(e.getMessage().contains(PROVIDED_CONTEXT_EXCEPTION)); + } + } + +} From c7da8fb7ab0307a788f54ee5ecea1133b68d57e2 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Fri, 26 Aug 2016 15:28:21 -0700 Subject: [PATCH 080/112] Update Dataflow Container Version --- .../java/org/apache/beam/runners/dataflow/DataflowRunner.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index e5b66147c382..d067b61064e6 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -206,9 +206,9 @@ public class DataflowRunner extends PipelineRunner { // Default Docker container images that execute Dataflow worker harness, residing in Google // Container Registry, separately for Batch and Streaming. public static final String BATCH_WORKER_HARNESS_CONTAINER_IMAGE = - "dataflow.gcr.io/v1beta3/beam-java-batch:beam-master-20160823"; + "dataflow.gcr.io/v1beta3/beam-java-batch:beam-master-20160826"; public static final String STREAMING_WORKER_HARNESS_CONTAINER_IMAGE = - "dataflow.gcr.io/v1beta3/beam-java-streaming:beam-master-20160823"; + "dataflow.gcr.io/v1beta3/beam-java-streaming:beam-master-20160826"; // The limit of CreateJob request size. private static final int CREATE_JOB_REQUEST_LIMIT_BYTES = 10 * 1024 * 1024; From a25b3589db738ea507047b502092b0d177ea8e1f Mon Sep 17 00:00:00 2001 From: Mark Liu Date: Fri, 26 Aug 2016 16:13:17 -0700 Subject: [PATCH 081/112] [BEAM-572] Remove Spark Reference in WordCount --- .../org/apache/beam/examples/WordCount.java | 24 +------------------ .../org/apache/beam/examples/WordCountIT.java | 9 +++++++ 2 files changed, 10 insertions(+), 23 deletions(-) diff --git a/examples/java/src/main/java/org/apache/beam/examples/WordCount.java b/examples/java/src/main/java/org/apache/beam/examples/WordCount.java index 0275651bf288..793ee4baceae 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/WordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/WordCount.java @@ -18,7 +18,6 @@ package org.apache.beam.examples; import com.google.common.base.Strings; -import com.google.common.io.Resources; import java.io.IOException; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.TextIO; @@ -165,7 +164,7 @@ public PCollection> apply(PCollection lines) { */ public static interface WordCountOptions extends PipelineOptions { @Description("Path of the file to read from") - @Default.InstanceFactory(InputFactory.class) + @Default.String("gs://apache-beam-samples/shakespeare/kinglear.txt") String getInputFile(); void setInputFile(String value); @@ -194,27 +193,6 @@ public String create(PipelineOptions options) { } } } - - /** - * Return default input file path according to runner type. - * - *

                - *
              • SparkRunner: - * .../src/test/resources/LICENSE
              • - *
              • other runners: - * gs://apache-beam-samples/apache/LICENSE
              • - *
              - */ - public static class InputFactory implements DefaultValueFactory { - @Override - public String create(PipelineOptions options) { - if (options.getRunner().getName().contains("SparkRunner")) { - return Resources.getResource("LICENSE").getPath(); - } else { - return "gs://apache-beam-samples/apache/LICENSE"; - } - } - } } public static void main(String[] args) { diff --git a/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java b/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java index ca0c9d6bda09..b0e0fe00628a 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java +++ b/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java @@ -18,6 +18,7 @@ package org.apache.beam.examples; +import com.google.common.io.Resources; import java.util.Date; import org.apache.beam.examples.WordCount.WordCountOptions; import org.apache.beam.sdk.options.Default; @@ -61,6 +62,14 @@ public void testE2EWordCount() throws Exception { options.setOnSuccessMatcher( new FileChecksumMatcher(options.getOutputChecksum(), options.getOutput() + "*")); + String e2eTestInputPath = "gs://apache-beam-samples/apache/LICENSE"; + // Spark runner currently doesn't support GCS I/O, change default input to: + // .../src/test/resources/LICENSE + if (options.getRunner().getName().contains("SparkRunner")) { + e2eTestInputPath = Resources.getResource("LICENSE").getPath(); + } + options.setInputFile(e2eTestInputPath); + WordCount.main(TestPipeline.convertToArgs(options)); } } From 20b5fe41c314d4fa16c4c16d2287bc5019642f4a Mon Sep 17 00:00:00 2001 From: Aviem Zur Date: Sun, 28 Aug 2016 11:41:58 +0300 Subject: [PATCH 082/112] Returned KafkaIO getWatermark log line in debug mode --- .../src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java | 1 + 1 file changed, 1 insertion(+) diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java index f6394220ceba..f16a5aba5815 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java @@ -1072,6 +1072,7 @@ private void updateLatestOffsets() { @Override public Instant getWatermark() { if (curRecord == null) { + LOG.debug("{}: getWatermark() : no records have been read yet.", name); return initialWatermark; } From 56c70027424cf665099ae64aca32f8076222033d Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Mon, 29 Aug 2016 13:09:58 -0700 Subject: [PATCH 083/112] take advantage of setup/teardown for KafkaWriter --- .../org/apache/beam/sdk/io/kafka/KafkaIO.java | 25 +++++++++---------- 1 file changed, 12 insertions(+), 13 deletions(-) diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java index f16a5aba5815..6769b3162a01 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java @@ -1325,16 +1325,12 @@ public KV apply(V element) { private static class KafkaWriter extends DoFn, Void> { - @StartBundle - public void startBundle(Context c) throws Exception { - // Producer initialization is fairly costly. Move this to future initialization api to avoid - // creating a producer for each bundle. - if (producer == null) { - if (producerFactoryFnOpt.isPresent()) { - producer = producerFactoryFnOpt.get().apply(producerConfig); - } else { - producer = new KafkaProducer(producerConfig); - } + @Setup + public void setup() { + if (producerFactoryFnOpt.isPresent()) { + producer = producerFactoryFnOpt.get().apply(producerConfig); + } else { + producer = new KafkaProducer(producerConfig); } } @@ -1349,13 +1345,16 @@ public void processElement(ProcessContext ctx) throws Exception { } @FinishBundle - public void finishBundle(Context c) throws Exception { + public void finishBundle(Context c) throws IOException { producer.flush(); - producer.close(); - producer = null; checkForFailures(); } + @Teardown + public void teardown() { + producer.close(); + } + /////////////////////////////////////////////////////////////////////////////////// private final String topic; From 5cf6592109cb9a8cadf431eb19900bf3887f34f2 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Fri, 5 Aug 2016 14:20:56 -0700 Subject: [PATCH 084/112] Add LeaderBoardTest This test exercises the PTransforms that make up the LeaderBoard example. This includes speculative and late trigger firings to produce team and individual scores on a global and fixed window basis. Refactor LeaderBoard to expose the team and user score calculations as composite PTransforms to enable this testing. --- .../examples/complete/game/LeaderBoard.java | 113 ++++-- .../complete/game/LeaderBoardTest.java | 362 ++++++++++++++++++ 2 files changed, 440 insertions(+), 35 deletions(-) create mode 100644 examples/java8/src/test/java/org/apache/beam/examples/complete/game/LeaderBoardTest.java diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java index 8dd4e39bee01..13bbf4482c6c 100644 --- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java +++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java @@ -17,6 +17,7 @@ */ package org.apache.beam.examples.complete.game; +import com.google.common.annotations.VisibleForTesting; import java.util.HashMap; import java.util.Map; import java.util.TimeZone; @@ -32,6 +33,7 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.StreamingOptions; import org.apache.beam.sdk.options.Validation; +import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime; import org.apache.beam.sdk.transforms.windowing.AfterWatermark; @@ -68,7 +70,7 @@ * here we're using an unbounded data source, which lets us provide speculative results, and allows * handling of late data, at much lower latency. We can use the early/speculative results to keep a * 'leaderboard' updated in near-realtime. Our handling of late data lets us generate correct - * results, e.g. for 'team prizes'. We're now outputing window results as they're + * results, e.g. for 'team prizes'. We're now outputting window results as they're * calculated, giving us much lower latency than with the previous batch examples. * *

              Run {@link injector.Injector} to generate pubsub data for this pipeline. The Injector @@ -186,50 +188,91 @@ public static void main(String[] args) throws Exception { .apply(PubsubIO.Read.timestampLabel(TIMESTAMP_ATTRIBUTE).topic(options.getTopic())) .apply("ParseGameEvent", ParDo.of(new ParseEventFn())); - // [START DocInclude_WindowAndTrigger] - // Extract team/score pairs from the event stream, using hour-long windows by default. - gameEvents - .apply("LeaderboardTeamFixedWindows", Window.into( - FixedWindows.of(Duration.standardMinutes(options.getTeamWindowDuration()))) - // We will get early (speculative) results as well as cumulative - // processing of late data. - .triggering( - AfterWatermark.pastEndOfWindow() - .withEarlyFirings(AfterProcessingTime.pastFirstElementInPane() - .plusDelayOf(FIVE_MINUTES)) - .withLateFirings(AfterProcessingTime.pastFirstElementInPane() - .plusDelayOf(TEN_MINUTES))) - .withAllowedLateness(Duration.standardMinutes(options.getAllowedLateness())) - .accumulatingFiredPanes()) - // Extract and sum teamname/score pairs from the event data. - .apply("ExtractTeamScore", new ExtractAndSumScore("team")) + gameEvents.apply("CalculateTeamScores", + new CalculateTeamScores( + Duration.standardMinutes(options.getTeamWindowDuration()), + Duration.standardMinutes(options.getAllowedLateness()))) // Write the results to BigQuery. .apply("WriteTeamScoreSums", new WriteWindowedToBigQuery>( options.getTableName() + "_team", configureWindowedTableWrite())); - // [END DocInclude_WindowAndTrigger] - - // [START DocInclude_ProcTimeTrigger] - // Extract user/score pairs from the event stream using processing time, via global windowing. - // Get periodic updates on all users' running scores. gameEvents - .apply("LeaderboardUserGlobalWindow", Window.into(new GlobalWindows()) - // Get periodic results every ten minutes. - .triggering(Repeatedly.forever(AfterProcessingTime.pastFirstElementInPane() - .plusDelayOf(TEN_MINUTES))) - .accumulatingFiredPanes() - .withAllowedLateness(Duration.standardMinutes(options.getAllowedLateness()))) - // Extract and sum username/score pairs from the event data. - .apply("ExtractUserScore", new ExtractAndSumScore("user")) + .apply( + "CalculateUserScores", + new CalculateUserScores(Duration.standardMinutes(options.getAllowedLateness()))) // Write the results to BigQuery. - .apply("WriteUserScoreSums", - new WriteToBigQuery>( - options.getTableName() + "_user", configureGlobalWindowBigQueryWrite())); - // [END DocInclude_ProcTimeTrigger] + .apply( + "WriteUserScoreSums", + new WriteToBigQuery>( + options.getTableName() + "_user", configureGlobalWindowBigQueryWrite())); // Run the pipeline and wait for the pipeline to finish; capture cancellation requests from the // command line. PipelineResult result = pipeline.run(); exampleUtils.waitToFinish(result); } + + /** + * Calculates scores for each team within the configured window duration. + */ + // [START DocInclude_WindowAndTrigger] + // Extract team/score pairs from the event stream, using hour-long windows by default. + @VisibleForTesting + static class CalculateTeamScores + extends PTransform, PCollection>> { + private final Duration teamWindowDuration; + private final Duration allowedLateness; + + CalculateTeamScores(Duration teamWindowDuration, Duration allowedLateness) { + this.teamWindowDuration = teamWindowDuration; + this.allowedLateness = allowedLateness; + } + + @Override + public PCollection> apply(PCollection infos) { + return infos.apply("LeaderboardTeamFixedWindows", + Window.into(FixedWindows.of(teamWindowDuration)) + // We will get early (speculative) results as well as cumulative + // processing of late data. + .triggering(AfterWatermark.pastEndOfWindow() + .withEarlyFirings(AfterProcessingTime.pastFirstElementInPane() + .plusDelayOf(FIVE_MINUTES)) + .withLateFirings(AfterProcessingTime.pastFirstElementInPane() + .plusDelayOf(TEN_MINUTES))) + .withAllowedLateness(allowedLateness) + .accumulatingFiredPanes()) + // Extract and sum teamname/score pairs from the event data. + .apply("ExtractTeamScore", new ExtractAndSumScore("team")); + } + } + // [END DocInclude_WindowAndTrigger] + + // [START DocInclude_ProcTimeTrigger] + /** + * Extract user/score pairs from the event stream using processing time, via global windowing. + * Get periodic updates on all users' running scores. + */ + @VisibleForTesting + static class CalculateUserScores + extends PTransform, PCollection>> { + private final Duration allowedLateness; + + CalculateUserScores(Duration allowedLateness) { + this.allowedLateness = allowedLateness; + } + + @Override + public PCollection> apply(PCollection input) { + return input.apply("LeaderboardUserGlobalWindow", + Window.into(new GlobalWindows()) + // Get periodic results every ten minutes. + .triggering(Repeatedly.forever(AfterProcessingTime.pastFirstElementInPane() + .plusDelayOf(TEN_MINUTES))) + .accumulatingFiredPanes() + .withAllowedLateness(allowedLateness)) + // Extract and sum username/score pairs from the event data. + .apply("ExtractUserScore", new ExtractAndSumScore("user")); + } + } + // [END DocInclude_ProcTimeTrigger] } diff --git a/examples/java8/src/test/java/org/apache/beam/examples/complete/game/LeaderBoardTest.java b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/LeaderBoardTest.java new file mode 100644 index 000000000000..40cac36f5a6e --- /dev/null +++ b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/LeaderBoardTest.java @@ -0,0 +1,362 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.examples.complete.game; + +import static org.apache.beam.sdk.testing.PAssert.that; +import static org.hamcrest.Matchers.hasItem; +import static org.junit.Assert.assertThat; + +import com.google.common.collect.ImmutableMap; +import java.io.Serializable; +import org.apache.beam.examples.complete.game.LeaderBoard.CalculateTeamScores; +import org.apache.beam.examples.complete.game.LeaderBoard.CalculateUserScores; +import org.apache.beam.examples.complete.game.UserScore.GameActionInfo; +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.testing.TestStream; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.GlobalWindow; +import org.apache.beam.sdk.transforms.windowing.IntervalWindow; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TimestampedValue; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Tests for {@link LeaderBoard}. + */ +@RunWith(JUnit4.class) +public class LeaderBoardTest implements Serializable { + private static final Duration ALLOWED_LATENESS = Duration.standardHours(1); + private static final Duration TEAM_WINDOW_DURATION = Duration.standardMinutes(20); + private Instant baseTime = new Instant(0); + + /** + * Some example users, on two separate teams. + */ + private enum TestUser { + RED_ONE("scarlet", "red"), RED_TWO("burgundy", "red"), + BLUE_ONE("navy", "blue"), BLUE_TWO("sky", "blue"); + + private final String userName; + private final String teamName; + + TestUser(String userName, String teamName) { + this.userName = userName; + this.teamName = teamName; + } + + public String getUser() { + return userName; + } + + public String getTeam() { + return teamName; + } + } + + /** + * A test of the {@link CalculateTeamScores} {@link PTransform} when all of the elements arrive + * on time (ahead of the watermark). + */ + @Test + public void testTeamScoresOnTime() { + TestPipeline p = TestPipeline.create(); + + TestStream createEvents = TestStream.create(AvroCoder.of(GameActionInfo.class)) + // Start at the epoch + .advanceWatermarkTo(baseTime) + // add some elements ahead of the watermark + .addElements(event(TestUser.BLUE_ONE, 3, Duration.standardSeconds(3)), + event(TestUser.BLUE_ONE, 2, Duration.standardMinutes(1)), + event(TestUser.RED_TWO, 3, Duration.standardSeconds(22)), + event(TestUser.BLUE_TWO, 5, Duration.standardMinutes(3))) + // The watermark advances slightly, but not past the end of the window + .advanceWatermarkTo(baseTime.plus(Duration.standardMinutes(3))) + // Add some more on time elements + .addElements(event(TestUser.RED_ONE, 1, Duration.standardMinutes(4)), + event(TestUser.BLUE_ONE, 2, Duration.standardSeconds(270))) + // The window should close and emit an ON_TIME pane + .advanceWatermarkToInfinity(); + + PCollection> teamScores = p.apply(createEvents) + .apply(new CalculateTeamScores(TEAM_WINDOW_DURATION, ALLOWED_LATENESS)); + + String blueTeam = TestUser.BLUE_ONE.getTeam(); + String redTeam = TestUser.RED_ONE.getTeam(); + that(teamScores) + .inOnTimePane(new IntervalWindow(baseTime, TEAM_WINDOW_DURATION)) + .containsInAnyOrder(KV.of(blueTeam, 12), KV.of(redTeam, 4)); + + p.run(); + } + + /** + * A test of the {@link CalculateTeamScores} {@link PTransform} when all of the elements arrive + * on time, and the processing time advances far enough for speculative panes. + */ + @Test + public void testTeamScoresSpeculative() { + TestPipeline p = TestPipeline.create(); + + TestStream createEvents = TestStream.create(AvroCoder.of(GameActionInfo.class)) + // Start at the epoch + .advanceWatermarkTo(baseTime) + .addElements(event(TestUser.BLUE_ONE, 3, Duration.standardSeconds(3)), + event(TestUser.BLUE_ONE, 2, Duration.standardMinutes(1))) + // Some time passes within the runner, which causes a speculative pane containing the blue + // team's score to be emitted + .advanceProcessingTime(Duration.standardMinutes(10)) + .addElements(event(TestUser.RED_TWO, 5, Duration.standardMinutes(3))) + // Some additional time passes and we get a speculative pane for the red team + .advanceProcessingTime(Duration.standardMinutes(12)) + .addElements(event(TestUser.BLUE_TWO, 3, Duration.standardSeconds(22))) + // More time passes and a speculative pane containing a refined value for the blue pane is + // emitted + .advanceProcessingTime(Duration.standardMinutes(10)) + // Some more events occur + .addElements(event(TestUser.RED_ONE, 4, Duration.standardMinutes(4)), + event(TestUser.BLUE_TWO, 2, Duration.standardMinutes(2))) + // The window closes and we get an ON_TIME pane that contains all of the updates + .advanceWatermarkToInfinity(); + + PCollection> teamScores = p.apply(createEvents) + .apply(new CalculateTeamScores(TEAM_WINDOW_DURATION, ALLOWED_LATENESS)); + + String blueTeam = TestUser.BLUE_ONE.getTeam(); + String redTeam = TestUser.RED_ONE.getTeam(); + IntervalWindow window = new IntervalWindow(baseTime, TEAM_WINDOW_DURATION); + // The window contains speculative panes alongside the on-time pane + PAssert.that(teamScores) + .inWindow(window) + .containsInAnyOrder(KV.of(blueTeam, 10) /* The on-time blue pane */, + KV.of(redTeam, 9) /* The on-time red pane */, + KV.of(blueTeam, 5) /* The first blue speculative pane */, + KV.of(blueTeam, 8) /* The second blue speculative pane */, + KV.of(redTeam, 5) /* The red speculative pane */); + PAssert.that(teamScores) + .inOnTimePane(window) + .containsInAnyOrder(KV.of(blueTeam, 10), KV.of(redTeam, 9)); + + p.run(); + } + + /** + * A test where elements arrive behind the watermark (late data), but before the end of the + * window. These elements are emitted on time. + */ + @Test + public void testTeamScoresUnobservablyLate() { + TestPipeline p = TestPipeline.create(); + + BoundedWindow window = new IntervalWindow(baseTime, TEAM_WINDOW_DURATION); + TestStream createEvents = TestStream.create(AvroCoder.of(GameActionInfo.class)) + .advanceWatermarkTo(baseTime) + .addElements(event(TestUser.BLUE_ONE, 3, Duration.standardSeconds(3)), + event(TestUser.BLUE_TWO, 5, Duration.standardMinutes(8)), + event(TestUser.RED_ONE, 4, Duration.standardMinutes(2)), + event(TestUser.BLUE_ONE, 3, Duration.standardMinutes(5))) + .advanceWatermarkTo(baseTime.plus(TEAM_WINDOW_DURATION).minus(Duration.standardMinutes(1))) + // These events are late, but the window hasn't closed yet, so the elements are in the + // on-time pane + .addElements(event(TestUser.RED_TWO, 2, Duration.ZERO), + event(TestUser.RED_TWO, 5, Duration.standardMinutes(1)), + event(TestUser.BLUE_TWO, 2, Duration.standardSeconds(90)), + event(TestUser.RED_TWO, 3, Duration.standardMinutes(3))) + .advanceWatermarkTo(baseTime.plus(TEAM_WINDOW_DURATION).plus(Duration.standardMinutes(1))) + .advanceWatermarkToInfinity(); + PCollection> teamScores = p.apply(createEvents) + .apply(new CalculateTeamScores(TEAM_WINDOW_DURATION, ALLOWED_LATENESS)); + + String blueTeam = TestUser.BLUE_ONE.getTeam(); + String redTeam = TestUser.RED_ONE.getTeam(); + // The On Time pane contains the late elements that arrived before the end of the window + PAssert.that(teamScores) + .inOnTimePane(window) + .containsInAnyOrder(KV.of(redTeam, 14), KV.of(blueTeam, 13)); + + p.run(); + } + + /** + * A test where elements arrive behind the watermark (late data) after the watermark passes the + * end of the window, but before the maximum allowed lateness. These elements are emitted in a + * late pane. + */ + @Test + public void testTeamScoresObservablyLate() { + TestPipeline p = TestPipeline.create(); + + Instant firstWindowCloses = baseTime.plus(ALLOWED_LATENESS).plus(TEAM_WINDOW_DURATION); + TestStream createEvents = TestStream.create(AvroCoder.of(GameActionInfo.class)) + .advanceWatermarkTo(baseTime) + .addElements(event(TestUser.BLUE_ONE, 3, Duration.standardSeconds(3)), + event(TestUser.BLUE_TWO, 5, Duration.standardMinutes(8))) + .advanceProcessingTime(Duration.standardMinutes(10)) + .advanceWatermarkTo(baseTime.plus(Duration.standardMinutes(3))) + .addElements(event(TestUser.RED_ONE, 3, Duration.standardMinutes(1)), + event(TestUser.RED_ONE, 4, Duration.standardMinutes(2)), + event(TestUser.BLUE_ONE, 3, Duration.standardMinutes(5))) + .advanceWatermarkTo(firstWindowCloses.minus(Duration.standardMinutes(1))) + // These events are late but should still appear in a late pane + .addElements(event(TestUser.RED_TWO, 2, Duration.ZERO), + event(TestUser.RED_TWO, 5, Duration.standardMinutes(1)), + event(TestUser.RED_TWO, 3, Duration.standardMinutes(3))) + // A late refinement is emitted due to the advance in processing time, but the window has + // not yet closed because the watermark has not advanced + .advanceProcessingTime(Duration.standardMinutes(12)) + // These elements should appear in the final pane + .addElements(event(TestUser.RED_TWO, 9, Duration.standardMinutes(1)), + event(TestUser.RED_TWO, 1, Duration.standardMinutes(3))) + .advanceWatermarkToInfinity(); + + PCollection> teamScores = p.apply(createEvents) + .apply(new CalculateTeamScores(TEAM_WINDOW_DURATION, ALLOWED_LATENESS)); + + BoundedWindow window = new IntervalWindow(baseTime, TEAM_WINDOW_DURATION); + String blueTeam = TestUser.BLUE_ONE.getTeam(); + String redTeam = TestUser.RED_ONE.getTeam(); + PAssert.that(teamScores) + .inWindow(window) + .satisfies((SerializableFunction>, Void>) input -> { + // The final sums need not exist in the same pane, but must appear in the output + // PCollection + assertThat(input, hasItem(KV.of(blueTeam, 11))); + assertThat(input, hasItem(KV.of(redTeam, 27))); + return null; + }); + PAssert.thatMap(teamScores) + // The closing behavior of CalculateTeamScores precludes an inFinalPane matcher + .inOnTimePane(window) + .isEqualTo(ImmutableMap.builder().put(redTeam, 7) + .put(blueTeam, 11) + .build()); + + // No final pane is emitted for the blue team, as all of their updates have been taken into + // account in earlier panes + PAssert.that(teamScores).inFinalPane(window).containsInAnyOrder(KV.of(redTeam, 27)); + + p.run(); + } + + /** + * A test where elements arrive beyond the maximum allowed lateness. These elements are dropped + * within {@link CalculateTeamScores} and do not impact the final result. + */ + @Test + public void testTeamScoresDroppablyLate() { + TestPipeline p = TestPipeline.create(); + + BoundedWindow window = new IntervalWindow(baseTime, TEAM_WINDOW_DURATION); + TestStream infos = TestStream.create(AvroCoder.of(GameActionInfo.class)) + .addElements(event(TestUser.BLUE_ONE, 12, Duration.ZERO), + event(TestUser.RED_ONE, 3, Duration.ZERO)) + .advanceWatermarkTo(window.maxTimestamp()) + .addElements(event(TestUser.RED_ONE, 4, Duration.standardMinutes(2)), + event(TestUser.BLUE_TWO, 3, Duration.ZERO), + event(TestUser.BLUE_ONE, 3, Duration.standardMinutes(3))) + // Move the watermark past the end of the allowed lateness plus the end of the window + .advanceWatermarkTo(baseTime.plus(ALLOWED_LATENESS) + .plus(TEAM_WINDOW_DURATION).plus(Duration.standardMinutes(1))) + // These elements within the expired window are droppably late, and will not appear in the + // output + .addElements( + event(TestUser.BLUE_TWO, 3, TEAM_WINDOW_DURATION.minus(Duration.standardSeconds(5))), + event(TestUser.RED_ONE, 7, Duration.standardMinutes(4))) + .advanceWatermarkToInfinity(); + PCollection> teamScores = p.apply(infos) + .apply(new CalculateTeamScores(TEAM_WINDOW_DURATION, ALLOWED_LATENESS)); + + String blueTeam = TestUser.BLUE_ONE.getTeam(); + String redTeam = TestUser.RED_ONE.getTeam(); + // Only one on-time pane and no late panes should be emitted + PAssert.that(teamScores) + .inWindow(window) + .containsInAnyOrder(KV.of(redTeam, 7), KV.of(blueTeam, 18)); + // No elements are added before the watermark passes the end of the window plus the allowed + // lateness, so no refinement should be emitted + PAssert.that(teamScores).inFinalPane(window).empty(); + } + + /** + * A test where elements arrive both on-time and late in {@link CalculateUserScores}, which emits + * output into the {@link GlobalWindow}. All elements that arrive should be taken into account, + * even if they arrive later than the maximum allowed lateness. + */ + @Test + public void testUserScore() { + TestPipeline p = TestPipeline.create(); + + TestStream infos = + TestStream.create(AvroCoder.of(GameActionInfo.class)) + .addElements( + event(TestUser.BLUE_ONE, 12, Duration.ZERO), + event(TestUser.RED_ONE, 3, Duration.ZERO)) + .advanceProcessingTime(Duration.standardMinutes(7)) + .addElements( + event(TestUser.RED_ONE, 4, Duration.standardMinutes(2)), + event(TestUser.BLUE_TWO, 3, Duration.ZERO), + event(TestUser.BLUE_ONE, 3, Duration.standardMinutes(3))) + .advanceProcessingTime(Duration.standardMinutes(5)) + .advanceWatermarkTo(baseTime.plus(ALLOWED_LATENESS).plus(Duration.standardHours(12))) + // Late elements are always observable within the global window - they arrive before + // the window closes, so they will appear in a pane, even if they arrive after the + // allowed lateness, and are taken into account alongside on-time elements + .addElements( + event(TestUser.RED_ONE, 3, Duration.standardMinutes(7)), + event(TestUser.RED_ONE, 2, (ALLOWED_LATENESS).plus(Duration.standardHours(13)))) + .advanceProcessingTime(Duration.standardMinutes(6)) + .addElements(event(TestUser.BLUE_TWO, 5, Duration.standardMinutes(12))) + .advanceProcessingTime(Duration.standardMinutes(20)) + .advanceWatermarkToInfinity(); + + PCollection> userScores = + p.apply(infos).apply(new CalculateUserScores(ALLOWED_LATENESS)); + + // User scores are emitted in speculative panes in the Global Window - this matcher choice + // ensures that panes emitted by the watermark advancing to positive infinity are not included, + // as that will not occur outside of tests + that(userScores) + .inEarlyGlobalWindowPanes() + .containsInAnyOrder(KV.of(TestUser.BLUE_ONE.getUser(), 15), + KV.of(TestUser.RED_ONE.getUser(), 7), + KV.of(TestUser.RED_ONE.getUser(), 12), + KV.of(TestUser.BLUE_TWO.getUser(), 3), + KV.of(TestUser.BLUE_TWO.getUser(), 8)); + + p.run(); + } + + private TimestampedValue event( + TestUser user, + int score, + Duration baseTimeOffset) { + return TimestampedValue.of(new GameActionInfo(user.getUser(), + user.getTeam(), + score, + baseTime.plus(baseTimeOffset).getMillis()), baseTime.plus(baseTimeOffset)); + } +} From 800c09870a732a71c897705fbecdedec4a961804 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Sat, 27 Aug 2016 14:01:34 +0200 Subject: [PATCH 085/112] [BEAM-569] Define maxNumRecords default value to Long.MAX_VALUE in JmsIO --- .../src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java b/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java index 29d0c5fac1d8..3107aab9fcff 100644 --- a/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java +++ b/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java @@ -101,11 +101,11 @@ public class JmsIO { private static final Logger LOG = LoggerFactory.getLogger(JmsIO.class); public static Read read() { - return new Read(); + return new Read(null, null, null, Long.MAX_VALUE, null); } public static Write write() { - return new Write(); + return new Write(null, null, null); } /** @@ -185,8 +185,6 @@ public void populateDisplayData(DisplayData.Builder builder) { protected long maxNumRecords; protected Duration maxReadTime; - private Read() {} - private Read( ConnectionFactory connectionFactory, String queue, @@ -428,8 +426,6 @@ public Write withTopic(String topic) { return new Write(connectionFactory, queue, topic); } - private Write() {} - private Write(ConnectionFactory connectionFactory, String queue, String topic) { this.connectionFactory = connectionFactory; this.queue = queue; From cae96380f15fc293d00b444148e5d08c3f14d909 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Thu, 25 Aug 2016 11:00:39 +0200 Subject: [PATCH 086/112] Address comments of Flink Side-Input PR --- .../beam/runners/core/SideInputHandler.java | 6 +- .../beam/runners/flink/FlinkRunner.java | 86 +++++++++++++++++-- .../wrappers/streaming/DoFnOperator.java | 13 ++- .../streaming/WindowDoFnOperator.java | 2 - 4 files changed, 89 insertions(+), 18 deletions(-) diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java index a97d3f306d0e..851ed37902ea 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java @@ -60,7 +60,11 @@ public class SideInputHandler implements ReadyCheckingSideInputReader { /** The list of side inputs that we're handling. */ protected final Collection> sideInputs; - /** State internals that are scoped not to the key of a value but instead to one key group. */ + /** + * State internals that are scoped not to the key of a value but are global. The state can still + * be keep locally but if side inputs are broadcast to all parallel operators then all will + * have the same view of the state. + */ private final StateInternals stateInternals; /** diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java index 8b1f42e5447c..d3c65c00773c 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java @@ -25,8 +25,13 @@ import java.net.URLClassLoader; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.SortedSet; +import java.util.TreeSet; + import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; @@ -35,6 +40,7 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsValidator; import org.apache.beam.sdk.runners.PipelineRunner; +import org.apache.beam.sdk.runners.TransformTreeNode; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.PTransform; @@ -47,6 +53,8 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; +import org.apache.beam.sdk.values.PValue; + import org.apache.flink.api.common.JobExecutionResult; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -108,6 +116,7 @@ public static FlinkRunner fromOptions(PipelineOptions options) { private FlinkRunner(FlinkPipelineOptions options) { this.options = options; + this.ptransformViewsWithNonDeterministicKeyCoders = new HashSet<>(); ImmutableMap.Builder, Class> builder = ImmutableMap., Class>builder(); if (options.isStreaming()) { @@ -124,6 +133,8 @@ private FlinkRunner(FlinkPipelineOptions options) { @Override public FlinkRunnerResult run(Pipeline pipeline) { + logWarningIfPCollectionViewHasNonDeterministicKeyCoder(pipeline); + LOG.info("Executing pipeline using FlinkRunner."); FlinkPipelineExecutionEnvironment env = new FlinkPipelineExecutionEnvironment(options); @@ -176,6 +187,7 @@ public OutputT apply( PTransform customTransform = InstanceBuilder.ofType(customTransformClass) + .withArg(FlinkRunner.class, this) .withArg(transformClass, transform) .build(); @@ -223,6 +235,59 @@ protected static List detectClassPathResourcesToStage( return files; } + /** A set of {@link View}s with non-deterministic key coders. */ + Set> ptransformViewsWithNonDeterministicKeyCoders; + + /** + * Records that the {@link PTransform} requires a deterministic key coder. + */ + private void recordViewUsesNonDeterministicKeyCoder(PTransform ptransform) { + ptransformViewsWithNonDeterministicKeyCoders.add(ptransform); + } + + /** Outputs a warning about PCollection views without deterministic key coders. */ + private void logWarningIfPCollectionViewHasNonDeterministicKeyCoder(Pipeline pipeline) { + // We need to wait till this point to determine the names of the transforms since only + // at this time do we know the hierarchy of the transforms otherwise we could + // have just recorded the full names during apply time. + if (!ptransformViewsWithNonDeterministicKeyCoders.isEmpty()) { + final SortedSet ptransformViewNamesWithNonDeterministicKeyCoders = new TreeSet<>(); + pipeline.traverseTopologically(new Pipeline.PipelineVisitor() { + @Override + public void visitValue(PValue value, TransformTreeNode producer) { + } + + @Override + public void visitPrimitiveTransform(TransformTreeNode node) { + if (ptransformViewsWithNonDeterministicKeyCoders.contains(node.getTransform())) { + ptransformViewNamesWithNonDeterministicKeyCoders.add(node.getFullName()); + } + } + + @Override + public CompositeBehavior enterCompositeTransform(TransformTreeNode node) { + if (ptransformViewsWithNonDeterministicKeyCoders.contains(node.getTransform())) { + ptransformViewNamesWithNonDeterministicKeyCoders.add(node.getFullName()); + } + return CompositeBehavior.ENTER_TRANSFORM; + } + + @Override + public void leaveCompositeTransform(TransformTreeNode node) { + } + }); + + LOG.warn("Unable to use indexed implementation for View.AsMap and View.AsMultimap for {} " + + "because the key coder is not deterministic. Falling back to singleton implementation " + + "which may cause memory and/or performance problems. Future major versions of " + + "the Flink runner will require deterministic key coders.", + ptransformViewNamesWithNonDeterministicKeyCoders); + } + } + + + ///////////////////////////////////////////////////////////////////////////// + /** * Specialized implementation for * {@link org.apache.beam.sdk.transforms.View.AsMap View.AsMap} @@ -231,8 +296,11 @@ protected static List detectClassPathResourcesToStage( private static class StreamingViewAsMap extends PTransform>, PCollectionView>> { + private final FlinkRunner runner; + @SuppressWarnings("unused") // used via reflection in FlinkRunner#apply() - public StreamingViewAsMap(View.AsMap transform) { + public StreamingViewAsMap(FlinkRunner runner, View.AsMap transform) { + this.runner = runner; } @Override @@ -248,7 +316,7 @@ public PCollectionView> apply(PCollection> input) { try { inputCoder.getKeyCoder().verifyDeterministic(); } catch (Coder.NonDeterministicException e) { -// runner.recordViewUsesNonDeterministicKeyCoder(this); + runner.recordViewUsesNonDeterministicKeyCoder(this); } return input @@ -270,11 +338,14 @@ protected String getKindString() { private static class StreamingViewAsMultimap extends PTransform>, PCollectionView>>> { + private final FlinkRunner runner; + /** * Builds an instance of this class from the overridden transform. */ @SuppressWarnings("unused") // used via reflection in FlinkRunner#apply() - public StreamingViewAsMultimap(View.AsMultimap transform) { + public StreamingViewAsMultimap(FlinkRunner runner, View.AsMultimap transform) { + this.runner = runner; } @Override @@ -290,7 +361,7 @@ public PCollectionView>> apply(PCollection> input) { try { inputCoder.getKeyCoder().verifyDeterministic(); } catch (Coder.NonDeterministicException e) { -// runner.recordViewUsesNonDeterministicKeyCoder(this); + runner.recordViewUsesNonDeterministicKeyCoder(this); } return input @@ -315,7 +386,7 @@ private static class StreamingViewAsList * Builds an instance of this class from the overridden transform. */ @SuppressWarnings("unused") // used via reflection in FlinkRunner#apply() - public StreamingViewAsList(View.AsList transform) {} + public StreamingViewAsList(FlinkRunner runner, View.AsList transform) {} @Override public PCollectionView> apply(PCollection input) { @@ -346,7 +417,7 @@ private static class StreamingViewAsIterable * Builds an instance of this class from the overridden transform. */ @SuppressWarnings("unused") // used via reflection in FlinkRunner#apply() - public StreamingViewAsIterable(View.AsIterable transform) { } + public StreamingViewAsIterable(FlinkRunner runner, View.AsIterable transform) { } @Override public PCollectionView> apply(PCollection input) { @@ -386,7 +457,7 @@ private static class StreamingViewAsSingleton * Builds an instance of this class from the overridden transform. */ @SuppressWarnings("unused") // used via reflection in FlinkRunner#apply() - public StreamingViewAsSingleton(View.AsSingleton transform) { + public StreamingViewAsSingleton(FlinkRunner runner, View.AsSingleton transform) { this.transform = transform; } @@ -443,6 +514,7 @@ private static class StreamingCombineGloballyAsSingletonView */ @SuppressWarnings("unused") // used via reflection in FlinkRunner#apply() public StreamingCombineGloballyAsSingletonView( + FlinkRunner runner, Combine.GloballyAsSingletonView transform) { this.transform = transform; } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java index 000d69f34823..2c7ebc697709 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java @@ -75,11 +75,12 @@ import org.apache.flink.streaming.runtime.tasks.StreamTaskState; /** - * Flink operator for executing {@link DoFn DoFns}. + * Flink operator for executing {@link OldDoFn DoFns}. * - * @param - * @param - * @param + * @param the input type of the {@link OldDoFn} + * @param the output type of the {@link OldDoFn} + * @param the output type of the operator, this can be different from the fn output type when we have + * side outputs */ public class DoFnOperator extends AbstractStreamOperator @@ -95,8 +96,6 @@ public class DoFnOperator protected final Collection> sideInputs; protected final Map> sideInputTagMapping; - protected final boolean hasSideInputs; - protected final WindowingStrategy windowingStrategy; protected final OutputManagerFactory outputManagerFactory; @@ -136,8 +135,6 @@ public DoFnOperator( this.windowingStrategy = windowingStrategy; this.outputManagerFactory = outputManagerFactory; - this.hasSideInputs = !sideInputs.isEmpty(); - this.pushedBackWatermarkDescriptor = new ReducingStateDescriptor<>( "pushed-back-elements-watermark-hold", diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java index c6dde5197dce..01cfa5b6a26b 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java @@ -201,8 +201,6 @@ public void processWatermark1(Watermark mark) throws Exception { if (timer != null && timer.f1.getTimestamp().getMillis() < actualInputWatermark) { fire = true; - System.out.println("FIRING: " + timer); - watermarkTimersQueue.remove(); watermarkTimers.remove(timer); From 98da6e8fb014d2a93b7441f6b2b131968d874ab6 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Wed, 31 Aug 2016 13:42:30 +0200 Subject: [PATCH 087/112] Fix condition in FlinkStreamingPipelineTranslator --- .../flink/translation/FlinkStreamingPipelineTranslator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java index b12745512c93..284cd2321e20 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java @@ -84,7 +84,7 @@ public void visitPrimitiveTransform(TransformTreeNode node) { StreamTransformTranslator translator = FlinkStreamingTransformTranslators.getTranslator(transform); - if (translator == null && applyCanTranslate(transform, node, translator)) { + if (translator == null || !applyCanTranslate(transform, node, translator)) { LOG.info(node.getTransform().getClass().toString()); throw new UnsupportedOperationException( "The transform " + transform + " is currently not supported."); From f70aa49e2babc79a65a339309776837be2a45126 Mon Sep 17 00:00:00 2001 From: bchambers Date: Wed, 3 Aug 2016 13:38:43 -0700 Subject: [PATCH 088/112] Correct some accidental renames IDE over-eagerly replaced some occurrences of createAggregator with createAggregatorForDoFn. This corrects that. --- .../java/org/apache/beam/sdk/util/DoFnRunnerBase.java | 2 +- .../java/org/apache/beam/sdk/transforms/Aggregator.java | 8 ++++---- .../java/org/apache/beam/sdk/transforms/DoFnTester.java | 2 +- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java index 04a0978b6030..f0cfd7455de5 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java @@ -344,7 +344,7 @@ public void sideOutputWithTimestamp(TupleTag tag, T output, Instant times @Override protected Aggregator createAggregatorInternal( String name, CombineFn combiner) { - checkNotNull(combiner, "Combiner passed to createAggregatorForDoFn cannot be null"); + checkNotNull(combiner, "Combiner passed to createAggregatorInternal cannot be null"); return aggregatorFactory.createAggregatorForDoFn(fn.getClass(), stepContext, name, combiner); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Aggregator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Aggregator.java index 67d399fbc820..e8f624792439 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Aggregator.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Aggregator.java @@ -25,7 +25,7 @@ * to be combined across all bundles. * *

              Aggregators are created by calling - * {@link DoFn#createAggregator DoFn.createAggregatorForDoFn}, + * {@link DoFn#createAggregator DoFn.createAggregator}, * typically from the {@link DoFn} constructor. Elements can be added to the * {@code Aggregator} by calling {@link Aggregator#addValue}. * @@ -41,7 +41,7 @@ * private Aggregator myAggregator; * * public MyDoFn() { - * myAggregator = createAggregatorForDoFn("myAggregator", new Sum.SumIntegerFn()); + * myAggregator = createAggregator("myAggregator", new Sum.SumIntegerFn()); * } * * @ProcessElement @@ -89,9 +89,9 @@ Aggregator createAggregatorForDoFn( } // TODO: Consider the following additional API conveniences: - // - In addition to createAggregatorForDoFn(), consider adding getAggregator() to + // - In addition to createAggregator(), consider adding getAggregator() to // avoid the need to store the aggregator locally in a DoFn, i.e., create // if not already present. // - Add a shortcut for the most common aggregator: - // c.createAggregatorForDoFn("name", new Sum.SumIntegerFn()). + // c.createAggregator("name", new Sum.SumIntegerFn()). } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java index 6801768c1096..b867a5582b97 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java @@ -667,7 +667,7 @@ protected Aggregator createAggreg String name, CombineFn combiner) { throw new IllegalStateException("Aggregators should not be created within ProcessContext. " + "Instead, create an aggregator at OldDoFn construction time with" - + " createAggregatorForDoFn, and ensure they are set up by the time startBundle is" + + " createAggregator, and ensure they are set up by the time startBundle is" + " called with setupDelegateAggregators."); } } From 89680975b5a89351ccc4bf99a3a6bd8772d87f40 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Tue, 30 Aug 2016 14:17:50 -0700 Subject: [PATCH 089/112] Test that multiple instances of TestStream are supported Add KeyedResourcePool This interface represents some shared pool of values that may be used by at most one caller at a time. Add LockedKeyedResourcePool which has at most one value per key and at most one user per value at a time. Use KeyedResourcePool in TestStream --- .../runners/direct/KeyedResourcePool.java | 47 ++++ .../direct/LockedKeyedResourcePool.java | 95 ++++++++ .../direct/TestStreamEvaluatorFactory.java | 141 +++++++----- .../direct/LockedKeyedResourcePoolTest.java | 163 ++++++++++++++ .../TestStreamEvaluatorFactoryTest.java | 206 ++++++++++++++++++ .../beam/sdk/testing/TestStreamTest.java | 29 +++ 6 files changed, 623 insertions(+), 58 deletions(-) create mode 100644 runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedResourcePool.java create mode 100644 runners/direct-java/src/main/java/org/apache/beam/runners/direct/LockedKeyedResourcePool.java create mode 100644 runners/direct-java/src/test/java/org/apache/beam/runners/direct/LockedKeyedResourcePoolTest.java create mode 100644 runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedResourcePool.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedResourcePool.java new file mode 100644 index 000000000000..b976b696925d --- /dev/null +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedResourcePool.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.runners.direct; + +import com.google.common.base.Optional; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; + +/** + * A pool of resources associated with specific keys. Implementations enforce specific use patterns, + * such as limiting the the number of outstanding elements available per key. + */ +interface KeyedResourcePool { + /** + * Tries to acquire a value for the provided key, loading it via the provided loader if necessary. + * + *

              If the returned {@link Optional} contains a value, the caller obtains ownership of that + * value. The value should be released back to this {@link KeyedResourcePool} after the + * caller no longer has use of it using {@link #release(Object, Object)}. + * + *

              The provided {@link Callable} must not return null; it may either return a non-null + * value or throw an exception. + */ + Optional tryAcquire(K key, Callable loader) throws ExecutionException; + + /** + * Release the provided value, relinquishing ownership of it. Future calls to + * {@link #tryAcquire(Object, Callable)} may return the released value. + */ + void release(K key, V value); +} diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/LockedKeyedResourcePool.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/LockedKeyedResourcePool.java new file mode 100644 index 000000000000..8b1e0b17e618 --- /dev/null +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/LockedKeyedResourcePool.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.runners.direct; + +import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.base.Preconditions.checkState; + +import com.google.common.base.Optional; +import com.google.common.util.concurrent.ExecutionError; +import com.google.common.util.concurrent.UncheckedExecutionException; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutionException; + +/** + * A {@link KeyedResourcePool} which is limited to at most one outstanding instance at a time for + * each key. + */ +class LockedKeyedResourcePool implements KeyedResourcePool { + /** + * A map from each key to an {@link Optional} of the associated value. At most one value is stored + * per key, and it is obtained by at most one thread at a time. + * + *

              For each key in this map: + * + *

                + *
              • If there is no associated value, then no value has been stored yet. + *
              • If the value is {@code Optional.absent()} then the value is currently in use. + *
              • If the value is {@code Optional.present()} then the contained value is available for use. + *
              + */ + public static LockedKeyedResourcePool create() { + return new LockedKeyedResourcePool<>(); + } + + private final ConcurrentMap> cache; + + private LockedKeyedResourcePool() { + cache = new ConcurrentHashMap<>(); + } + + @Override + public Optional tryAcquire(K key, Callable loader) throws ExecutionException { + Optional value = cache.replace(key, Optional.absent()); + if (value == null) { + // No value already existed, so populate the cache with the value returned by the loader + cache.putIfAbsent(key, Optional.of(load(loader))); + // Some other thread may obtain the result after the putIfAbsent, so retry acquisition + value = cache.replace(key, Optional.absent()); + } + return value; + } + + private V load(Callable loader) throws ExecutionException { + try { + return loader.call(); + } catch (Error t) { + throw new ExecutionError(t); + } catch (RuntimeException e) { + throw new UncheckedExecutionException(e); + } catch (Exception e) { + throw new ExecutionException(e); + } + } + + @Override + public void release(K key, V value) { + Optional replaced = cache.replace(key, Optional.of(value)); + checkNotNull(replaced, "Tried to release before a value was acquired"); + checkState( + !replaced.isPresent(), + "Released a value to a %s where there is already a value present for key %s (%s). " + + "At most one value may be present at a time.", + LockedKeyedResourcePool.class.getSimpleName(), + key, + replaced); + } +} diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java index e9f37bab7567..3dbd886fc75e 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java @@ -22,12 +22,12 @@ import com.google.common.base.Supplier; import java.util.List; -import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicReference; import javax.annotation.Nullable; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.testing.TestStream; import org.apache.beam.sdk.testing.TestStream.ElementEvent; @@ -49,43 +49,52 @@ import org.joda.time.Duration; import org.joda.time.Instant; -/** - * The {@link TransformEvaluatorFactory} for the {@link TestStream} primitive. - */ +/** The {@link TransformEvaluatorFactory} for the {@link TestStream} primitive. */ class TestStreamEvaluatorFactory implements TransformEvaluatorFactory { - private final AtomicBoolean inUse = new AtomicBoolean(false); - private final AtomicReference> evaluator = new AtomicReference<>(); + private final KeyedResourcePool, Evaluator> evaluators = + LockedKeyedResourcePool.create(); @Nullable @Override public TransformEvaluator forApplication( AppliedPTransform application, @Nullable CommittedBundle inputBundle, - EvaluationContext evaluationContext) throws Exception { + EvaluationContext evaluationContext) + throws Exception { return createEvaluator((AppliedPTransform) application, evaluationContext); } @Override public void cleanup() throws Exception {} + /** + * Returns the evaluator for the provided application of {@link TestStream}, or null if it is + * already in use. + * + *

              The documented behavior of {@link TestStream} requires the output of one event to travel + * completely through the pipeline before any additional event, so additional instances that have + * a separate collection of events cannot be created. + */ private TransformEvaluator createEvaluator( AppliedPTransform, TestStream> application, - EvaluationContext evaluationContext) { - if (evaluator.get() == null) { - Evaluator createdEvaluator = new Evaluator<>(application, evaluationContext, inUse); - evaluator.compareAndSet(null, createdEvaluator); - } - if (inUse.compareAndSet(false, true)) { - return evaluator.get(); - } else { - return null; - } + EvaluationContext evaluationContext) + throws ExecutionException { + return evaluators + .tryAcquire(application, new CreateEvaluator<>(application, evaluationContext, evaluators)) + .orNull(); } + /** + * Release the provided {@link Evaluator} after completing an evaluation. The next call to {@link + * #createEvaluator(AppliedPTransform, EvaluationContext)} with the {@link AppliedPTransform} will + * return this evaluator. + */ + private void completeEvaluation(Evaluator evaluator) {} + private static class Evaluator implements TransformEvaluator { private final AppliedPTransform, TestStream> application; private final EvaluationContext context; - private final AtomicBoolean inUse; + private final KeyedResourcePool, Evaluator> cache; private final List> events; private int index; private Instant currentWatermark; @@ -93,49 +102,48 @@ private static class Evaluator implements TransformEvaluator { private Evaluator( AppliedPTransform, TestStream> application, EvaluationContext context, - AtomicBoolean inUse) { + KeyedResourcePool, Evaluator> cache) { this.application = application; this.context = context; - this.inUse = inUse; + this.cache = cache; this.events = application.getTransform().getEvents(); index = 0; currentWatermark = BoundedWindow.TIMESTAMP_MIN_VALUE; } @Override - public void processElement(WindowedValue element) throws Exception { - } + public void processElement(WindowedValue element) throws Exception {} @Override public TransformResult finishBundle() throws Exception { - if (index >= events.size()) { - return StepTransformResult.withHold(application, BoundedWindow.TIMESTAMP_MAX_VALUE).build(); - } - Event event = events.get(index); - if (event.getType().equals(EventType.WATERMARK)) { - currentWatermark = ((WatermarkEvent) event).getWatermark(); - } - StepTransformResult.Builder result = - StepTransformResult.withHold(application, currentWatermark); - if (event.getType().equals(EventType.ELEMENT)) { - UncommittedBundle bundle = context.createRootBundle(application.getOutput()); - for (TimestampedValue elem : ((ElementEvent) event).getElements()) { - bundle.add(WindowedValue.timestampedValueInGlobalWindow(elem.getValue(), - elem.getTimestamp())); + try { + if (index >= events.size()) { + return StepTransformResult.withHold(application, BoundedWindow.TIMESTAMP_MAX_VALUE) + .build(); } - result.addOutput(bundle); - } - if (event.getType().equals(EventType.PROCESSING_TIME)) { - ((TestClock) context.getClock()) - .advance(((ProcessingTimeEvent) event).getProcessingTimeAdvance()); + Event event = events.get(index); + if (event.getType().equals(EventType.WATERMARK)) { + currentWatermark = ((WatermarkEvent) event).getWatermark(); + } + StepTransformResult.Builder result = + StepTransformResult.withHold(application, currentWatermark); + if (event.getType().equals(EventType.ELEMENT)) { + UncommittedBundle bundle = context.createRootBundle(application.getOutput()); + for (TimestampedValue elem : ((ElementEvent) event).getElements()) { + bundle.add( + WindowedValue.timestampedValueInGlobalWindow(elem.getValue(), elem.getTimestamp())); + } + result.addOutput(bundle); + } + if (event.getType().equals(EventType.PROCESSING_TIME)) { + ((TestClock) context.getClock()) + .advance(((ProcessingTimeEvent) event).getProcessingTimeAdvance()); + } + index++; + return result.build(); + } finally { + cache.release(application, this); } - index++; - checkState(inUse.compareAndSet(true, false), - "The InUse flag of a %s was changed while the source evaluator was executing. " - + "%s cannot be split or evaluated in parallel.", - TestStream.class.getSimpleName(), - TestStream.class.getSimpleName()); - return result.build(); } } @@ -181,20 +189,37 @@ private DirectTestStream(TestStream transform) { @Override public PCollection apply(PBegin input) { - setup(input.getPipeline()); - return PCollection.createPrimitiveOutputInternal( - input.getPipeline(), WindowingStrategy.globalDefault(), IsBounded.UNBOUNDED) - .setCoder(original.getValueCoder()); - } - - private void setup(Pipeline p) { - PipelineRunner runner = p.getRunner(); - checkState(runner instanceof DirectRunner, + PipelineRunner runner = input.getPipeline().getRunner(); + checkState( + runner instanceof DirectRunner, "%s can only be used when running with the %s", getClass().getSimpleName(), DirectRunner.class.getSimpleName()); ((DirectRunner) runner).setClockSupplier(new TestClockSupplier()); + return PCollection.createPrimitiveOutputInternal( + input.getPipeline(), WindowingStrategy.globalDefault(), IsBounded.UNBOUNDED) + .setCoder(original.getValueCoder()); } } } + + private static class CreateEvaluator implements Callable> { + private final AppliedPTransform, TestStream> application; + private final EvaluationContext evaluationContext; + private final KeyedResourcePool, Evaluator> evaluators; + + public CreateEvaluator( + AppliedPTransform, TestStream> application, + EvaluationContext evaluationContext, + KeyedResourcePool, Evaluator> evaluators) { + this.application = application; + this.evaluationContext = evaluationContext; + this.evaluators = evaluators; + } + + @Override + public Evaluator call() throws Exception { + return new Evaluator<>(application, evaluationContext, evaluators); + } + } } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/LockedKeyedResourcePoolTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/LockedKeyedResourcePoolTest.java new file mode 100644 index 000000000000..e1e24a37276a --- /dev/null +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/LockedKeyedResourcePoolTest.java @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.runners.direct; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThat; + +import com.google.common.base.Optional; +import com.google.common.util.concurrent.ExecutionError; +import com.google.common.util.concurrent.UncheckedExecutionException; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Tests for {@link LockedKeyedResourcePool}. + */ +@RunWith(JUnit4.class) +public class LockedKeyedResourcePoolTest { + @Rule public ExpectedException thrown = ExpectedException.none(); + private LockedKeyedResourcePool cache = + LockedKeyedResourcePool.create(); + + @Test + public void acquireReleaseAcquireLastLoadedOrReleased() throws ExecutionException { + Optional returned = cache.tryAcquire("foo", new Callable() { + @Override + public Integer call() throws Exception { + return 3; + } + }); + assertThat(returned.get(), equalTo(3)); + + cache.release("foo", 4); + Optional reacquired = cache.tryAcquire("foo", new Callable() { + @Override + public Integer call() throws Exception { + return 5; + } + }); + assertThat(reacquired.get(), equalTo(4)); + } + + @Test + public void acquireReleaseReleaseThrows() throws ExecutionException { + Optional returned = cache.tryAcquire("foo", new Callable() { + @Override + public Integer call() throws Exception { + return 3; + } + }); + assertThat(returned.get(), equalTo(3)); + + cache.release("foo", 4); + thrown.expect(IllegalStateException.class); + thrown.expectMessage("already a value present"); + thrown.expectMessage("At most one"); + cache.release("foo", 4); + } + + @Test + public void releaseBeforeAcquireThrows() { + thrown.expect(NullPointerException.class); + thrown.expectMessage("before a value was acquired"); + cache.release("bar", 3); + } + + @Test + public void multipleAcquireWithoutReleaseAbsent() throws ExecutionException { + Optional returned = cache.tryAcquire("foo", new Callable() { + @Override + public Integer call() throws Exception { + return 3; + } + }); + Optional secondReturned = cache.tryAcquire("foo", new Callable() { + @Override + public Integer call() throws Exception { + return 3; + } + }); + assertThat(secondReturned.isPresent(), is(false)); + } + + @Test + public void acquireMultipleKeysSucceeds() throws ExecutionException { + Optional returned = cache.tryAcquire("foo", new Callable() { + @Override + public Integer call() throws Exception { + return 3; + } + }); + Optional secondReturned = cache.tryAcquire("bar", new Callable() { + @Override + public Integer call() throws Exception { + return 4; + } + }); + + assertThat(returned.get(), equalTo(3)); + assertThat(secondReturned.get(), equalTo(4)); + } + + @Test + public void acquireThrowsExceptionWrapped() throws ExecutionException { + final Exception cause = new Exception("checkedException"); + thrown.expect(ExecutionException.class); + thrown.expectCause(equalTo(cause)); + cache.tryAcquire("foo", new Callable() { + @Override + public Integer call() throws Exception { + throw cause; + } + }); + } + + @Test + public void acquireThrowsRuntimeExceptionWrapped() throws ExecutionException { + final RuntimeException cause = new RuntimeException("UncheckedException"); + thrown.expect(UncheckedExecutionException.class); + thrown.expectCause(equalTo(cause)); + cache.tryAcquire("foo", new Callable() { + @Override + public Integer call() throws Exception { + throw cause; + } + }); + } + + @Test + public void acquireThrowsErrorWrapped() throws ExecutionException { + final Error cause = new Error("Error"); + thrown.expect(ExecutionError.class); + thrown.expectCause(equalTo(cause)); + cache.tryAcquire("foo", new Callable() { + @Override + public Integer call() throws Exception { + throw cause; + } + }); + } +} diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java new file mode 100644 index 000000000000..7703881abf33 --- /dev/null +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java @@ -0,0 +1,206 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.runners.direct; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.nullValue; +import static org.junit.Assert.assertThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import com.google.common.collect.Iterables; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.coders.VarIntCoder; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.testing.TestStream; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.PCollection; +import org.hamcrest.Matchers; +import org.joda.time.Instant; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for {@link TestStreamEvaluatorFactory}. */ +@RunWith(JUnit4.class) +public class TestStreamEvaluatorFactoryTest { + private TestStreamEvaluatorFactory factory = new TestStreamEvaluatorFactory(); + private BundleFactory bundleFactory = ImmutableListBundleFactory.create(); + + /** Demonstrates that returned evaluators produce elements in sequence. */ + @Test + public void producesElementsInSequence() throws Exception { + TestPipeline p = TestPipeline.create(); + PCollection streamVals = + p.apply( + TestStream.create(VarIntCoder.of()) + .addElements(1, 2, 3) + .addElements(4, 5, 6) + .advanceWatermarkToInfinity()); + + EvaluationContext context = mock(EvaluationContext.class); + when(context.createRootBundle(streamVals)) + .thenReturn( + bundleFactory.createRootBundle(streamVals), bundleFactory.createRootBundle(streamVals)); + + TransformEvaluator firstEvaluator = + factory.forApplication(streamVals.getProducingTransformInternal(), null, context); + TransformResult firstResult = firstEvaluator.finishBundle(); + + TransformEvaluator secondEvaluator = + factory.forApplication(streamVals.getProducingTransformInternal(), null, context); + TransformResult secondResult = secondEvaluator.finishBundle(); + + TransformEvaluator thirdEvaluator = + factory.forApplication(streamVals.getProducingTransformInternal(), null, context); + TransformResult thirdResult = thirdEvaluator.finishBundle(); + + assertThat( + Iterables.getOnlyElement(firstResult.getOutputBundles()) + .commit(Instant.now()) + .getElements(), + Matchers.>containsInAnyOrder( + WindowedValue.valueInGlobalWindow(1), + WindowedValue.valueInGlobalWindow(2), + WindowedValue.valueInGlobalWindow(3))); + assertThat(firstResult.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE)); + + assertThat( + Iterables.getOnlyElement(secondResult.getOutputBundles()) + .commit(Instant.now()) + .getElements(), + Matchers.>containsInAnyOrder( + WindowedValue.valueInGlobalWindow(4), + WindowedValue.valueInGlobalWindow(5), + WindowedValue.valueInGlobalWindow(6))); + assertThat(secondResult.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE)); + + assertThat(Iterables.isEmpty(thirdResult.getOutputBundles()), is(true)); + assertThat(thirdResult.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE)); + } + + /** Demonstrates that at most one evaluator for an application is available at a time. */ + @Test + public void onlyOneEvaluatorAtATime() throws Exception { + TestPipeline p = TestPipeline.create(); + PCollection streamVals = + p.apply( + TestStream.create(VarIntCoder.of()).addElements(4, 5, 6).advanceWatermarkToInfinity()); + + EvaluationContext context = mock(EvaluationContext.class); + TransformEvaluator firstEvaluator = + factory.forApplication(streamVals.getProducingTransformInternal(), null, context); + + // create a second evaluator before the first is finished. The evaluator should not be available + TransformEvaluator secondEvaluator = + factory.forApplication(streamVals.getProducingTransformInternal(), null, context); + assertThat(secondEvaluator, is(nullValue())); + } + + /** + * Demonstrates that multiple applications of the same {@link TestStream} produce separate + * evaluators. + */ + @Test + public void multipleApplicationsMultipleEvaluators() throws Exception { + TestPipeline p = TestPipeline.create(); + TestStream stream = + TestStream.create(VarIntCoder.of()).addElements(2).advanceWatermarkToInfinity(); + PCollection firstVals = p.apply("Stream One", stream); + PCollection secondVals = p.apply("Stream A", stream); + + EvaluationContext context = mock(EvaluationContext.class); + when(context.createRootBundle(firstVals)).thenReturn(bundleFactory.createRootBundle(firstVals)); + when(context.createRootBundle(secondVals)) + .thenReturn(bundleFactory.createRootBundle(secondVals)); + + TransformEvaluator firstEvaluator = + factory.forApplication(firstVals.getProducingTransformInternal(), null, context); + // The two evaluators can exist independently + TransformEvaluator secondEvaluator = + factory.forApplication(secondVals.getProducingTransformInternal(), null, context); + + TransformResult firstResult = firstEvaluator.finishBundle(); + TransformResult secondResult = secondEvaluator.finishBundle(); + + assertThat( + Iterables.getOnlyElement(firstResult.getOutputBundles()) + .commit(Instant.now()) + .getElements(), + Matchers.>containsInAnyOrder(WindowedValue.valueInGlobalWindow(2))); + assertThat(firstResult.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE)); + + // They both produce equal results, and don't interfere with each other + assertThat( + Iterables.getOnlyElement(secondResult.getOutputBundles()) + .commit(Instant.now()) + .getElements(), + Matchers.>containsInAnyOrder(WindowedValue.valueInGlobalWindow(2))); + assertThat(secondResult.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE)); + } + + /** + * Demonstrates that multiple applications of different {@link TestStream} produce independent + * evaluators. + */ + @Test + public void multipleStreamsMultipleEvaluators() throws Exception { + TestPipeline p = TestPipeline.create(); + PCollection firstVals = + p.apply( + "Stream One", + TestStream.create(VarIntCoder.of()).addElements(2).advanceWatermarkToInfinity()); + PCollection secondVals = + p.apply( + "Stream A", + TestStream.create(StringUtf8Coder.of()) + .addElements("Two") + .advanceWatermarkToInfinity()); + + EvaluationContext context = mock(EvaluationContext.class); + when(context.createRootBundle(firstVals)).thenReturn(bundleFactory.createRootBundle(firstVals)); + when(context.createRootBundle(secondVals)) + .thenReturn(bundleFactory.createRootBundle(secondVals)); + + TransformEvaluator firstEvaluator = + factory.forApplication(firstVals.getProducingTransformInternal(), null, context); + // The two evaluators can exist independently + TransformEvaluator secondEvaluator = + factory.forApplication(secondVals.getProducingTransformInternal(), null, context); + + TransformResult firstResult = firstEvaluator.finishBundle(); + TransformResult secondResult = secondEvaluator.finishBundle(); + + assertThat( + Iterables.getOnlyElement(firstResult.getOutputBundles()) + .commit(Instant.now()) + .getElements(), + Matchers.>containsInAnyOrder(WindowedValue.valueInGlobalWindow(2))); + assertThat(firstResult.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE)); + + assertThat( + Iterables.getOnlyElement(secondResult.getOutputBundles()) + .commit(Instant.now()) + .getElements(), + Matchers.>containsInAnyOrder(WindowedValue.valueInGlobalWindow("Two"))); + assertThat(secondResult.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE)); + } +} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java index 6457f910308f..a1b4e4a8b3ed 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java @@ -264,6 +264,35 @@ public void testElementsAtAlmostPositiveInfinity() { p.run(); } + @Test + @Category(NeedsRunner.class) + public void testMultipleStreams() { + TestStream stream = TestStream.create(StringUtf8Coder.of()) + .addElements("foo", "bar") + .advanceWatermarkToInfinity(); + + TestStream other = + TestStream.create(VarIntCoder.of()).addElements(1, 2, 3, 4).advanceWatermarkToInfinity(); + + TestPipeline p = TestPipeline.create(); + PCollection createStrings = + p.apply("CreateStrings", stream) + .apply("WindowStrings", + Window.triggering(AfterPane.elementCountAtLeast(2)) + .withAllowedLateness(Duration.ZERO) + .accumulatingFiredPanes()); + PAssert.that(createStrings).containsInAnyOrder("foo", "bar"); + PCollection createInts = + p.apply("CreateInts", other) + .apply("WindowInts", + Window.triggering(AfterPane.elementCountAtLeast(4)) + .withAllowedLateness(Duration.ZERO) + .accumulatingFiredPanes()); + PAssert.that(createInts).containsInAnyOrder(1, 2, 3, 4); + + p.run(); + } + @Test public void testElementAtPositiveInfinityThrows() { Builder stream = From ccfb78eac4d9b992d8694ad1f6347f50d80169c1 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Wed, 31 Aug 2016 15:34:21 -0700 Subject: [PATCH 090/112] Remove empty unused method in TestStreamEvaluatorFactory --- .../beam/runners/direct/TestStreamEvaluatorFactory.java | 7 ------- 1 file changed, 7 deletions(-) diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java index 3dbd886fc75e..5fe771c52121 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java @@ -84,13 +84,6 @@ private TransformEvaluator createEvaluator( .orNull(); } - /** - * Release the provided {@link Evaluator} after completing an evaluation. The next call to {@link - * #createEvaluator(AppliedPTransform, EvaluationContext)} with the {@link AppliedPTransform} will - * return this evaluator. - */ - private void completeEvaluation(Evaluator evaluator) {} - private static class Evaluator implements TransformEvaluator { private final AppliedPTransform, TestStream> application; private final EvaluationContext context; From 0d3f2f2b7300f7a03b7f94c927f8160b8cd81798 Mon Sep 17 00:00:00 2001 From: Scott Wegner Date: Thu, 18 Aug 2016 13:56:34 -0700 Subject: [PATCH 091/112] Add Latest CombineFn and PTransforms Add DoFnTester support for specifying input timestamps --- .../apache/beam/sdk/coders/NullableCoder.java | 7 + .../beam/sdk/transforms/DoFnTester.java | 33 ++- .../apache/beam/sdk/transforms/Latest.java | 203 +++++++++++++++ .../beam/sdk/values/TimestampedValue.java | 14 ++ .../beam/sdk/transforms/DoFnTesterTest.java | 34 ++- .../beam/sdk/transforms/LatestFnTests.java | 233 ++++++++++++++++++ .../beam/sdk/transforms/LatestTest.java | 146 +++++++++++ .../beam/sdk/values/TimestampedValueTest.java | 83 +++++++ 8 files changed, 747 insertions(+), 6 deletions(-) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Latest.java create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestFnTests.java create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestTest.java create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/values/TimestampedValueTest.java diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/NullableCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/NullableCoder.java index 44aadbdd88cc..9c6c7c0f694e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/NullableCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/NullableCoder.java @@ -65,6 +65,13 @@ private NullableCoder(Coder valueCoder) { this.valueCoder = valueCoder; } + /** + * Returns the inner {@link Coder} wrapped by this {@link NullableCoder} instance. + */ + public Coder getValueCoder() { + return valueCoder; + } + @Override public void encode(@Nullable T value, OutputStream outStream, Context context) throws IOException, CoderException { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java index b867a5582b97..0e018ba2fa42 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java @@ -17,6 +17,9 @@ */ package org.apache.beam.sdk.transforms; +import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.base.Preconditions.checkState; + import com.google.common.base.Function; import com.google.common.base.MoreObjects; import com.google.common.collect.ImmutableList; @@ -221,9 +224,26 @@ private static void unwrapUserCodeException(UserCodeException e) throws Exceptio * been finished */ public void processElement(InputT element) throws Exception { - if (state == State.FINISHED) { - throw new IllegalStateException("finishBundle() has already been called"); - } + processTimestampedElement(TimestampedValue.atMinimumTimestamp(element)); + } + + /** + * Calls {@link OldDoFn#processElement} on the {@code OldDoFn} under test, in a + * context where {@link OldDoFn.ProcessContext#element} returns the + * given element and timestamp. + * + *

              Will call {@link #startBundle} automatically, if it hasn't + * already been called. + * + *

              If the input timestamp is {@literal null}, the minimum timestamp will be used. + * + * @throws IllegalStateException if the {@code OldDoFn} under test has already + * been finished + */ + public void processTimestampedElement(TimestampedValue element) throws Exception { + checkNotNull(element, "Timestamped element cannot be null"); + checkState(state != State.FINISHED, "finishBundle() has already been called"); + if (state == State.UNSTARTED) { startBundle(); } @@ -522,10 +542,13 @@ private List> getOutputList(TupleTag tag) { private TestProcessContext createProcessContext( OldDoFn fn, - InputT elem) { + TimestampedValue elem) { + WindowedValue windowedValue = WindowedValue.timestampedValueInGlobalWindow( + elem.getValue(), elem.getTimestamp()); + return new TestProcessContext<>(fn, createContext(fn), - WindowedValue.valueInGlobalWindow(elem), + windowedValue, mainOutputTag, sideInputs); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Latest.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Latest.java new file mode 100644 index 000000000000..7f13649b783a --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Latest.java @@ -0,0 +1,203 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.transforms; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.base.Preconditions.checkState; + +import java.util.Iterator; +import org.apache.beam.sdk.coders.CannotProvideCoderException; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.NullableCoder; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TimestampedValue; + +/** + * {@link PTransform} and {@link Combine.CombineFn} for computing the latest element + * in a {@link PCollection}. + * + *

              Example 1: compute the latest value for each session: + *

              
              + * PCollection input = ...;
              + * PCollection sessioned = input
              + *    .apply(Window.into(Sessions.withGapDuration(Duration.standardMinutes(5)));
              + * PCollection latestValues = sessioned.apply(Latest.globally());
              + * 
              + * + *

              Example 2: track a latest computed value in an aggregator: + *

              
              + * class MyDoFn extends DoFn {
              + *  private Aggregator, Double> latestValue =
              + *    createAggregator("latestValue", new Latest.LatestFn());
              + *
              + *  {@literal @}ProcessElement
              + *  public void processElement(ProcessContext c) {
              + *    double val = // ..
              + *    latestValue.addValue(TimestampedValue.of(val, c.timestamp()));
              + *    // ..
              + *  }
              + * }
              + * 
              + * + *

              For elements with the same timestamp, the element chosen for output is arbitrary. + */ +public class Latest { + // Do not instantiate + private Latest() {} + + /** + * A {@link Combine.CombineFn} that computes the latest element from a set of inputs. This is + * particularly useful as an {@link Aggregator}. + * + * @param Type of input element. + * @see Latest + */ + public static class LatestFn + extends Combine.CombineFn, TimestampedValue, T> { + /** Construct a new {@link LatestFn} instance. */ + public LatestFn() {} + + @Override + public TimestampedValue createAccumulator() { + return TimestampedValue.atMinimumTimestamp(null); + } + + @Override + public TimestampedValue addInput(TimestampedValue accumulator, + TimestampedValue input) { + checkNotNull(accumulator, "accumulator must be non-null"); + checkNotNull(input, "input must be non-null"); + + if (input.getTimestamp().isBefore(accumulator.getTimestamp())) { + return accumulator; + } else { + return input; + } + } + + @Override + public Coder> getAccumulatorCoder(CoderRegistry registry, + Coder> inputCoder) throws CannotProvideCoderException { + return NullableCoder.of(inputCoder); + } + + @Override + public Coder getDefaultOutputCoder(CoderRegistry registry, + Coder> inputCoder) throws CannotProvideCoderException { + checkState(inputCoder instanceof TimestampedValue.TimestampedValueCoder, + "inputCoder must be a TimestampedValueCoder, but was %s", inputCoder); + + TimestampedValue.TimestampedValueCoder inputTVCoder = + (TimestampedValue.TimestampedValueCoder) inputCoder; + return NullableCoder.of(inputTVCoder.getValueCoder()); + } + + @Override + public TimestampedValue mergeAccumulators(Iterable> accumulators) { + checkNotNull(accumulators, "accumulators must be non-null"); + + Iterator> iter = accumulators.iterator(); + if (!iter.hasNext()) { + return createAccumulator(); + } + + TimestampedValue merged = iter.next(); + while (iter.hasNext()) { + merged = addInput(merged, iter.next()); + } + + return merged; + } + + @Override + public T extractOutput(TimestampedValue accumulator) { + return accumulator.getValue(); + } + } + + /** + * Returns a {@link PTransform} that takes as input a {@link PCollection} and returns a + * {@link PCollection} whose contents is the latest element according to its event time, or + * {@literal null} if there are no elements. + * + * @param The type of the elements being combined. + */ + public static PTransform, PCollection> globally() { + return new Globally<>(); + } + + /** + * Returns a {@link PTransform} that takes as input a {@code PCollection>} and returns a + * {@code PCollection>} whose contents is the latest element per-key according to its + * event time. + * + * @param The key type of the elements being combined. + * @param The value type of the elements being combined. + */ + public static PTransform>, PCollection>> perKey() { + return new PerKey<>(); + } + + private static class Globally extends PTransform, PCollection> { + @Override + public PCollection apply(PCollection input) { + Coder inputCoder = input.getCoder(); + + return input + .apply("Reify Timestamps", ParDo.of( + new DoFn>() { + @ProcessElement + public void processElement(ProcessContext c) { + c.output(TimestampedValue.of(c.element(), c.timestamp())); + } + })).setCoder(TimestampedValue.TimestampedValueCoder.of(inputCoder)) + .apply("Latest Value", Combine.globally(new LatestFn())) + .setCoder(NullableCoder.of(inputCoder)); + } + } + + private static class PerKey + extends PTransform>, PCollection>> { + @Override + public PCollection> apply(PCollection> input) { + checkNotNull(input); + checkArgument(input.getCoder() instanceof KvCoder, + "Input specifiedCoder must be an instance of KvCoder, but was %s", input.getCoder()); + + @SuppressWarnings("unchecked") + KvCoder inputCoder = (KvCoder) input.getCoder(); + return input + .apply("Reify Timestamps", ParDo.of( + new DoFn, KV>>() { + @ProcessElement + public void processElement(ProcessContext c) { + c.output(KV.of(c.element().getKey(), TimestampedValue.of(c.element().getValue(), + c.timestamp()))); + } + })).setCoder(KvCoder.of( + inputCoder.getKeyCoder(), + TimestampedValue.TimestampedValueCoder.of(inputCoder.getValueCoder()))) + .apply("Latest Value", Combine., V>perKey(new LatestFn())) + .setCoder(inputCoder); + } + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java index f2ad6168bb65..dd80fb2d06c5 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java @@ -31,6 +31,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.InstantCoder; import org.apache.beam.sdk.coders.StandardCoder; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.PropertyNames; import org.joda.time.Instant; @@ -43,6 +44,13 @@ * @param the type of the value */ public class TimestampedValue { + /** + * Returns a new {@link TimestampedValue} with the + * {@link BoundedWindow#TIMESTAMP_MIN_VALUE minimum timestamp}. + */ + public static TimestampedValue atMinimumTimestamp(V value) { + return of(value, BoundedWindow.TIMESTAMP_MIN_VALUE); + } /** * Returns a new {@code TimestampedValue} with the given value and timestamp. @@ -136,6 +144,10 @@ public List> getCoderArguments() { return Arrays.>asList(valueCoder); } + public Coder getValueCoder() { + return valueCoder; + } + public static List getInstanceComponents(TimestampedValue exampleValue) { return Arrays.asList(exampleValue.getValue()); } @@ -147,6 +159,8 @@ public static List getInstanceComponents(TimestampedValue example private final Instant timestamp; protected TimestampedValue(V value, Instant timestamp) { + checkNotNull(timestamp, "timestamp must be non-null"); + this.value = value; this.timestamp = timestamp; } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java index 2649be5aadf3..3ed30fdd0c67 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.transforms; +import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasItems; @@ -35,7 +36,9 @@ import org.apache.beam.sdk.values.TimestampedValue; import org.hamcrest.Matchers; import org.joda.time.Instant; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -44,6 +47,7 @@ */ @RunWith(JUnit4.class) public class DoFnTesterTest { + @Rule public ExpectedException thrown = ExpectedException.none(); @Test public void processElement() throws Exception { @@ -125,6 +129,16 @@ public void processElementsWithPeeks() throws Exception { assertTrue(deserializedDoFn.wasFinishBundleCalled()); } + @Test + public void processElementAfterFinish() throws Exception { + DoFnTester tester = DoFnTester.of(new CounterDoFn()); + tester.finishBundle(); + + thrown.expect(IllegalStateException.class); + thrown.expectMessage("finishBundle() has already been called"); + tester.processElement(1L); + } + @Test public void processBatch() throws Exception { CounterDoFn counterDoFn = new CounterDoFn(); @@ -145,7 +159,25 @@ public void processBatch() throws Exception { } @Test - public void processElementWithTimestamp() throws Exception { + public void processTimestampedElement() throws Exception { + DoFn> reifyTimestamps = new ReifyTimestamps(); + + DoFnTester> tester = DoFnTester.of(reifyTimestamps); + + TimestampedValue input = TimestampedValue.of(1L, new Instant(100)); + tester.processTimestampedElement(input); + assertThat(tester.takeOutputElements(), contains(input)); + } + + static class ReifyTimestamps extends DoFn> { + @ProcessElement + public void processElement(ProcessContext c) { + c.output(TimestampedValue.of(c.element(), c.timestamp())); + } + } + + @Test + public void processElementWithOutputTimestamp() throws Exception { CounterDoFn counterDoFn = new CounterDoFn(); DoFnTester tester = DoFnTester.of(counterDoFn); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestFnTests.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestFnTests.java new file mode 100644 index 000000000000..84b5b68d6b4f --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestFnTests.java @@ -0,0 +1,233 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.transforms; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.isOneOf; +import static org.hamcrest.Matchers.nullValue; +import static org.junit.Assert.assertEquals; + +import com.google.common.collect.Lists; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Objects; +import org.apache.beam.sdk.coders.CannotProvideCoderException; +import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.coders.NullableCoder; +import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.values.TimestampedValue; +import org.joda.time.Instant; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Unit tests for {@link Latest.LatestFn}. + * */ +@RunWith(JUnit4.class) +public class LatestFnTests { + private static final Instant INSTANT = new Instant(100); + private static final long VALUE = 100 * INSTANT.getMillis(); + + private static final TimestampedValue TV = TimestampedValue.of(VALUE, INSTANT); + private static final TimestampedValue TV_MINUS_TEN = + TimestampedValue.of(VALUE - 10, INSTANT.minus(10)); + private static final TimestampedValue TV_PLUS_TEN = + TimestampedValue.of(VALUE + 10, INSTANT.plus(10)); + + @Rule + public final ExpectedException thrown = ExpectedException.none(); + + private final Latest.LatestFn fn = new Latest.LatestFn<>(); + private final Instant baseTimestamp = Instant.now(); + + @Test + public void testDefaultValue() { + assertThat(fn.defaultValue(), nullValue()); + } + + @Test + public void testCreateAccumulator() { + assertEquals(TimestampedValue.atMinimumTimestamp(null), fn.createAccumulator()); + } + + @Test + public void testAddInputInitialAdd() { + TimestampedValue input = TV; + assertEquals(input, fn.addInput(fn.createAccumulator(), input)); + } + + @Test + public void testAddInputMinTimestamp() { + TimestampedValue input = TimestampedValue.atMinimumTimestamp(1234L); + assertEquals(input, fn.addInput(fn.createAccumulator(), input)); + } + + @Test + public void testAddInputEarlierValue() { + assertEquals(TV, fn.addInput(TV, TV_MINUS_TEN)); + } + + @Test + public void testAddInputLaterValue() { + assertEquals(TV_PLUS_TEN, fn.addInput(TV, TV_PLUS_TEN)); + } + + @Test + public void testAddInputSameTimestamp() { + TimestampedValue accum = TimestampedValue.of(100L, INSTANT); + TimestampedValue input = TimestampedValue.of(200L, INSTANT); + + assertThat("Latest for values with the same timestamp is chosen arbitrarily", + fn.addInput(accum, input), isOneOf(accum, input)); + } + + @Test + public void testAddInputNullAccumulator() { + thrown.expect(NullPointerException.class); + thrown.expectMessage("accumulators"); + fn.addInput(null, TV); + } + + @Test + public void testAddInputNullInput() { + thrown.expect(NullPointerException.class); + thrown.expectMessage("input"); + fn.addInput(TV, null); + } + + @Test + public void testAddInputNullValue() { + TimestampedValue input = TimestampedValue.of(null, INSTANT.plus(10)); + assertEquals("Null values are allowed", input, fn.addInput(TV, input)); + } + + @Test + public void testMergeAccumulatorsMultipleValues() { + Iterable> accums = Lists.newArrayList( + TV, + TV_PLUS_TEN, + TV_MINUS_TEN + ); + + assertEquals(TV_PLUS_TEN, fn.mergeAccumulators(accums)); + } + + @Test + public void testMergeAccumulatorsSingleValue() { + assertEquals(TV, fn.mergeAccumulators(Lists.newArrayList(TV))); + } + + @Test + public void testMergeAccumulatorsEmptyIterable() { + ArrayList> emptyAccums = Lists.newArrayList(); + assertEquals(TimestampedValue.atMinimumTimestamp(null), fn.mergeAccumulators(emptyAccums)); + } + + @Test + public void testMergeAccumulatorsDefaultAccumulator() { + TimestampedValue defaultAccum = fn.createAccumulator(); + assertEquals(TV, fn.mergeAccumulators(Lists.newArrayList(TV, defaultAccum))); + } + + @Test + public void testMergeAccumulatorsAllDefaultAccumulators() { + TimestampedValue defaultAccum = fn.createAccumulator(); + assertEquals(defaultAccum, fn.mergeAccumulators( + Lists.newArrayList(defaultAccum, defaultAccum))); + } + + @Test + public void testMergeAccumulatorsNullIterable() { + thrown.expect(NullPointerException.class); + thrown.expectMessage("accumulators"); + fn.mergeAccumulators(null); + } + + @Test + public void testExtractOutput() { + assertEquals(TV.getValue(), fn.extractOutput(TV)); + } + + @Test + public void testExtractOutputDefaultAggregator() { + TimestampedValue accum = fn.createAccumulator(); + assertThat(fn.extractOutput(accum), nullValue()); + } + + @Test + public void testExtractOutputNullValue() { + TimestampedValue accum = TimestampedValue.of(null, baseTimestamp); + assertEquals(null, fn.extractOutput(accum)); + } + + @Test + public void testAggregator() throws Exception { + LatestAggregatorsFn doFn = new LatestAggregatorsFn<>(TV_MINUS_TEN.getValue()); + DoFnTester harness = DoFnTester.of(doFn); + for (TimestampedValue element : Arrays.asList(TV, TV_PLUS_TEN, TV_MINUS_TEN)) { + harness.processTimestampedElement(element); + } + + assertEquals(TV_PLUS_TEN.getValue(), harness.getAggregatorValue(doFn.allValuesAgg)); + assertEquals(TV_MINUS_TEN.getValue(), harness.getAggregatorValue(doFn.specialValueAgg)); + assertThat(harness.getAggregatorValue(doFn.noValuesAgg), nullValue()); + } + + @Test + public void testDefaultCoderHandlesNull() throws CannotProvideCoderException { + Latest.LatestFn fn = new Latest.LatestFn<>(); + + CoderRegistry registry = new CoderRegistry(); + TimestampedValue.TimestampedValueCoder inputCoder = + TimestampedValue.TimestampedValueCoder.of(VarLongCoder.of()); + + assertThat("Default output coder should handle null values", + fn.getDefaultOutputCoder(registry, inputCoder), instanceOf(NullableCoder.class)); + assertThat("Default accumulator coder should handle null values", + fn.getAccumulatorCoder(registry, inputCoder), instanceOf(NullableCoder.class)); + } + + static class LatestAggregatorsFn extends DoFn { + private final T specialValue; + LatestAggregatorsFn(T specialValue) { + this.specialValue = specialValue; + } + + Aggregator, T> allValuesAgg = + createAggregator("allValues", new Latest.LatestFn()); + + Aggregator, T> specialValueAgg = + createAggregator("oneValue", new Latest.LatestFn()); + + Aggregator, T> noValuesAgg = + createAggregator("noValues", new Latest.LatestFn()); + + @ProcessElement + public void processElement(ProcessContext c) { + TimestampedValue val = TimestampedValue.of(c.element(), c.timestamp()); + allValuesAgg.addValue(val); + if (Objects.equals(c.element(), specialValue)) { + specialValueAgg.addValue(val); + } + } + } +} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestTest.java new file mode 100644 index 000000000000..ce9ae376ad34 --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestTest.java @@ -0,0 +1,146 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.transforms; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.instanceOf; +import static org.junit.Assert.assertEquals; + +import java.io.Serializable; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.coders.BigEndianLongCoder; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.NullableCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.testing.NeedsRunner; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TimestampedValue; + +import org.joda.time.Instant; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Unit tests for {@link Latest} {@link PTransform} and {@link Combine.CombineFn}. + */ +@RunWith(JUnit4.class) +public class LatestTest implements Serializable { + @Rule public transient ExpectedException thrown = ExpectedException.none(); + + @Test + @Category(NeedsRunner.class) + public void testGloballyEventTimestamp() { + TestPipeline p = TestPipeline.create(); + PCollection output = + p.apply(Create.timestamped( + TimestampedValue.of("foo", new Instant(100)), + TimestampedValue.of("bar", new Instant(300)), + TimestampedValue.of("baz", new Instant(200)) + )) + .apply(Latest.globally()); + + PAssert.that(output).containsInAnyOrder("bar"); + p.run(); + } + + @Test + public void testGloballyOutputCoder() { + TestPipeline p = TestPipeline.create(); + BigEndianLongCoder inputCoder = BigEndianLongCoder.of(); + + PCollection output = + p.apply(Create.of(1L, 2L).withCoder(inputCoder)) + .apply(Latest.globally()); + + Coder outputCoder = output.getCoder(); + assertThat(outputCoder, instanceOf(NullableCoder.class)); + assertEquals(inputCoder, ((NullableCoder) outputCoder).getValueCoder()); + } + + @Test + @Category(NeedsRunner.class) + public void testGloballyEmptyCollection() { + TestPipeline p = TestPipeline.create(); + PCollection emptyInput = p.apply(Create.of() + // Explicitly set coder such that then runner enforces encodability. + .withCoder(VarLongCoder.of())); + PCollection output = emptyInput.apply(Latest.globally()); + + PAssert.that(output).containsInAnyOrder((Long) null); + p.run(); + } + + @Test + @Category(NeedsRunner.class) + public void testPerKeyEventTimestamp() { + TestPipeline p = TestPipeline.create(); + PCollection> output = + p.apply(Create.timestamped( + TimestampedValue.of(KV.of("A", "foo"), new Instant(100)), + TimestampedValue.of(KV.of("B", "bar"), new Instant(300)), + TimestampedValue.of(KV.of("A", "baz"), new Instant(200)) + )) + .apply(Latest.perKey()); + + PAssert.that(output).containsInAnyOrder(KV.of("B", "bar"), KV.of("A", "baz")); + p.run(); + } + + @Test + public void testPerKeyOutputCoder() { + TestPipeline p = TestPipeline.create(); + KvCoder inputCoder = KvCoder.of( + AvroCoder.of(String.class), AvroCoder.of(Long.class)); + + PCollection> output = + p.apply(Create.of(KV.of("foo", 1L)).withCoder(inputCoder)) + .apply(Latest.perKey()); + + assertEquals("Should use input coder for outputs", inputCoder, output.getCoder()); + } + + @Test + @Category(NeedsRunner.class) + public void testPerKeyEmptyCollection() { + TestPipeline p = TestPipeline.create(); + PCollection> output = + p.apply(Create.>of().withCoder(KvCoder.of( + StringUtf8Coder.of(), StringUtf8Coder.of()))) + .apply(Latest.perKey()); + + PAssert.that(output).empty(); + p.run(); + } + + /** Helper method to easily create a timestamped value. */ + private static TimestampedValue timestamped(Instant timestamp) { + return TimestampedValue.of(uniqueLong.incrementAndGet(), timestamp); + } + private static final AtomicLong uniqueLong = new AtomicLong(); +} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TimestampedValueTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TimestampedValueTest.java new file mode 100644 index 000000000000..a982f3139ae9 --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TimestampedValueTest.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.sdk.values; + +import static org.junit.Assert.assertEquals; + +import com.google.common.testing.EqualsTester; + +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; + +import org.joda.time.Instant; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Unit tests for {@link TimestampedValue}. + */ +@RunWith(JUnit4.class) +public class TimestampedValueTest { + @Rule + public ExpectedException thrown = ExpectedException.none(); + + @Test + public void testValues() { + Instant now = Instant.now(); + TimestampedValue tsv = TimestampedValue.of("foobar", now); + + assertEquals(now, tsv.getTimestamp()); + assertEquals("foobar", tsv.getValue()); + } + + @Test + public void testAtMinimumTimestamp() { + TimestampedValue tsv = TimestampedValue.atMinimumTimestamp("foobar"); + assertEquals(BoundedWindow.TIMESTAMP_MIN_VALUE, tsv.getTimestamp()); + } + + @Test + public void testNullTimestamp() { + thrown.expect(NullPointerException.class); + thrown.expectMessage("timestamp"); + TimestampedValue.of("foobar", null); + } + + @Test + public void testNullValue() { + TimestampedValue tsv = TimestampedValue.atMinimumTimestamp(null); + assertEquals(null, tsv.getValue()); + } + + @Test + public void testEquality() { + new EqualsTester() + .addEqualityGroup( + TimestampedValue.of("foo", new Instant(1000)), + TimestampedValue.of("foo", new Instant(1000))) + .addEqualityGroup(TimestampedValue.of("foo", new Instant(2000))) + .addEqualityGroup(TimestampedValue.of("bar", new Instant(1000))) + .addEqualityGroup( + TimestampedValue.of("foo", BoundedWindow.TIMESTAMP_MIN_VALUE), + TimestampedValue.atMinimumTimestamp("foo")) + .testEquals(); + } +} From 446e0f27e3ac83f1baaae4538b1c34fdaff72035 Mon Sep 17 00:00:00 2001 From: Vikas Kedigehalli Date: Mon, 29 Aug 2016 13:55:32 -0700 Subject: [PATCH 092/112] DatastoreIO SplitQueryFn integration test --- .../sdk/io/gcp/datastore/DatastoreV1.java | 9 +- .../sdk/io/gcp/datastore/DatastoreV1Test.java | 6 +- .../sdk/io/gcp/datastore/SplitQueryFnIT.java | 97 +++++++++++++++++++ 3 files changed, 107 insertions(+), 5 deletions(-) create mode 100644 sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/SplitQueryFnIT.java diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java index 8456e0287def..e24bc809c1eb 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java @@ -24,6 +24,7 @@ import static com.google.datastore.v1.PropertyFilter.Operator.EQUAL; import static com.google.datastore.v1.PropertyOrder.Direction.DESCENDING; import static com.google.datastore.v1.QueryResultBatch.MoreResultsType.NOT_FINISHED; +import static com.google.datastore.v1.client.DatastoreHelper.makeAndFilter; import static com.google.datastore.v1.client.DatastoreHelper.makeDelete; import static com.google.datastore.v1.client.DatastoreHelper.makeFilter; import static com.google.datastore.v1.client.DatastoreHelper.makeOrder; @@ -290,7 +291,7 @@ static long getEstimatedSizeBytes(Datastore datastore, Query query, @Nullable St throws DatastoreException { String ourKind = query.getKind(0).getName(); long latestTimestamp = queryLatestStatisticsTimestamp(datastore, namespace); - LOG.info("Latest stats timestamp : {}", latestTimestamp); + LOG.info("Latest stats timestamp for kind {} is {}", ourKind, latestTimestamp); Query.Builder queryBuilder = Query.newBuilder(); if (namespace == null) { @@ -298,8 +299,10 @@ static long getEstimatedSizeBytes(Datastore datastore, Query query, @Nullable St } else { queryBuilder.addKindBuilder().setName("__Stat_Ns_Kind__"); } - queryBuilder.setFilter(makeFilter("kind_name", EQUAL, makeValue(ourKind).build())); - queryBuilder.setFilter(makeFilter("timestamp", EQUAL, makeValue(latestTimestamp).build())); + + queryBuilder.setFilter(makeAndFilter( + makeFilter("kind_name", EQUAL, makeValue(ourKind).build()).build(), + makeFilter("timestamp", EQUAL, makeValue(latestTimestamp).build()).build())); RunQueryRequest request = makeRequest(queryBuilder.build(), namespace); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java index 138671d4a007..d96c320dd94c 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java @@ -19,6 +19,7 @@ import static com.google.datastore.v1.PropertyFilter.Operator.EQUAL; import static com.google.datastore.v1.PropertyOrder.Direction.DESCENDING; +import static com.google.datastore.v1.client.DatastoreHelper.makeAndFilter; import static com.google.datastore.v1.client.DatastoreHelper.makeDelete; import static com.google.datastore.v1.client.DatastoreHelper.makeFilter; import static com.google.datastore.v1.client.DatastoreHelper.makeKey; @@ -805,8 +806,9 @@ private static Query makeStatKindQuery(String namespace, long timestamp) { } else { statQuery.addKindBuilder().setName("__Stat_Ns_Kind__"); } - statQuery.setFilter(makeFilter("kind_name", EQUAL, makeValue(KIND)).build()); - statQuery.setFilter(makeFilter("timestamp", EQUAL, makeValue(timestamp * 1000000L)).build()); + statQuery.setFilter(makeAndFilter( + makeFilter("kind_name", EQUAL, makeValue(KIND).build()).build(), + makeFilter("timestamp", EQUAL, makeValue(timestamp * 1000000L).build()).build())); return statQuery.build(); } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/SplitQueryFnIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/SplitQueryFnIT.java new file mode 100644 index 000000000000..72ab7c2535e4 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/SplitQueryFnIT.java @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.sdk.io.gcp.datastore; + +import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.NUM_QUERY_SPLITS_MIN; +import static org.junit.Assert.assertEquals; + +import com.google.datastore.v1.Query; +import java.util.List; +import javax.annotation.Nullable; +import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.SplitQueryFn; +import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.V1Options; +import org.apache.beam.sdk.transforms.DoFnTester; +import org.apache.beam.sdk.values.KV; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Integration tests for {@link DatastoreV1.Read.SplitQueryFn}. + * + *

              It is hard to mock the exact behavior of Cloud Datastore, especially for the statistics + * queries. Also the fact that DatastoreIO falls back gracefully when querying statistics fails, + * makes it hard to catch these issues in production. This test here ensures we interact with + * the Cloud Datastore directly, query the actual stats and verify that the SplitQueryFn generates + * the expected number of query splits. + * + *

              These tests are brittle as they rely on statistics data in Cloud Datastore. If the data + * gets lost or changes then they will begin failing and this test should be disabled. + * At the time of writing, the Cloud Datastore has the following statistics, + *

                + *
              • kind = sort_1G, entity_bytes = 2130000000, count = 10000000 + *
              • kind = shakespeare, entity_bytes = 26383451, count = 172948 + *
              + */ +@RunWith(JUnit4.class) +public class SplitQueryFnIT { + /** + * Tests {@link SplitQueryFn} to generate expected number of splits for a large dataset. + */ + @Test + public void testSplitQueryFnWithLargeDataset() throws Exception { + String projectId = "apache-beam-testing"; + String kind = "sort_1G"; + String namespace = null; + // Num splits is computed based on the entity_bytes size of the input_sort_1G kind reported by + // Datastore stats. + int expectedNumSplits = 32; + testSplitQueryFn(projectId, kind, namespace, expectedNumSplits); + } + + /** + * Tests {@link SplitQueryFn} to fallback to NUM_QUERY_SPLITS_MIN for a small dataset. + */ + @Test + public void testSplitQueryFnWithSmallDataset() throws Exception { + String projectId = "apache-beam-testing"; + String kind = "shakespeare"; + String namespace = null; + int expectedNumSplits = NUM_QUERY_SPLITS_MIN; + testSplitQueryFn(projectId, kind, namespace, expectedNumSplits); + } + + /** + * A helper method to test {@link SplitQueryFn} to generate the expected number of splits. + */ + private void testSplitQueryFn(String projectId, String kind, @Nullable String namespace, + int expectedNumSplits) throws Exception { + Query.Builder query = Query.newBuilder(); + query.addKindBuilder().setName(kind); + + SplitQueryFn splitQueryFn = new SplitQueryFn( + V1Options.from(projectId, query.build(), namespace), 0); + DoFnTester> doFnTester = DoFnTester.of(splitQueryFn); + + List> queries = doFnTester.processBundle(query.build()); + assertEquals(queries.size(), expectedNumSplits); + } + + // TODO (vikasrk): Create datasets under a different namespace and add tests. +} From d0a4a0d6bdea846b3947e539592a4c63af1c66eb Mon Sep 17 00:00:00 2001 From: Vikas Kedigehalli Date: Thu, 1 Sep 2016 13:31:35 -0700 Subject: [PATCH 093/112] Cloud Datastore naming clean-up --- .../beam/examples/complete/AutoComplete.java | 12 ++--- .../examples/cookbook/DatastoreWordCount.java | 18 +++---- .../sdk/io/gcp/datastore/DatastoreIO.java | 6 +-- .../sdk/io/gcp/datastore/DatastoreV1.java | 52 +++++++++---------- .../sdk/io/gcp/datastore/package-info.java | 2 +- .../sdk/io/gcp/datastore/DatastoreV1Test.java | 6 +-- 6 files changed, 48 insertions(+), 48 deletions(-) diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java b/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java index 56c7855e45e0..2182e6dd0ea5 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java @@ -96,7 +96,7 @@ * --streaming * } * - *

              This will update the datastore every 10 seconds based on the last + *

              This will update the Cloud Datastore every 10 seconds based on the last * 30 minutes of data received. */ public class AutoComplete { @@ -380,7 +380,7 @@ static TableSchema getSchema() { /** * Takes as input a the top candidates per prefix, and emits an entity - * suitable for writing to Datastore. + * suitable for writing to Cloud Datastore. * *

              Note: We use ancestor keys for strong consistency. See the Cloud Datastore documentation on * @@ -431,7 +431,7 @@ private static interface Options Boolean getRecursive(); void setRecursive(Boolean value); - @Description("Datastore entity kind") + @Description("Cloud Datastore entity kind") @Default.String("autocomplete-demo") String getKind(); void setKind(String value); @@ -441,17 +441,17 @@ private static interface Options Boolean getOutputToBigQuery(); void setOutputToBigQuery(Boolean value); - @Description("Whether output to Datastore") + @Description("Whether output to Cloud Datastore") @Default.Boolean(false) Boolean getOutputToDatastore(); void setOutputToDatastore(Boolean value); - @Description("Datastore ancestor key") + @Description("Cloud Datastore ancestor key") @Default.String("root") String getDatastoreAncestorKey(); void setDatastoreAncestorKey(String value); - @Description("Datastore output project ID, defaults to project ID") + @Description("Cloud Datastore output project ID, defaults to project ID") String getOutputProject(); void setOutputProject(String value); } diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java index eb2165f91e19..434e9fb94b26 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java @@ -47,11 +47,11 @@ /** * A WordCount example using DatastoreIO. * - *

              This example shows how to use DatastoreIO to read from Datastore and + *

              This example shows how to use DatastoreIO to read from Cloud Datastore and * write the results to Cloud Storage. Note that this example will write - * data to Datastore, which may incur charge for Datastore operations. + * data to Cloud Datastore, which may incur charge for Cloud Datastore operations. * - *

              To run this example, users need to use gcloud to get credential for Datastore: + *

              To run this example, users need to use gcloud to get credential for Cloud Datastore: *

              {@code
                * $ gcloud auth login
                * }
              @@ -150,7 +150,7 @@ public void processElement(ProcessContext c) { *

              Inherits standard configuration options. */ public static interface Options extends PipelineOptions { - @Description("Path of the file to read from and store to Datastore") + @Description("Path of the file to read from and store to Cloud Datastore") @Default.String("gs://apache-beam-samples/shakespeare/kinglear.txt") String getInput(); void setInput(String value); @@ -160,17 +160,17 @@ public static interface Options extends PipelineOptions { String getOutput(); void setOutput(String value); - @Description("Project ID to read from datastore") + @Description("Project ID to read from Cloud Datastore") @Validation.Required String getProject(); void setProject(String value); - @Description("Datastore Entity kind") + @Description("Cloud Datastore Entity kind") @Default.String("shakespeare-demo") String getKind(); void setKind(String value); - @Description("Datastore Namespace") + @Description("Cloud Datastore Namespace") String getNamespace(); void setNamespace(@Nullable String value); @@ -186,7 +186,7 @@ public static interface Options extends PipelineOptions { /** * An example that creates a pipeline to populate DatastoreIO from a - * text input. Forces use of DirectRunner for local execution mode. + * text input. Forces use of DirectRunner for local execution mode. */ public static void writeDataToDatastore(Options options) { Pipeline p = Pipeline.create(options); @@ -217,7 +217,7 @@ static Query makeAncestorKindQuery(Options options) { } /** - * An example that creates a pipeline to do DatastoreIO.Read from Datastore. + * An example that creates a pipeline to do DatastoreIO.Read from Cloud Datastore. */ public static void readDataFromDatastore(Options options) { Query query = makeAncestorKindQuery(options); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreIO.java index 5abf0153ac10..c50c23a070da 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreIO.java @@ -22,7 +22,7 @@ /** *

              {@link DatastoreIO} provides an API for reading from and writing to * Google Cloud Datastore over different - * versions of the Datastore Client libraries. + * versions of the Cloud Datastore Client libraries. * *

              To use the v1 version see {@link DatastoreV1}. */ @@ -32,8 +32,8 @@ public class DatastoreIO { private DatastoreIO() {} /** - * Returns a {@link DatastoreV1} that provides an API for accessing Datastore through v1 version - * of Datastore Client library. + * Returns a {@link DatastoreV1} that provides an API for accessing Cloud Datastore through v1 + * version of Datastore Client library. */ public static DatastoreV1 v1() { return new DatastoreV1(); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java index e24bc809c1eb..6bd03b508268 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java @@ -91,13 +91,13 @@ * {@link Entity} objects. * *

              This API currently requires an authentication workaround. To use {@link DatastoreV1}, users - * must use the {@code gcloud} command line tool to get credentials for Datastore: + * must use the {@code gcloud} command line tool to get credentials for Cloud Datastore: *

                * $ gcloud auth login
                * 
              * - *

              To read a {@link PCollection} from a query to Datastore, use {@link DatastoreV1#read} and - * its methods {@link DatastoreV1.Read#withProjectId} and {@link DatastoreV1.Read#withQuery} to + *

              To read a {@link PCollection} from a query to Cloud Datastore, use {@link DatastoreV1#read} + * and its methods {@link DatastoreV1.Read#withProjectId} and {@link DatastoreV1.Read#withQuery} to * specify the project to query and the query to read from. You can optionally provide a namespace * to query within using {@link DatastoreV1.Read#withNamespace}. You could also optionally specify * how many splits you want for the query using {@link DatastoreV1.Read#withNumQuerySplits}. @@ -122,7 +122,7 @@ * {@link com.google.datastore.v1.Query.Builder#setLimit(Int32Value)}, then * all returned results will be read by a single Dataflow worker in order to ensure correct data. * - *

              To write a {@link PCollection} to a Datastore, use {@link DatastoreV1#write}, + *

              To write a {@link PCollection} to a Cloud Datastore, use {@link DatastoreV1#write}, * specifying the Cloud Datastore project to write to: * *

               {@code
              @@ -131,7 +131,7 @@
                * p.run();
                * } 
              * - *

              To delete a {@link PCollection} of {@link Entity Entities} from Datastore, use + *

              To delete a {@link PCollection} of {@link Entity Entities} from Cloud Datastore, use * {@link DatastoreV1#deleteEntity()}, specifying the Cloud Datastore project to write to: * *

               {@code
              @@ -140,8 +140,8 @@
                * p.run();
                * } 
              * - *

              To delete entities associated with a {@link PCollection} of {@link Key Keys} from Datastore, - * use {@link DatastoreV1#deleteKey}, specifying the Cloud Datastore project to write to: + *

              To delete entities associated with a {@link PCollection} of {@link Key Keys} from Cloud + * Datastore, use {@link DatastoreV1#deleteKey}, specifying the Cloud Datastore project to write to: * *

               {@code
                * PCollection entities = ...;
              @@ -169,7 +169,7 @@
                * more details.
                *
                * 

              Please see Cloud Datastore Sign Up - * for security and permission related information specific to Datastore. + * for security and permission related information specific to Cloud Datastore. * * @see org.apache.beam.sdk.runners.PipelineRunner */ @@ -180,7 +180,7 @@ public class DatastoreV1 { DatastoreV1() {} /** - * Datastore has a limit of 500 mutations per batch operation, so we flush + * Cloud Datastore has a limit of 500 mutations per batch operation, so we flush * changes to Datastore every 500 entities. */ @VisibleForTesting @@ -197,7 +197,7 @@ public DatastoreV1.Read read() { } /** - * A {@link PTransform} that reads the result rows of a Datastore query as {@code Entity} + * A {@link PTransform} that reads the result rows of a Cloud Datastore query as {@code Entity} * objects. * * @see DatastoreIO @@ -234,7 +234,7 @@ public static class Read extends PTransform> { /** * Computes the number of splits to be performed on the given query by querying the estimated - * size from Datastore. + * size from Cloud Datastore. */ static int getEstimatedNumSplits(Datastore datastore, Query query, @Nullable String namespace) { int numSplits; @@ -252,7 +252,7 @@ static int getEstimatedNumSplits(Datastore datastore, Query query, @Nullable Str } /** - * Datastore system tables with statistics are periodically updated. This method fetches + * Cloud Datastore system tables with statistics are periodically updated. This method fetches * the latest timestamp (in microseconds) of statistics update using the {@code __Stat_Total__} * table. */ @@ -281,7 +281,7 @@ private static long queryLatestStatisticsTimestamp(Datastore datastore, /** * Get the estimated size of the data returned by the given query. * - *

              Datastore provides no way to get a good estimate of how large the result of a query + *

              Cloud Datastore provides no way to get a good estimate of how large the result of a query * entity kind being queried, using the __Stat_Kind__ system table, assuming exactly 1 kind * is specified in the query. * @@ -357,7 +357,7 @@ private Read(@Nullable String projectId, @Nullable Query query, @Nullable String } /** - * Returns a new {@link DatastoreV1.Read} that reads from the Datastore for the specified + * Returns a new {@link DatastoreV1.Read} that reads from the Cloud Datastore for the specified * project. */ public DatastoreV1.Read withProjectId(String projectId) { @@ -400,7 +400,7 @@ public DatastoreV1.Read withNamespace(String namespace) { *

            3. If the {@code query} has a user limit set, then {@code numQuerySplits} will be * ignored and no split will be performed. *
            4. Under certain cases Cloud Datastore is unable to split query to the requested number of - * splits. In such cases we just use whatever the Datastore returns. + * splits. In such cases we just use whatever the Cloud Datastore returns. * */ public DatastoreV1.Read withNumQuerySplits(int numQuerySplits) { @@ -492,7 +492,7 @@ public String toString() { } /** - * A class for v1 Datastore related options. + * A class for v1 Cloud Datastore related options. */ @VisibleForTesting static class V1Options implements Serializable { @@ -608,7 +608,7 @@ public void populateDisplayData(Builder builder) { } /** - * A {@link DoFn} that reads entities from Datastore for each query. + * A {@link DoFn} that reads entities from Cloud Datastore for each query. */ @VisibleForTesting static class ReadFn extends DoFn { @@ -908,8 +908,8 @@ public void finishBundle(Context c) throws Exception { * *

              If a commit fails, it will be retried (up to {@link DatastoreWriterFn#MAX_RETRIES} * times). All mutations in the batch will be committed again, even if the commit was partially - * successful. If the retry limit is exceeded, the last exception from the Datastore will be - * thrown. + * successful. If the retry limit is exceeded, the last exception from the Cloud Datastore will + * be thrown. * * @throws DatastoreException if the commit fails or IOException or InterruptedException if * backing off between retries fails. @@ -953,7 +953,7 @@ public void populateDisplayData(Builder builder) { } /** - * Returns true if a Datastore key is complete. A key is complete if its last element + * Returns true if a Cloud Datastore key is complete. A key is complete if its last element * has either an id or a name. */ static boolean isValidKey(Key key) { @@ -974,7 +974,7 @@ static class UpsertFn extends SimpleFunction { public Mutation apply(Entity entity) { // Verify that the entity to write has a complete key. checkArgument(isValidKey(entity.getKey()), - "Entities to be written to the Datastore must have complete keys:\n%s", entity); + "Entities to be written to the Cloud Datastore must have complete keys:\n%s", entity); return makeUpsert(entity).build(); } @@ -995,7 +995,7 @@ static class DeleteEntityFn extends SimpleFunction { public Mutation apply(Entity entity) { // Verify that the entity to delete has a complete key. checkArgument(isValidKey(entity.getKey()), - "Entities to be deleted from the Datastore must have complete keys:\n%s", entity); + "Entities to be deleted from the Cloud Datastore must have complete keys:\n%s", entity); return makeDelete(entity.getKey()).build(); } @@ -1016,7 +1016,7 @@ static class DeleteKeyFn extends SimpleFunction { public Mutation apply(Key key) { // Verify that the entity to delete has a complete key. checkArgument(isValidKey(key), - "Keys to be deleted from the Datastore must be complete:\n%s", key); + "Keys to be deleted from the Cloud Datastore must be complete:\n%s", key); return makeDelete(key).build(); } @@ -1029,7 +1029,7 @@ public void populateDisplayData(Builder builder) { } /** - * A wrapper factory class for Datastore singleton classes {@link DatastoreFactory} and + * A wrapper factory class for Cloud Datastore singleton classes {@link DatastoreFactory} and * {@link QuerySplitter} * *

              {@link DatastoreFactory} and {@link QuerySplitter} are not java serializable, hence @@ -1038,7 +1038,7 @@ public void populateDisplayData(Builder builder) { @VisibleForTesting static class V1DatastoreFactory implements Serializable { - /** Builds a Datastore client for the given pipeline options and project. */ + /** Builds a Cloud Datastore client for the given pipeline options and project. */ public Datastore getDatastore(PipelineOptions pipelineOptions, String projectId) { DatastoreOptions.Builder builder = new DatastoreOptions.Builder() @@ -1055,7 +1055,7 @@ public Datastore getDatastore(PipelineOptions pipelineOptions, String projectId) return DatastoreFactory.get().create(builder.build()); } - /** Builds a Datastore {@link QuerySplitter}. */ + /** Builds a Cloud Datastore {@link QuerySplitter}. */ public QuerySplitter getQuerySplitter() { return DatastoreHelper.getQuerySplitter(); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/package-info.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/package-info.java index 1ca02668f3b4..4e5fbc30898c 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/package-info.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/package-info.java @@ -19,6 +19,6 @@ /** *

              Provides an API for reading from and writing to * Google Cloud Datastore over different - * versions of the Datastore Client libraries. + * versions of the Cloud Datastore Client libraries. */ package org.apache.beam.sdk.io.gcp.datastore; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java index d96c320dd94c..dd1904ae48a7 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java @@ -430,7 +430,7 @@ public void testAddEntitiesWithIncompleteKeys() throws Exception { UpsertFn upsertFn = new UpsertFn(); thrown.expect(IllegalArgumentException.class); - thrown.expectMessage("Entities to be written to the Datastore must have complete keys"); + thrown.expectMessage("Entities to be written to the Cloud Datastore must have complete keys"); upsertFn.apply(entity); } @@ -458,7 +458,7 @@ public void testDeleteEntitiesWithIncompleteKeys() throws Exception { DeleteEntityFn deleteEntityFn = new DeleteEntityFn(); thrown.expect(IllegalArgumentException.class); - thrown.expectMessage("Entities to be deleted from the Datastore must have complete keys"); + thrown.expectMessage("Entities to be deleted from the Cloud Datastore must have complete keys"); deleteEntityFn.apply(entity); } @@ -485,7 +485,7 @@ public void testDeleteIncompleteKeys() throws Exception { DeleteKeyFn deleteKeyFn = new DeleteKeyFn(); thrown.expect(IllegalArgumentException.class); - thrown.expectMessage("Keys to be deleted from the Datastore must be complete"); + thrown.expectMessage("Keys to be deleted from the Cloud Datastore must be complete"); deleteKeyFn.apply(key); } From 6adaebf76a6f6caef66dc60a56b94e2734689723 Mon Sep 17 00:00:00 2001 From: Ian Zhou Date: Thu, 18 Aug 2016 13:50:52 -0700 Subject: [PATCH 094/112] Fixed Combine display data --- .../apache/beam/sdk/transforms/Combine.java | 53 +++++++++++++++++-- .../beam/sdk/transforms/CombineTest.java | 19 +++++++ 2 files changed, 68 insertions(+), 4 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java index 26f0f660f074..d432e15dc71e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java @@ -51,6 +51,7 @@ import org.apache.beam.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext; import org.apache.beam.sdk.transforms.CombineWithContext.RequiresContextInternal; import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.transforms.display.DisplayData.Builder; import org.apache.beam.sdk.transforms.display.HasDisplayData; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; @@ -1815,7 +1816,14 @@ public PerKeyWithHotKeyFanout withHotKeyFanout( */ public PerKeyWithHotKeyFanout withHotKeyFanout(final int hotKeyFanout) { return new PerKeyWithHotKeyFanout<>(name, fn, fnDisplayData, - new SerializableFunction() { + new SimpleFunction() { + @Override + public void populateDisplayData(Builder builder) { + super.populateDisplayData(builder); + builder.addIfNotDefault(DisplayData.item("fanout", hotKeyFanout) + .withLabel("Key Fanout Size"), 0); + } + @Override public Integer apply(K unused) { return hotKeyFanout; @@ -1904,7 +1912,7 @@ private PCollection> applyHelper(PCollection( inputCoder.getValueCoder(), accumCoder); - // A CombineFn's mergeAccumulator can be applied in a tree-like fashon. + // A CombineFn's mergeAccumulator can be applied in a tree-like fashion. // Here we shard the key using an integer nonce, combine on that partial // set of values, then drop the nonce and do a final combine of the // aggregates. We do this by splitting the original CombineFn into two, @@ -1944,6 +1952,16 @@ public Coder getAccumulatorCoder( throws CannotProvideCoderException { return accumCoder; } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + builder.add(DisplayData.item("fanoutFn", hotKeyFanout.getClass()) + .withLabel("Fanout Function")); + if (hotKeyFanout instanceof HasDisplayData) { + ((HasDisplayData) hotKeyFanout).populateDisplayData(builder); + } + } }; postCombine = new KeyedCombineFn, AccumT, OutputT>() { @@ -1988,6 +2006,15 @@ public Coder getAccumulatorCoder(CoderRegistry registry, Coder keyCod throws CannotProvideCoderException { return accumCoder; } + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + builder.add(DisplayData.item("fanoutFn", hotKeyFanout.getClass()) + .withLabel("Fanout Function")); + if (hotKeyFanout instanceof HasDisplayData) { + ((HasDisplayData) hotKeyFanout).populateDisplayData(builder); + } + } }; } else { final KeyedCombineFnWithContext keyedFnWithContext = @@ -2028,6 +2055,15 @@ public Coder getAccumulatorCoder( throws CannotProvideCoderException { return accumCoder; } + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + builder.add(DisplayData.item("fanoutFn", hotKeyFanout.getClass()) + .withLabel("Fanout Function")); + if (hotKeyFanout instanceof HasDisplayData) { + ((HasDisplayData) hotKeyFanout).populateDisplayData(builder); + } + } }; postCombine = new KeyedCombineFnWithContext, AccumT, OutputT>() { @@ -2073,6 +2109,15 @@ public Coder getAccumulatorCoder(CoderRegistry registry, Coder keyCod throws CannotProvideCoderException { return accumCoder; } + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + builder.add(DisplayData.item("fanoutFn", hotKeyFanout.getClass()) + .withLabel("Fanout Function")); + if (hotKeyFanout instanceof HasDisplayData) { + ((HasDisplayData) hotKeyFanout).populateDisplayData(builder); + } + } }; } @@ -2117,7 +2162,7 @@ public void processElement(ProcessContext c) { .setCoder(KvCoder.of(KvCoder.of(inputCoder.getKeyCoder(), VarIntCoder.of()), inputCoder.getValueCoder())) .setWindowingStrategyInternal(preCombineStrategy) - .apply("PreCombineHot", Combine.perKey(hotPreCombine)) + .apply("PreCombineHot", Combine.perKey(hotPreCombine, fnDisplayData)) .apply("StripNonce", MapElements.via( new SimpleFunction, AccumT>, KV>>() { @@ -2147,7 +2192,7 @@ public KV> apply(KV element) { // Combine the union of the pre-processed hot and cold key results. return PCollectionList.of(precombinedHot).and(preprocessedCold) .apply(Flatten.>>pCollections()) - .apply("PostCombine", Combine.perKey(postCombine)); + .apply("PostCombine", Combine.perKey(postCombine, fnDisplayData)); } @Override diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java index 77a1d6b03e69..be061af51c3b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java @@ -731,6 +731,25 @@ public void testCombinePerKeyPrimitiveDisplayData() { displayData, hasItem(hasDisplayItem("combineFn", combineFn.getClass()))); } + @Test + @Category(RunnableOnService.class) + public void testCombinePerKeyWithHotKeyFanoutPrimitiveDisplayData() { + int hotKeyFanout = 2; + DisplayDataEvaluator evaluator = DisplayDataEvaluator.create(); + + CombineTest.UniqueInts combineFn = new CombineTest.UniqueInts(); + PTransform>, PCollection>>> combine = + Combine.>perKey(combineFn).withHotKeyFanout(hotKeyFanout); + + Set displayData = evaluator.displayDataForPrimitiveTransforms(combine, + KvCoder.of(VarIntCoder.of(), VarIntCoder.of())); + + assertThat("Combine.perKey.withHotKeyFanout should include the combineFn in its primitive " + + "transform", displayData, hasItem(hasDisplayItem("combineFn", combineFn.getClass()))); + assertThat("Combine.perKey.withHotKeyFanout(int) should include the fanout in its primitive " + + "transform", displayData, hasItem(hasDisplayItem("fanout", hotKeyFanout))); + } + //////////////////////////////////////////////////////////////////////////// // Test classes, for different kinds of combining fns. From d24d51fb210f0dd7254f6ddd9608eb1b53f7300c Mon Sep 17 00:00:00 2001 From: Scott Wegner Date: Fri, 19 Aug 2016 13:32:45 -0700 Subject: [PATCH 095/112] Delegate populateDipslayData to wrapped combineFn's --- .../apache/beam/sdk/transforms/Combine.java | 39 ++++++------------- 1 file changed, 11 insertions(+), 28 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java index d432e15dc71e..2b89372a47a8 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java @@ -1740,7 +1740,7 @@ protected SimpleCombineFn(SerializableFunction, V> combiner) { public static class PerKey extends PTransform>, PCollection>> { - private final transient PerKeyCombineFn fn; + private final PerKeyCombineFn fn; private final DisplayData.Item> fnDisplayData; private final boolean fewKeys; private final List> sideInputs; @@ -1820,8 +1820,8 @@ public PerKeyWithHotKeyFanout withHotKeyFanout(final int hot @Override public void populateDisplayData(Builder builder) { super.populateDisplayData(builder); - builder.addIfNotDefault(DisplayData.item("fanout", hotKeyFanout) - .withLabel("Key Fanout Size"), 0); + builder.add(DisplayData.item("fanout", hotKeyFanout) + .withLabel("Key Fanout Size")); } @Override @@ -1866,7 +1866,7 @@ public void populateDisplayData(DisplayData.Builder builder) { public static class PerKeyWithHotKeyFanout extends PTransform>, PCollection>> { - private final transient PerKeyCombineFn fn; + private final PerKeyCombineFn fn; private final DisplayData.Item> fnDisplayData; private final SerializableFunction hotKeyFanout; @@ -1955,12 +1955,7 @@ public Coder getAccumulatorCoder( @Override public void populateDisplayData(DisplayData.Builder builder) { - super.populateDisplayData(builder); - builder.add(DisplayData.item("fanoutFn", hotKeyFanout.getClass()) - .withLabel("Fanout Function")); - if (hotKeyFanout instanceof HasDisplayData) { - ((HasDisplayData) hotKeyFanout).populateDisplayData(builder); - } + builder.include(PerKeyWithHotKeyFanout.this); } }; postCombine = @@ -2008,12 +2003,7 @@ public Coder getAccumulatorCoder(CoderRegistry registry, Coder keyCod } @Override public void populateDisplayData(DisplayData.Builder builder) { - super.populateDisplayData(builder); - builder.add(DisplayData.item("fanoutFn", hotKeyFanout.getClass()) - .withLabel("Fanout Function")); - if (hotKeyFanout instanceof HasDisplayData) { - ((HasDisplayData) hotKeyFanout).populateDisplayData(builder); - } + builder.include(PerKeyWithHotKeyFanout.this); } }; } else { @@ -2057,12 +2047,7 @@ public Coder getAccumulatorCoder( } @Override public void populateDisplayData(DisplayData.Builder builder) { - super.populateDisplayData(builder); - builder.add(DisplayData.item("fanoutFn", hotKeyFanout.getClass()) - .withLabel("Fanout Function")); - if (hotKeyFanout instanceof HasDisplayData) { - ((HasDisplayData) hotKeyFanout).populateDisplayData(builder); - } + builder.include(PerKeyWithHotKeyFanout.this); } }; postCombine = @@ -2111,12 +2096,7 @@ public Coder getAccumulatorCoder(CoderRegistry registry, Coder keyCod } @Override public void populateDisplayData(DisplayData.Builder builder) { - super.populateDisplayData(builder); - builder.add(DisplayData.item("fanoutFn", hotKeyFanout.getClass()) - .withLabel("Fanout Function")); - if (hotKeyFanout instanceof HasDisplayData) { - ((HasDisplayData) hotKeyFanout).populateDisplayData(builder); - } + builder.include(PerKeyWithHotKeyFanout.this); } }; } @@ -2200,6 +2180,9 @@ public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); Combine.populateDisplayData(builder, fn, fnDisplayData); + if (hotKeyFanout instanceof HasDisplayData) { + builder.include((HasDisplayData) hotKeyFanout); + } builder.add(DisplayData.item("fanoutFn", hotKeyFanout.getClass()) .withLabel("Fanout Function")); } From a62e5018d6fa0a4e6ed05b872a53a4eb36415d1d Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 25 Aug 2016 14:57:26 -0700 Subject: [PATCH 096/112] Put classes in runners-core package into runners.core namespace --- .../util => runners/core}/AssignWindows.java | 2 +- .../core}/AssignWindowsDoFn.java | 4 +++- .../core}/BatchTimerInternals.java | 5 ++++- .../util => runners/core}/DoFnRunner.java | 4 +++- .../util => runners/core}/DoFnRunnerBase.java | 12 +++++++++-- .../util => runners/core}/DoFnRunners.java | 9 +++++++-- .../core}/ElementByteSizeObservable.java | 4 +++- .../GroupAlsoByWindowViaWindowSetDoFn.java | 5 +---- .../core}/GroupAlsoByWindowsDoFn.java | 5 ++++- ...GroupAlsoByWindowsViaOutputBufferDoFn.java | 5 ++++- .../core}/GroupByKeyViaGroupByKeyOnly.java | 5 ++++- .../core}/LateDataDroppingDoFnRunner.java | 8 +++++++- .../util => runners/core}/NonEmptyPanes.java | 3 ++- .../core}/PaneInfoTracker.java | 4 +++- .../core}/PeekingReiterator.java | 3 ++- .../core}/PushbackSideInputDoFnRunner.java | 5 ++++- .../{sdk/util => runners/core}/ReduceFn.java | 4 +++- .../core}/ReduceFnContextFactory.java | 8 +++++++- .../util => runners/core}/ReduceFnRunner.java | 18 +++++++++++++---- .../core}/SimpleDoFnRunner.java | 9 ++++++--- .../util => runners/core}/SystemReduceFn.java | 3 ++- .../util => runners/core}/TriggerRunner.java | 8 +++++++- .../util => runners/core}/WatermarkHold.java | 5 ++++- .../beam/sdk/util/common/package-info.java | 20 ------------------- .../apache/beam/sdk/util/package-info.java | 20 ------------------- .../core}/BatchTimerInternalsTest.java | 3 ++- .../core}/GroupAlsoByWindowsProperties.java | 4 +++- ...pAlsoByWindowsViaOutputBufferDoFnTest.java | 7 ++++--- .../core}/LateDataDroppingDoFnRunnerTest.java | 7 +++++-- .../PushbackSideInputDoFnRunnerTest.java | 6 +++++- .../core}/ReduceFnRunnerTest.java | 6 +++++- .../util => runners/core}/ReduceFnTester.java | 14 ++++++++++++- .../core}/SimpleDoFnRunnerTest.java | 6 +++++- .../GroupAlsoByWindowEvaluatorFactory.java | 8 ++++---- .../GroupByKeyOnlyEvaluatorFactory.java | 4 ++-- .../beam/runners/direct/ParDoEvaluator.java | 8 ++++---- .../UncommittedBundleOutputManager.java | 2 +- .../FlinkStreamingTransformTranslators.java | 2 +- .../wrappers/streaming/DoFnOperator.java | 6 +++--- .../streaming/WindowDoFnOperator.java | 2 +- .../beam/runners/spark/SparkRunner.java | 2 +- .../translation/TransformTranslator.java | 10 +++++----- .../StreamingTransformTranslator.java | 2 +- .../main/resources/beam/findbugs-filter.xml | 2 +- .../org/apache/beam/sdk/util/BitSetCoder.java | 2 +- 45 files changed, 172 insertions(+), 109 deletions(-) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util => runners/core}/AssignWindows.java (97%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util => runners/core}/AssignWindowsDoFn.java (95%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util => runners/core}/BatchTimerInternals.java (97%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util => runners/core}/DoFnRunner.java (94%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util => runners/core}/DoFnRunnerBase.java (97%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util => runners/core}/DoFnRunners.java (94%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util/common => runners/core}/ElementByteSizeObservable.java (94%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util => runners/core}/GroupAlsoByWindowsDoFn.java (93%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util => runners/core}/GroupAlsoByWindowsViaOutputBufferDoFn.java (95%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util => runners/core}/GroupByKeyViaGroupByKeyOnly.java (98%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util => runners/core}/LateDataDroppingDoFnRunner.java (94%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util => runners/core}/NonEmptyPanes.java (98%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util => runners/core}/PaneInfoTracker.java (98%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util/common => runners/core}/PeekingReiterator.java (96%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util => runners/core}/PushbackSideInputDoFnRunner.java (96%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util => runners/core}/ReduceFn.java (97%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util => runners/core}/ReduceFnContextFactory.java (97%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util => runners/core}/ReduceFnRunner.java (98%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util => runners/core}/SimpleDoFnRunner.java (89%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util => runners/core}/SystemReduceFn.java (98%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util => runners/core}/TriggerRunner.java (96%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util => runners/core}/WatermarkHold.java (99%) delete mode 100644 runners/core-java/src/main/java/org/apache/beam/sdk/util/common/package-info.java delete mode 100644 runners/core-java/src/main/java/org/apache/beam/sdk/util/package-info.java rename runners/core-java/src/test/java/org/apache/beam/{sdk/util => runners/core}/BatchTimerInternalsTest.java (98%) rename runners/core-java/src/test/java/org/apache/beam/{sdk/util => runners/core}/GroupAlsoByWindowsProperties.java (99%) rename runners/core-java/src/test/java/org/apache/beam/{sdk/util => runners/core}/GroupAlsoByWindowsViaOutputBufferDoFnTest.java (94%) rename runners/core-java/src/test/java/org/apache/beam/{sdk/util => runners/core}/LateDataDroppingDoFnRunnerTest.java (93%) rename runners/core-java/src/test/java/org/apache/beam/{sdk/util => runners/core}/PushbackSideInputDoFnRunnerTest.java (97%) rename runners/core-java/src/test/java/org/apache/beam/{sdk/util => runners/core}/ReduceFnRunnerTest.java (99%) rename runners/core-java/src/test/java/org/apache/beam/{sdk/util => runners/core}/ReduceFnTester.java (98%) rename runners/core-java/src/test/java/org/apache/beam/{sdk/util => runners/core}/SimpleDoFnRunnerTest.java (93%) diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/AssignWindows.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/AssignWindows.java similarity index 97% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/AssignWindows.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/AssignWindows.java index af2805235af9..f2387f57ba39 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/AssignWindows.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/AssignWindows.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/AssignWindowsDoFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/AssignWindowsDoFn.java similarity index 95% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/AssignWindowsDoFn.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/AssignWindowsDoFn.java index 7e26253a829a..0eb1667c3a1c 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/AssignWindowsDoFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/AssignWindowsDoFn.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import static com.google.common.base.Preconditions.checkNotNull; @@ -26,6 +26,8 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.WindowFn; +import org.apache.beam.sdk.util.SystemDoFnInternal; +import org.apache.beam.sdk.values.PCollection; import org.joda.time.Instant; /** diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/BatchTimerInternals.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/BatchTimerInternals.java similarity index 97% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/BatchTimerInternals.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/BatchTimerInternals.java index f3e84a6861cb..829dbde84a4c 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/BatchTimerInternals.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/BatchTimerInternals.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import static com.google.common.base.Preconditions.checkState; @@ -25,6 +25,9 @@ import java.util.Set; import javax.annotation.Nullable; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.TimeDomain; +import org.apache.beam.sdk.util.TimerInternals; + import org.joda.time.Instant; /** diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java similarity index 94% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunner.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java index 49206d19a214..f4c8eea34e1e 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java @@ -15,11 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.OldDoFn.ProcessContext; +import org.apache.beam.sdk.util.KeyedWorkItem; +import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; /** diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunnerBase.java similarity index 97% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunnerBase.java index f0cfd7455de5..71472da760c7 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunnerBase.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import static com.google.common.base.Preconditions.checkNotNull; @@ -30,6 +30,8 @@ import java.util.List; import java.util.Map; import java.util.Set; + +import org.apache.beam.runners.core.DoFnRunners.OutputManager; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.options.PipelineOptions; @@ -43,8 +45,14 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.DoFnRunners.OutputManager; import org.apache.beam.sdk.util.ExecutionContext.StepContext; +import org.apache.beam.sdk.util.SideInputReader; +import org.apache.beam.sdk.util.SystemDoFnInternal; +import org.apache.beam.sdk.util.TimerInternals; +import org.apache.beam.sdk.util.UserCodeException; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.WindowingInternals; +import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunners.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java similarity index 94% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunners.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java index c4df7b222036..7726374d85e4 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunners.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java @@ -15,15 +15,20 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import java.util.List; + +import org.apache.beam.runners.core.DoFnRunner.ReduceFnExecutor; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.DoFnRunner.ReduceFnExecutor; import org.apache.beam.sdk.util.ExecutionContext.StepContext; +import org.apache.beam.sdk.util.KeyedWorkItem; +import org.apache.beam.sdk.util.SideInputReader; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObservable.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ElementByteSizeObservable.java similarity index 94% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObservable.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/ElementByteSizeObservable.java index 613aa4b2bdd5..2380ba904df3 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObservable.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ElementByteSizeObservable.java @@ -15,7 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util.common; +package org.apache.beam.runners.core; + +import org.apache.beam.sdk.util.common.ElementByteSizeObserver; /** * An interface for things that allow observing the size in bytes of diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetDoFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetDoFn.java index 7cdab005e73d..b427037ef008 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetDoFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetDoFn.java @@ -17,16 +17,13 @@ */ package org.apache.beam.runners.core; +import org.apache.beam.runners.core.DoFnRunner.ReduceFnExecutor; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.DoFnRunner.ReduceFnExecutor; -import org.apache.beam.sdk.util.GroupAlsoByWindowsDoFn; import org.apache.beam.sdk.util.KeyedWorkItem; -import org.apache.beam.sdk.util.ReduceFnRunner; import org.apache.beam.sdk.util.SystemDoFnInternal; -import org.apache.beam.sdk.util.SystemReduceFn; import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.TimerInternals.TimerData; import org.apache.beam.sdk.util.WindowingStrategy; diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsDoFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsDoFn.java similarity index 93% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsDoFn.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsDoFn.java index f386dfba1220..985144948402 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsDoFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsDoFn.java @@ -15,13 +15,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.SystemDoFnInternal; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.state.StateInternalsFactory; import org.apache.beam.sdk.values.KV; diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFn.java similarity index 95% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFn.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFn.java index cc418daec7e9..091ad33bd8de 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFn.java @@ -15,12 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import com.google.common.collect.Iterables; import java.util.List; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.SystemDoFnInternal; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.util.state.StateInternalsFactory; import org.apache.beam.sdk.values.KV; diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupByKeyViaGroupByKeyOnly.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupByKeyViaGroupByKeyOnly.java similarity index 98% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupByKeyViaGroupByKeyOnly.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/GroupByKeyViaGroupByKeyOnly.java index fdad17a8bb0c..b5214259a86b 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupByKeyViaGroupByKeyOnly.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupByKeyViaGroupByKeyOnly.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import static com.google.common.base.Preconditions.checkArgument; @@ -31,7 +31,10 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.ReifyTimestampsAndWindows; +import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; +import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.state.StateInternalsFactory; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java similarity index 94% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunner.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java index 08c670e4753f..63a80d2294e9 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; @@ -24,6 +24,12 @@ import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.KeyedWorkItem; +import org.apache.beam.sdk.util.KeyedWorkItems; +import org.apache.beam.sdk.util.TimerInternals; +import org.apache.beam.sdk.util.WindowTracing; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.joda.time.Instant; diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/NonEmptyPanes.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/NonEmptyPanes.java similarity index 98% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/NonEmptyPanes.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/NonEmptyPanes.java index e809c24062fd..3e51dfb7e6d3 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/NonEmptyPanes.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/NonEmptyPanes.java @@ -15,11 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode; import org.apache.beam.sdk.util.state.AccumulatorCombiningState; import org.apache.beam.sdk.util.state.MergingStateAccessor; diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/PaneInfoTracker.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/PaneInfoTracker.java similarity index 98% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/PaneInfoTracker.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/PaneInfoTracker.java index 90c10b5c7449..114f5e6313a4 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/PaneInfoTracker.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/PaneInfoTracker.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import static com.google.common.base.Preconditions.checkState; @@ -25,6 +25,8 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.PaneInfo.PaneInfoCoder; import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing; +import org.apache.beam.sdk.util.TimerInternals; +import org.apache.beam.sdk.util.WindowTracing; import org.apache.beam.sdk.util.state.ReadableState; import org.apache.beam.sdk.util.state.StateAccessor; import org.apache.beam.sdk.util.state.StateTag; diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/common/PeekingReiterator.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/PeekingReiterator.java similarity index 96% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/common/PeekingReiterator.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/PeekingReiterator.java index 1e3c17f1a6e3..fcdff3bac906 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/common/PeekingReiterator.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/PeekingReiterator.java @@ -15,12 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util.common; +package org.apache.beam.runners.core; import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; import java.util.NoSuchElementException; +import org.apache.beam.sdk.util.common.Reiterator; /** * A {@link Reiterator} that supports one-element lookahead during iteration. diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunner.java similarity index 96% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunner.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunner.java index d9f1fbffccb3..deeac3c2ee8d 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunner.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; @@ -23,7 +23,10 @@ import java.util.Collections; import java.util.HashSet; import java.util.Set; + import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; +import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; /** diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFn.java similarity index 97% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFn.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFn.java index 8135a5beca53..bb20226fcb33 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFn.java @@ -15,11 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import java.io.Serializable; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.util.Timers; +import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.state.MergingStateAccessor; import org.apache.beam.sdk.util.state.ReadableState; import org.apache.beam.sdk.util.state.StateAccessor; diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnContextFactory.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java similarity index 97% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnContextFactory.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java index 5d27d51ad2f4..2043f143609e 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnContextFactory.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; @@ -28,7 +28,13 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.util.ActiveWindowSet; +import org.apache.beam.sdk.util.TimeDomain; +import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.TimerInternals.TimerData; +import org.apache.beam.sdk.util.Timers; +import org.apache.beam.sdk.util.WindowingInternals; +import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.state.MergingStateAccessor; import org.apache.beam.sdk.util.state.ReadableState; import org.apache.beam.sdk.util.state.State; diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java similarity index 98% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java index 7c3e4d749a1f..96d764aa62ea 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; @@ -31,6 +31,9 @@ import java.util.Map; import java.util.Set; import javax.annotation.Nullable; +import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly; +import org.apache.beam.runners.core.ReduceFnContextFactory.OnTriggerCallbacks; +import org.apache.beam.runners.core.ReduceFnContextFactory.StateStyle; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.windowing.AfterWatermark; @@ -41,10 +44,17 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing; import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly; -import org.apache.beam.sdk.util.ReduceFnContextFactory.OnTriggerCallbacks; -import org.apache.beam.sdk.util.ReduceFnContextFactory.StateStyle; +import org.apache.beam.sdk.util.ActiveWindowSet; +import org.apache.beam.sdk.util.MergingActiveWindowSet; +import org.apache.beam.sdk.util.NonMergingActiveWindowSet; +import org.apache.beam.sdk.util.TimeDomain; +import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.TimerInternals.TimerData; +import org.apache.beam.sdk.util.TriggerContextFactory; +import org.apache.beam.sdk.util.WindowTracing; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.WindowingInternals; +import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode; import org.apache.beam.sdk.util.state.ReadableState; import org.apache.beam.sdk.util.state.StateInternals; diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/SimpleDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java similarity index 89% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/SimpleDoFnRunner.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java index 1ebe5a874e74..df74ed3b9a7b 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/SimpleDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java @@ -15,14 +15,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import java.util.List; +import org.apache.beam.runners.core.DoFnRunners.OutputManager; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory; import org.apache.beam.sdk.transforms.OldDoFn; -import org.apache.beam.sdk.util.DoFnRunners.OutputManager; import org.apache.beam.sdk.util.ExecutionContext.StepContext; +import org.apache.beam.sdk.util.SideInputReader; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.TupleTag; /** @@ -31,7 +34,7 @@ * @param the type of the {@link OldDoFn} (main) input elements * @param the type of the {@link OldDoFn} (main) output elements */ -public class SimpleDoFnRunner extends DoFnRunnerBase{ +public class SimpleDoFnRunner extends DoFnRunnerBase { protected SimpleDoFnRunner(PipelineOptions options, OldDoFn fn, SideInputReader sideInputReader, diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/SystemReduceFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SystemReduceFn.java similarity index 98% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/SystemReduceFn.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/SystemReduceFn.java index 28177a811cd3..3948d9e61ece 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/SystemReduceFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SystemReduceFn.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; @@ -25,6 +25,7 @@ import org.apache.beam.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.AppliedCombineFn; import org.apache.beam.sdk.util.state.AccumulatorCombiningState; import org.apache.beam.sdk.util.state.BagState; import org.apache.beam.sdk.util.state.CombiningState; diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/TriggerRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/TriggerRunner.java similarity index 96% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/TriggerRunner.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/TriggerRunner.java index a53fb8c17513..8d0f32254585 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/TriggerRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/TriggerRunner.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import static com.google.common.base.Preconditions.checkState; @@ -28,6 +28,12 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.DefaultTrigger; import org.apache.beam.sdk.transforms.windowing.Trigger; +import org.apache.beam.sdk.util.BitSetCoder; +import org.apache.beam.sdk.util.ExecutableTrigger; +import org.apache.beam.sdk.util.FinishedTriggers; +import org.apache.beam.sdk.util.FinishedTriggersBitSet; +import org.apache.beam.sdk.util.Timers; +import org.apache.beam.sdk.util.TriggerContextFactory; import org.apache.beam.sdk.util.state.MergingStateAccessor; import org.apache.beam.sdk.util.state.StateAccessor; import org.apache.beam.sdk.util.state.StateTag; diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java similarity index 99% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java index 7404e1bc8ba3..7d0b608f8cd9 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import static com.google.common.base.Preconditions.checkState; @@ -28,6 +28,9 @@ import org.apache.beam.sdk.transforms.windowing.OutputTimeFns; import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing; import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior; +import org.apache.beam.sdk.util.TimerInternals; +import org.apache.beam.sdk.util.WindowTracing; +import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.state.MergingStateAccessor; import org.apache.beam.sdk.util.state.ReadableState; import org.apache.beam.sdk.util.state.StateMerging; diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/common/package-info.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/common/package-info.java deleted file mode 100644 index 1ea8af812cd4..000000000000 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/common/package-info.java +++ /dev/null @@ -1,20 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -/** Defines utilities shared by multiple PipelineRunner implementations. */ -package org.apache.beam.sdk.util.common; diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/package-info.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/package-info.java deleted file mode 100644 index b4772f35d33c..000000000000 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/package-info.java +++ /dev/null @@ -1,20 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -/** Defines utilities that can be used by Beam runners. */ -package org.apache.beam.sdk.util; diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/BatchTimerInternalsTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/BatchTimerInternalsTest.java similarity index 98% rename from runners/core-java/src/test/java/org/apache/beam/sdk/util/BatchTimerInternalsTest.java rename to runners/core-java/src/test/java/org/apache/beam/runners/core/BatchTimerInternalsTest.java index 20a9852b0958..122e60ce6131 100644 --- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/BatchTimerInternalsTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/BatchTimerInternalsTest.java @@ -15,8 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; +import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.TimerInternals.TimerData; import org.apache.beam.sdk.util.state.StateNamespace; import org.apache.beam.sdk.util.state.StateNamespaceForTest; diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsProperties.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowsProperties.java similarity index 99% rename from runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsProperties.java rename to runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowsProperties.java index 215cd4c38cb3..d1e0c6814392 100644 --- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsProperties.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowsProperties.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -41,6 +41,8 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.Sessions; import org.apache.beam.sdk.transforms.windowing.SlidingWindows; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.state.InMemoryStateInternals; import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.util.state.StateInternalsFactory; diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFnTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFnTest.java similarity index 94% rename from runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFnTest.java rename to runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFnTest.java index a1586c8bf2d9..1fad1fbe5d6c 100644 --- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFnTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFnTest.java @@ -15,12 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; +import org.apache.beam.runners.core.GroupAlsoByWindowsProperties.GroupAlsoByWindowsDoFnFactory; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.GroupAlsoByWindowsProperties.GroupAlsoByWindowsDoFnFactory; +import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.state.StateInternalsFactory; import org.junit.Test; import org.junit.runner.RunWith; @@ -43,7 +44,7 @@ public BufferingGABWViaOutputBufferDoFnFactory(Coder inputCoder) { @Override public - GroupAlsoByWindowsDoFn, W> forStrategy( + GroupAlsoByWindowsDoFn, W> forStrategy( WindowingStrategy windowingStrategy, StateInternalsFactory stateInternalsFactory) { return new GroupAlsoByWindowsViaOutputBufferDoFn, W>( diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunnerTest.java similarity index 93% rename from runners/core-java/src/test/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunnerTest.java rename to runners/core-java/src/test/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunnerTest.java index c63e43ef55b0..1cf05b670729 100644 --- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunnerTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.junit.Assert.assertEquals; @@ -25,12 +25,15 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import java.util.Arrays; +import org.apache.beam.runners.core.LateDataDroppingDoFnRunner.LateDataFilter; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.LateDataDroppingDoFnRunner.LateDataFilter; +import org.apache.beam.sdk.util.TimerInternals; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.WindowingStrategy; import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Before; diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunnerTest.java similarity index 97% rename from runners/core-java/src/test/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunnerTest.java rename to runners/core-java/src/test/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunnerTest.java index f8ad291d9bc9..59a7c92e7d21 100644 --- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunnerTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.emptyIterable; @@ -27,6 +27,7 @@ import com.google.common.collect.ImmutableList; import java.util.ArrayList; import java.util.List; + import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.Sum; @@ -35,6 +36,9 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.util.IdentitySideInputWindowFn; +import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; +import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.hamcrest.Matchers; diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java similarity index 99% rename from runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java rename to runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java index 647495cc344a..4d5680c5ea97 100644 --- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import static com.google.common.base.Preconditions.checkArgument; import static org.apache.beam.sdk.WindowMatchers.isSingleWindowedValue; @@ -65,6 +65,10 @@ import org.apache.beam.sdk.transforms.windowing.Trigger; import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior; import org.apache.beam.sdk.transforms.windowing.WindowFn; +import org.apache.beam.sdk.util.SideInputReader; +import org.apache.beam.sdk.util.TimeDomain; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TimestampedValue; diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java similarity index 98% rename from runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java rename to runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java index 24e33ddf310c..45062fbb8c82 100644 --- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; @@ -43,6 +43,7 @@ import java.util.PriorityQueue; import java.util.Set; import javax.annotation.Nullable; + import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.coders.IterableCoder; @@ -63,7 +64,18 @@ import org.apache.beam.sdk.transforms.windowing.Trigger; import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior; import org.apache.beam.sdk.transforms.windowing.WindowFn; +import org.apache.beam.sdk.util.AppliedCombineFn; +import org.apache.beam.sdk.util.ExecutableTrigger; +import org.apache.beam.sdk.util.NullSideInputReader; +import org.apache.beam.sdk.util.SerializableUtils; +import org.apache.beam.sdk.util.SideInputReader; +import org.apache.beam.sdk.util.TimeDomain; +import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.TimerInternals.TimerData; +import org.apache.beam.sdk.util.WindowTracing; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.WindowingInternals; +import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode; import org.apache.beam.sdk.util.state.InMemoryStateInternals; import org.apache.beam.sdk.util.state.State; diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/SimpleDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java similarity index 93% rename from runners/core-java/src/test/java/org/apache/beam/sdk/util/SimpleDoFnRunnerTest.java rename to runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java index 156b4a92ca17..adb0aac7db2c 100644 --- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/SimpleDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java @@ -15,15 +15,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import static org.hamcrest.Matchers.is; import static org.mockito.Mockito.mock; import java.util.Arrays; import java.util.List; + import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.util.BaseExecutionContext.StepContext; +import org.apache.beam.sdk.util.SystemDoFnInternal; +import org.apache.beam.sdk.util.UserCodeException; +import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; import org.junit.Rule; import org.junit.Test; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java index 2da70bbe5628..c08c229964f9 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java @@ -20,6 +20,10 @@ import com.google.common.collect.ImmutableMap; import java.util.Collections; import org.apache.beam.runners.core.GroupAlsoByWindowViaWindowSetDoFn; +import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly; +import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly.GroupAlsoByWindow; +import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly; +import org.apache.beam.runners.core.SystemReduceFn; import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext; import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupAlsoByWindow; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; @@ -28,11 +32,7 @@ import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly; -import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly.GroupAlsoByWindow; -import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly; import org.apache.beam.sdk.util.KeyedWorkItem; -import org.apache.beam.sdk.util.SystemReduceFn; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.state.StateInternals; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java index f085a39fda9b..17dc0bea3c9c 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java @@ -25,6 +25,8 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly; +import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly; import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; @@ -34,8 +36,6 @@ import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly; -import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly; import org.apache.beam.sdk.util.KeyedWorkItem; import org.apache.beam.sdk.util.KeyedWorkItems; import org.apache.beam.sdk.util.WindowedValue; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java index 85a1c6af1385..99ab22ab12bc 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java @@ -23,15 +23,15 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import org.apache.beam.runners.core.DoFnRunner; +import org.apache.beam.runners.core.DoFnRunners; +import org.apache.beam.runners.core.DoFnRunners.OutputManager; +import org.apache.beam.runners.core.PushbackSideInputDoFnRunner; import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.OldDoFn; -import org.apache.beam.sdk.util.DoFnRunner; -import org.apache.beam.sdk.util.DoFnRunners; -import org.apache.beam.sdk.util.DoFnRunners.OutputManager; -import org.apache.beam.sdk.util.PushbackSideInputDoFnRunner; import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UncommittedBundleOutputManager.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UncommittedBundleOutputManager.java index 41f7e8d64b39..d40dc110df59 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UncommittedBundleOutputManager.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UncommittedBundleOutputManager.java @@ -18,9 +18,9 @@ package org.apache.beam.runners.direct; import java.util.Map; +import org.apache.beam.runners.core.DoFnRunners.OutputManager; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; -import org.apache.beam.sdk.util.DoFnRunners.OutputManager; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java index 07e2191d4e0c..3719fa8e5d4c 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java @@ -63,7 +63,7 @@ import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.AppliedCombineFn; import org.apache.beam.sdk.util.Reshuffle; -import org.apache.beam.sdk.util.SystemReduceFn; +import org.apache.beam.runners.core.SystemReduceFn; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java index 2c7ebc697709..3b0fccc2b365 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java @@ -40,11 +40,11 @@ import org.apache.beam.sdk.transforms.join.RawUnionValue; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.DoFnRunner; -import org.apache.beam.sdk.util.DoFnRunners; +import org.apache.beam.runners.core.DoFnRunner; +import org.apache.beam.runners.core.DoFnRunners; import org.apache.beam.sdk.util.ExecutionContext; import org.apache.beam.sdk.util.NullSideInputReader; -import org.apache.beam.sdk.util.PushbackSideInputDoFnRunner; +import org.apache.beam.runners.core.PushbackSideInputDoFnRunner; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.WindowedValue; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java index 01cfa5b6a26b..b8931161045e 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java @@ -43,7 +43,7 @@ import org.apache.beam.sdk.util.ExecutionContext; import org.apache.beam.sdk.util.KeyedWorkItem; import org.apache.beam.sdk.util.KeyedWorkItems; -import org.apache.beam.sdk.util.SystemReduceFn; +import org.apache.beam.runners.core.SystemReduceFn; import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.WindowedValue; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java index 9f1a83996fb7..03db8117aae4 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java @@ -18,6 +18,7 @@ package org.apache.beam.runners.spark; +import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly; import org.apache.beam.runners.spark.translation.EvaluationContext; import org.apache.beam.runners.spark.translation.SparkContextFactory; import org.apache.beam.runners.spark.translation.SparkPipelineEvaluator; @@ -36,7 +37,6 @@ import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java index eaceb852ce34..8341c6d84a24 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java @@ -37,6 +37,11 @@ import org.apache.avro.mapred.AvroKey; import org.apache.avro.mapreduce.AvroJob; import org.apache.avro.mapreduce.AvroKeyInputFormat; +import org.apache.beam.runners.core.AssignWindowsDoFn; +import org.apache.beam.runners.core.GroupAlsoByWindowsViaOutputBufferDoFn; +import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly.GroupAlsoByWindow; +import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly; +import org.apache.beam.runners.core.SystemReduceFn; import org.apache.beam.runners.spark.coders.CoderHelpers; import org.apache.beam.runners.spark.io.hadoop.HadoopIO; import org.apache.beam.runners.spark.io.hadoop.ShardNameTemplateHelper; @@ -61,11 +66,6 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.AssignWindowsDoFn; -import org.apache.beam.sdk.util.GroupAlsoByWindowsViaOutputBufferDoFn; -import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly.GroupAlsoByWindow; -import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly; -import org.apache.beam.sdk.util.SystemReduceFn; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; import org.apache.beam.sdk.util.WindowingStrategy; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java index 43dcef665fb2..c55be3dee2d9 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java @@ -28,6 +28,7 @@ import java.util.Map; import java.util.Set; import kafka.serializer.Decoder; +import org.apache.beam.runners.core.AssignWindowsDoFn; import org.apache.beam.runners.spark.io.ConsoleIO; import org.apache.beam.runners.spark.io.CreateStream; import org.apache.beam.runners.spark.io.KafkaIO; @@ -49,7 +50,6 @@ import org.apache.beam.sdk.transforms.windowing.SlidingWindows; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.AssignWindowsDoFn; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionList; diff --git a/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml b/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml index cd50408fd395..3c016907a3b3 100644 --- a/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml +++ b/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml @@ -110,7 +110,7 @@ - + diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java index fde90af68369..72524bd446e8 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java @@ -28,7 +28,7 @@ /** * Coder for the BitSet used to track child-trigger finished states. */ -class BitSetCoder extends AtomicCoder { +public class BitSetCoder extends AtomicCoder { private static final BitSetCoder INSTANCE = new BitSetCoder(); private static final ByteArrayCoder byteArrayCoder = ByteArrayCoder.of(); From 963bedf320ad4ed138fdbae2ec3293e3243f9080 Mon Sep 17 00:00:00 2001 From: Pei He Date: Thu, 25 Aug 2016 14:20:30 -0700 Subject: [PATCH 097/112] Remove the DataflowRunner instructions from examples --- .../beam/examples/DebuggingWordCount.java | 16 ++++++------ .../beam/examples/MinimalWordCount.java | 7 +++--- .../beam/examples/WindowedWordCount.java | 22 ++++++---------- .../org/apache/beam/examples/WordCount.java | 22 +++++----------- .../beam/examples/complete/AutoComplete.java | 25 +++++++------------ .../complete/StreamingWordExtract.java | 4 +-- .../apache/beam/examples/complete/TfIdf.java | 18 +++++-------- .../complete/TopWikipediaSessions.java | 12 +++------ .../examples/complete/TrafficMaxLaneFlow.java | 4 +-- .../beam/examples/complete/TrafficRoutes.java | 4 +-- .../examples/cookbook/BigQueryTornadoes.java | 18 +++---------- .../cookbook/CombinePerKeyExamples.java | 18 +++---------- .../examples/cookbook/DatastoreWordCount.java | 17 ++++++------- .../beam/examples/cookbook/DeDupExample.java | 16 ++++++------ .../examples/cookbook/FilterExamples.java | 21 +++------------- .../beam/examples/cookbook/JoinExamples.java | 18 +++---------- .../examples/cookbook/MaxPerKeyExamples.java | 19 +++----------- .../examples/cookbook/TriggerExample.java | 16 ++++++------ 18 files changed, 92 insertions(+), 185 deletions(-) diff --git a/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java index be3aa419b537..eb382278346e 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java @@ -46,12 +46,12 @@ * *

              Basic concepts, also in the MinimalWordCount and WordCount examples: * Reading text files; counting a PCollection; executing a Pipeline both locally - * and using the Dataflow service; defining DoFns. + * and using a selected runner; defining DoFns. * *

              New Concepts: *

                *   1. Logging to Cloud Logging
              - *   2. Controlling Dataflow worker log levels
              + *   2. Controlling worker log levels
                *   3. Creating a custom aggregator
                *   4. Testing your Pipeline via PAssert
                * 
              @@ -62,12 +62,14 @@ * } *
            5. * - *

              To execute this pipeline using the Dataflow service and the additional logging discussed - * below, specify pipeline configuration: + *

              To change the runner, specify: + *

              {@code
              + *   --runner=YOUR_SELECTED_RUNNER
              + * }
              + * 
              + * + *

              To use the additional logging discussed below, specify: *

              {@code
              - *   --project=YOUR_PROJECT_ID
              - *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
              - *   --runner=BlockingDataflowRunner
                *   --workerLogLevelOverrides={"org.apache.beam.examples":"DEBUG"}
                * }
                * 
              diff --git a/examples/java/src/main/java/org/apache/beam/examples/MinimalWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/MinimalWordCount.java index f28a20cf9449..f772dd5bc768 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/MinimalWordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/MinimalWordCount.java @@ -66,12 +66,11 @@ public static void main(String[] args) { // In order to run your pipeline, you need to make following runner specific changes: // - // CHANGE 1/3: Select a Beam runner, such as BlockingDataflowRunner - // or FlinkRunner. + // CHANGE 1/3: Select a Beam runner, such as DataflowRunner or FlinkRunner. // CHANGE 2/3: Specify runner-required options. - // For BlockingDataflowRunner, set project and temp location as follows: + // For DataflowRunner, set project and temp location as follows: // DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class); - // dataflowOptions.setRunner(BlockingDataflowRunner.class); + // dataflowOptions.setRunner(DataflowRunner.class); // dataflowOptions.setProject("SET_YOUR_PROJECT_ID_HERE"); // dataflowOptions.setTempLocation("gs://SET_YOUR_BUCKET_NAME_HERE/AND_TEMP_DIRECTORY"); // For FlinkRunner, set the runner as follows. See {@code FlinkPipelineOptions} diff --git a/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java index 7af354cee0fa..c8bd9d322003 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java @@ -54,7 +54,7 @@ * *

              Basic concepts, also in the MinimalWordCount, WordCount, and DebuggingWordCount examples: * Reading text files; counting a PCollection; writing to GCS; executing a Pipeline both locally - * and using the Dataflow service; defining DoFns; creating a custom aggregator; + * and using a selected runner; defining DoFns; creating a custom aggregator; * user-defined PTransforms; defining PipelineOptions. * *

              New Concepts: @@ -66,19 +66,13 @@ * 5. Writing to BigQuery * * - *

              To execute this pipeline locally, specify general pipeline configuration: + *

              By default, the examples will run with the {@code DirectRunner}. + * To change the runner, specify: *

              {@code
              - *   --project=YOUR_PROJECT_ID
              - * }
              - * 
              - * - *

              To execute this pipeline using the Dataflow service, specify pipeline configuration: - *

              {@code
              - *   --project=YOUR_PROJECT_ID
              - *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
              - *   --runner=BlockingDataflowRunner
              + *   --runner=YOUR_SELECTED_RUNNER
                * }
                * 
              + * See examples/java/README.md for instructions about how to configure different runners. * *

              Optionally specify the input file path via: * {@code --inputFile=gs://INPUT_PATH}, @@ -86,7 +80,7 @@ * *

              Specify an output BigQuery dataset and optionally, a table for the output. If you don't * specify the table, one will be created for you using the job name. If you don't specify the - * dataset, a dataset called {@code dataflow-examples} must already exist in your project. + * dataset, a dataset called {@code beam_examples} must already exist in your project. * {@code --bigQueryDataset=YOUR-DATASET --bigQueryTable=YOUR-NEW-TABLE-NAME}. * *

              By default, the pipeline will do fixed windowing, on 1-minute windows. You can @@ -190,7 +184,7 @@ public static void main(String[] args) throws IOException { Pipeline pipeline = Pipeline.create(options); /** - * Concept #1: the Dataflow SDK lets us run the same pipeline with either a bounded or + * Concept #1: the Beam SDK lets us run the same pipeline with either a bounded or * unbounded input source. */ PCollection input = pipeline @@ -229,7 +223,7 @@ public static void main(String[] args) throws IOException { PipelineResult result = pipeline.run(); - // dataflowUtils will try to cancel the pipeline before the program exists. + // ExampleUtils will try to cancel the pipeline before the program exists. exampleUtils.waitToFinish(result); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/WordCount.java b/examples/java/src/main/java/org/apache/beam/examples/WordCount.java index 793ee4baceae..498b0693877e 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/WordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/WordCount.java @@ -48,8 +48,8 @@ * pipeline, for introduction of additional concepts. * *

              For a detailed walkthrough of this example, see - * - * https://cloud.google.com/dataflow/java-sdk/wordcount-example + * + * http://beam.incubator.apache.org/use/walkthroughs/ * * *

              Basic concepts, also in the MinimalWordCount example: @@ -66,27 +66,17 @@ *

              Concept #1: you can execute this pipeline either locally or using the selected runner. * These are now command-line options and not hard-coded as they were in the MinimalWordCount * example. - * To execute this pipeline locally, specify general pipeline configuration: - *

              {@code
              - *   --project=YOUR_PROJECT_ID
              - * }
              - * 
              - * and a local output file or output prefix on GCS: + * To execute this pipeline locally, specify a local output file or output prefix on GCS: *
              {@code
                *   --output=[YOUR_LOCAL_FILE | gs://YOUR_OUTPUT_PREFIX]
                * }
              * - *

              To execute this pipeline using the Dataflow service, specify pipeline configuration: + *

              To change the runner, specify: *

              {@code
              - *   --project=YOUR_PROJECT_ID
              - *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
              - *   --runner=BlockingDataflowRunner
              + *   --runner=YOUR_SELECTED_RUNNER
                * }
                * 
              - * and an output prefix on GCS: - *
              {@code
              - *   --output=gs://YOUR_OUTPUT_PREFIX
              - * }
              + * See examples/java/README.md for instructions about how to configure different runners. * *

              The input file defaults to {@code gs://apache-beam-samples/shakespeare/kinglear.txt} * and can be overridden with {@code --inputFile}. diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java b/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java index 2182e6dd0ea5..c3ac6149c461 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java @@ -77,24 +77,17 @@ *

              Concepts: Using the same pipeline in both streaming and batch, combiners, * composite transforms. * - *

              To execute this pipeline using the Dataflow service in batch mode, - * specify pipeline configuration: + *

              To execute this pipeline in streaming mode, specify: *

              {@code
              - *   --project=YOUR_PROJECT_ID
              - *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
              - *   --runner=DataflowRunner
              - *   --inputFile=gs://path/to/input*.txt
              + *   --streaming
                * }
              * - *

              To execute this pipeline using the Dataflow service in streaming mode, - * specify pipeline configuration: + *

              To change the runner, specify: *

              {@code
              - *   --project=YOUR_PROJECT_ID
              - *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
              - *   --runner=DataflowRunner
              - *   --inputFile=gs://YOUR_INPUT_DIRECTORY/*.txt
              - *   --streaming
              - * }
              + * --runner=YOUR_SELECTED_RUNNER + * } + * + * See examples/java/README.md for instructions about how to configure different runners. * *

              This will update the Cloud Datastore every 10 seconds based on the last * 30 minutes of data received. @@ -417,7 +410,7 @@ public void processElement(ProcessContext c) { /** * Options supported by this class. * - *

              Inherits standard Dataflow configuration options. + *

              Inherits standard Beam example configuration options. */ private static interface Options extends ExampleOptions, ExampleBigQueryTableOptions, StreamingOptions { @@ -510,7 +503,7 @@ public static void main(String[] args) throws IOException { // Run the pipeline. PipelineResult result = p.run(); - // dataflowUtils will try to cancel the pipeline and the injector before the program exists. + // ExampleUtils will try to cancel the pipeline and the injector before the program exists. exampleUtils.waitToFinish(result); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java b/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java index 869ea69b8ae4..e8d895057fd5 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java @@ -44,7 +44,7 @@ * a BigQuery table. * *

              The example is configured to use the default BigQuery table from the example common package - * (there are no defaults for a general Dataflow pipeline). + * (there are no defaults for a general Beam pipeline). * You can override them by using the {@literal --bigQueryDataset}, and {@literal --bigQueryTable} * options. If the BigQuery table do not exist, the example will try to create them. * @@ -141,7 +141,7 @@ public static void main(String[] args) throws IOException { PipelineResult result = pipeline.run(); - // dataflowUtils will try to cancel the pipeline before the program exists. + // ExampleUtils will try to cancel the pipeline before the program exists. exampleUtils.waitToFinish(result); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java index 6684553c41eb..59bbd49b55e3 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java @@ -65,23 +65,17 @@ * *

              Concepts: joining data; side inputs; logging * - *

              To execute this pipeline locally, specify general pipeline configuration: - *

              {@code
              - *   --project=YOUR_PROJECT_ID
              - * }
              - * and a local output file or output prefix on GCS: + *

              To execute this pipeline locally, specify a local output file or output prefix on GCS: *

              {@code
                *   --output=[YOUR_LOCAL_FILE | gs://YOUR_OUTPUT_PREFIX]
                * }
              * - *

              To execute this pipeline using the Dataflow service, specify pipeline configuration: + *

              To change the runner, specify: *

              {@code
              - *   --project=YOUR_PROJECT_ID
              - *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
              - *   --runner=BlockingDataflowRunner
              - * and an output prefix on GCS:
              - *   --output=gs://YOUR_OUTPUT_PREFIX
              - * }
              + * --runner=YOUR_SELECTED_RUNNER + * } + * + * See examples/java/README.md for instructions about how to configure different runners. * *

              The default input is {@code gs://apache-beam-samples/shakespeare/} and can be overridden with * {@code --input}. diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java index d597258d3b92..0f594d7ff973 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java @@ -52,17 +52,13 @@ *

              It is not recommended to execute this pipeline locally, given the size of the default input * data. * - *

              To execute this pipeline using the Dataflow service, specify pipeline configuration: + *

              To execute this pipeline using a selected runner and an output prefix on GCS, specify: *

              {@code
              - *   --project=YOUR_PROJECT_ID
              - *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
              - *   --runner=BlockingDataflowRunner
              + *   --runner=YOUR_SELECTED_RUNNER
              + *   --output=gs://YOUR_OUTPUT_PREFIX
                * }
                * 
              - * and an output prefix on GCS: - *
              {@code
              - *   --output=gs://YOUR_OUTPUT_PREFIX
              - * }
              + * See examples/java/README.md for instructions about how to configure different runners. * *

              The default input is {@code gs://apache-beam-samples/wikipedia_edits/*.json} and can be * overridden with {@code --input}. diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java index e4569600c05c..0c367d484a79 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java @@ -66,7 +66,7 @@ *

              The pipeline reads traffic sensor data from {@literal --inputFile}. * *

              The example is configured to use the default BigQuery table from the example common package - * (there are no defaults for a general Dataflow pipeline). + * (there are no defaults for a general Beam pipeline). * You can override them by using the {@literal --bigQueryDataset}, and {@literal --bigQueryTable} * options. If the BigQuery table do not exist, the example will try to create them. * @@ -354,7 +354,7 @@ public static void main(String[] args) throws IOException { // Run the pipeline. PipelineResult result = pipeline.run(); - // dataflowUtils will try to cancel the pipeline and the injector before the program exists. + // ExampleUtils will try to cancel the pipeline and the injector before the program exists. exampleUtils.waitToFinish(result); } diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java index 95336c644a79..14cee4dd2946 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java @@ -69,7 +69,7 @@ *

              The pipeline reads traffic sensor data from {@literal --inputFile}. * *

              The example is configured to use the default BigQuery table from the example common package - * (there are no defaults for a general Dataflow pipeline). + * (there are no defaults for a general Beam pipeline). * You can override them by using the {@literal --bigQueryDataset}, and {@literal --bigQueryTable} * options. If the BigQuery table do not exist, the example will try to create them. * @@ -365,7 +365,7 @@ public static void main(String[] args) throws IOException { // Run the pipeline. PipelineResult result = pipeline.run(); - // dataflowUtils will try to cancel the pipeline and the injector before the program exists. + // ExampleUtils will try to cancel the pipeline and the injector before the program exists. exampleUtils.waitToFinish(result); } diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java index 439cf020aaa9..1e4918d37025 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java @@ -45,27 +45,17 @@ *

              Note: Before running this example, you must create a BigQuery dataset to contain your output * table. * - *

              To execute this pipeline locally, specify general pipeline configuration: - *

              {@code
              - *   --project=YOUR_PROJECT_ID
              - * }
              - * 
              - * and the BigQuery table for the output, with the form + *

              To execute this pipeline locally, specify the BigQuery table for the output with the form: *

              {@code
                *   --output=YOUR_PROJECT_ID:DATASET_ID.TABLE_ID
                * }
              * - *

              To execute this pipeline using the Dataflow service, specify pipeline configuration: + *

              To change the runner, specify: *

              {@code
              - *   --project=YOUR_PROJECT_ID
              - *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
              - *   --runner=BlockingDataflowRunner
              + *   --runner=YOUR_SELECTED_RUNNER
                * }
                * 
              - * and the BigQuery table for the output: - *
              {@code
              - *   --output=YOUR_PROJECT_ID:DATASET_ID.TABLE_ID
              - * }
              + * See examples/java/README.md for instructions about how to configure different runners. * *

              The BigQuery input table defaults to {@code clouddataflow-readonly:samples.weather_stations} * and can be overridden with {@code --input}. diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java index 1d280a6e154b..fc11ac9c2c9d 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java @@ -52,27 +52,17 @@ *

              Note: Before running this example, you must create a BigQuery dataset to contain your output * table. * - *

              To execute this pipeline locally, specify general pipeline configuration: - *

              {@code
              - *   --project=YOUR_PROJECT_ID
              - * }
              - * 
              - * and the BigQuery table for the output: + *

              To execute this pipeline locally, specify the BigQuery table for the output: *

              {@code
                *   --output=YOUR_PROJECT_ID:DATASET_ID.TABLE_ID
                * }
              * - *

              To execute this pipeline using the Dataflow service, specify pipeline configuration: + *

              To change the runner, specify: *

              {@code
              - *   --project=YOUR_PROJECT_ID
              - *   --tempLocation=gs://
              - *   --runner=BlockingDataflowRunner
              + *   --runner=YOUR_SELECTED_RUNNER
                * }
                * 
              - * and the BigQuery table for the output: - *
              {@code
              - *   --output=YOUR_PROJECT_ID:DATASET_ID.TABLE_ID
              - * }
              + * See examples/java/README.md for instructions about how to configure different runners. * *

              The BigQuery input table defaults to {@code publicdata:samples.shakespeare} and can * be overridden with {@code --input}. diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java index 434e9fb94b26..c0066e698ed3 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java @@ -58,14 +58,15 @@ * *

              To run this pipeline locally, the following options must be provided: *

              {@code
              - *   --project=YOUR_PROJECT_ID
                *   --output=[YOUR_LOCAL_FILE | gs://YOUR_OUTPUT_PATH]
                * }
              * - *

              To run this example using Dataflow service, you must additionally - * provide either {@literal --tempLocation} or {@literal --tempLocation}, and - * select one of the Dataflow pipeline runners, eg - * {@literal --runner=BlockingDataflowRunner}. + *

              To change the runner, specify: + *

              {@code
              + *   --runner=YOUR_SELECTED_RUNNER
              + * }
              + * 
              + * See examples/java/README.md for instructions about how to configure different runners. * *

              Note: this example creates entities with Ancestor keys to ensure that all * entities created are in the same entity group. Similarly, the query used to read from the Cloud @@ -239,13 +240,9 @@ public static void readDataFromDatastore(Options options) { } /** - * An example to demo how to use {@link DatastoreIO}. The runner here is - * customizable, which means users could pass either {@code DirectRunner} - * or {@code DataflowRunner} in the pipeline options. + * An example to demo how to use {@link DatastoreIO}. */ public static void main(String args[]) { - // The options are used in two places, for Dataflow service, and - // building DatastoreIO.Read object Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); if (!options.isReadOnly()) { diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java index 57917109addd..594d52dab489 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java @@ -35,17 +35,15 @@ * Demonstrates {@link org.apache.beam.sdk.io.TextIO.Read}/ * {@link RemoveDuplicates}/{@link org.apache.beam.sdk.io.TextIO.Write}. * - *

              To execute this pipeline locally, specify general pipeline configuration: - * --project=YOUR_PROJECT_ID - * and a local output file or output prefix on GCS: + *

              To execute this pipeline locally, specify a local output file or output prefix on GCS: * --output=[YOUR_LOCAL_FILE | gs://YOUR_OUTPUT_PREFIX] * - *

              To execute this pipeline using the Dataflow service, specify pipeline configuration: - * --project=YOUR_PROJECT_ID - * --tempLocation=gs://YOUR_TEMP_DIRECTORY - * --runner=BlockingDataflowRunner - * and an output prefix on GCS: - * --output=gs://YOUR_OUTPUT_PREFIX + *

              To change the runner, specify: + *

              {@code
              + *   --runner=YOUR_SELECTED_RUNNER
              + * }
              + * 
              + * See examples/java/README.md for instructions about how to configure different runners. * *

              The input defaults to {@code gs://apache-beam-samples/shakespeare/*} and can be * overridden with {@code --input}. diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java index 6c42520ff7c6..01d668bb43ac 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java @@ -54,12 +54,7 @@ *

              Note: Before running this example, you must create a BigQuery dataset to contain your output * table. * - *

              To execute this pipeline locally, specify general pipeline configuration: - *

              {@code
              - *   --project=YOUR_PROJECT_ID
              - * }
              - * 
              - * and the BigQuery table for the output: + *

              To execute this pipeline locally, specify the BigQuery table for the output: *

              {@code
                *   --output=YOUR_PROJECT_ID:DATASET_ID.TABLE_ID
                *   [--monthFilter=]
              @@ -67,20 +62,12 @@
                * 
              * where optional parameter {@code --monthFilter} is set to a number 1-12. * - *

              To execute this pipeline using the Dataflow service, specify pipeline configuration: + *

              To change the runner, specify: *

              {@code
              - *   --project=YOUR_PROJECT_ID
              - *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
              - *   --runner=BlockingDataflowRunner
              + *   --runner=YOUR_SELECTED_RUNNER
                * }
                * 
              - * and the BigQuery table for the output: - *
              {@code
              - *   --output=YOUR_PROJECT_ID:DATASET_ID.TABLE_ID
              - *   [--monthFilter=]
              - * }
              - * 
              - * where optional parameter {@code --monthFilter} is set to a number 1-12. + * See examples/java/README.md for instructions about how to configure different runners. * *

              The BigQuery input table defaults to {@code clouddataflow-readonly:samples.weather_stations} * and can be overridden with {@code --input}. diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java index 1b91bf1e0615..799cad3f1e9e 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java @@ -41,27 +41,17 @@ * *

              Concepts: Join operation; multiple input sources. * - *

              To execute this pipeline locally, specify general pipeline configuration: - *

              {@code
              - *   --project=YOUR_PROJECT_ID
              - * }
              - * 
              - * and a local output file or output prefix on GCS: + *

              To execute this pipeline locally, specify a local output file or output prefix on GCS: *

              {@code
                *   --output=[YOUR_LOCAL_FILE | gs://YOUR_OUTPUT_PREFIX]
                * }
              * - *

              To execute this pipeline using the Dataflow service, specify pipeline configuration: + *

              To change the runner, specify: *

              {@code
              - *   --project=YOUR_PROJECT_ID
              - *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
              - *   --runner=BlockingDataflowRunner
              + *   --runner=YOUR_SELECTED_RUNNER
                * }
                * 
              - * and an output prefix on GCS: - *
              {@code
              - *   --output=gs://YOUR_OUTPUT_PREFIX
              - * }
              + * See examples/java/README.md for instructions about how to configure different runners. */ public class JoinExamples { diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java index 3772a7bc5b86..3a4fa2676f96 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java @@ -46,27 +46,16 @@ *

              Note: Before running this example, you must create a BigQuery dataset to contain your output * table. * - *

              To execute this pipeline locally, specify general pipeline configuration: - *

              {@code
              - *   --project=YOUR_PROJECT_ID
              - * }
              - * 
              - * and the BigQuery table for the output, with the form + *

              To execute this pipeline locally, specify the BigQuery table for the output with the form: *

              {@code
                *   --output=YOUR_PROJECT_ID:DATASET_ID.TABLE_ID
                * }
              * - *

              To execute this pipeline using the Dataflow service, specify pipeline configuration: - *

              {@code
              - *   --project=YOUR_PROJECT_ID
              - *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
              - *   --runner=BlockingDataflowRunner
              - * }
              - * 
              - * and the BigQuery table for the output: + *

              To change the runner, specify: *

              {@code
              - *   --output=YOUR_PROJECT_ID:DATASET_ID.TABLE_ID
              + *   --runner=YOUR_SELECTED_RUNNER
                * }
              + * See examples/java/README.md for instructions about how to configure different runners. * *

              The BigQuery input table defaults to {@code clouddataflow-readonly:samples.weather_stations } * and can be overridden with {@code --input}. diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java index 263054138dd6..68d4d329f302 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java @@ -73,15 +73,13 @@ * 4. Combining late data and speculative estimates * * - *

              Before running this example, it will be useful to familiarize yourself with Dataflow triggers + *

              Before running this example, it will be useful to familiarize yourself with Beam triggers * and understand the concept of 'late data', - * See: - * https://cloud.google.com/dataflow/model/triggers and - * - * https://cloud.google.com/dataflow/model/windowing#Advanced + * See: + * http://beam.incubator.apache.org/use/walkthroughs/ * *

              The example is configured to use the default BigQuery table from the example common package - * (there are no defaults for a general Dataflow pipeline). + * (there are no defaults for a general Beam pipeline). * You can override them by using the {@code --bigQueryDataset}, and {@code --bigQueryTable} * options. If the BigQuery table do not exist, the example will try to create them. * @@ -155,7 +153,7 @@ public class TriggerExample { * 5 | 60 | 10:27:20 | 10:27:25 * 5 | 60 | 10:29:00 | 11:11:00 * - *

              Dataflow tracks a watermark which records up to what point in event time the data is + *

              Beam tracks a watermark which records up to what point in event time the data is * complete. For the purposes of the example, we'll assume the watermark is approximately 15m * behind the current processing time. In practice, the actual value would vary over time based * on the systems knowledge of the current delay and contents of the backlog (data @@ -176,7 +174,7 @@ static class CalculateTotalFlow public PCollectionList apply(PCollection> flowInfo) { // Concept #1: The default triggering behavior - // By default Dataflow uses a trigger which fires when the watermark has passed the end of the + // By default Beam uses a trigger which fires when the watermark has passed the end of the // window. This would be written {@code Repeatedly.forever(AfterWatermark.pastEndOfWindow())}. // The system also defaults to dropping late data -- data which arrives after the watermark @@ -459,7 +457,7 @@ public static void main(String[] args) throws Exception { PipelineResult result = pipeline.run(); - // dataflowUtils will try to cancel the pipeline and the injector before the program exits. + // ExampleUtils will try to cancel the pipeline and the injector before the program exits. exampleUtils.waitToFinish(result); } From dbbcbe604e167b306feac2443bec85f2da3c1dd6 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Wed, 24 Aug 2016 22:35:26 -0700 Subject: [PATCH 098/112] FluentBackoff: a replacement for a variety of custom backoff implementations We have 3 different backoff classes, which don't really have that much different functionality. Add a single, flexible backoff implementation that can be used to replace all three classes. Additionally, this new backoff actually supports more functionality than any of the other three did -- you can limit retries, cap the exponential growth of an individual backoff, and cap the cumulative time spent in backoff; prior implementations did not allow all 3. This also makes the parameters self-obvious (Duration, not number-that-is-also-millis) where appropriate. This initial PR should have no functional changes. * Implement FluentBackoff * Replace other custom BackOff implementations with FluentBackoff --- .../beam/examples/common/ExampleUtils.java | 7 +- .../runners/dataflow/DataflowPipelineJob.java | 72 +++--- .../runners/dataflow/util/PackageUtil.java | 31 ++- .../dataflow/DataflowPipelineJobTest.java | 32 ++- .../io/BoundedReadFromUnboundedSource.java | 8 +- ...temptAndTimeBoundedExponentialBackOff.java | 172 ------------- .../AttemptBoundedExponentialBackOff.java | 85 ------- .../apache/beam/sdk/util/FluentBackoff.java | 229 ++++++++++++++++++ .../org/apache/beam/sdk/util/GcsUtil.java | 18 +- .../IntervalBoundedExponentialBackOff.java | 1 + ...tAndTimeBoundedExponentialBackOffTest.java | 212 ---------------- .../AttemptBoundedExponentialBackOffTest.java | 84 ------- .../beam/sdk/util/FluentBackoffTest.java | 226 +++++++++++++++++ .../org/apache/beam/sdk/util/GcsUtilTest.java | 10 +- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 21 +- .../io/gcp/bigquery/BigQueryServicesImpl.java | 152 ++++++------ .../bigquery/BigQueryTableRowIterator.java | 5 +- .../sdk/io/gcp/datastore/DatastoreV1.java | 26 +- .../bigquery/BigQueryServicesImplTest.java | 17 +- .../beam/sdk/io/gcp/datastore/V1TestUtil.java | 9 +- 20 files changed, 675 insertions(+), 742 deletions(-) delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOff.java delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOff.java create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/util/FluentBackoff.java delete mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOffTest.java delete mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOffTest.java create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/util/FluentBackoffTest.java diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/ExampleUtils.java b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleUtils.java index eadb580a257c..2e8dcf696887 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/common/ExampleUtils.java +++ b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleUtils.java @@ -45,8 +45,9 @@ import org.apache.beam.sdk.options.BigQueryOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PubsubOptions; -import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; +import org.apache.beam.sdk.util.FluentBackoff; import org.apache.beam.sdk.util.Transport; +import org.joda.time.Duration; /** * The utility class that sets up and tears down external resources, @@ -79,7 +80,9 @@ public ExampleUtils(PipelineOptions options) { */ public void setup() throws IOException { Sleeper sleeper = Sleeper.DEFAULT; - BackOff backOff = new AttemptBoundedExponentialBackOff(3, 200); + BackOff backOff = + FluentBackoff.DEFAULT + .withMaxRetries(3).withInitialBackoff(Duration.millis(200)).backoff(); Throwable lastException = null; try { do { diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java index 9a515fa6140f..dad59f279741 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java @@ -34,7 +34,6 @@ import java.net.SocketTimeoutException; import java.util.List; import java.util.Map; -import java.util.concurrent.TimeUnit; import javax.annotation.Nullable; import org.apache.beam.runners.dataflow.internal.DataflowAggregatorTransforms; import org.apache.beam.runners.dataflow.internal.DataflowMetricUpdateExtractor; @@ -44,8 +43,7 @@ import org.apache.beam.sdk.AggregatorValues; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.transforms.Aggregator; -import org.apache.beam.sdk.util.AttemptAndTimeBoundedExponentialBackOff; -import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; +import org.apache.beam.sdk.util.FluentBackoff; import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -95,14 +93,27 @@ public class DataflowPipelineJob implements PipelineResult { /** * The polling interval for job status and messages information. */ - static final long MESSAGES_POLLING_INTERVAL = TimeUnit.SECONDS.toMillis(2); - static final long STATUS_POLLING_INTERVAL = TimeUnit.SECONDS.toMillis(2); + static final Duration MESSAGES_POLLING_INTERVAL = Duration.standardSeconds(2); + static final Duration STATUS_POLLING_INTERVAL = Duration.standardSeconds(2); + + static final double DEFAULT_BACKOFF_EXPONENT = 1.5; /** - * The amount of polling attempts for job status and messages information. + * The amount of polling retries for job status and messages information. */ - static final int MESSAGES_POLLING_ATTEMPTS = 12; - static final int STATUS_POLLING_ATTEMPTS = 5; + static final int MESSAGES_POLLING_RETRIES = 11; + static final int STATUS_POLLING_RETRIES = 4; + + private static final FluentBackoff MESSAGES_BACKOFF_FACTORY = + FluentBackoff.DEFAULT + .withInitialBackoff(MESSAGES_POLLING_INTERVAL) + .withMaxRetries(MESSAGES_POLLING_RETRIES) + .withExponent(DEFAULT_BACKOFF_EXPONENT); + protected static final FluentBackoff STATUS_BACKOFF_FACTORY = + FluentBackoff.DEFAULT + .withInitialBackoff(STATUS_POLLING_INTERVAL) + .withMaxRetries(STATUS_POLLING_RETRIES) + .withExponent(DEFAULT_BACKOFF_EXPONENT); /** * Constructs the job. @@ -214,21 +225,23 @@ State waitUntilFinish( MonitoringUtil monitor = new MonitoringUtil(projectId, dataflowOptions.getDataflowClient()); long lastTimestamp = 0; - BackOff backoff = - duration.getMillis() > 0 - ? new AttemptAndTimeBoundedExponentialBackOff( - MESSAGES_POLLING_ATTEMPTS, - MESSAGES_POLLING_INTERVAL, - duration.getMillis(), - AttemptAndTimeBoundedExponentialBackOff.ResetPolicy.ATTEMPTS, - nanoClock) - : new AttemptBoundedExponentialBackOff( - MESSAGES_POLLING_ATTEMPTS, MESSAGES_POLLING_INTERVAL); + BackOff backoff; + if (!duration.isLongerThan(Duration.ZERO)) { + backoff = MESSAGES_BACKOFF_FACTORY.backoff(); + } else { + backoff = MESSAGES_BACKOFF_FACTORY.withMaxCumulativeBackoff(duration).backoff(); + } + + // This function tracks the cumulative time from the *first request* to enforce the wall-clock + // limit. Any backoff instance could, at best, track the the time since the first attempt at a + // given request. Thus, we need to track the cumulative time ourselves. + long startNanos = nanoClock.nanoTime(); + State state; do { // Get the state of the job before listing messages. This ensures we always fetch job // messages after the job finishes to ensure we have all them. - state = getStateWithRetries(1, sleeper); + state = getStateWithRetries(STATUS_BACKOFF_FACTORY.withMaxRetries(0).backoff(), sleeper); boolean hasError = state == State.UNKNOWN; if (messageHandler != null && !hasError) { @@ -250,7 +263,16 @@ State waitUntilFinish( } if (!hasError) { + // Reset the backoff. backoff.reset(); + // If duration is set, update the new cumulative sleep time to be the remaining + // part of the total input sleep duration. + if (duration.isLongerThan(Duration.ZERO)) { + long nanosConsumed = nanoClock.nanoTime() - startNanos; + Duration consumed = Duration.millis((nanosConsumed + 999999) / 1000000); + backoff = + MESSAGES_BACKOFF_FACTORY.withMaxCumulativeBackoff(duration.minus(consumed)).backoff(); + } // Check if the job is done. if (state.isTerminal()) { return state; @@ -287,7 +309,7 @@ public State getState() { return terminalState; } - return getStateWithRetries(STATUS_POLLING_ATTEMPTS, Sleeper.DEFAULT); + return getStateWithRetries(STATUS_BACKOFF_FACTORY.backoff(), Sleeper.DEFAULT); } /** @@ -299,7 +321,7 @@ public State getState() { * @return The state of the job or State.UNKNOWN in case of failure. */ @VisibleForTesting - State getStateWithRetries(int attempts, Sleeper sleeper) { + State getStateWithRetries(BackOff attempts, Sleeper sleeper) { if (terminalState != null) { return terminalState; } @@ -318,17 +340,13 @@ State getStateWithRetries(int attempts, Sleeper sleeper) { * Attempts to get the underlying {@link Job}. Uses exponential backoff on failure up to the * maximum number of passed in attempts. * - * @param attempts The amount of attempts to make. + * @param backoff the {@link BackOff} used to control retries. * @param sleeper Object used to do the sleeps between attempts. * @return The underlying {@link Job} object. * @throws IOException When the maximum number of retries is exhausted, the last exception is * thrown. */ - @VisibleForTesting - Job getJobWithRetries(int attempts, Sleeper sleeper) throws IOException { - AttemptBoundedExponentialBackOff backoff = - new AttemptBoundedExponentialBackOff(attempts, STATUS_POLLING_INTERVAL); - + private Job getJobWithRetries(BackOff backoff, Sleeper sleeper) throws IOException { // Retry loop ends in return or throw while (true) { try { diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java index bf1f666c32c5..6d910baba372 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java @@ -18,7 +18,7 @@ package org.apache.beam.runners.dataflow.util; import com.fasterxml.jackson.core.Base64Variants; -import com.google.api.client.util.BackOffUtils; +import com.google.api.client.util.BackOff; import com.google.api.client.util.Sleeper; import com.google.api.services.dataflow.model.DataflowPackage; import com.google.cloud.hadoop.util.ApiErrorExtractor; @@ -37,10 +37,11 @@ import java.util.Collection; import java.util.List; import java.util.Objects; -import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; +import org.apache.beam.sdk.util.FluentBackoff; import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.util.MimeTypes; import org.apache.beam.sdk.util.ZipFiles; +import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -54,11 +55,15 @@ public class PackageUtil { /** * The initial interval to use between package staging attempts. */ - private static final long INITIAL_BACKOFF_INTERVAL_MS = 5000L; + private static final Duration INITIAL_BACKOFF_INTERVAL = Duration.standardSeconds(5); /** - * The maximum number of attempts when staging a file. + * The maximum number of retries when staging a file. */ - private static final int MAX_ATTEMPTS = 5; + private static final int MAX_RETRIES = 4; + + private static final FluentBackoff BACKOFF_FACTORY = + FluentBackoff.DEFAULT + .withMaxRetries(MAX_RETRIES).withInitialBackoff(INITIAL_BACKOFF_INTERVAL); /** * Translates exceptions from API calls. @@ -199,9 +204,7 @@ static List stageClasspathElements( } // Upload file, retrying on failure. - AttemptBoundedExponentialBackOff backoff = new AttemptBoundedExponentialBackOff( - MAX_ATTEMPTS, - INITIAL_BACKOFF_INTERVAL_MS); + BackOff backoff = BACKOFF_FACTORY.backoff(); while (true) { try { LOG.debug("Uploading classpath element {} to {}", classpathElement, target); @@ -219,15 +222,17 @@ static List stageClasspathElements( + "'gcloud auth login'.", classpathElement, target); LOG.error(errorMessage); throw new IOException(errorMessage, e); - } else if (!backoff.atMaxAttempts()) { - LOG.warn("Upload attempt failed, sleeping before retrying staging of classpath: {}", - classpathElement, e); - BackOffUtils.next(retrySleeper, backoff); - } else { + } + long sleep = backoff.nextBackOffMillis(); + if (sleep == BackOff.STOP) { // Rethrow last error, to be included as a cause in the catch below. LOG.error("Upload failed, will NOT retry staging of classpath: {}", classpathElement, e); throw e; + } else { + LOG.warn("Upload attempt failed, sleeping before retrying staging of classpath: {}", + classpathElement, e); + retrySleeper.sleep(sleep); } } } diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java index 22b5400b0fdf..226140a67245 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java @@ -60,7 +60,6 @@ import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.Sum; -import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.joda.time.Duration; @@ -111,22 +110,21 @@ public void setup() { * AttemptBoundedExponentialBackOff given the number of retries and * an initial polling interval. * - * @param pollingIntervalMillis The initial polling interval given. - * @param attempts The number of attempts made + * @param pollingInterval The initial polling interval given. + * @param retries The number of retries made * @param timeSleptMillis The amount of time slept by the clock. This is checked * against the valid interval. */ - void checkValidInterval(long pollingIntervalMillis, int attempts, long timeSleptMillis) { + private void checkValidInterval(Duration pollingInterval, int retries, long timeSleptMillis) { long highSum = 0; long lowSum = 0; - for (int i = 1; i < attempts; i++) { + for (int i = 0; i < retries; i++) { double currentInterval = - pollingIntervalMillis - * Math.pow(AttemptBoundedExponentialBackOff.DEFAULT_MULTIPLIER, i - 1); - double offset = - AttemptBoundedExponentialBackOff.DEFAULT_RANDOMIZATION_FACTOR * currentInterval; - highSum += Math.round(currentInterval + offset); - lowSum += Math.round(currentInterval - offset); + pollingInterval.getMillis() + * Math.pow(DataflowPipelineJob.DEFAULT_BACKOFF_EXPONENT, i); + double randomOffset = 0.5 * currentInterval; + highSum += Math.round(currentInterval + randomOffset); + lowSum += Math.round(currentInterval - randomOffset); } assertThat(timeSleptMillis, allOf(greaterThanOrEqualTo(lowSum), lessThanOrEqualTo(highSum))); } @@ -228,7 +226,7 @@ public void testWaitToFinishFail() throws Exception { assertEquals(null, state); long timeDiff = TimeUnit.NANOSECONDS.toMillis(fastClock.nanoTime() - startTime); checkValidInterval(DataflowPipelineJob.MESSAGES_POLLING_INTERVAL, - DataflowPipelineJob.MESSAGES_POLLING_ATTEMPTS, timeDiff); + DataflowPipelineJob.MESSAGES_POLLING_RETRIES, timeDiff); } @Test @@ -246,8 +244,8 @@ public void testWaitToFinishTimeFail() throws Exception { State state = job.waitUntilFinish(Duration.millis(4), null, fastClock, fastClock); assertEquals(null, state); long timeDiff = TimeUnit.NANOSECONDS.toMillis(fastClock.nanoTime() - startTime); - // Should only sleep for the 4 ms remaining. - assertEquals(timeDiff, 4L); + // Should only have slept for the 4 ms allowed. + assertEquals(4L, timeDiff); } @Test @@ -268,7 +266,7 @@ public void testGetStateReturnsServiceState() throws Exception { assertEquals( State.RUNNING, - job.getStateWithRetries(DataflowPipelineJob.STATUS_POLLING_ATTEMPTS, fastClock)); + job.getStateWithRetries(DataflowPipelineJob.STATUS_BACKOFF_FACTORY.backoff(), fastClock)); } @Test @@ -286,10 +284,10 @@ public void testGetStateWithExceptionReturnsUnknown() throws Exception { long startTime = fastClock.nanoTime(); assertEquals( State.UNKNOWN, - job.getStateWithRetries(DataflowPipelineJob.STATUS_POLLING_ATTEMPTS, fastClock)); + job.getStateWithRetries(DataflowPipelineJob.STATUS_BACKOFF_FACTORY.backoff(), fastClock)); long timeDiff = TimeUnit.NANOSECONDS.toMillis(fastClock.nanoTime() - startTime); checkValidInterval(DataflowPipelineJob.STATUS_POLLING_INTERVAL, - DataflowPipelineJob.STATUS_POLLING_ATTEMPTS, timeDiff); + DataflowPipelineJob.STATUS_POLLING_RETRIES, timeDiff); } @Test diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java index 28d77468bbc0..b41c655c31bc 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java @@ -34,7 +34,7 @@ import org.apache.beam.sdk.transforms.RemoveDuplicates; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.util.IntervalBoundedExponentialBackOff; +import org.apache.beam.sdk.util.FluentBackoff; import org.apache.beam.sdk.util.ValueWithRecordId; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; @@ -52,6 +52,10 @@ class BoundedReadFromUnboundedSource extends PTransform source; private final long maxNumRecords; private final Duration maxReadTime; + private static final FluentBackoff BACKOFF_FACTORY = + FluentBackoff.DEFAULT + .withInitialBackoff(Duration.millis(10)) + .withMaxBackoff(Duration.standardSeconds(10)); /** * Returns a new {@link BoundedReadFromUnboundedSource} that reads a bounded amount @@ -241,7 +245,7 @@ public boolean advance() throws IOException { private boolean advanceWithBackoff() throws IOException { // Try reading from the source with exponential backoff - BackOff backoff = new IntervalBoundedExponentialBackOff(10000L, 10L); + BackOff backoff = BACKOFF_FACTORY.backoff(); long nextSleep = backoff.nextBackOffMillis(); while (nextSleep != BackOff.STOP) { if (endTime != null && Instant.now().isAfter(endTime)) { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOff.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOff.java deleted file mode 100644 index d8050e0a3219..000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOff.java +++ /dev/null @@ -1,172 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.util; - -import static com.google.common.base.Preconditions.checkArgument; - -import com.google.api.client.util.BackOff; -import com.google.api.client.util.NanoClock; -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; -import java.util.concurrent.TimeUnit; - -/** - * Extension of {@link AttemptBoundedExponentialBackOff} that bounds the total time that the backoff - * is happening as well as the amount of retries. Acts exactly as a AttemptBoundedExponentialBackOff - * unless the time interval has expired since the object was created. At this point, it will always - * return BackOff.STOP. Calling reset() resets both the timer and the number of retry attempts, - * unless a custom ResetPolicy (ResetPolicy.ATTEMPTS or ResetPolicy.TIMER) is passed to the - * constructor. - * - *

              Implementation is not thread-safe. - */ -public class AttemptAndTimeBoundedExponentialBackOff extends AttemptBoundedExponentialBackOff { - private long endTimeMillis; - private long maximumTotalWaitTimeMillis; - private ResetPolicy resetPolicy; - private final NanoClock nanoClock; - // NanoClock.SYSTEM has a max elapsed time of 292 years or 2^63 ns. Here, we choose 2^53 ns as - // a smaller but still huge limit. - private static final long MAX_ELAPSED_TIME_MILLIS = 1L << 53; - - /** - * A ResetPolicy controls the behavior of this BackOff when reset() is called. By default, both - * the number of attempts and the time bound for the BackOff are reset, but an alternative - * ResetPolicy may be set to only reset one of these two. - */ - public static enum ResetPolicy { - ALL, - ATTEMPTS, - TIMER - } - - /** - * Constructs an instance of AttemptAndTimeBoundedExponentialBackoff. - * - * @param maximumNumberOfAttempts The maximum number of attempts it will make. - * @param initialIntervalMillis The original interval to wait between attempts in milliseconds. - * @param maximumTotalWaitTimeMillis The maximum total time that this object will - * allow more attempts in milliseconds. - */ - public AttemptAndTimeBoundedExponentialBackOff( - int maximumNumberOfAttempts, long initialIntervalMillis, long maximumTotalWaitTimeMillis) { - this( - maximumNumberOfAttempts, - initialIntervalMillis, - maximumTotalWaitTimeMillis, - ResetPolicy.ALL, - NanoClock.SYSTEM); - } - - /** - * Constructs an instance of AttemptAndTimeBoundedExponentialBackoff. - * - * @param maximumNumberOfAttempts The maximum number of attempts it will make. - * @param initialIntervalMillis The original interval to wait between attempts in milliseconds. - * @param maximumTotalWaitTimeMillis The maximum total time that this object will - * allow more attempts in milliseconds. - * @param resetPolicy The ResetPolicy specifying the properties of this BackOff that are subject - * to being reset. - */ - public AttemptAndTimeBoundedExponentialBackOff( - int maximumNumberOfAttempts, - long initialIntervalMillis, - long maximumTotalWaitTimeMillis, - ResetPolicy resetPolicy) { - this( - maximumNumberOfAttempts, - initialIntervalMillis, - maximumTotalWaitTimeMillis, - resetPolicy, - NanoClock.SYSTEM); - } - - /** - * Constructs an instance of AttemptAndTimeBoundedExponentialBackoff. - * - * @param maximumNumberOfAttempts The maximum number of attempts it will make. - * @param initialIntervalMillis The original interval to wait between attempts in milliseconds. - * @param maximumTotalWaitTimeMillis The maximum total time that this object will - * allow more attempts in milliseconds. - * @param resetPolicy The ResetPolicy specifying the properties of this BackOff that are subject - * to being reset. - * @param nanoClock clock used to measure the time that has passed. - */ - public AttemptAndTimeBoundedExponentialBackOff( - int maximumNumberOfAttempts, - long initialIntervalMillis, - long maximumTotalWaitTimeMillis, - ResetPolicy resetPolicy, - NanoClock nanoClock) { - super(maximumNumberOfAttempts, initialIntervalMillis); - checkArgument( - maximumTotalWaitTimeMillis > 0, "Maximum total wait time must be greater than zero."); - checkArgument( - maximumTotalWaitTimeMillis < MAX_ELAPSED_TIME_MILLIS, - "Maximum total wait time must be less than " + MAX_ELAPSED_TIME_MILLIS + " milliseconds"); - checkArgument(resetPolicy != null, "resetPolicy may not be null"); - checkArgument(nanoClock != null, "nanoClock may not be null"); - this.maximumTotalWaitTimeMillis = maximumTotalWaitTimeMillis; - this.resetPolicy = resetPolicy; - this.nanoClock = nanoClock; - // Set the end time for this BackOff. Note that we cannot simply call reset() here since the - // resetPolicy may not be set to reset the time bound. - endTimeMillis = getTimeMillis() + maximumTotalWaitTimeMillis; - } - - @Override - @SuppressFBWarnings(value = "UR_UNINIT_READ_CALLED_FROM_SUPER_CONSTRUCTOR", - justification = "Explicitly handled in implementation.") - public void reset() { - // reset() is called in the constructor of the parent class before resetPolicy and nanoClock are - // set. In this case, we call the parent class's reset() method and return. - if (resetPolicy == null) { - super.reset(); - return; - } - // Reset the number of attempts. - if (resetPolicy == ResetPolicy.ALL || resetPolicy == ResetPolicy.ATTEMPTS) { - super.reset(); - } - // Reset the time bound. - if (resetPolicy == ResetPolicy.ALL || resetPolicy == ResetPolicy.TIMER) { - endTimeMillis = getTimeMillis() + maximumTotalWaitTimeMillis; - } - } - - public void setEndtimeMillis(long endTimeMillis) { - this.endTimeMillis = endTimeMillis; - } - - @Override - public long nextBackOffMillis() { - if (atMaxAttempts()) { - return BackOff.STOP; - } - long backoff = Math.min(super.nextBackOffMillis(), endTimeMillis - getTimeMillis()); - return (backoff > 0 ? backoff : BackOff.STOP); - } - - private long getTimeMillis() { - return TimeUnit.NANOSECONDS.toMillis(nanoClock.nanoTime()); - } - - @Override - public boolean atMaxAttempts() { - return super.atMaxAttempts() || getTimeMillis() >= endTimeMillis; - } -} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOff.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOff.java deleted file mode 100644 index 5707293bad06..000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOff.java +++ /dev/null @@ -1,85 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.util; - -import static com.google.common.base.Preconditions.checkArgument; - -import com.google.api.client.util.BackOff; - - -/** - * Implementation of {@link BackOff} that increases the back off period for each retry attempt - * using a randomization function that grows exponentially. - * - *

              Example: The initial interval is .5 seconds and the maximum number of retries is 10. - * For 10 tries the sequence will be (values in seconds): - * - *

              - * retry#      retry_interval     randomized_interval
              - * 1             0.5                [0.25,   0.75]
              - * 2             0.75               [0.375,  1.125]
              - * 3             1.125              [0.562,  1.687]
              - * 4             1.687              [0.8435, 2.53]
              - * 5             2.53               [1.265,  3.795]
              - * 6             3.795              [1.897,  5.692]
              - * 7             5.692              [2.846,  8.538]
              - * 8             8.538              [4.269, 12.807]
              - * 9            12.807              [6.403, 19.210]
              - * 10           {@link BackOff#STOP}
              - * 
              - * - *

              Implementation is not thread-safe. - */ -public class AttemptBoundedExponentialBackOff implements BackOff { - public static final double DEFAULT_MULTIPLIER = 1.5; - public static final double DEFAULT_RANDOMIZATION_FACTOR = 0.5; - private final int maximumNumberOfAttempts; - private final long initialIntervalMillis; - private int currentAttempt; - - public AttemptBoundedExponentialBackOff(int maximumNumberOfAttempts, long initialIntervalMillis) { - checkArgument(maximumNumberOfAttempts > 0, - "Maximum number of attempts must be greater than zero."); - checkArgument(initialIntervalMillis > 0, "Initial interval must be greater than zero."); - this.maximumNumberOfAttempts = maximumNumberOfAttempts; - this.initialIntervalMillis = initialIntervalMillis; - reset(); - } - - @Override - public void reset() { - currentAttempt = 1; - } - - @Override - public long nextBackOffMillis() { - if (currentAttempt >= maximumNumberOfAttempts) { - return BackOff.STOP; - } - double currentIntervalMillis = initialIntervalMillis - * Math.pow(DEFAULT_MULTIPLIER, currentAttempt - 1); - double randomOffset = (Math.random() * 2 - 1) - * DEFAULT_RANDOMIZATION_FACTOR * currentIntervalMillis; - currentAttempt += 1; - return Math.round(currentIntervalMillis + randomOffset); - } - - public boolean atMaxAttempts() { - return currentAttempt >= maximumNumberOfAttempts; - } -} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FluentBackoff.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FluentBackoff.java new file mode 100644 index 000000000000..479d7a81451b --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FluentBackoff.java @@ -0,0 +1,229 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.util; + +import static com.google.common.base.Preconditions.checkArgument; + +import com.google.api.client.util.BackOff; +import com.google.common.base.MoreObjects; +import org.joda.time.Duration; + +/** + * A fluent builder for {@link BackOff} objects that allows customization of the retry algorithm. + * + * @see #DEFAULT for the default configuration parameters. + */ +public final class FluentBackoff { + + private static final double DEFAULT_EXPONENT = 1.5; + private static final double DEFAULT_RANDOMIZATION_FACTOR = 0.5; + private static final Duration DEFAULT_MIN_BACKOFF = Duration.standardSeconds(1); + private static final Duration DEFAULT_MAX_BACKOFF = Duration.standardDays(1000); + private static final int DEFAULT_MAX_RETRIES = Integer.MAX_VALUE; + private static final Duration DEFAULT_MAX_CUM_BACKOFF = Duration.standardDays(1000); + + private final double exponent; + private final Duration initialBackoff; + private final Duration maxBackoff; + private final Duration maxCumulativeBackoff; + private final int maxRetries; + + /** + * By default the {@link BackOff} created by this builder will use exponential backoff (base + * exponent 1.5) with an initial backoff of 1 second. These parameters can be overridden with + * {@link #withExponent(double)} and {@link #withInitialBackoff(Duration)}, + * respectively, and the maximum backoff after exponential increase can be capped using {@link + * FluentBackoff#withMaxBackoff(Duration)}. + * + *

              The default {@link BackOff} does not limit the number of retries. To limit the backoff, the + * maximum total number of retries can be set using {@link #withMaxRetries(int)}. The + * total time spent in backoff can be time-bounded as well by configuring {@link + * #withMaxCumulativeBackoff(Duration)}. If either of these limits are reached, calls + * to {@link BackOff#nextBackOffMillis()} will return {@link BackOff#STOP} to signal that no more + * retries should continue. + */ + public static final FluentBackoff DEFAULT = new FluentBackoff( + DEFAULT_EXPONENT, + DEFAULT_MIN_BACKOFF, DEFAULT_MAX_BACKOFF, DEFAULT_MAX_CUM_BACKOFF, + DEFAULT_MAX_RETRIES); + + /** + * Instantiates a {@link BackOff} that will obey the current configuration. + * + * @see FluentBackoff + */ + public BackOff backoff() { + return new BackoffImpl(this); + } + + /** + * Returns a copy of this {@link FluentBackoff} that instead uses the specified exponent to + * control the exponential growth of delay. + * + *

              Does not modify this object. + * + * @see FluentBackoff + */ + public FluentBackoff withExponent(double exponent) { + checkArgument(exponent > 0, "exponent %s must be greater than 0", exponent); + return new FluentBackoff( + exponent, initialBackoff, maxBackoff, maxCumulativeBackoff, maxRetries); + } + + /** + * Returns a copy of this {@link FluentBackoff} that instead uses the specified initial backoff + * duration. + * + *

              Does not modify this object. + * + * @see FluentBackoff + */ + public FluentBackoff withInitialBackoff(Duration initialBackoff) { + checkArgument( + initialBackoff.isLongerThan(Duration.ZERO), + "initialBackoff %s must be at least 1 millisecond", + initialBackoff); + return new FluentBackoff( + exponent, initialBackoff, maxBackoff, maxCumulativeBackoff, maxRetries); + } + + /** + * Returns a copy of this {@link FluentBackoff} that limits the maximum backoff of an individual + * attempt to the specified duration. + * + *

              Does not modify this object. + * + * @see FluentBackoff + */ + public FluentBackoff withMaxBackoff(Duration maxBackoff) { + checkArgument( + maxBackoff.getMillis() > 0, + "maxBackoff %s must be at least 1 millisecond", + maxBackoff); + return new FluentBackoff( + exponent, initialBackoff, maxBackoff, maxCumulativeBackoff, maxRetries); + } + + /** + * Returns a copy of this {@link FluentBackoff} that limits the total time spent in backoff + * returned across all calls to {@link BackOff#nextBackOffMillis()}. + * + *

              Does not modify this object. + * + * @see FluentBackoff + */ + public FluentBackoff withMaxCumulativeBackoff(Duration maxCumulativeBackoff) { + checkArgument(maxCumulativeBackoff.isLongerThan(Duration.ZERO), + "maxCumulativeBackoff %s must be at least 1 millisecond", maxCumulativeBackoff); + return new FluentBackoff( + exponent, initialBackoff, maxBackoff, maxCumulativeBackoff, maxRetries); + } + + /** + * Returns a copy of this {@link FluentBackoff} that limits the total number of retries, aka + * the total number of calls to {@link BackOff#nextBackOffMillis()} before returning + * {@link BackOff#STOP}. + * + *

              Does not modify this object. + * + * @see FluentBackoff + */ + public FluentBackoff withMaxRetries(int maxRetries) { + checkArgument(maxRetries >= 0, "maxRetries %s cannot be negative", maxRetries); + return new FluentBackoff( + exponent, initialBackoff, maxBackoff, maxCumulativeBackoff, maxRetries); + } + + public String toString() { + return MoreObjects.toStringHelper(FluentBackoff.class) + .add("exponent", exponent) + .add("initialBackoff", initialBackoff) + .add("maxBackoff", maxBackoff) + .add("maxRetries", maxRetries) + .add("maxCumulativeBackoff", maxCumulativeBackoff) + .toString(); + } + + private static class BackoffImpl implements BackOff { + + // Customization of this backoff. + private final FluentBackoff backoffConfig; + // Current state + private Duration currentCumulativeBackoff; + private int currentRetry; + + @Override + public void reset() { + currentRetry = 0; + currentCumulativeBackoff = Duration.ZERO; + } + + @Override + public long nextBackOffMillis() { + // Maximum number of retries reached. + if (currentRetry >= backoffConfig.maxRetries) { + return BackOff.STOP; + } + // Maximum cumulative backoff reached. + if (currentCumulativeBackoff.compareTo(backoffConfig.maxCumulativeBackoff) >= 0) { + return BackOff.STOP; + } + + double currentIntervalMillis = + Math.min( + backoffConfig.initialBackoff.getMillis() + * Math.pow(backoffConfig.exponent, currentRetry), + backoffConfig.maxBackoff.getMillis()); + double randomOffset = + (Math.random() * 2 - 1) * DEFAULT_RANDOMIZATION_FACTOR * currentIntervalMillis; + long nextBackoffMillis = Math.round(currentIntervalMillis + randomOffset); + // Cap to limit on cumulative backoff + Duration remainingCumulative = + backoffConfig.maxCumulativeBackoff.minus(currentCumulativeBackoff); + nextBackoffMillis = Math.min(nextBackoffMillis, remainingCumulative.getMillis()); + + // Update state and return backoff. + currentCumulativeBackoff = currentCumulativeBackoff.plus(nextBackoffMillis); + currentRetry += 1; + return nextBackoffMillis; + } + + private BackoffImpl(FluentBackoff backoffConfig) { + this.backoffConfig = backoffConfig; + this.reset(); + } + + public String toString() { + return MoreObjects.toStringHelper(BackoffImpl.class) + .add("backoffConfig", backoffConfig) + .add("currentRetry", currentRetry) + .add("currentCumulativeBackoff", currentCumulativeBackoff) + .toString(); + } + } + + private FluentBackoff( + double exponent, Duration initialBackoff, Duration maxBackoff, Duration maxCumulativeBackoff, + int maxRetries) { + this.exponent = exponent; + this.initialBackoff = initialBackoff; + this.maxBackoff = maxBackoff; + this.maxRetries = maxRetries; + this.maxCumulativeBackoff = maxCumulativeBackoff; + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java index 44a182eeb3d8..41c372e9bc0c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java @@ -66,6 +66,7 @@ import org.apache.beam.sdk.options.GcsOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.gcsfs.GcsPath; +import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -119,6 +120,9 @@ public GcsUtil create(PipelineOptions options) { */ private static final int MAX_CONCURRENT_BATCHES = 256; + private static final FluentBackoff BACKOFF_FACTORY = + FluentBackoff.DEFAULT.withMaxRetries(3).withInitialBackoff(Duration.millis(200)); + ///////////////////////////////////////////////////////////////////////////// /** Client for the GCS API. */ @@ -177,7 +181,7 @@ public List expand(GcsPath gcsPattern) throws IOException { // the request has strong global consistency. ResilientOperation.retry( ResilientOperation.getGoogleRequestCallable(getObject), - new AttemptBoundedExponentialBackOff(3, 200), + BACKOFF_FACTORY.backoff(), RetryDeterminer.SOCKET_ERRORS, IOException.class); return ImmutableList.of(gcsPattern); @@ -216,7 +220,7 @@ public List expand(GcsPath gcsPattern) throws IOException { try { objects = ResilientOperation.retry( ResilientOperation.getGoogleRequestCallable(listObject), - new AttemptBoundedExponentialBackOff(3, 200), + BACKOFF_FACTORY.backoff(), RetryDeterminer.SOCKET_ERRORS, IOException.class); } catch (Exception e) { @@ -257,7 +261,10 @@ Integer getUploadBufferSizeBytes() { * if the resource does not exist. */ public long fileSize(GcsPath path) throws IOException { - return fileSize(path, new AttemptBoundedExponentialBackOff(4, 200), Sleeper.DEFAULT); + return fileSize( + path, + BACKOFF_FACTORY.backoff(), + Sleeper.DEFAULT); } /** @@ -335,7 +342,10 @@ public WritableByteChannel create(GcsPath path, * be accessible otherwise the permissions exception will be propagated. */ public boolean bucketExists(GcsPath path) throws IOException { - return bucketExists(path, new AttemptBoundedExponentialBackOff(4, 200), Sleeper.DEFAULT); + return bucketExists( + path, + BACKOFF_FACTORY.backoff(), + Sleeper.DEFAULT); } /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOff.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOff.java index 519776a550da..6fac6dca67a6 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOff.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOff.java @@ -49,6 +49,7 @@ * *

              Implementation is not thread-safe. */ +@Deprecated public class IntervalBoundedExponentialBackOff implements BackOff { public static final double DEFAULT_MULTIPLIER = 1.5; public static final double DEFAULT_RANDOMIZATION_FACTOR = 0.5; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOffTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOffTest.java deleted file mode 100644 index 59e0fb7dab28..000000000000 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOffTest.java +++ /dev/null @@ -1,212 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.util; - -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.allOf; -import static org.hamcrest.Matchers.greaterThan; -import static org.hamcrest.Matchers.lessThan; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -import com.google.api.client.util.BackOff; -import org.apache.beam.sdk.testing.FastNanoClockAndSleeper; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** Unit tests for {@link AttemptAndTimeBoundedExponentialBackOff}. */ -@RunWith(JUnit4.class) -public class AttemptAndTimeBoundedExponentialBackOffTest { - @Rule public ExpectedException exception = ExpectedException.none(); - @Rule public FastNanoClockAndSleeper fastClock = new FastNanoClockAndSleeper(); - - @Test - public void testUsingInvalidInitialInterval() throws Exception { - exception.expect(IllegalArgumentException.class); - exception.expectMessage("Initial interval must be greater than zero."); - new AttemptAndTimeBoundedExponentialBackOff(10, 0L, 1000L); - } - - @Test - public void testUsingInvalidTimeInterval() throws Exception { - exception.expect(IllegalArgumentException.class); - exception.expectMessage("Maximum total wait time must be greater than zero."); - new AttemptAndTimeBoundedExponentialBackOff(10, 2L, 0L); - } - - @Test - public void testUsingInvalidMaximumNumberOfRetries() throws Exception { - exception.expect(IllegalArgumentException.class); - exception.expectMessage("Maximum number of attempts must be greater than zero."); - new AttemptAndTimeBoundedExponentialBackOff(-1, 10L, 1000L); - } - - @Test - public void testThatFixedNumberOfAttemptsExits() throws Exception { - BackOff backOff = - new AttemptAndTimeBoundedExponentialBackOff( - 3, - 500L, - 1000L, - AttemptAndTimeBoundedExponentialBackOff.ResetPolicy.ALL, - fastClock); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L))); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(374L), lessThan(1126L))); - assertEquals(BackOff.STOP, backOff.nextBackOffMillis()); - } - - @Test - public void testThatResettingAllowsReuse() throws Exception { - AttemptBoundedExponentialBackOff backOff = - new AttemptAndTimeBoundedExponentialBackOff( - 3, - 500, - 1000L, - AttemptAndTimeBoundedExponentialBackOff.ResetPolicy.ALL, - fastClock); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L))); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(374L), lessThan(1126L))); - assertEquals(BackOff.STOP, backOff.nextBackOffMillis()); - backOff.reset(); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L))); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(374L), lessThan(1126L))); - assertEquals(BackOff.STOP, backOff.nextBackOffMillis()); - - backOff = - new AttemptAndTimeBoundedExponentialBackOff( - 30, - 500, - 1000L, - AttemptAndTimeBoundedExponentialBackOff.ResetPolicy.ALL, - fastClock); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L))); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(374L), lessThan(1126L))); - fastClock.sleep(2000L); - assertEquals(BackOff.STOP, backOff.nextBackOffMillis()); - backOff.reset(); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L))); - } - - @Test - public void testThatResettingAttemptsAllowsReuse() throws Exception { - AttemptBoundedExponentialBackOff backOff = - new AttemptAndTimeBoundedExponentialBackOff( - 3, - 500, - 1000, - AttemptAndTimeBoundedExponentialBackOff.ResetPolicy.ATTEMPTS, - fastClock); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L))); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(374L), lessThan(1126L))); - assertEquals(BackOff.STOP, backOff.nextBackOffMillis()); - backOff.reset(); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L))); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(374L), lessThan(1126L))); - assertEquals(BackOff.STOP, backOff.nextBackOffMillis()); - } - - @Test - public void testThatResettingAttemptsDoesNotAllowsReuse() throws Exception { - AttemptBoundedExponentialBackOff backOff = - new AttemptAndTimeBoundedExponentialBackOff( - 30, - 500, - 1000L, - AttemptAndTimeBoundedExponentialBackOff.ResetPolicy.ATTEMPTS, - fastClock); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L))); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(374L), lessThan(1126L))); - fastClock.sleep(2000L); - assertEquals(BackOff.STOP, backOff.nextBackOffMillis()); - backOff.reset(); - assertEquals(BackOff.STOP, backOff.nextBackOffMillis()); - } - - @Test - public void testThatResettingTimerAllowsReuse() throws Exception { - AttemptBoundedExponentialBackOff backOff = - new AttemptAndTimeBoundedExponentialBackOff( - 30, - 500, - 1000L, - AttemptAndTimeBoundedExponentialBackOff.ResetPolicy.TIMER, - fastClock); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L))); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(374L), lessThan(1126L))); - fastClock.sleep(2000L); - assertEquals(BackOff.STOP, backOff.nextBackOffMillis()); - backOff.reset(); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(561L), lessThan(1688L))); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(843L), lessThan(2531L))); - } - - @Test - public void testThatResettingTimerDoesNotAllowReuse() throws Exception { - AttemptBoundedExponentialBackOff backOff = - new AttemptAndTimeBoundedExponentialBackOff( - 3, - 500, - 1000L, - AttemptAndTimeBoundedExponentialBackOff.ResetPolicy.TIMER, - fastClock); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L))); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(374L), lessThan(1126L))); - assertEquals(BackOff.STOP, backOff.nextBackOffMillis()); - backOff.reset(); - assertEquals(BackOff.STOP, backOff.nextBackOffMillis()); - } - - @Test - public void testTimeBound() throws Exception { - AttemptBoundedExponentialBackOff backOff = - new AttemptAndTimeBoundedExponentialBackOff( - 3, 500L, 5L, AttemptAndTimeBoundedExponentialBackOff.ResetPolicy.ALL, fastClock); - assertEquals(backOff.nextBackOffMillis(), 5L); - } - - @Test - public void testAtMaxAttempts() throws Exception { - AttemptBoundedExponentialBackOff backOff = - new AttemptAndTimeBoundedExponentialBackOff( - 3, - 500L, - 1000L, - AttemptAndTimeBoundedExponentialBackOff.ResetPolicy.ALL, - fastClock); - assertFalse(backOff.atMaxAttempts()); - backOff.nextBackOffMillis(); - assertFalse(backOff.atMaxAttempts()); - backOff.nextBackOffMillis(); - assertTrue(backOff.atMaxAttempts()); - assertEquals(BackOff.STOP, backOff.nextBackOffMillis()); - } - - @Test - public void testAtMaxTime() throws Exception { - AttemptBoundedExponentialBackOff backOff = - new AttemptAndTimeBoundedExponentialBackOff( - 3, 500L, 1L, AttemptAndTimeBoundedExponentialBackOff.ResetPolicy.ALL, fastClock); - fastClock.sleep(2); - assertTrue(backOff.atMaxAttempts()); - assertEquals(BackOff.STOP, backOff.nextBackOffMillis()); - } -} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOffTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOffTest.java deleted file mode 100644 index 3cfa961865fb..000000000000 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOffTest.java +++ /dev/null @@ -1,84 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.util; - -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.allOf; -import static org.hamcrest.Matchers.greaterThan; -import static org.hamcrest.Matchers.lessThan; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -import com.google.api.client.util.BackOff; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** Unit tests for {@link AttemptBoundedExponentialBackOff}. */ -@RunWith(JUnit4.class) -public class AttemptBoundedExponentialBackOffTest { - @Rule public ExpectedException exception = ExpectedException.none(); - - @Test - public void testUsingInvalidInitialInterval() throws Exception { - exception.expect(IllegalArgumentException.class); - exception.expectMessage("Initial interval must be greater than zero."); - new AttemptBoundedExponentialBackOff(10, 0L); - } - - @Test - public void testUsingInvalidMaximumNumberOfRetries() throws Exception { - exception.expect(IllegalArgumentException.class); - exception.expectMessage("Maximum number of attempts must be greater than zero."); - new AttemptBoundedExponentialBackOff(-1, 10L); - } - - @Test - public void testThatFixedNumberOfAttemptsExits() throws Exception { - BackOff backOff = new AttemptBoundedExponentialBackOff(3, 500); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L))); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(374L), lessThan(1126L))); - assertEquals(BackOff.STOP, backOff.nextBackOffMillis()); - } - - @Test - public void testThatResettingAllowsReuse() throws Exception { - BackOff backOff = new AttemptBoundedExponentialBackOff(3, 500); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L))); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(374L), lessThan(1126L))); - assertEquals(BackOff.STOP, backOff.nextBackOffMillis()); - backOff.reset(); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L))); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(374L), lessThan(1126L))); - assertEquals(BackOff.STOP, backOff.nextBackOffMillis()); - } - - @Test - public void testAtMaxAttempts() throws Exception { - AttemptBoundedExponentialBackOff backOff = new AttemptBoundedExponentialBackOff(3, 500); - assertFalse(backOff.atMaxAttempts()); - backOff.nextBackOffMillis(); - assertFalse(backOff.atMaxAttempts()); - backOff.nextBackOffMillis(); - assertTrue(backOff.atMaxAttempts()); - assertEquals(BackOff.STOP, backOff.nextBackOffMillis()); - } -} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FluentBackoffTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FluentBackoffTest.java new file mode 100644 index 000000000000..20b03cf603a3 --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FluentBackoffTest.java @@ -0,0 +1,226 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.util; + +import static org.hamcrest.Matchers.allOf; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.lessThan; +import static org.hamcrest.Matchers.lessThanOrEqualTo; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; + +import com.google.api.client.util.BackOff; +import java.io.IOException; +import org.joda.time.Duration; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Tests for {@link FluentBackoff}. + */ +@RunWith(JUnit4.class) +public class FluentBackoffTest { + + @Rule + public ExpectedException thrown = ExpectedException.none(); + private final FluentBackoff defaultBackoff = FluentBackoff.DEFAULT; + + @Test + public void testInvalidExponent() { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("exponent -2.0 must be greater than 0"); + defaultBackoff.withExponent(-2.0); + } + + @Test + public void testInvalidInitialBackoff() { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("initialBackoff PT0S must be at least 1 millisecond"); + defaultBackoff.withInitialBackoff(Duration.ZERO); + } + + @Test + public void testInvalidMaxBackoff() { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("maxBackoff PT0S must be at least 1 millisecond"); + defaultBackoff.withMaxBackoff(Duration.ZERO); + } + + @Test + public void testInvalidMaxRetries() { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("maxRetries -1 cannot be negative"); + defaultBackoff.withMaxRetries(-1); + } + + @Test + public void testInvalidCumulativeBackoff() { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("maxCumulativeBackoff PT-0.002S must be at least 1 millisecond"); + defaultBackoff.withMaxCumulativeBackoff(Duration.millis(-2)); + } + + /** + * Tests with bounded interval, custom exponent, and unlimited retries. + */ + @Test + public void testBoundedIntervalWithReset() throws Exception { + BackOff backOff = + FluentBackoff.DEFAULT + .withInitialBackoff(Duration.millis(500)) + .withMaxBackoff(Duration.standardSeconds(1)).backoff(); + assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(249L), lessThan(751L))); + assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(374L), lessThan(1126L))); + assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(500L), + lessThanOrEqualTo(1500L))); + assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(500L), + lessThanOrEqualTo(1500L))); + assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(500L), + lessThanOrEqualTo(1500L))); + assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(500L), + lessThanOrEqualTo(1500L))); + assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(500L), + lessThanOrEqualTo(1500L))); + + // Reset, should go back to short times. + backOff.reset(); + assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(249L), lessThan(751L))); + assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(374L), lessThan(1126L))); + assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(500L), + lessThanOrEqualTo(1500L))); + assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(500L), + lessThanOrEqualTo(1500L))); + + } + + /** + * Tests with bounded interval, custom exponent, limited retries, and a reset. + */ + @Test + public void testMaxRetriesWithReset() throws Exception { + BackOff backOff = + FluentBackoff.DEFAULT + .withInitialBackoff(Duration.millis(500)) + .withMaxRetries(1) + .backoff(); + assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(249L), lessThan(751L))); + assertThat(backOff.nextBackOffMillis(), equalTo(BackOff.STOP)); + assertThat(backOff.nextBackOffMillis(), equalTo(BackOff.STOP)); + assertThat(backOff.nextBackOffMillis(), equalTo(BackOff.STOP)); + assertThat(backOff.nextBackOffMillis(), equalTo(BackOff.STOP)); + + backOff.reset(); + assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(249L), lessThan(751L))); + assertThat(backOff.nextBackOffMillis(), equalTo(BackOff.STOP)); + } + + private static long countMaximumBackoff(BackOff backOff) throws IOException { + long cumulativeBackoffMillis = 0; + long currentBackoffMillis = backOff.nextBackOffMillis(); + while (currentBackoffMillis != BackOff.STOP) { + cumulativeBackoffMillis += currentBackoffMillis; + currentBackoffMillis = backOff.nextBackOffMillis(); + } + return cumulativeBackoffMillis; + } + + /** + * Tests with bounded interval, custom exponent, limited cumulative time, and a reset. + */ + @Test + public void testBoundedIntervalAndCumTimeWithReset() throws Exception { + BackOff backOff = + FluentBackoff.DEFAULT + .withInitialBackoff(Duration.millis(500)) + .withMaxBackoff(Duration.standardSeconds(1)) + .withMaxCumulativeBackoff(Duration.standardMinutes(1)).backoff(); + + assertThat(countMaximumBackoff(backOff), equalTo(Duration.standardMinutes(1).getMillis())); + + backOff.reset(); + assertThat(countMaximumBackoff(backOff), equalTo(Duration.standardMinutes(1).getMillis())); + // sanity check: should get 0 if we don't reset + assertThat(countMaximumBackoff(backOff), equalTo(0L)); + + backOff.reset(); + assertThat(countMaximumBackoff(backOff), equalTo(Duration.standardMinutes(1).getMillis())); + } + + /** + * Tests with bounded interval, custom exponent, limited cumulative time and retries. + */ + @Test + public void testBoundedIntervalAndCumTimeAndRetriesWithReset() throws Exception { + BackOff backOff = + FluentBackoff.DEFAULT + .withInitialBackoff(Duration.millis(500)) + .withMaxBackoff(Duration.standardSeconds(1)) + .withMaxCumulativeBackoff(Duration.standardMinutes(1)) + .backoff(); + + long cumulativeBackoffMillis = 0; + long currentBackoffMillis = backOff.nextBackOffMillis(); + while (currentBackoffMillis != BackOff.STOP) { + cumulativeBackoffMillis += currentBackoffMillis; + currentBackoffMillis = backOff.nextBackOffMillis(); + } + assertThat(cumulativeBackoffMillis, equalTo(Duration.standardMinutes(1).getMillis())); + } + + @Test + public void testFluentBackoffToString() throws IOException { + FluentBackoff config = FluentBackoff.DEFAULT + .withExponent(3.4) + .withMaxRetries(4) + .withInitialBackoff(Duration.standardSeconds(3)) + .withMaxBackoff(Duration.standardHours(1)) + .withMaxCumulativeBackoff(Duration.standardDays(1)); + + assertEquals( + "FluentBackoff{exponent=3.4, initialBackoff=PT3S, maxBackoff=PT3600S," + + " maxRetries=4, maxCumulativeBackoff=PT86400S}", + config.toString()); + } + @Test + public void testBackoffImplToString() throws IOException { + FluentBackoff config = FluentBackoff.DEFAULT + .withExponent(3.4) + .withMaxRetries(4) + .withInitialBackoff(Duration.standardSeconds(3)) + .withMaxBackoff(Duration.standardHours(1)) + .withMaxCumulativeBackoff(Duration.standardDays(1)); + BackOff backOff = config.backoff(); + + assertEquals( + "BackoffImpl{backoffConfig=" + config.toString() + "," + + " currentRetry=0, currentCumulativeBackoff=PT0S}", + backOff.toString()); + + // backoff once, ignoring result + backOff.nextBackOffMillis(); + + // currentRetry is exact, we can test it. + assertThat(backOff.toString(), containsString("currentRetry=1")); + // currentCumulativeBackoff is not exact; we cannot even check that it's non-zero (randomness). + } +} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java index 681b0aaaa059..9504b4c0c07c 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java @@ -365,7 +365,7 @@ public void testRetryFileSize() throws IOException { Storage.Objects mockStorageObjects = Mockito.mock(Storage.Objects.class); Storage.Objects.Get mockStorageGet = Mockito.mock(Storage.Objects.Get.class); - BackOff mockBackOff = new AttemptBoundedExponentialBackOff(3, 200); + BackOff mockBackOff = FluentBackoff.DEFAULT.withMaxRetries(2).backoff(); when(mockStorage.objects()).thenReturn(mockStorageObjects); when(mockStorageObjects.get("testbucket", "testobject")).thenReturn(mockStorageGet); @@ -376,7 +376,7 @@ public void testRetryFileSize() throws IOException { assertEquals(1000, gcsUtil.fileSize(GcsPath.fromComponents("testbucket", "testobject"), mockBackOff, new FastNanoClockAndSleeper())); - assertEquals(mockBackOff.nextBackOffMillis(), BackOff.STOP); + assertEquals(BackOff.STOP, mockBackOff.nextBackOffMillis()); } @Test @@ -390,7 +390,7 @@ public void testBucketExists() throws IOException { Storage.Buckets mockStorageObjects = Mockito.mock(Storage.Buckets.class); Storage.Buckets.Get mockStorageGet = Mockito.mock(Storage.Buckets.Get.class); - BackOff mockBackOff = new AttemptBoundedExponentialBackOff(3, 200); + BackOff mockBackOff = FluentBackoff.DEFAULT.backoff(); when(mockStorage.buckets()).thenReturn(mockStorageObjects); when(mockStorageObjects.get("testbucket")).thenReturn(mockStorageGet); @@ -413,7 +413,7 @@ public void testBucketDoesNotExistBecauseOfAccessError() throws IOException { Storage.Buckets mockStorageObjects = Mockito.mock(Storage.Buckets.class); Storage.Buckets.Get mockStorageGet = Mockito.mock(Storage.Buckets.Get.class); - BackOff mockBackOff = new AttemptBoundedExponentialBackOff(3, 200); + BackOff mockBackOff = FluentBackoff.DEFAULT.backoff(); GoogleJsonResponseException expectedException = googleJsonResponseException(HttpStatusCodes.STATUS_CODE_FORBIDDEN, "Waves hand mysteriously", "These aren't the buckets your looking for"); @@ -438,7 +438,7 @@ public void testBucketDoesNotExist() throws IOException { Storage.Buckets mockStorageObjects = Mockito.mock(Storage.Buckets.class); Storage.Buckets.Get mockStorageGet = Mockito.mock(Storage.Buckets.Get.class); - BackOff mockBackOff = new AttemptBoundedExponentialBackOff(3, 200); + BackOff mockBackOff = FluentBackoff.DEFAULT.backoff(); when(mockStorage.buckets()).thenReturn(mockStorageObjects); when(mockStorageObjects.get("testbucket")).thenReturn(mockStorageGet); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java index 304dc820ac9d..6dde581ab4d5 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java @@ -24,9 +24,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.api.client.json.JsonFactory; -import com.google.api.client.util.BackOff; -import com.google.api.client.util.BackOffUtils; -import com.google.api.client.util.Sleeper; import com.google.api.services.bigquery.Bigquery; import com.google.api.services.bigquery.model.Job; import com.google.api.services.bigquery.model.JobConfigurationExtract; @@ -69,7 +66,6 @@ import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ThreadLocalRandom; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -108,7 +104,6 @@ import org.apache.beam.sdk.transforms.windowing.DefaultTrigger; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; import org.apache.beam.sdk.util.FileIOChannelFactory; import org.apache.beam.sdk.util.GcsIOChannelFactory; import org.apache.beam.sdk.util.GcsUtil; @@ -129,6 +124,7 @@ import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; +import org.joda.time.Duration; import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -294,7 +290,7 @@ public class BigQueryIO { * *

              If the project id is omitted, the default project id is used. */ - public static TableReference parseTableSpec(String tableSpec) { + static TableReference parseTableSpec(String tableSpec) { Matcher match = TABLE_SPEC.matcher(tableSpec); if (!match.matches()) { throw new IllegalArgumentException( @@ -953,14 +949,14 @@ private void readObject(ObjectInputStream in) throws ClassNotFoundException, IOE * ... */ private abstract static class BigQuerySourceBase extends BoundedSource { - // The maximum number of attempts to verify temp files. - private static final int MAX_FILES_VERIFY_ATTEMPTS = 10; + // The maximum number of retries to verify temp files. + private static final int MAX_FILES_VERIFY_RETRIES = 9; // The maximum number of retries to poll a BigQuery job. protected static final int JOB_POLL_MAX_RETRIES = Integer.MAX_VALUE; // The initial backoff for verifying temp files. - private static final long INITIAL_FILES_VERIFY_BACKOFF_MILLIS = TimeUnit.SECONDS.toMillis(1); + private static final Duration INITIAL_FILES_VERIFY_BACKOFF = Duration.standardSeconds(1); protected final String jobIdToken; protected final String extractDestinationDir; @@ -1055,14 +1051,7 @@ public TableRow apply(GenericRecord input) { }}; List> avroSources = Lists.newArrayList(); - BackOff backoff = new AttemptBoundedExponentialBackOff( - MAX_FILES_VERIFY_ATTEMPTS, INITIAL_FILES_VERIFY_BACKOFF_MILLIS); for (String fileName : files) { - while (BackOffUtils.next(Sleeper.DEFAULT, backoff)) { - if (IOChannelUtils.getFactory(fileName).getSizeBytes(fileName) != -1) { - break; - } - } avroSources.add(new TransformingSource<>( AvroSource.from(fileName), function, getDefaultOutputCoder())); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java index 6aff3b097710..8b5e8c28aeb2 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java @@ -53,14 +53,13 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; import javax.annotation.Nullable; import org.apache.beam.sdk.options.BigQueryOptions; import org.apache.beam.sdk.options.GcsOptions; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; -import org.apache.beam.sdk.util.IntervalBoundedExponentialBackOff; +import org.apache.beam.sdk.util.FluentBackoff; import org.apache.beam.sdk.util.Transport; +import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -72,14 +71,14 @@ class BigQueryServicesImpl implements BigQueryServices { private static final Logger LOG = LoggerFactory.getLogger(BigQueryServicesImpl.class); - // The maximum number of attempts to execute a BigQuery RPC. - private static final int MAX_RPC_ATTEMPTS = 10; + // The maximum number of retries to execute a BigQuery RPC. + private static final int MAX_RPC_RETRIES = 9; // The initial backoff for executing a BigQuery RPC. - private static final long INITIAL_RPC_BACKOFF_MILLIS = TimeUnit.SECONDS.toMillis(1); + private static final Duration INITIAL_RPC_BACKOFF = Duration.standardSeconds(1); // The initial backoff for polling the status of a BigQuery job. - private static final long INITIAL_JOB_STATUS_POLL_BACKOFF_MILLIS = TimeUnit.SECONDS.toMillis(1); + private static final Duration INITIAL_JOB_STATUS_POLL_BACKOFF = Duration.standardSeconds(1); @Override public JobService getJobService(BigQueryOptions options) { @@ -121,9 +120,9 @@ private JobServiceImpl(BigQueryOptions options) { /** * {@inheritDoc} * - *

              Tries executing the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds. + *

              Tries executing the RPC for at most {@code MAX_RPC_RETRIES} times until it succeeds. * - * @throws IOException if it exceeds {@code MAX_RPC_ATTEMPTS} attempts. + * @throws IOException if it exceeds {@code MAX_RPC_RETRIES} attempts. */ @Override public void startLoadJob( @@ -139,9 +138,9 @@ public void startLoadJob( /** * {@inheritDoc} * - *

              Tries executing the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds. + *

              Tries executing the RPC for at most {@code MAX_RPC_RETRIES} times until it succeeds. * - * @throws IOException if it exceeds {@code MAX_RPC_ATTEMPTS} attempts. + * @throws IOException if it exceeds {@code MAX_RPC_RETRIES} attempts. */ @Override public void startExtractJob(JobReference jobRef, JobConfigurationExtract extractConfig) @@ -157,9 +156,9 @@ public void startExtractJob(JobReference jobRef, JobConfigurationExtract extract /** * {@inheritDoc} * - *

              Tries executing the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds. + *

              Tries executing the RPC for at most {@code MAX_RPC_RETRIES} times until it succeeds. * - * @throws IOException if it exceeds {@code MAX_RPC_ATTEMPTS} attempts. + * @throws IOException if it exceeds {@code MAX_RPC_RETRIES} attempts. */ @Override public void startQueryJob(JobReference jobRef, JobConfigurationQuery queryConfig) @@ -175,9 +174,9 @@ public void startQueryJob(JobReference jobRef, JobConfigurationQuery queryConfig /** * {@inheritDoc} * - *

              Tries executing the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds. + *

              Tries executing the RPC for at most {@code MAX_RPC_RETRIES} times until it succeeds. * - * @throws IOException if it exceeds {@code MAX_RPC_ATTEMPTS} attempts. + * @throws IOException if it exceeds {@code MAX_RPC_RETRIES} attempts. */ @Override public void startCopyJob(JobReference jobRef, JobConfigurationTableCopy copyConfig) @@ -194,7 +193,8 @@ private static void startJob(Job job, ApiErrorExtractor errorExtractor, Bigquery client) throws IOException, InterruptedException { BackOff backoff = - new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS); + FluentBackoff.DEFAULT + .withMaxRetries(MAX_RPC_RETRIES).withInitialBackoff(INITIAL_RPC_BACKOFF).backoff(); startJob(job, errorExtractor, client, Sleeper.DEFAULT, backoff); } @@ -227,15 +227,17 @@ static void startJob( throw new IOException( String.format( "Unable to insert job: %s, aborting after %d .", - jobRef.getJobId(), MAX_RPC_ATTEMPTS), + jobRef.getJobId(), MAX_RPC_RETRIES), lastException); } @Override public Job pollJob(JobReference jobRef, int maxAttempts) throws InterruptedException { - BackOff backoff = new AttemptBoundedExponentialBackOff( - maxAttempts, INITIAL_JOB_STATUS_POLL_BACKOFF_MILLIS); + BackOff backoff = + FluentBackoff.DEFAULT + .withMaxRetries(maxAttempts).withInitialBackoff(INITIAL_JOB_STATUS_POLL_BACKOFF) + .backoff(); return pollJob(jobRef, Sleeper.DEFAULT, backoff); } @@ -270,12 +272,13 @@ public JobStatistics dryRunQuery(String projectId, String query) .setQuery(query)) .setDryRun(true)); BackOff backoff = - new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS); + FluentBackoff.DEFAULT + .withMaxRetries(MAX_RPC_RETRIES).withInitialBackoff(INITIAL_RPC_BACKOFF).backoff(); return executeWithRetries( client.jobs().insert(projectId, job), String.format( "Unable to dry run query: %s, aborting after %d retries.", - query, MAX_RPC_ATTEMPTS), + query, MAX_RPC_RETRIES), Sleeper.DEFAULT, backoff).getStatistics(); } @@ -289,15 +292,14 @@ static class DatasetServiceImpl implements DatasetService { // The maximum number of rows to upload per InsertAll request. private static final long MAX_ROWS_PER_BATCH = 500; - // The maximum number of times to retry inserting rows into BigQuery. - private static final int MAX_INSERT_ATTEMPTS = 5; - - // The initial backoff after a failure inserting rows into BigQuery. - private static final long INITIAL_INSERT_BACKOFF_INTERVAL_MS = 200L; + private static final FluentBackoff INSERT_BACKOFF_FACTORY = + FluentBackoff.DEFAULT.withInitialBackoff(Duration.millis(200)).withMaxRetries(5); - // Backoff time bounds for rate limit exceeded errors. - private static final long INITIAL_RATE_LIMIT_EXCEEDED_BACKOFF_MS = TimeUnit.SECONDS.toMillis(1); - private static final long MAX_RATE_LIMIT_EXCEEDED_BACKOFF_MS = TimeUnit.MINUTES.toMillis(2); + // A backoff for rate limit exceeded errors. Retries forever. + private static final FluentBackoff DEFAULT_BACKOFF_FACTORY = + FluentBackoff.DEFAULT + .withInitialBackoff(Duration.standardSeconds(1)) + .withMaxBackoff(Duration.standardMinutes(2)); private final ApiErrorExtractor errorExtractor; private final Bigquery client; @@ -335,20 +337,21 @@ private DatasetServiceImpl(BigQueryOptions bqOptions) { /** * {@inheritDoc} * - *

              Tries executing the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds. + *

              Tries executing the RPC for at most {@code MAX_RPC_RETRIES} times until it succeeds. * - * @throws IOException if it exceeds {@code MAX_RPC_ATTEMPTS} attempts. + * @throws IOException if it exceeds {@code MAX_RPC_RETRIES} attempts. */ @Override public Table getTable(String projectId, String datasetId, String tableId) throws IOException, InterruptedException { BackOff backoff = - new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS); + FluentBackoff.DEFAULT + .withMaxRetries(MAX_RPC_RETRIES).withInitialBackoff(INITIAL_RPC_BACKOFF).backoff(); return executeWithRetries( client.tables().get(projectId, datasetId, tableId), String.format( "Unable to get table: %s, aborting after %d retries.", - tableId, MAX_RPC_ATTEMPTS), + tableId, MAX_RPC_RETRIES), Sleeper.DEFAULT, backoff); } @@ -356,20 +359,21 @@ public Table getTable(String projectId, String datasetId, String tableId) /** * {@inheritDoc} * - *

              Tries executing the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds. + *

              Tries executing the RPC for at most {@code MAX_RPC_RETRIES} times until it succeeds. * - * @throws IOException if it exceeds {@code MAX_RPC_ATTEMPTS} attempts. + * @throws IOException if it exceeds {@code MAX_RPC_RETRIES} attempts. */ @Override public void deleteTable(String projectId, String datasetId, String tableId) throws IOException, InterruptedException { BackOff backoff = - new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS); + FluentBackoff.DEFAULT + .withMaxRetries(MAX_RPC_RETRIES).withInitialBackoff(INITIAL_RPC_BACKOFF).backoff(); executeWithRetries( client.tables().delete(projectId, datasetId, tableId), String.format( "Unable to delete table: %s, aborting after %d retries.", - tableId, MAX_RPC_ATTEMPTS), + tableId, MAX_RPC_RETRIES), Sleeper.DEFAULT, backoff); } @@ -378,12 +382,13 @@ public void deleteTable(String projectId, String datasetId, String tableId) public boolean isTableEmpty(String projectId, String datasetId, String tableId) throws IOException, InterruptedException { BackOff backoff = - new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS); + FluentBackoff.DEFAULT + .withMaxRetries(MAX_RPC_RETRIES).withInitialBackoff(INITIAL_RPC_BACKOFF).backoff(); TableDataList dataList = executeWithRetries( client.tabledata().list(projectId, datasetId, tableId), String.format( "Unable to list table data: %s, aborting after %d retries.", - tableId, MAX_RPC_ATTEMPTS), + tableId, MAX_RPC_RETRIES), Sleeper.DEFAULT, backoff); return dataList.getRows() == null || dataList.getRows().isEmpty(); @@ -392,20 +397,21 @@ public boolean isTableEmpty(String projectId, String datasetId, String tableId) /** * {@inheritDoc} * - *

              Tries executing the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds. + *

              Tries executing the RPC for at most {@code MAX_RPC_RETRIES} times until it succeeds. * - * @throws IOException if it exceeds {@code MAX_RPC_ATTEMPTS} attempts. + * @throws IOException if it exceeds {@code MAX_RPC_RETRIES} attempts. */ @Override public Dataset getDataset(String projectId, String datasetId) throws IOException, InterruptedException { BackOff backoff = - new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS); + FluentBackoff.DEFAULT + .withMaxRetries(MAX_RPC_RETRIES).withInitialBackoff(INITIAL_RPC_BACKOFF).backoff(); return executeWithRetries( client.datasets().get(projectId, datasetId), String.format( "Unable to get dataset: %s, aborting after %d retries.", - datasetId, MAX_RPC_ATTEMPTS), + datasetId, MAX_RPC_RETRIES), Sleeper.DEFAULT, backoff); } @@ -413,21 +419,21 @@ public Dataset getDataset(String projectId, String datasetId) /** * {@inheritDoc} * - *

              Tries executing the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds. + *

              Tries executing the RPC for at most {@code MAX_RPC_RETRIES} times until it succeeds. * - * @throws IOException if it exceeds {@code MAX_RPC_ATTEMPTS} attempts. + * @throws IOException if it exceeds {@code MAX_RPC_RETRIES} attempts. */ @Override public void createDataset( String projectId, String datasetId, String location, String description) throws IOException, InterruptedException { BackOff backoff = - new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS); + FluentBackoff.DEFAULT + .withMaxRetries(MAX_RPC_RETRIES).withInitialBackoff(INITIAL_RPC_BACKOFF).backoff(); createDataset(projectId, datasetId, location, description, Sleeper.DEFAULT, backoff); } - @VisibleForTesting - void createDataset( + private void createDataset( String projectId, String datasetId, String location, @@ -464,27 +470,28 @@ void createDataset( throw new IOException( String.format( "Unable to create dataset: %s, aborting after %d .", - datasetId, MAX_RPC_ATTEMPTS), + datasetId, MAX_RPC_RETRIES), lastException); } /** * {@inheritDoc} * - *

              Tries executing the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds. + *

              Tries executing the RPC for at most {@code MAX_RPC_RETRIES} times until it succeeds. * - * @throws IOException if it exceeds {@code MAX_RPC_ATTEMPTS} attempts. + * @throws IOException if it exceeds {@code MAX_RPC_RETRIES} attempts. */ @Override public void deleteDataset(String projectId, String datasetId) throws IOException, InterruptedException { BackOff backoff = - new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS); + FluentBackoff.DEFAULT + .withMaxRetries(MAX_RPC_RETRIES).withInitialBackoff(INITIAL_RPC_BACKOFF).backoff(); executeWithRetries( client.datasets().delete(projectId, datasetId), String.format( "Unable to delete table: %s, aborting after %d retries.", - datasetId, MAX_RPC_ATTEMPTS), + datasetId, MAX_RPC_RETRIES), Sleeper.DEFAULT, backoff); } @@ -502,9 +509,7 @@ public long insertAll( + "as many elements as rowList"); } - AttemptBoundedExponentialBackOff backoff = new AttemptBoundedExponentialBackOff( - MAX_INSERT_ATTEMPTS, - INITIAL_INSERT_BACKOFF_INTERVAL_MS); + BackOff backoff = INSERT_BACKOFF_FACTORY.backoff(); long retTotalDataSize = 0; List allErrors = new ArrayList<>(); @@ -547,8 +552,7 @@ public long insertAll( executor.submit(new Callable>() { @Override public List call() throws IOException { - BackOff backoff = new IntervalBoundedExponentialBackOff( - MAX_RATE_LIMIT_EXCEEDED_BACKOFF_MS, INITIAL_RATE_LIMIT_EXCEEDED_BACKOFF_MS); + BackOff backoff = DEFAULT_BACKOFF_FACTORY.backoff(); while (true) { try { return insert.execute().getInsertErrors(); @@ -603,21 +607,24 @@ public List call() throws IOException { throw new RuntimeException(e.getCause()); } - if (!allErrors.isEmpty() && !backoff.atMaxAttempts()) { - try { - Thread.sleep(backoff.nextBackOffMillis()); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new IOException( - "Interrupted while waiting before retrying insert of " + retryRows); - } - LOG.info("Retrying failed inserts to BigQuery"); - rowsToPublish = retryRows; - idsToPublish = retryIds; - allErrors.clear(); - } else { + if (allErrors.isEmpty()) { + break; + } + long nextBackoffMillis = backoff.nextBackOffMillis(); + if (nextBackoffMillis == BackOff.STOP) { break; } + try { + Thread.sleep(backoff.nextBackOffMillis()); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IOException( + "Interrupted while waiting before retrying insert of " + retryRows); + } + LOG.info("Retrying failed inserts to BigQuery"); + rowsToPublish = retryRows; + idsToPublish = retryIds; + allErrors.clear(); } if (!allErrors.isEmpty()) { throw new IOException("Insert failed: " + allErrors); @@ -628,7 +635,7 @@ public List call() throws IOException { } private static class BigQueryJsonReaderImpl implements BigQueryJsonReader { - BigQueryTableRowIterator iterator; + private BigQueryTableRowIterator iterator; private BigQueryJsonReaderImpl(BigQueryTableRowIterator iterator) { this.iterator = iterator; @@ -706,7 +713,6 @@ static T executeWithRetries( /** * Identical to {@link BackOffUtils#next} but without checked IOException. - * @throws InterruptedException */ private static boolean nextBackOff(Sleeper sleeper, BackOff backoff) throws InterruptedException { try { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableRowIterator.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableRowIterator.java index 729da97f7f7e..677c661b8ac8 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableRowIterator.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableRowIterator.java @@ -58,7 +58,7 @@ import java.util.Random; import java.util.concurrent.TimeUnit; import javax.annotation.Nullable; -import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; +import org.apache.beam.sdk.util.FluentBackoff; import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -426,7 +426,8 @@ public static T executeWithBackOff(AbstractGoogleClientRequest client, St Object... errorArgs) throws IOException, InterruptedException { Sleeper sleeper = Sleeper.DEFAULT; BackOff backOff = - new AttemptBoundedExponentialBackOff(MAX_RETRIES, INITIAL_BACKOFF_TIME.getMillis()); + FluentBackoff.DEFAULT + .withMaxRetries(MAX_RETRIES).withInitialBackoff(INITIAL_BACKOFF_TIME).backoff(); T result = null; while (true) { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java index 6bd03b508268..45871f139769 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java @@ -76,12 +76,13 @@ import org.apache.beam.sdk.transforms.Values; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.display.DisplayData.Builder; -import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; +import org.apache.beam.sdk.util.FluentBackoff; import org.apache.beam.sdk.util.RetryHttpRequestInitializer; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; +import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -862,16 +863,11 @@ static class DatastoreWriterFn extends DoFn { private final V1DatastoreFactory datastoreFactory; // Current batch of mutations to be written. private final List mutations = new ArrayList<>(); - /** - * Since a bundle is written in batches, we should retry the commit of a batch in order to - * prevent transient errors from causing the bundle to fail. - */ - private static final int MAX_RETRIES = 5; - /** - * Initial backoff time for exponential backoff for retry attempts. - */ - private static final int INITIAL_BACKOFF_MILLIS = 5000; + private static final int MAX_RETRIES = 5; + private static final FluentBackoff BUNDLE_WRITE_BACKOFF = + FluentBackoff.DEFAULT + .withMaxRetries(MAX_RETRIES).withInitialBackoff(Duration.standardSeconds(5)); DatastoreWriterFn(String projectId) { this(projectId, new V1DatastoreFactory()); @@ -906,10 +902,10 @@ public void finishBundle(Context c) throws Exception { /** * Writes a batch of mutations to Cloud Datastore. * - *

              If a commit fails, it will be retried (up to {@link DatastoreWriterFn#MAX_RETRIES} - * times). All mutations in the batch will be committed again, even if the commit was partially - * successful. If the retry limit is exceeded, the last exception from the Cloud Datastore will - * be thrown. + *

              If a commit fails, it will be retried up to {@link #MAX_RETRIES} times. All + * mutations in the batch will be committed again, even if the commit was partially + * successful. If the retry limit is exceeded, the last exception from Cloud Datastore will be + * thrown. * * @throws DatastoreException if the commit fails or IOException or InterruptedException if * backing off between retries fails. @@ -917,7 +913,7 @@ public void finishBundle(Context c) throws Exception { private void flushBatch() throws DatastoreException, IOException, InterruptedException { LOG.debug("Writing batch of {} mutations", mutations.size()); Sleeper sleeper = Sleeper.DEFAULT; - BackOff backoff = new AttemptBoundedExponentialBackOff(MAX_RETRIES, INITIAL_BACKOFF_MILLIS); + BackOff backoff = BUNDLE_WRITE_BACKOFF.backoff(); while (true) { // Batch upsert entities. diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java index eb5fbe6e05f5..16cb004e318a 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java @@ -56,7 +56,7 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.ExpectedLogs; import org.apache.beam.sdk.testing.FastNanoClockAndSleeper; -import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; +import org.apache.beam.sdk.util.FluentBackoff; import org.apache.beam.sdk.util.RetryHttpRequestInitializer; import org.apache.beam.sdk.util.Transport; import org.junit.Before; @@ -117,9 +117,8 @@ public void testStartLoadJobSucceeds() throws IOException, InterruptedException when(response.getContent()).thenReturn(toStream(testJob)); Sleeper sleeper = new FastNanoClockAndSleeper(); - BackOff backoff = new AttemptBoundedExponentialBackOff( - 5 /* attempts */, 1000 /* initialIntervalMillis */); - JobServiceImpl.startJob(testJob, new ApiErrorExtractor(), bigquery, sleeper, backoff); + JobServiceImpl.startJob( + testJob, new ApiErrorExtractor(), bigquery, sleeper, FluentBackoff.DEFAULT.backoff()); verify(response, times(1)).getStatusCode(); verify(response, times(1)).getContent(); @@ -141,9 +140,8 @@ public void testStartLoadJobSucceedsAlreadyExists() throws IOException, Interrup when(response.getStatusCode()).thenReturn(409); // 409 means already exists Sleeper sleeper = new FastNanoClockAndSleeper(); - BackOff backoff = new AttemptBoundedExponentialBackOff( - 5 /* attempts */, 1000 /* initialIntervalMillis */); - JobServiceImpl.startJob(testJob, new ApiErrorExtractor(), bigquery, sleeper, backoff); + JobServiceImpl.startJob( + testJob, new ApiErrorExtractor(), bigquery, sleeper, FluentBackoff.DEFAULT.backoff()); verify(response, times(1)).getStatusCode(); verify(response, times(1)).getContent(); @@ -169,9 +167,8 @@ public void testStartLoadJobRetry() throws IOException, InterruptedException { .thenReturn(toStream(testJob)); Sleeper sleeper = new FastNanoClockAndSleeper(); - BackOff backoff = new AttemptBoundedExponentialBackOff( - 5 /* attempts */, 1000 /* initialIntervalMillis */); - JobServiceImpl.startJob(testJob, new ApiErrorExtractor(), bigquery, sleeper, backoff); + JobServiceImpl.startJob( + testJob, new ApiErrorExtractor(), bigquery, sleeper, FluentBackoff.DEFAULT.backoff()); verify(response, times(2)).getStatusCode(); verify(response, times(2)).getContent(); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java index a596bb3b7686..b680a0e23e75 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java @@ -54,8 +54,9 @@ import org.apache.beam.sdk.options.GcpOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; +import org.apache.beam.sdk.util.FluentBackoff; import org.apache.beam.sdk.util.RetryHttpRequestInitializer; +import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -232,7 +233,7 @@ static class V1TestWriter { // Number of times to retry on update failure private static final int MAX_RETRIES = 5; //Initial backoff time for exponential backoff for retry attempts. - private static final int INITIAL_BACKOFF_MILLIS = 5000; + private static final Duration INITIAL_BACKOFF = Duration.standardSeconds(5); // Returns true if a Datastore key is complete. A key is complete if its last element // has either an id or a name. @@ -279,7 +280,9 @@ void close() throws Exception { private void flushBatch() throws DatastoreException, IOException, InterruptedException { LOG.info("Writing batch of {} entities", entities.size()); Sleeper sleeper = Sleeper.DEFAULT; - BackOff backoff = new AttemptBoundedExponentialBackOff(MAX_RETRIES, INITIAL_BACKOFF_MILLIS); + BackOff backoff = + FluentBackoff.DEFAULT + .withMaxRetries(MAX_RETRIES).withInitialBackoff(INITIAL_BACKOFF).backoff(); while (true) { // Batch mutate entities. From 4b60e36816df66915061f2c834dabf3ca4ac1b89 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Fri, 15 Jul 2016 18:44:26 +0200 Subject: [PATCH 099/112] [BEAM-456] Add MongoDbIO --- sdks/java/io/mongodb/pom.xml | 129 ++++ .../apache/beam/sdk/io/mongodb/MongoDbIO.java | 553 ++++++++++++++++++ .../beam/sdk/io/mongodb/package-info.java | 22 + .../beam/sdk/io/mongodb/MongoDbIOTest.java | 209 +++++++ .../beam/sdk/io/mongodb/package-info.java | 22 + sdks/java/io/pom.xml | 1 + 6 files changed, 936 insertions(+) create mode 100644 sdks/java/io/mongodb/pom.xml create mode 100644 sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java create mode 100644 sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/package-info.java create mode 100644 sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java create mode 100644 sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/package-info.java diff --git a/sdks/java/io/mongodb/pom.xml b/sdks/java/io/mongodb/pom.xml new file mode 100644 index 000000000000..60f1d1e108ff --- /dev/null +++ b/sdks/java/io/mongodb/pom.xml @@ -0,0 +1,129 @@ + + + + + 4.0.0 + + + org.apache.beam + beam-sdks-java-io-parent + 0.3.0-incubating-SNAPSHOT + ../pom.xml + + + beam-sdks-java-io-mongodb + Apache Beam :: SDKs :: Java :: IO :: MongoDB + IO to read and write on MongoDB. + + + + + org.apache.maven.plugins + maven-compiler-plugin + + + org.apache.maven.plugins + maven-source-plugin + + + org.apache.maven.plugins + maven-surefire-plugin + + + org.apache.maven.plugins + maven-jar-plugin + + + org.apache.maven.plugins + maven-checkstyle-plugin + + + org.apache.maven.plugins + maven-javadoc-plugin + + + + + + 3.2.2 + + + + + org.apache.beam + beam-sdks-java-core + + + + org.slf4j + slf4j-api + + + + com.google.guava + guava + + + + com.google.code.findbugs + annotations + + + + org.mongodb + mongo-java-driver + ${mongo-java-driver.version} + + + + + de.flapdoodle.embed + de.flapdoodle.embed.mongo + 1.50.1 + test + + + de.flapdoodle.embed + de.flapdoodle.embed.process + 1.50.1 + test + + + junit + junit + test + + + org.slf4j + slf4j-jdk14 + test + + + org.apache.beam + beam-runners-direct-java + ${project.version} + test + + + org.hamcrest + hamcrest-all + test + + + + \ No newline at end of file diff --git a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java new file mode 100644 index 000000000000..77246143757c --- /dev/null +++ b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java @@ -0,0 +1,553 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.mongodb; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; + +import com.mongodb.BasicDBObject; +import com.mongodb.MongoClient; +import com.mongodb.MongoClientURI; +import com.mongodb.client.MongoCollection; +import com.mongodb.client.MongoCursor; +import com.mongodb.client.MongoDatabase; + +import java.util.ArrayList; +import java.util.List; + +import javax.annotation.Nullable; + +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.SerializableCoder; +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PDone; + +import org.bson.Document; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * IO to read and write data on MongoDB. + *

              + *

              Reading from MongoDB

              + *

              + *

              MongoDbIO source returns a bounded collection of String as {@code PCollection}. + * The String is the JSON form of the MongoDB Document.

              + *

              + *

              To configure the MongoDB source, you have to provide the connection URI, the database name + * and the collection name. The following example illustrates various options for configuring the + * source:

              + *

              + *

              {@code
              + *
              + * pipeline.apply(MongoDbIO.read()
              + *   .withUri("mongodb://localhost:27017")
              + *   .withDatabase("my-database")
              + *   .withCollection("my-collection"))
              + *   // above three are required configuration, returns PCollection
              + *
              + *   // rest of the settings are optional
              + *
              + * }
              + *

              + *

              The source also accepts an optional configuration: {@code withFilter()} allows you to + * define a JSON filter to get subset of data.

              + *

              + *

              Writing to MongoDB

              + *

              + *

              MongoDB sink supports writing of Document (as JSON String) in a MongoDB.

              + *

              + *

              To configure a MongoDB sink, you must specify a connection {@code URI}, a {@code Database} + * name, a {@code Collection} name. For instance:

              + *

              + *

              {@code
              + *
              + * pipeline
              + *   .apply(...)
              + *   .apply(MongoDbIO.write()
              + *     .withUri("mongodb://localhost:27017")
              + *     .withDatabase("my-database")
              + *     .withCollection("my-collection")
              + *     .withNumSplits(30))
              + *
              + * }
              + */ +// TODO instead of JSON String, does it make sense to populate the PCollection with BSON Document or +// DBObject ?? +public class MongoDbIO { + + private static final Logger LOGGER = LoggerFactory.getLogger(MongoDbIO.class); + + /** Read data from MongoDB. */ + public static Read read() { + return new Read(new BoundedMongoDbSource(null, null, null, null, 0)); + } + + /** Write data to MongoDB. */ + public static Write write() { + return new Write(new Write.MongoDbWriter(null, null, null, 1024L)); + } + + private MongoDbIO() { + } + + /** + * A {@link PTransform} to read data from MongoDB. + */ + public static class Read extends PTransform> { + + public Read withUri(String uri) { + return new Read(source.withUri(uri)); + } + + public Read withDatabase(String database) { + return new Read(source.withDatabase(database)); + } + + public Read withCollection(String collection) { + return new Read(source.withCollection(collection)); + } + + public Read withFilter(String filter) { + return new Read(source.withFilter(filter)); + } + + public Read withNumSplits(int numSplits) { + return new Read(source.withNumSplits(numSplits)); + } + + private final BoundedMongoDbSource source; + + private Read(BoundedMongoDbSource source) { + this.source = source; + } + + @Override + public PCollection apply(PBegin input) { + return input.apply(org.apache.beam.sdk.io.Read.from(getSource())); + } + + /** + * Creates a {@link BoundedSource} with the configuration in {@link Read}. + */ + @VisibleForTesting + BoundedSource getSource() { + return source; + } + + @Override + public void validate(PBegin input) { + source.validate(); + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + + source.populateDisplayData(builder); + } + + } + + private static class BoundedMongoDbSource extends BoundedSource { + + public BoundedMongoDbSource withUri(String uri) { + return new BoundedMongoDbSource(uri, database, collection, filter, numSplits); + } + + public BoundedMongoDbSource withDatabase(String database) { + return new BoundedMongoDbSource(uri, database, collection, filter, numSplits); + } + + public BoundedMongoDbSource withCollection(String collection) { + return new BoundedMongoDbSource(uri, database, collection, filter, numSplits); + } + + public BoundedMongoDbSource withFilter(String filter) { + return new BoundedMongoDbSource(uri, database, collection, filter, numSplits); + } + + public BoundedMongoDbSource withNumSplits(int numSplits) { + return new BoundedMongoDbSource(uri, database, collection, filter, numSplits); + } + + private final String uri; + private final String database; + private final String collection; + @Nullable + private final String filter; + private final int numSplits; + + public BoundedMongoDbSource(String uri, String database, String collection, String filter, + int numSplits) { + this.uri = uri; + this.database = database; + this.collection = collection; + this.filter = filter; + this.numSplits = numSplits; + } + + @Override + public Coder getDefaultOutputCoder() { + return SerializableCoder.of(String.class); + } + + @Override + public void validate() { + Preconditions.checkNotNull(uri, "uri"); + Preconditions.checkNotNull(database, "database"); + Preconditions.checkNotNull(collection, "collection"); + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder.add(DisplayData.item("uri", uri)); + builder.add(DisplayData.item("database", database)); + builder.add(DisplayData.item("collection", collection)); + builder.addIfNotNull(DisplayData.item("filter", filter)); + builder.add(DisplayData.item("numSplit", numSplits)); + } + + @Override + public boolean producesSortedKeys(PipelineOptions options) { + return false; + } + + @Override + public BoundedReader createReader(PipelineOptions options) { + return new BoundedMongoDbReader(this); + } + + @Override + public long getEstimatedSizeBytes(PipelineOptions pipelineOptions) { + long estimatedSizeBytes = 0L; + + MongoClient mongoClient = new MongoClient(); + MongoDatabase mongoDatabase = mongoClient.getDatabase(database); + MongoCollection mongoCollection = mongoDatabase.getCollection(collection); + + // get the Mongo collStats object + // it gives the size for the entire collection + BasicDBObject stat = new BasicDBObject(); + stat.append("collStats", collection); + Document stats = mongoDatabase.runCommand(stat); + estimatedSizeBytes = Long.valueOf(stats.get("size").toString()); + return estimatedSizeBytes; + } + + @Override + public List> splitIntoBundles(long desiredBundleSizeBytes, + PipelineOptions options) { + MongoClient mongoClient = new MongoClient(); + MongoDatabase mongoDatabase = mongoClient.getDatabase(database); + + List splitKeys = null; + if (numSplits > 0) { + // the user defines his desired number of splits + // calculate the batch size + long estimatedSizeBytes = getEstimatedSizeBytes(options); + desiredBundleSizeBytes = estimatedSizeBytes / numSplits; + } + + // the desired batch size is small, using default chunk size of 1MB + if (desiredBundleSizeBytes < 1024 * 1024) { + desiredBundleSizeBytes = 1 * 1024 * 1024; + } + + // now we have the batch size (provided by user or provided by the runner) + // we use Mongo splitVector command to get the split keys + BasicDBObject splitVectorCommand = new BasicDBObject(); + splitVectorCommand.append("splitVector", database + "." + collection); + splitVectorCommand.append("keyPattern", new BasicDBObject().append("_id", 1)); + splitVectorCommand.append("force", false); + // maxChunkSize is the Mongo partition size in MB + LOGGER.debug("Splitting in chunk of {} MB", desiredBundleSizeBytes / 1024 / 1024); + splitVectorCommand.append("maxChunkSize", desiredBundleSizeBytes / 1024 / 1024); + Document splitVectorCommandResult = mongoDatabase.runCommand(splitVectorCommand); + splitKeys = (List) splitVectorCommandResult.get("splitKeys"); + + List> sources = new ArrayList<>(); + if (splitKeys.size() < 1) { + LOGGER.debug("Split keys is low, using an unique source"); + sources.add(this); + return sources; + } + + LOGGER.debug("Number of splits is {}", splitKeys.size()); + for (String shardFilter : splitKeysToFilters(splitKeys, filter)) { + sources.add(this.withFilter(shardFilter)); + } + + return sources; + } + + /** + * Transform a list of split keys as a list of filters containing corresponding range. + * + *

              The list of split keys contains BSon Document basically containing for example: + *

                + *
              • _id: 56
              • + *
              • _id: 109
              • + *
              • _id: 256
              • + *
              + *

              + * + * This method will generate a list of range filters performing the following splits: + *
                + *
              • from the beginning of the collection up to _id 56, so basically data with + * _id lower than 56
              • + *
              • from _id 57 up to _id 109
              • + *
              • from _id 110 up to _id 256
              • + *
              • from _id 257 up to the end of the collection, so basically data with _id greater + * than 257
              • + *
              + * + * @param splitKeys The list of split keys. + * @param additionalFilter A custom (user) additional filter to append to the range filters. + * @return A list of filters containing the ranges. + */ + private static List splitKeysToFilters(List splitKeys, String + additionalFilter) { + ArrayList filters = new ArrayList<>(); + String lowestBound = null; // lower boundary (previous split in the iteration) + for (int i = 0; i < splitKeys.size(); i++) { + String splitKey = splitKeys.get(i).toString(); + String rangeFilter = null; + if (i == 0) { + // this is the first split in the list, the filter defines + // the range from the beginning up to this split + rangeFilter = String.format("{ $and: [ {\"_id\":{$lte:Objectd(\"%s\")}}", + splitKey); + } else if (i == splitKeys.size() - 1) { + // this is the last split in the list, the filter defines + // the range from the split up to the end + rangeFilter = String.format("{ $and: [ {\"_id\":{$gt:ObjectId(\"%s\")}}", + splitKey); + } else { + // we are between two splits + rangeFilter = String.format("{ $and: [ {\"_id\":{$gt:ObjectId(\"%s\")," + + "$lte:ObjectId(\"%s\")}}", lowestBound, splitKey); + } + if (additionalFilter != null && !additionalFilter.isEmpty()) { + // user provided a filter, we append the user filter to the range filter + rangeFilter = String.format("%s,%s ]}", rangeFilter, additionalFilter); + } else { + // user didn't provide a filter, just cleany close the range filter + rangeFilter = String.format("%s ]}", rangeFilter); + } + + filters.add(rangeFilter); + + lowestBound = splitKey; + } + return filters; + } + } + + private static class BoundedMongoDbReader extends BoundedSource.BoundedReader { + + private final BoundedMongoDbSource source; + + private MongoClient client; + private MongoCursor cursor; + private String current; + + public BoundedMongoDbReader(BoundedMongoDbSource source) { + this.source = source; + } + + @Override + public boolean start() { + client = new MongoClient(new MongoClientURI(source.uri)); + + MongoDatabase mongoDatabase = client.getDatabase(source.database); + + MongoCollection mongoCollection = mongoDatabase.getCollection(source.collection); + + if (source.filter == null) { + cursor = mongoCollection.find().iterator(); + } else { + Document bson = Document.parse(source.filter); + cursor = mongoCollection.find(bson).iterator(); + } + + return advance(); + } + + @Override + public boolean advance() { + if (cursor.hasNext()) { + current = cursor.next().toJson(); + return true; + } else { + return false; + } + } + + @Override + public BoundedSource getCurrentSource() { + return source; + } + + @Override + public String getCurrent() { + return current; + } + + @Override + public void close() { + try { + if (cursor != null) { + cursor.close(); + } + } catch (Exception e) { + LOGGER.warn("Error closing MongoDB cursor", e); + } + try { + client.close(); + } catch (Exception e) { + LOGGER.warn("Error closing MongoDB client", e); + } + } + + } + + /** + * A {@link PTransform} to write to a MongoDB database. + */ + public static class Write extends PTransform, PDone> { + + public Write withUri(String uri) { + return new Write(writer.withUri(uri)); + } + + public Write withDatabase(String database) { + return new Write(writer.withDatabase(database)); + } + + public Write withCollection(String collection) { + return new Write(writer.withCollection(collection)); + } + + public Write withBatchSize(long batchSize) { + return new Write(writer.withBatchSize(batchSize)); + } + + private final MongoDbWriter writer; + + private Write(MongoDbWriter writer) { + this.writer = writer; + } + + @Override + public PDone apply(PCollection input) { + input.apply(ParDo.of(writer)); + return PDone.in(input.getPipeline()); + } + + @Override + public void validate(PCollection input) { + writer.validate(); + } + + private static class MongoDbWriter extends DoFn { + + private final String uri; + private final String database; + private final String collection; + private final long batchSize; + + private MongoClient client; + private List batch; + + public MongoDbWriter(String uri, String database, String collection, long batchSize) { + this.uri = uri; + this.database = database; + this.collection = collection; + this.batchSize = batchSize; + } + + public MongoDbWriter withUri(String uri) { + return new MongoDbWriter(uri, database, collection, batchSize); + } + + public MongoDbWriter withDatabase(String database) { + return new MongoDbWriter(uri, database, collection, batchSize); + } + + public MongoDbWriter withCollection(String collection) { + return new MongoDbWriter(uri, database, collection, batchSize); + } + + public MongoDbWriter withBatchSize(long batchSize) { + return new MongoDbWriter(uri, database, collection, batchSize); + } + + public void validate() { + Preconditions.checkNotNull(uri, "uri"); + Preconditions.checkNotNull(database, "database"); + Preconditions.checkNotNull(collection, "collection"); + Preconditions.checkNotNull(batchSize, "batchSize"); + } + + @Setup + public void createMongoClient() throws Exception { + client = new MongoClient(new MongoClientURI(uri)); + } + + @StartBundle + public void startBundle(Context ctx) throws Exception { + batch = new ArrayList<>(); + } + + @ProcessElement + public void processElement(ProcessContext ctx) throws Exception { + String value = ctx.element(); + + batch.add(Document.parse(ctx.element())); + if (batch.size() >= batchSize) { + finishBundle(ctx); + } + } + + @FinishBundle + public void finishBundle(Context ctx) throws Exception { + MongoDatabase mongoDatabase = client.getDatabase(database); + MongoCollection mongoCollection = mongoDatabase.getCollection(collection); + + mongoCollection.insertMany(batch); + + batch.clear(); + } + + @Teardown + public void closeMongoClient() throws Exception { + client.close(); + client = null; + } + } + } +} diff --git a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/package-info.java b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/package-info.java new file mode 100644 index 000000000000..fd08b5854d3f --- /dev/null +++ b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/package-info.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Transforms for reading and writing from MongoDB. + */ +package org.apache.beam.sdk.io.mongodb; diff --git a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java new file mode 100644 index 000000000000..308e0719ff5f --- /dev/null +++ b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java @@ -0,0 +1,209 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.mongodb; + +import static org.junit.Assert.assertEquals; + +import com.mongodb.MongoClient; +import com.mongodb.client.MongoCollection; +import com.mongodb.client.MongoCursor; +import com.mongodb.client.MongoDatabase; + +import de.flapdoodle.embed.mongo.MongodExecutable; +import de.flapdoodle.embed.mongo.MongodStarter; +import de.flapdoodle.embed.mongo.config.IMongodConfig; +import de.flapdoodle.embed.mongo.config.MongoCmdOptionsBuilder; +import de.flapdoodle.embed.mongo.config.MongodConfigBuilder; +import de.flapdoodle.embed.mongo.config.Net; +import de.flapdoodle.embed.mongo.config.Storage; +import de.flapdoodle.embed.mongo.distribution.Version; +import de.flapdoodle.embed.process.io.file.Files; +import de.flapdoodle.embed.process.runtime.Network; + +import java.io.File; +import java.io.Serializable; +import java.util.ArrayList; + +import org.apache.beam.sdk.testing.NeedsRunner; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Count; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; + +import org.bson.Document; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Test on the MongoDbIO. + */ +public class MongoDbIOTest implements Serializable { + + private static final Logger LOGGER = LoggerFactory.getLogger(MongoDbIOTest.class); + + private static final String MONGODB_LOCATION = "target/mongodb"; + private static final int PORT = 27017; + private static final String DATABASE = "beam"; + private static final String COLLECTION = "test"; + + private transient MongodExecutable mongodExecutable; + + @Before + public void setup() throws Exception { + LOGGER.info("Starting MongoDB embedded instance"); + try { + Files.forceDelete(new File(MONGODB_LOCATION)); + } catch (Exception e) { + + } + new File(MONGODB_LOCATION).mkdirs(); + IMongodConfig mongodConfig = new MongodConfigBuilder() + .version(Version.Main.PRODUCTION) + .configServer(false) + .replication(new Storage(MONGODB_LOCATION, null, 0)) + .net(new Net("localhost", PORT, Network.localhostIsIPv6())) + .cmdOptions(new MongoCmdOptionsBuilder() + .syncDelay(10) + .useNoPrealloc(true) + .useSmallFiles(true) + .useNoJournal(true) + .build()) + .build(); + mongodExecutable = MongodStarter.getDefaultInstance().prepare(mongodConfig); + mongodExecutable.start(); + + LOGGER.info("Insert test data"); + + MongoClient client = new MongoClient("localhost", PORT); + MongoDatabase database = client.getDatabase(DATABASE); + + MongoCollection collection = database.getCollection(COLLECTION); + + String[] scientists = {"Einstein", "Darwin", "Copernicus", "Pasteur", "Curie", "Faraday", + "Newton", "Bohr", "Galilei", "Maxwell"}; + for (int i = 1; i <= 1000; i++) { + int index = i % scientists.length; + Document document = new Document(); + document.append("_id", i); + document.append("scientist", scientists[index]); + collection.insertOne(document); + } + + } + + @After + public void stop() throws Exception { + LOGGER.info("Stopping MongoDB instance"); + mongodExecutable.stop(); + } + + @Test + @Category(NeedsRunner.class) + public void testFullRead() throws Exception { + TestPipeline pipeline = TestPipeline.create(); + + PCollection output = pipeline.apply( + MongoDbIO.read() + .withUri("mongodb://localhost:" + PORT) + .withDatabase(DATABASE) + .withCollection(COLLECTION)); + + PAssert.thatSingleton(output.apply("Count All", Count.globally())) + .isEqualTo(1000L); + + PAssert.that(output + .apply("Map Scientist", MapElements.via(new SimpleFunction>() { + public KV apply(String input) { + Document bson = Document.parse(input); + return KV.of(bson.getString("scientist"), null); + } + })) + .apply("Count Scientist", Count.perKey()) + ).satisfies(new SerializableFunction>, Void>() { + @Override + public Void apply(Iterable> input) { + for (KV element : input) { + assertEquals(100L, element.getValue().longValue()); + } + return null; + } + }); + + pipeline.run(); + } + + @Test + @Category(NeedsRunner.class) + public void testReadWithFilter() throws Exception { + TestPipeline pipeline = TestPipeline.create(); + + PCollection output = pipeline.apply( + MongoDbIO.read() + .withUri("mongodb://localhost:" + PORT) + .withDatabase(DATABASE) + .withCollection(COLLECTION) + .withFilter("{\"scientist\":\"Einstein\"}")); + + PAssert.thatSingleton(output.apply("Count", Count.globally())) + .isEqualTo(100L); + + pipeline.run(); + } + + @Test + @Category(NeedsRunner.class) + public void testWrite() throws Exception { + TestPipeline pipeline = TestPipeline.create(); + + ArrayList data = new ArrayList<>(); + for (int i = 0; i < 10000; i++) { + data.add(String.format("{\"scientist\":\"Test %s\"}", i)); + } + pipeline.apply(Create.of(data)) + .apply(MongoDbIO.write().withUri("mongodb://localhost:" + PORT).withDatabase("test") + .withCollection("test")); + + pipeline.run(); + + MongoClient client = new MongoClient("localhost", PORT); + MongoDatabase database = client.getDatabase("test"); + MongoCollection collection = database.getCollection("test"); + + MongoCursor cursor = collection.find().iterator(); + + int count = 0; + while (cursor.hasNext()) { + count = count + 1; + cursor.next(); + } + + Assert.assertEquals(10000, count); + + } + +} diff --git a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/package-info.java b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/package-info.java new file mode 100644 index 000000000000..fd08b5854d3f --- /dev/null +++ b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/package-info.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Transforms for reading and writing from MongoDB. + */ +package org.apache.beam.sdk.io.mongodb; diff --git a/sdks/java/io/pom.xml b/sdks/java/io/pom.xml index 6cbd615a1d6e..c4c32ed0864f 100644 --- a/sdks/java/io/pom.xml +++ b/sdks/java/io/pom.xml @@ -38,6 +38,7 @@ jms kafka kinesis + mongodb From 951455746cebe6f42d2e7e85c02fd7c7be16767e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Thu, 25 Aug 2016 16:19:54 +0200 Subject: [PATCH 100/112] [BEAM-242] Enable checkstyle and fix checkstyle errors in Flink runner --- runners/flink/runner/pom.xml | 2 - .../FlinkPipelineExecutionEnvironment.java | 17 +- .../runners/flink/FlinkPipelineOptions.java | 27 +-- .../runners/flink/FlinkRunnerRegistrar.java | 6 + .../beam/runners/flink/FlinkRunnerResult.java | 17 +- .../beam/runners/flink/TestFlinkRunner.java | 8 +- .../beam/runners/flink/package-info.java | 22 +++ .../FlinkBatchPipelineTranslator.java | 15 +- .../FlinkBatchTranslationContext.java | 10 +- .../translation/FlinkPipelineTranslator.java | 2 +- .../FlinkStreamingTransformTranslators.java | 5 +- .../flink/translation/TranslationMode.java | 8 +- .../translation/functions/package-info.java | 22 +++ .../flink/translation/package-info.java | 22 +++ .../types/CoderTypeSerializer.java | 2 +- .../types/EncodedValueSerializer.java | 162 +++++++++--------- .../flink/translation/types/package-info.java | 22 +++ .../utils/SerializedPipelineOptions.java | 2 +- .../flink/translation/utils/package-info.java | 22 +++ .../wrappers/DataOutputViewWrapper.java | 2 +- .../translation/wrappers/package-info.java | 22 +++ .../wrappers/streaming/DoFnOperator.java | 12 +- .../streaming/SingletonKeyedWorkItem.java | 5 + .../SingletonKeyedWorkItemCoder.java | 14 +- .../streaming/WindowDoFnOperator.java | 2 +- .../streaming/WorkItemKeySelector.java | 3 +- .../streaming/io/UnboundedFlinkSink.java | 13 +- .../streaming/io/UnboundedFlinkSource.java | 29 ++-- .../streaming/io/UnboundedSocketSource.java | 46 +++-- .../wrappers/streaming/io/package-info.java | 22 +++ .../wrappers/streaming/package-info.java | 22 +++ .../runners/flink/PipelineOptionsTest.java | 3 + .../beam/runners/flink/WriteSinkITCase.java | 3 +- .../beam/runners/flink/package-info.java | 22 +++ .../streaming/FlinkStateInternalsTest.java | 3 +- .../flink/streaming/GroupByNullKeyTest.java | 6 + .../streaming/TopWikipediaSessionsITCase.java | 2 +- .../streaming/UnboundedSourceWrapperTest.java | 33 ++-- .../runners/flink/streaming/package-info.java | 22 +++ 39 files changed, 490 insertions(+), 189 deletions(-) create mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/package-info.java create mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/package-info.java create mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/package-info.java create mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/package-info.java create mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/package-info.java create mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/package-info.java create mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/package-info.java create mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/package-info.java create mode 100644 runners/flink/runner/src/test/java/org/apache/beam/runners/flink/package-info.java create mode 100644 runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/package-info.java diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml index 08adc607abf6..7c322804e5b9 100644 --- a/runners/flink/runner/pom.xml +++ b/runners/flink/runner/pom.xml @@ -234,12 +234,10 @@ - org.apache.maven.plugins diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java index d1977a445adf..a5d33b40b8f6 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java @@ -37,14 +37,15 @@ /** * The class that instantiates and manages the execution of a given job. * Depending on if the job is a Streaming or Batch processing one, it creates - * the adequate execution environment ({@link ExecutionEnvironment} or {@link StreamExecutionEnvironment}), - * the necessary {@link FlinkPipelineTranslator} ({@link FlinkBatchPipelineTranslator} or - * {@link FlinkStreamingPipelineTranslator}) to transform the Beam job into a Flink one, and - * executes the (translated) job. + * the adequate execution environment ({@link ExecutionEnvironment} + * or {@link StreamExecutionEnvironment}), the necessary {@link FlinkPipelineTranslator} + * ({@link FlinkBatchPipelineTranslator} or {@link FlinkStreamingPipelineTranslator}) to + * transform the Beam job into a Flink one, and executes the (translated) job. */ public class FlinkPipelineExecutionEnvironment { - private static final Logger LOG = LoggerFactory.getLogger(FlinkPipelineExecutionEnvironment.class); + private static final Logger LOG = + LoggerFactory.getLogger(FlinkPipelineExecutionEnvironment.class); private final FlinkPipelineOptions options; @@ -79,8 +80,8 @@ public class FlinkPipelineExecutionEnvironment { * Depending on if the job is a Streaming or a Batch one, this method creates * the necessary execution environment and pipeline translator, and translates * the {@link org.apache.beam.sdk.values.PCollection} program into - * a {@link org.apache.flink.api.java.DataSet} or {@link org.apache.flink.streaming.api.datastream.DataStream} - * one. + * a {@link org.apache.flink.api.java.DataSet} + * or {@link org.apache.flink.streaming.api.datastream.DataStream} one. * */ public void translate(Pipeline pipeline) { this.flinkBatchEnv = null; @@ -213,7 +214,7 @@ private StreamExecutionEnvironment createStreamExecutionEnvironment() { // If the value is not -1, then the validity checks are applied. // By default, checkpointing is disabled. long checkpointInterval = options.getCheckpointingInterval(); - if(checkpointInterval != -1) { + if (checkpointInterval != -1) { if (checkpointInterval < 1) { throw new IllegalArgumentException("The checkpoint interval must be positive"); } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java index 6561fa5ef996..1fb23ec6dcec 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java @@ -29,7 +29,8 @@ /** * Options which can be used to configure a Flink PipelineRunner. */ -public interface FlinkPipelineOptions extends PipelineOptions, ApplicationNameOptions, StreamingOptions { +public interface FlinkPipelineOptions + extends PipelineOptions, ApplicationNameOptions, StreamingOptions { /** * List of local files to make available to workers. @@ -38,8 +39,8 @@ public interface FlinkPipelineOptions extends PipelineOptions, ApplicationNameOp *

              * The default value is the list of jars from the main program's classpath. */ - @Description("Jar-Files to send to all workers and put on the classpath. " + - "The default value is all files from the classpath.") + @Description("Jar-Files to send to all workers and put on the classpath. " + + "The default value is all files from the classpath.") @JsonIgnore List getFilesToStage(); void setFilesToStage(List value); @@ -51,9 +52,9 @@ public interface FlinkPipelineOptions extends PipelineOptions, ApplicationNameOp * Cluster in the JVM, "[collection]" will execute the pipeline on Java Collections while * "[auto]" will let the system decide where to execute the pipeline based on the environment. */ - @Description("Address of the Flink Master where the Pipeline should be executed. Can" + - " either be of the form \"host:port\" or one of the special values [local], " + - "[collection] or [auto].") + @Description("Address of the Flink Master where the Pipeline should be executed. Can" + + " either be of the form \"host:port\" or one of the special values [local], " + + "[collection] or [auto].") String getFlinkMaster(); void setFlinkMaster(String value); @@ -62,21 +63,23 @@ public interface FlinkPipelineOptions extends PipelineOptions, ApplicationNameOp Integer getParallelism(); void setParallelism(Integer value); - @Description("The interval between consecutive checkpoints (i.e. snapshots of the current pipeline state used for " + - "fault tolerance).") + @Description("The interval between consecutive checkpoints (i.e. snapshots of the current" + + "pipeline state used for fault tolerance).") @Default.Long(-1L) Long getCheckpointingInterval(); void setCheckpointingInterval(Long interval); - @Description("Sets the number of times that failed tasks are re-executed. " + - "A value of zero effectively disables fault tolerance. A value of -1 indicates " + - "that the system default value (as defined in the configuration) should be used.") + @Description("Sets the number of times that failed tasks are re-executed. " + + "A value of zero effectively disables fault tolerance. A value of -1 indicates " + + "that the system default value (as defined in the configuration) should be used.") @Default.Integer(-1) Integer getNumberOfExecutionRetries(); void setNumberOfExecutionRetries(Integer retries); - @Description("Sets the delay between executions. A value of {@code -1} indicates that the default value should be used.") + @Description("Sets the delay between executions. A value of {@code -1} " + + "indicates that the default value should be used.") @Default.Long(-1L) Long getExecutionRetryDelay(); void setExecutionRetryDelay(Long delay); + } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java index f328279bfba0..0e4b5130be91 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java @@ -36,6 +36,9 @@ public class FlinkRunnerRegistrar { private FlinkRunnerRegistrar() { } + /** + * Pipeline runner registrar. + */ @AutoService(PipelineRunnerRegistrar.class) public static class Runner implements PipelineRunnerRegistrar { @Override @@ -46,6 +49,9 @@ public Iterable>> getPipelineRunners() { } } + /** + * Pipeline options registrar. + */ @AutoService(PipelineOptionsRegistrar.class) public static class Options implements PipelineOptionsRegistrar { @Override diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java index dd0733a671c9..90bb64d31046 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java @@ -35,9 +35,9 @@ public class FlinkRunnerResult implements PipelineResult { private final Map aggregators; private final long runtime; public FlinkRunnerResult(Map aggregators, long runtime) { - this.aggregators = (aggregators == null || aggregators.isEmpty()) ? - Collections.emptyMap() : - Collections.unmodifiableMap(aggregators); + this.aggregators = (aggregators == null || aggregators.isEmpty()) + ? Collections.emptyMap() + : Collections.unmodifiableMap(aggregators); this.runtime = runtime; } @@ -47,7 +47,8 @@ public State getState() { } @Override - public AggregatorValues getAggregatorValues(final Aggregator aggregator) throws AggregatorRetrievalException { + public AggregatorValues getAggregatorValues(final Aggregator aggregator) + throws AggregatorRetrievalException { // TODO provide a list of all accumulator step values Object value = aggregators.get(aggregator.getName()); if (value != null) { @@ -65,10 +66,10 @@ public Map getValuesAtSteps() { @Override public String toString() { - return "FlinkRunnerResult{" + - "aggregators=" + aggregators + - ", runtime=" + runtime + - '}'; + return "FlinkRunnerResult{" + + "aggregators=" + aggregators + + ", runtime=" + runtime + + '}'; } @Override diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java index dd231d6e6bdd..67a7d3891d3d 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java @@ -26,6 +26,9 @@ import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; +/** + * Test Flink runner. + */ public class TestFlinkRunner extends PipelineRunner { private FlinkRunner delegate; @@ -37,7 +40,8 @@ private TestFlinkRunner(FlinkPipelineOptions options) { } public static TestFlinkRunner fromOptions(PipelineOptions options) { - FlinkPipelineOptions flinkOptions = PipelineOptionsValidator.validate(FlinkPipelineOptions.class, options); + FlinkPipelineOptions flinkOptions = + PipelineOptionsValidator.validate(FlinkPipelineOptions.class, options); return new TestFlinkRunner(flinkOptions); } @@ -50,7 +54,7 @@ public static TestFlinkRunner create(boolean streaming) { @Override public - OutputT apply(PTransform transform, InputT input) { + OutputT apply(PTransform transform, InputT input) { return delegate.apply(transform, input); } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/package-info.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/package-info.java new file mode 100644 index 000000000000..57f1e599ee50 --- /dev/null +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/package-info.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Internal implementation of the Beam runner for Apache Flink. + */ +package org.apache.beam.runners.flink; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java index 66c48b07e23b..aa38bfbbf5d2 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java @@ -91,15 +91,20 @@ public void visitPrimitiveTransform(TransformTreeNode node) { // get the transformation corresponding to the node we are // currently visiting and translate it into its Flink alternative. PTransform transform = node.getTransform(); - BatchTransformTranslator translator = FlinkBatchTransformTranslators.getTranslator(transform); + BatchTransformTranslator translator = + FlinkBatchTransformTranslators.getTranslator(transform); if (translator == null) { LOG.info(node.getTransform().getClass().toString()); - throw new UnsupportedOperationException("The transform " + transform + " is currently not supported."); + throw new UnsupportedOperationException("The transform " + transform + + " is currently not supported."); } applyBatchTransform(transform, node, translator); } - private > void applyBatchTransform(PTransform transform, TransformTreeNode node, BatchTransformTranslator translator) { + private > void applyBatchTransform( + PTransform transform, + TransformTreeNode node, + BatchTransformTranslator translator) { @SuppressWarnings("unchecked") T typedTransform = (T) transform; @@ -116,8 +121,8 @@ public void visitPrimitiveTransform(TransformTreeNode node) { /** * A translator of a {@link PTransform}. */ - public interface BatchTransformTranslator { - void translateNode(Type transform, FlinkBatchTranslationContext context); + public interface BatchTransformTranslator { + void translateNode(TransformT transform, FlinkBatchTranslationContext context); } /** diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java index 835648e43ea8..611f5e6d9321 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java @@ -40,7 +40,7 @@ * {@link FlinkBatchTransformTranslators}. */ public class FlinkBatchTranslationContext { - + private final Map> dataSets; private final Map, DataSet> broadcastDataSets; @@ -55,9 +55,9 @@ public class FlinkBatchTranslationContext { private final PipelineOptions options; private AppliedPTransform currentTransform; - + // ------------------------------------------------------------------------ - + public FlinkBatchTranslationContext(ExecutionEnvironment env, PipelineOptions options) { this.env = env; this.options = options; @@ -66,7 +66,7 @@ public FlinkBatchTranslationContext(ExecutionEnvironment env, PipelineOptions op this.danglingDataSets = new HashMap<>(); } - + // ------------------------------------------------------------------------ public Map> getDanglingDataSets() { @@ -80,7 +80,7 @@ public ExecutionEnvironment getExecutionEnvironment() { public PipelineOptions getPipelineOptions() { return options; } - + @SuppressWarnings("unchecked") public DataSet> getInputDataSet(PValue value) { // assume that the DataSet is used as an input if retrieved here diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkPipelineTranslator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkPipelineTranslator.java index 4db929b0641b..cba28e48e01b 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkPipelineTranslator.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkPipelineTranslator.java @@ -39,7 +39,7 @@ public void translate(Pipeline pipeline) { } /** - * Utility formatting method + * Utility formatting method. * @param n number of spaces to generate * @return String with "|" followed by n spaces */ diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java index 3719fa8e5d4c..4b819b78f00a 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java @@ -18,9 +18,10 @@ package org.apache.beam.runners.flink.translation; -import com.google.api.client.util.Maps; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; + import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collection; @@ -29,6 +30,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import org.apache.beam.runners.core.SystemReduceFn; import org.apache.beam.runners.flink.FlinkRunner; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; import org.apache.beam.runners.flink.translation.types.FlinkCoder; @@ -63,7 +65,6 @@ import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.AppliedCombineFn; import org.apache.beam.sdk.util.Reshuffle; -import org.apache.beam.runners.core.SystemReduceFn; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/TranslationMode.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/TranslationMode.java index 71eb6552b710..57b69aa14753 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/TranslationMode.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/TranslationMode.java @@ -18,14 +18,14 @@ package org.apache.beam.runners.flink.translation; /** - * The translation mode of the Beam Pipeline + * The translation mode of the Beam Pipeline. */ public enum TranslationMode { - /** Uses the batch mode of Flink */ + /** Uses the batch mode of Flink. */ BATCH, - /** Uses the streaming mode of Flink */ + /** Uses the streaming mode of Flink. */ STREAMING -} \ No newline at end of file +} diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/package-info.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/package-info.java new file mode 100644 index 000000000000..9f1121225b88 --- /dev/null +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/package-info.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Internal implementation of the Beam runner for Apache Flink. + */ +package org.apache.beam.runners.flink.translation.functions; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/package-info.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/package-info.java new file mode 100644 index 000000000000..af4b35491ba4 --- /dev/null +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/package-info.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Internal implementation of the Beam runner for Apache Flink. + */ +package org.apache.beam.runners.flink.translation; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java index 4eda357f64eb..e210ed9d7b98 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java @@ -33,7 +33,7 @@ * Dataflow {@link org.apache.beam.sdk.coders.Coder Coders}. */ public class CoderTypeSerializer extends TypeSerializer { - + private Coder coder; public CoderTypeSerializer(Coder coder) { diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueSerializer.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueSerializer.java index f3e667d575a5..41db61edbb49 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueSerializer.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueSerializer.java @@ -18,7 +18,9 @@ package org.apache.beam.runners.flink.translation.types; import java.io.IOException; + import org.apache.beam.sdk.coders.Coder; + import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; @@ -28,84 +30,84 @@ */ public final class EncodedValueSerializer extends TypeSerializer { - private static final long serialVersionUID = 1L; - - private static final byte[] EMPTY = new byte[0]; - - @Override - public boolean isImmutableType() { - return true; - } - - @Override - public byte[] createInstance() { - return EMPTY; - } - - @Override - public byte[] copy(byte[] from) { - return from; - } - - @Override - public byte[] copy(byte[] from, byte[] reuse) { - return copy(from); - } - - @Override - public int getLength() { - return -1; - } - - - @Override - public void serialize(byte[] record, DataOutputView target) throws IOException { - if (record == null) { - throw new IllegalArgumentException("The record must not be null."); - } - - final int len = record.length; - target.writeInt(len); - target.write(record); - } - - @Override - public byte[] deserialize(DataInputView source) throws IOException { - final int len = source.readInt(); - byte[] result = new byte[len]; - source.readFully(result); - return result; - } - - @Override - public byte[] deserialize(byte[] reuse, DataInputView source) throws IOException { - return deserialize(source); - } - - @Override - public void copy(DataInputView source, DataOutputView target) throws IOException { - final int len = source.readInt(); - target.writeInt(len); - target.write(source, len); - } - - @Override - public boolean canEqual(Object obj) { - return obj instanceof EncodedValueSerializer; - } - - @Override - public int hashCode() { - return this.getClass().hashCode(); - } - - @Override - public boolean equals(Object obj) { - return obj instanceof EncodedValueSerializer; - } - - @Override - public TypeSerializer duplicate() { - return this; - } + private static final long serialVersionUID = 1L; + + private static final byte[] EMPTY = new byte[0]; + + @Override + public boolean isImmutableType() { + return true; + } + + @Override + public byte[] createInstance() { + return EMPTY; + } + + @Override + public byte[] copy(byte[] from) { + return from; + } + + @Override + public byte[] copy(byte[] from, byte[] reuse) { + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + + @Override + public void serialize(byte[] record, DataOutputView target) throws IOException { + if (record == null) { + throw new IllegalArgumentException("The record must not be null."); + } + + final int len = record.length; + target.writeInt(len); + target.write(record); + } + + @Override + public byte[] deserialize(DataInputView source) throws IOException { + final int len = source.readInt(); + byte[] result = new byte[len]; + source.readFully(result); + return result; + } + + @Override + public byte[] deserialize(byte[] reuse, DataInputView source) throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + final int len = source.readInt(); + target.writeInt(len); + target.write(source, len); + } + + @Override + public boolean canEqual(Object obj) { + return obj instanceof EncodedValueSerializer; + } + + @Override + public int hashCode() { + return this.getClass().hashCode(); + } + + @Override + public boolean equals(Object obj) { + return obj instanceof EncodedValueSerializer; + } + + @Override + public TypeSerializer duplicate() { + return this; + } } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/package-info.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/package-info.java new file mode 100644 index 000000000000..6fb3182f27c5 --- /dev/null +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/package-info.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Internal implementation of the Beam runner for Apache Flink. + */ +package org.apache.beam.runners.flink.translation.types; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java index 0c6cea8dfd29..fe2602b3377b 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java @@ -33,7 +33,7 @@ public class SerializedPipelineOptions implements Serializable { private final byte[] serializedOptions; - /** Lazily initialized copy of deserialized options */ + /** Lazily initialized copy of deserialized options. */ private transient PipelineOptions pipelineOptions; public SerializedPipelineOptions(PipelineOptions options) { diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/package-info.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/package-info.java new file mode 100644 index 000000000000..5dedd53c7752 --- /dev/null +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/package-info.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Internal implementation of the Beam runner for Apache Flink. + */ +package org.apache.beam.runners.flink.translation.utils; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java index 2cb9b188ff06..f2d9db222c56 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java @@ -28,7 +28,7 @@ * {@link java.io.OutputStream}. */ public class DataOutputViewWrapper extends OutputStream { - + private DataOutputView outputView; public DataOutputViewWrapper(DataOutputView outputView) { diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/package-info.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/package-info.java new file mode 100644 index 000000000000..72f7debb434f --- /dev/null +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/package-info.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Internal implementation of the Beam runner for Apache Flink. + */ +package org.apache.beam.runners.flink.translation.wrappers; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java index 3b0fccc2b365..3b917e2f4bdd 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java @@ -27,6 +27,9 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import org.apache.beam.runners.core.DoFnRunner; +import org.apache.beam.runners.core.DoFnRunners; +import org.apache.beam.runners.core.PushbackSideInputDoFnRunner; import org.apache.beam.runners.core.SideInputHandler; import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; import org.apache.beam.runners.flink.translation.wrappers.SerializableFnAggregatorWrapper; @@ -40,11 +43,8 @@ import org.apache.beam.sdk.transforms.join.RawUnionValue; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.runners.core.DoFnRunner; -import org.apache.beam.runners.core.DoFnRunners; import org.apache.beam.sdk.util.ExecutionContext; import org.apache.beam.sdk.util.NullSideInputReader; -import org.apache.beam.runners.core.PushbackSideInputDoFnRunner; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.WindowedValue; @@ -79,8 +79,8 @@ * * @param the input type of the {@link OldDoFn} * @param the output type of the {@link OldDoFn} - * @param the output type of the operator, this can be different from the fn output type when we have - * side outputs + * @param the output type of the operator, this can be different from the fn output + * type when we have side outputs */ public class DoFnOperator extends AbstractStreamOperator @@ -166,7 +166,7 @@ public void open() throws Exception { currentInputWatermark = Long.MIN_VALUE; currentOutputWatermark = currentInputWatermark; - Aggregator.AggregatorFactory aggregatorFactory = new Aggregator.AggregatorFactory() { + Aggregator.AggregatorFactory aggregatorFactory = new Aggregator.AggregatorFactory() { @Override public Aggregator createAggregatorForDoFn( Class fnClass, diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItem.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItem.java index 5751aac78123..6d2582b5da5b 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItem.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItem.java @@ -22,6 +22,11 @@ import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.WindowedValue; +/** + * Singleton keyed word item. + * @param + * @param + */ public class SingletonKeyedWorkItem implements KeyedWorkItem { final K key; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java index 5e583e9013c2..37454a31c4bf 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java @@ -35,7 +35,13 @@ import org.apache.beam.sdk.util.PropertyNames; import org.apache.beam.sdk.util.WindowedValue; -public class SingletonKeyedWorkItemCoder extends StandardCoder> { +/** + * Singleton keyed word iteam coder. + * @param + * @param + */ +public class SingletonKeyedWorkItemCoder + extends StandardCoder> { /** * Create a new {@link KeyedWorkItemCoder} with the provided key coder, element coder, and window * coder. @@ -68,7 +74,7 @@ private SingletonKeyedWorkItemCoder( this.keyCoder = keyCoder; this.elemCoder = elemCoder; this.windowCoder = windowCoder; - valueCoder= WindowedValue.FullWindowedValueCoder.of(elemCoder, windowCoder); + valueCoder = WindowedValue.FullWindowedValueCoder.of(elemCoder, windowCoder); } public Coder getKeyCoder() { @@ -80,7 +86,9 @@ public Coder getElementCoder() { } @Override - public void encode(SingletonKeyedWorkItem value, OutputStream outStream, Context context) + public void encode(SingletonKeyedWorkItem value, + OutputStream outStream, + Context context) throws CoderException, IOException { Context nestedContext = context.nested(); keyCoder.encode(value.key(), outStream, nestedContext); diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java index b8931161045e..29ae6ae63f8a 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java @@ -34,6 +34,7 @@ import java.util.Set; import javax.annotation.Nullable; import org.apache.beam.runners.core.GroupAlsoByWindowViaWindowSetDoFn; +import org.apache.beam.runners.core.SystemReduceFn; import org.apache.beam.runners.flink.translation.wrappers.DataInputViewWrapper; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; @@ -43,7 +44,6 @@ import org.apache.beam.sdk.util.ExecutionContext; import org.apache.beam.sdk.util.KeyedWorkItem; import org.apache.beam.sdk.util.KeyedWorkItems; -import org.apache.beam.runners.core.SystemReduceFn; import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.WindowedValue; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WorkItemKeySelector.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WorkItemKeySelector.java index 51d9e0c9399c..7829163e6a26 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WorkItemKeySelector.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WorkItemKeySelector.java @@ -33,7 +33,8 @@ * that all key comparisons/hashing happen on the encoded form. */ public class WorkItemKeySelector - implements KeySelector>, ByteBuffer>, ResultTypeQueryable { + implements KeySelector>, ByteBuffer>, + ResultTypeQueryable { private final Coder keyCoder; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSink.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSink.java index 2117e9d19353..5b01796fe1f9 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSink.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSink.java @@ -62,7 +62,8 @@ public void initialize(PipelineOptions options) throws Exception { } @Override - public void finalize(Iterable writerResults, PipelineOptions options) throws Exception { + public void finalize(Iterable writerResults, PipelineOptions options) + throws Exception { } @@ -70,12 +71,14 @@ public void finalize(Iterable writerResults, PipelineOptions options) th public Coder getWriterResultCoder() { return new Coder() { @Override - public void encode(Object value, OutputStream outStream, Context context) throws CoderException, IOException { + public void encode(Object value, OutputStream outStream, Context context) + throws CoderException, IOException { } @Override - public Object decode(InputStream inStream, Context context) throws CoderException, IOException { + public Object decode(InputStream inStream, Context context) + throws CoderException, IOException { return null; } @@ -110,7 +113,9 @@ public boolean isRegisterByteSizeObserverCheap(Object value, Context context) { } @Override - public void registerByteSizeObserver(Object value, ElementByteSizeObserver observer, Context context) throws Exception { + public void registerByteSizeObserver(Object value, + ElementByteSizeObserver observer, + Context context) throws Exception { } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSource.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSource.java index c6e0825648f4..ac20c34ff204 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSource.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSource.java @@ -36,17 +36,19 @@ public class UnboundedFlinkSource extends UnboundedSource flinkSource; - /** Coder set during translation */ + /** Coder set during translation. */ private Coder coder; - /** Timestamp / watermark assigner for source; defaults to ingestion time */ - private AssignerWithPeriodicWatermarks flinkTimestampAssigner = new IngestionTimeExtractor(); + /** Timestamp / watermark assigner for source; defaults to ingestion time. */ + private AssignerWithPeriodicWatermarks flinkTimestampAssigner = + new IngestionTimeExtractor(); public UnboundedFlinkSource(SourceFunction source) { flinkSource = checkNotNull(source); } - public UnboundedFlinkSource(SourceFunction source, AssignerWithPeriodicWatermarks timestampAssigner) { + public UnboundedFlinkSource(SourceFunction source, + AssignerWithPeriodicWatermarks timestampAssigner) { flinkSource = checkNotNull(source); flinkTimestampAssigner = checkNotNull(timestampAssigner); } @@ -60,19 +62,25 @@ public AssignerWithPeriodicWatermarks getFlinkTimestampAssigner() { } @Override - public List> generateInitialSplits(int desiredNumSplits, PipelineOptions options) throws Exception { - throw new RuntimeException("Flink Sources are supported only when running with the FlinkRunner."); + public List> generateInitialSplits( + int desiredNumSplits, + PipelineOptions options) throws Exception { + throw new RuntimeException("Flink Sources are supported only when " + + "running with the FlinkRunner."); } @Override - public UnboundedReader createReader(PipelineOptions options, @Nullable CheckpointMark checkpointMark) { - throw new RuntimeException("Flink Sources are supported only when running with the FlinkRunner."); + public UnboundedReader createReader(PipelineOptions options, + @Nullable CheckpointMark checkpointMark) { + throw new RuntimeException("Flink Sources are supported only when " + + "running with the FlinkRunner."); } @Nullable @Override public Coder getCheckpointMarkCoder() { - throw new RuntimeException("Flink Sources are supported only when running with the FlinkRunner."); + throw new RuntimeException("Flink Sources are supported only when " + + "running with the FlinkRunner."); } @@ -100,7 +108,8 @@ public void setFlinkTimestampAssigner(AssignerWithPeriodicWatermarks flinkTim * @param The type that the source function produces. * @return The wrapped source function. */ - public static UnboundedSource of(SourceFunction flinkSource) { + public static UnboundedSource of( + SourceFunction flinkSource) { return new UnboundedFlinkSource<>(flinkSource); } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java index 8d37fe70dfb8..96b51389deaf 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java @@ -38,9 +38,11 @@ import org.slf4j.LoggerFactory; /** - * An example unbounded Beam source that reads input from a socket. This is used mainly for testing and debugging. + * An example unbounded Beam source that reads input from a socket. + * This is used mainly for testing and debugging. * */ -public class UnboundedSocketSource extends UnboundedSource { +public class UnboundedSocketSource + extends UnboundedSource { private static final Coder DEFAULT_SOCKET_CODER = StringUtf8Coder.of(); @@ -60,7 +62,11 @@ public UnboundedSocketSource(String hostname, int port, char delimiter, long max this(hostname, port, delimiter, maxNumRetries, DEFAULT_CONNECTION_RETRY_SLEEP); } - public UnboundedSocketSource(String hostname, int port, char delimiter, long maxNumRetries, long delayBetweenRetries) { + public UnboundedSocketSource(String hostname, + int port, + char delimiter, + long maxNumRetries, + long delayBetweenRetries) { this.hostname = hostname; this.port = port; this.delimiter = delimiter; @@ -89,12 +95,15 @@ public long getDelayBetweenRetries() { } @Override - public List> generateInitialSplits(int desiredNumSplits, PipelineOptions options) throws Exception { - return Collections.>singletonList(this); + public List> generateInitialSplits( + int desiredNumSplits, + PipelineOptions options) throws Exception { + return Collections.>singletonList(this); } @Override - public UnboundedReader createReader(PipelineOptions options, @Nullable C checkpointMark) { + public UnboundedReader createReader(PipelineOptions options, + @Nullable CheckpointMarkT checkpointMark) { return new UnboundedSocketReader(this); } @@ -109,7 +118,8 @@ public Coder getCheckpointMarkCoder() { @Override public void validate() { checkArgument(port > 0 && port < 65536, "port is out of range"); - checkArgument(maxNumRetries >= -1, "maxNumRetries must be zero or larger (num retries), or -1 (infinite retries)"); + checkArgument(maxNumRetries >= -1, "maxNumRetries must be zero or larger (num retries), " + + "or -1 (infinite retries)"); checkArgument(delayBetweenRetries >= 0, "delayBetweenRetries must be zero or positive"); } @@ -118,7 +128,11 @@ public Coder getDefaultOutputCoder() { return DEFAULT_SOCKET_CODER; } - public static class UnboundedSocketReader extends UnboundedSource.UnboundedReader implements Serializable { + /** + * Unbounded socket reader. + */ + public static class UnboundedSocketReader extends UnboundedSource.UnboundedReader + implements Serializable { private static final long serialVersionUID = 7526472295622776147L; private static final Logger LOG = LoggerFactory.getLogger(UnboundedSocketReader.class); @@ -138,7 +152,8 @@ public UnboundedSocketReader(UnboundedSocketSource source) { private void openConnection() throws IOException { this.socket = new Socket(); - this.socket.connect(new InetSocketAddress(this.source.getHostname(), this.source.getPort()), CONNECTION_TIMEOUT_TIME); + this.socket.connect(new InetSocketAddress(this.source.getHostname(), this.source.getPort()), + CONNECTION_TIMEOUT_TIME); this.reader = new BufferedReader(new InputStreamReader(this.socket.getInputStream())); this.isRunning = true; } @@ -149,11 +164,14 @@ public boolean start() throws IOException { while (!isRunning) { try { openConnection(); - LOG.info("Connected to server socket " + this.source.getHostname() + ':' + this.source.getPort()); + LOG.info("Connected to server socket " + this.source.getHostname() + ':' + + this.source.getPort()); return advance(); } catch (IOException e) { - LOG.info("Lost connection to server socket " + this.source.getHostname() + ':' + this.source.getPort() + ". Retrying in " + this.source.getDelayBetweenRetries() + " msecs..."); + LOG.info("Lost connection to server socket " + this.source.getHostname() + ':' + + this.source.getPort() + ". Retrying in " + + this.source.getDelayBetweenRetries() + " msecs..."); if (this.source.getMaxNumRetries() == -1 || attempt++ < this.source.getMaxNumRetries()) { try { @@ -167,7 +185,8 @@ public boolean start() throws IOException { } } } - LOG.error("Unable to connect to host " + this.source.getHostname() + " : " + this.source.getPort()); + LOG.error("Unable to connect to host " + this.source.getHostname() + + " : " + this.source.getPort()); return false; } @@ -211,7 +230,8 @@ public void close() throws IOException { this.reader.close(); this.socket.close(); this.isRunning = false; - LOG.info("Closed connection to server socket at " + this.source.getHostname() + ":" + this.source.getPort() + "."); + LOG.info("Closed connection to server socket at " + this.source.getHostname() + ":" + + this.source.getPort() + "."); } @Override diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/package-info.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/package-info.java new file mode 100644 index 000000000000..b431ce745bde --- /dev/null +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/package-info.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Internal implementation of the Beam runner for Apache Flink. + */ +package org.apache.beam.runners.flink.translation.wrappers.streaming.io; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/package-info.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/package-info.java new file mode 100644 index 000000000000..0674871307e6 --- /dev/null +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/package-info.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Internal implementation of the Beam runner for Apache Flink. + */ +package org.apache.beam.runners.flink.translation.wrappers.streaming; diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java index 32339dce5165..3c30fed923ba 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java @@ -52,6 +52,9 @@ */ public class PipelineOptionsTest { + /** + * Pipeline options. + */ public interface MyOptions extends FlinkPipelineOptions { @Description("Bla bla bla") @Default.String("Hello") diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java index 09881463f6c2..37eedb220bac 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java @@ -118,7 +118,8 @@ public void initialize(PipelineOptions options) throws Exception { } @Override - public void finalize(Iterable writerResults, PipelineOptions options) throws Exception { + public void finalize(Iterable writerResults, PipelineOptions options) + throws Exception { } diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/package-info.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/package-info.java new file mode 100644 index 000000000000..57f1e599ee50 --- /dev/null +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/package-info.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Internal implementation of the Beam runner for Apache Flink. + */ +package org.apache.beam.runners.flink; diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java index 711ae0001f79..628212a4e5f1 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java @@ -56,7 +56,8 @@ import org.junit.runners.JUnit4; /** - * Tests for {@link FlinkStateInternals}. This is based on the tests for {@code InMemoryStateInternals}. + * Tests for {@link FlinkStateInternals}. This is based on the tests for + * {@code InMemoryStateInternals}. */ @RunWith(JUnit4.class) public class FlinkStateInternalsTest { diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java index ab98c27733b3..c6381ee4002e 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java @@ -36,6 +36,9 @@ import org.joda.time.Duration; import org.joda.time.Instant; +/** + * Test for GroupByNullKey. + */ public class GroupByNullKeyTest extends StreamingProgramTestBase implements Serializable { @@ -58,6 +61,9 @@ protected void postSubmit() throws Exception { compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath); } + /** + * DoFn extracting user and timestamp. + */ public static class ExtractUserAndTimestamp extends OldDoFn, String> { private static final long serialVersionUID = 0; diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java index 64f978fa0f23..9410481f054e 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java @@ -38,7 +38,7 @@ /** - * Session window test + * Session window test. */ public class TopWikipediaSessionsITCase extends StreamingProgramTestBase implements Serializable { protected String resultPath; diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceWrapperTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceWrapperTest.java index a70ad49b9f73..73124a9eed72 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceWrapperTest.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceWrapperTest.java @@ -56,14 +56,14 @@ public class UnboundedSourceWrapperTest { */ @Test public void testWithOneReader() throws Exception { - final int NUM_ELEMENTS = 20; + final int numElements = 20; final Object checkpointLock = new Object(); PipelineOptions options = PipelineOptionsFactory.create(); // this source will emit exactly NUM_ELEMENTS across all parallel readers, // afterwards it will stall. We check whether we also receive NUM_ELEMENTS // elements later. - TestCountingSource source = new TestCountingSource(NUM_ELEMENTS); + TestCountingSource source = new TestCountingSource(numElements); UnboundedSourceWrapper, TestCountingSource.CounterMark> flinkWrapper = new UnboundedSourceWrapper<>(options, source, 1); @@ -92,7 +92,7 @@ public void collect( StreamRecord>> windowedValueStreamRecord) { count++; - if (count >= NUM_ELEMENTS) { + if (count >= numElements) { throw new SuccessException(); } } @@ -116,14 +116,14 @@ public void close() { */ @Test public void testWithMultipleReaders() throws Exception { - final int NUM_ELEMENTS = 20; + final int numElements = 20; final Object checkpointLock = new Object(); PipelineOptions options = PipelineOptionsFactory.create(); // this source will emit exactly NUM_ELEMENTS across all parallel readers, // afterwards it will stall. We check whether we also receive NUM_ELEMENTS // elements later. - TestCountingSource source = new TestCountingSource(NUM_ELEMENTS); + TestCountingSource source = new TestCountingSource(numElements); UnboundedSourceWrapper, TestCountingSource.CounterMark> flinkWrapper = new UnboundedSourceWrapper<>(options, source, 4); @@ -149,10 +149,10 @@ public void emitWatermark(Watermark watermark) { @Override public void collect( - StreamRecord>> windowedValueStreamRecord) { + StreamRecord>> windowedValueStreamRecord) { count++; - if (count >= NUM_ELEMENTS) { + if (count >= numElements) { throw new SuccessException(); } } @@ -177,14 +177,14 @@ public void close() { */ @Test public void testRestore() throws Exception { - final int NUM_ELEMENTS = 20; + final int numElements = 20; final Object checkpointLock = new Object(); PipelineOptions options = PipelineOptionsFactory.create(); // this source will emit exactly NUM_ELEMENTS across all parallel readers, // afterwards it will stall. We check whether we also receive NUM_ELEMENTS // elements later. - TestCountingSource source = new TestCountingSource(NUM_ELEMENTS); + TestCountingSource source = new TestCountingSource(numElements); UnboundedSourceWrapper, TestCountingSource.CounterMark> flinkWrapper = new UnboundedSourceWrapper<>(options, source, 1); @@ -213,11 +213,11 @@ public void emitWatermark(Watermark watermark) { @Override public void collect( - StreamRecord>> windowedValueStreamRecord) { + StreamRecord>> windowedValueStreamRecord) { emittedElements.add(windowedValueStreamRecord.getValue().getValue()); count++; - if (count >= NUM_ELEMENTS / 2) { + if (count >= numElements / 2) { throw new SuccessException(); } } @@ -238,7 +238,7 @@ public void close() { byte[] snapshot = flinkWrapper.snapshotState(0, 0); // create a completely new source but restore from the snapshot - TestCountingSource restoredSource = new TestCountingSource(NUM_ELEMENTS); + TestCountingSource restoredSource = new TestCountingSource(numElements); UnboundedSourceWrapper< KV, TestCountingSource.CounterMark> restoredFlinkWrapper = new UnboundedSourceWrapper<>(options, restoredSource, 1); @@ -271,10 +271,10 @@ public void emitWatermark(Watermark watermark) { @Override public void collect( - StreamRecord>> windowedValueStreamRecord) { + StreamRecord>> windowedValueStreamRecord) { emittedElements.add(windowedValueStreamRecord.getValue().getValue()); count++; - if (count >= NUM_ELEMENTS / 2) { + if (count >= numElements / 2) { throw new SuccessException(); } } @@ -292,7 +292,7 @@ public void close() { assertTrue("Did not successfully read second batch of elements.", readSecondBatchOfElements); // verify that we saw all NUM_ELEMENTS elements - assertTrue(emittedElements.size() == NUM_ELEMENTS); + assertTrue(emittedElements.size() == numElements); } @SuppressWarnings("unchecked") @@ -310,7 +310,8 @@ private static void setupSourceOperator(StreamSource operator) { when(mockTask.getConfiguration()).thenReturn(cfg); when(mockTask.getEnvironment()).thenReturn(env); when(mockTask.getExecutionConfig()).thenReturn(executionConfig); - when(mockTask.getAccumulatorMap()).thenReturn(Collections.>emptyMap()); + when(mockTask.getAccumulatorMap()) + .thenReturn(Collections.>emptyMap()); operator.setup(mockTask, cfg, (Output< StreamRecord>) mock(Output.class)); } diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/package-info.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/package-info.java new file mode 100644 index 000000000000..08a1e03ec243 --- /dev/null +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/package-info.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Internal implementation of the Beam runner for Apache Flink. + */ +package org.apache.beam.runners.flink.streaming; From 8d306bbf5481ff1a5d27a2ae8d73e710532154a8 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Sun, 4 Sep 2016 14:54:42 -0700 Subject: [PATCH 101/112] BigQuery: limit max job polling time to 1 minute Before the backoff would grow unboundedly, so we could in principle wait 1.5x to 2x the actual job time. For long running jobs this is hours. Now, we just back off at most 1 minute between checking the job state. Note there should be no danger of QPS overload here because we should have very few concurrent outstanding jobs --- .../beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java index 8b5e8c28aeb2..20dadff23f9d 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java @@ -232,11 +232,12 @@ static void startJob( } @Override - public Job pollJob(JobReference jobRef, int maxAttempts) - throws InterruptedException { + public Job pollJob(JobReference jobRef, int maxAttempts) throws InterruptedException { BackOff backoff = FluentBackoff.DEFAULT - .withMaxRetries(maxAttempts).withInitialBackoff(INITIAL_JOB_STATUS_POLL_BACKOFF) + .withMaxRetries(maxAttempts) + .withInitialBackoff(INITIAL_JOB_STATUS_POLL_BACKOFF) + .withMaxBackoff(Duration.standardMinutes(1)) .backoff(); return pollJob(jobRef, Sleeper.DEFAULT, backoff); } From ae897c063bab07dcfba08ce164898688b257b674 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Fri, 2 Sep 2016 10:43:43 -0700 Subject: [PATCH 102/112] Be more accepting in UnboundedReadDeduplicatorTest Don't depend on all the threads failing. Instead, assert that at most one success was encountered, and we saw at most numThreads - 1 failures. --- .../runners/direct/UnboundedReadDeduplicatorTest.java | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadDeduplicatorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadDeduplicatorTest.java index 7d2a95cd167c..0aa2c493a821 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadDeduplicatorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadDeduplicatorTest.java @@ -19,6 +19,7 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.lessThan; import static org.junit.Assert.assertThat; import java.util.concurrent.CountDownLatch; @@ -60,18 +61,18 @@ public void cachedIdDeduplicatorMultithreaded() throws InterruptedException { byte[] id = new byte[] {-1, 2, 4, 22}; UnboundedReadDeduplicator dedupper = CachedIdDeduplicator.create(); final CountDownLatch startSignal = new CountDownLatch(1); - int numThreads = 1000; + int numThreads = 50; final CountDownLatch readyLatch = new CountDownLatch(numThreads); final CountDownLatch finishLine = new CountDownLatch(numThreads); ExecutorService executor = Executors.newCachedThreadPool(); AtomicInteger successCount = new AtomicInteger(); - AtomicInteger failureCount = new AtomicInteger(); + AtomicInteger noOutputCount = new AtomicInteger(); for (int i = 0; i < numThreads; i++) { executor.submit(new TryOutputIdRunnable(dedupper, id, successCount, - failureCount, + noOutputCount, readyLatch, startSignal, finishLine)); @@ -82,8 +83,10 @@ public void cachedIdDeduplicatorMultithreaded() throws InterruptedException { finishLine.await(10L, TimeUnit.SECONDS); executor.shutdownNow(); + // The first thread to run will succeed, and no others will assertThat(successCount.get(), equalTo(1)); - assertThat(failureCount.get(), equalTo(numThreads - 1)); + // The threads may not all complete; all of the threads that do not succeed must not output + assertThat(noOutputCount.get(), lessThan(numThreads)); } private static class TryOutputIdRunnable implements Runnable { From 88a6cf5e3793c9ecff664119be88349f94a37694 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Tue, 6 Sep 2016 11:13:07 -0700 Subject: [PATCH 103/112] Remove timeout in JAXBCoderTest The infrastructure should kill the test if the await does not return. --- .../org/apache/beam/sdk/coders/JAXBCoderTest.java | 13 +++++-------- 1 file changed, 5 insertions(+), 8 deletions(-) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/JAXBCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/JAXBCoderTest.java index 129be61ae7c1..36190f9523c6 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/JAXBCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/JAXBCoderTest.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.coders; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; import com.google.common.collect.ImmutableList; import java.io.IOException; @@ -28,7 +27,6 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executor; import java.util.concurrent.Executors; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import javax.xml.bind.annotation.XmlRootElement; import org.apache.beam.sdk.testing.CoderProperties; @@ -119,7 +117,7 @@ public void testEncodeDecodeNested() throws Exception { @Test public void testEncodeDecodeMultithreaded() throws Throwable { final JAXBCoder coder = JAXBCoder.of(TestType.class); - int numThreads = 1000; + int numThreads = 100; final CountDownLatch ready = new CountDownLatch(numThreads); final CountDownLatch start = new CountDownLatch(1); @@ -155,11 +153,10 @@ public void run() { ready.await(); start.countDown(); - if (!done.await(10L, TimeUnit.SECONDS)) { - fail("Should be able to clone " + numThreads + " elements in 10 seconds"); - } - if (thrown.get() != null) { - throw thrown.get(); + done.await(); + Throwable actuallyThrown = thrown.get(); + if (actuallyThrown != null) { + throw actuallyThrown; } } From dafb80556c1d984630c6ccf615ba982903f176df Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Tue, 6 Sep 2016 07:26:45 +0200 Subject: [PATCH 104/112] [BEAM-242] Enable and fix checkstyle in Flink runner examples --- runners/flink/examples/pom.xml | 2 -- .../runners/flink/examples/WordCount.java | 9 +++++ .../runners/flink/examples/package-info.java | 22 ++++++++++++ .../examples/streaming/AutoComplete.java | 5 +-- .../examples/streaming/JoinExamples.java | 3 +- .../examples/streaming/KafkaIOExamples.java | 34 +++++++++---------- .../KafkaWindowedWordCountExample.java | 27 ++++++++++++--- .../examples/streaming/WindowedWordCount.java | 19 +++++++---- .../examples/streaming/package-info.java | 22 ++++++++++++ 9 files changed, 110 insertions(+), 33 deletions(-) create mode 100644 runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/package-info.java create mode 100644 runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/package-info.java diff --git a/runners/flink/examples/pom.xml b/runners/flink/examples/pom.xml index 9f705db88dbe..b8489fc23871 100644 --- a/runners/flink/examples/pom.xml +++ b/runners/flink/examples/pom.xml @@ -109,12 +109,10 @@ - org.apache.maven.plugins diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java index ab9297f72d8e..9cce75798076 100644 --- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java +++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java @@ -36,8 +36,14 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +/** + * Wordcount pipeline. + */ public class WordCount { + /** + * Function to extract words. + */ public static class ExtractWordsFn extends DoFn { private final Aggregator emptyLines = createAggregator("emptyLines", new Sum.SumLongFn()); @@ -60,6 +66,9 @@ public void processElement(ProcessContext c) { } } + /** + * PTransform counting words. + */ public static class CountWords extends PTransform, PCollection>> { @Override diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/package-info.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/package-info.java new file mode 100644 index 000000000000..b0ecb5646605 --- /dev/null +++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/package-info.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Flink Beam runner exemple. + */ +package org.apache.beam.runners.flink.examples; diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/AutoComplete.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/AutoComplete.java index 9b5e31d08bbe..4636e3f3149f 100644 --- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/AutoComplete.java +++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/AutoComplete.java @@ -96,7 +96,8 @@ public PCollection>> apply(PCollection> elem = c.element(); - str.append(elem.getKey() +" @ "+ window +" -> "); + str.append(elem.getKey() + " @ " + window + " -> "); for (CompletionCandidate cand: elem.getValue()) { str.append(cand.toString() + " "); } diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/JoinExamples.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/JoinExamples.java index bf5dfc453e19..96638aab1db6 100644 --- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/JoinExamples.java +++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/JoinExamples.java @@ -132,7 +132,8 @@ public static void main(String[] args) throws Exception { options.setExecutionRetryDelay(3000L); options.setRunner(FlinkRunner.class); - WindowFn windowFn = FixedWindows.of(Duration.standardSeconds(options.getWindowSize())); + WindowFn windowFn = FixedWindows.of( + Duration.standardSeconds(options.getWindowSize())); Pipeline p = Pipeline.create(options); diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaIOExamples.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaIOExamples.java index 27faefe6d745..f0bf188e3228 100644 --- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaIOExamples.java +++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaIOExamples.java @@ -56,15 +56,15 @@ public class KafkaIOExamples { private static final String KAFKA_AVRO_TOPIC = "output"; // Default kafka topic to read from private static final String KAFKA_BROKER = "localhost:9092"; // Default kafka broker to contact private static final String GROUP_ID = "myGroup"; // Default groupId - private static final String ZOOKEEPER = "localhost:2181"; // Default zookeeper to connect to for Kafka + private static final String ZOOKEEPER = "localhost:2181"; // Default zookeeper to connect (Kafka) /** - * Read/Write String data to Kafka + * Read/Write String data to Kafka. */ public static class KafkaString { /** - * Read String data from Kafka + * Read String data from Kafka. */ public static class ReadStringFromKafka { @@ -88,7 +88,7 @@ public static void main(String[] args) { } /** - * Write String data to Kafka + * Write String data to Kafka. */ public static class WriteStringToKafka { @@ -113,12 +113,12 @@ public static void main(String[] args) { } /** - * Read/Write Avro data to Kafka + * Read/Write Avro data to Kafka. */ public static class KafkaAvro { /** - * Read Avro data from Kafka + * Read Avro data from Kafka. */ public static class ReadAvroFromKafka { @@ -142,7 +142,7 @@ public static void main(String[] args) { } /** - * Write Avro data to Kafka + * Write Avro data to Kafka. */ public static class WriteAvroToKafka { @@ -169,7 +169,7 @@ public static void main(String[] args) { } /** - * Serialiation/Deserialiation schema for Avro types + * Serialiation/Deserialiation schema for Avro types. * @param */ static class AvroSerializationDeserializationSchema @@ -217,7 +217,7 @@ public TypeInformation getProducedType() { } /** - * Custom type for Avro serialization + * Custom type for Avro serialization. */ static class MyType implements Serializable { @@ -233,10 +233,10 @@ public MyType() {} @Override public String toString() { - return "MyType{" + - "word='" + word + '\'' + - ", count=" + count + - '}'; + return "MyType{" + + "word='" + word + '\'' + + ", count=" + count + + '}'; } } } @@ -244,7 +244,7 @@ public String toString() { // -------------- Utilities -------------- /** - * Custom options for the Pipeline + * Custom options for the Pipeline. */ public interface KafkaOptions extends FlinkPipelineOptions { @Description("The Kafka topic to read from") @@ -279,7 +279,7 @@ public interface KafkaOptions extends FlinkPipelineOptions { } /** - * Initializes some options for the Flink runner + * Initializes some options for the Flink runner. * @param args The command line args * @return the pipeline */ @@ -298,7 +298,7 @@ private static Pipeline initializePipeline(String[] args) { } /** - * Gets KafkaOptions from the Pipeline + * Gets KafkaOptions from the Pipeline. * @param p the pipeline * @return KafkaOptions */ @@ -322,7 +322,7 @@ private static Properties getKafkaProps(KafkaOptions options) { } /** - * Print contents to stdout + * Print contents to stdout. * @param type of the input */ private static class PrintFn extends DoFn { diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java index 365fb7b48672..42c42f324198 100644 --- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java +++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java @@ -40,6 +40,9 @@ import org.apache.flink.streaming.util.serialization.SimpleStringSchema; import org.joda.time.Duration; +/** + * Wordcount example using Kafka topic. + */ public class KafkaWindowedWordCountExample { static final String KAFKA_TOPIC = "test"; // Default kafka topic to read from @@ -47,6 +50,9 @@ public class KafkaWindowedWordCountExample { static final String GROUP_ID = "myGroup"; // Default groupId static final String ZOOKEEPER = "localhost:2181"; // Default zookeeper to connect to for Kafka + /** + * Function to extract words. + */ public static class ExtractWordsFn extends DoFn { private final Aggregator emptyLines = createAggregator("emptyLines", new Sum.SumLongFn()); @@ -69,16 +75,24 @@ public void processElement(ProcessContext c) { } } + /** + * Function to format KV as String. + */ public static class FormatAsStringFn extends DoFn, String> { @ProcessElement public void processElement(ProcessContext c) { - String row = c.element().getKey() + " - " + c.element().getValue() + " @ " + c.timestamp().toString(); + String row = c.element().getKey() + " - " + c.element().getValue() + " @ " + + c.timestamp().toString(); System.out.println(row); c.output(row); } } - public interface KafkaStreamingWordCountOptions extends WindowedWordCount.StreamingWordCountOptions { + /** + * Pipeline options. + */ + public interface KafkaStreamingWordCountOptions + extends WindowedWordCount.StreamingWordCountOptions { @Description("The Kafka topic to read from") @Default.String(KAFKA_TOPIC) String getKafkaTopic(); @@ -107,7 +121,8 @@ public interface KafkaStreamingWordCountOptions extends WindowedWordCount.Stream public static void main(String[] args) { PipelineOptionsFactory.register(KafkaStreamingWordCountOptions.class); - KafkaStreamingWordCountOptions options = PipelineOptionsFactory.fromArgs(args).as(KafkaStreamingWordCountOptions.class); + KafkaStreamingWordCountOptions options = PipelineOptionsFactory.fromArgs(args) + .as(KafkaStreamingWordCountOptions.class); options.setJobName("KafkaExample - WindowSize: " + options.getWindowSize() + " seconds"); options.setStreaming(true); options.setCheckpointingInterval(1000L); @@ -115,7 +130,8 @@ public static void main(String[] args) { options.setExecutionRetryDelay(3000L); options.setRunner(FlinkRunner.class); - System.out.println(options.getKafkaTopic() +" "+ options.getZookeeper() +" "+ options.getBroker() +" "+ options.getGroup() ); + System.out.println(options.getKafkaTopic() + " " + options.getZookeeper() + " " + + options.getBroker() + " " + options.getGroup()); Pipeline pipeline = Pipeline.create(options); Properties p = new Properties(); @@ -132,7 +148,8 @@ public static void main(String[] args) { PCollection words = pipeline .apply("StreamingWordCount", Read.from(UnboundedFlinkSource.of(kafkaConsumer))) .apply(ParDo.of(new ExtractWordsFn())) - .apply(Window.into(FixedWindows.of(Duration.standardSeconds(options.getWindowSize()))) + .apply(Window.into(FixedWindows.of( + Duration.standardSeconds(options.getWindowSize()))) .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO) .discardingFiredPanes()); diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java index f3361c571697..0e250b868b72 100644 --- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java +++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java @@ -60,7 +60,8 @@ public class WindowedWordCount { static class FormatAsStringFn extends DoFn, String> { @ProcessElement public void processElement(ProcessContext c) { - String row = c.element().getKey() + " - " + c.element().getValue() + " @ " + c.timestamp().toString(); + String row = c.element().getKey() + " - " + c.element().getValue() + " @ " + + c.timestamp().toString(); c.output(row); } } @@ -87,7 +88,11 @@ public void processElement(ProcessContext c) { } } - public interface StreamingWordCountOptions extends org.apache.beam.runners.flink.examples.WordCount.Options { + /** + * Pipeline options. + */ + public interface StreamingWordCountOptions + extends org.apache.beam.runners.flink.examples.WordCount.Options { @Description("Sliding window duration, in seconds") @Default.Long(WINDOW_SIZE) Long getWindowSize(); @@ -102,7 +107,8 @@ public interface StreamingWordCountOptions extends org.apache.beam.runners.flink } public static void main(String[] args) throws IOException { - StreamingWordCountOptions options = PipelineOptionsFactory.fromArgs(args).withValidation().as(StreamingWordCountOptions.class); + StreamingWordCountOptions options = PipelineOptionsFactory.fromArgs(args).withValidation() + .as(StreamingWordCountOptions.class); options.setStreaming(true); options.setWindowSize(10L); options.setSlide(5L); @@ -111,8 +117,8 @@ public static void main(String[] args) throws IOException { options.setExecutionRetryDelay(3000L); options.setRunner(FlinkRunner.class); - LOG.info("Windpwed WordCount with Sliding Windows of " + options.getWindowSize() + - " sec. and a slide of " + options.getSlide()); + LOG.info("Windpwed WordCount with Sliding Windows of " + options.getWindowSize() + + " sec. and a slide of " + options.getSlide()); Pipeline pipeline = Pipeline.create(options); @@ -120,7 +126,8 @@ public static void main(String[] args) throws IOException { .apply("StreamingWordCount", Read.from(new UnboundedSocketSource<>("localhost", 9999, '\n', 3))) .apply(ParDo.of(new ExtractWordsFn())) - .apply(Window.into(SlidingWindows.of(Duration.standardSeconds(options.getWindowSize())) + .apply(Window.into(SlidingWindows.of( + Duration.standardSeconds(options.getWindowSize())) .every(Duration.standardSeconds(options.getSlide()))) .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO) .discardingFiredPanes()); diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/package-info.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/package-info.java new file mode 100644 index 000000000000..58f41b6a8bbd --- /dev/null +++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/package-info.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Flink Beam runner exemple. + */ +package org.apache.beam.runners.flink.examples.streaming; From 4691231b036950c5ca9c5c78b512933e61fae076 Mon Sep 17 00:00:00 2001 From: Stas Levin Date: Mon, 5 Sep 2016 20:26:12 +0300 Subject: [PATCH 105/112] Add header/footer support to TextIO.Write --- .../java/org/apache/beam/sdk/io/TextIO.java | 114 +++++++++++++++--- .../org/apache/beam/sdk/io/TextIOTest.java | 60 ++++++++- 2 files changed, 152 insertions(+), 22 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java index 242470b58792..c754a0ba0c4f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java @@ -448,7 +448,15 @@ public static Bound withoutValidation() { return new Bound<>(DEFAULT_TEXT_CODER).withoutValidation(); } - // TODO: appendingNewlines, header, footer, etc. + public static Bound withHeader(String header) { + return new Bound<>(DEFAULT_TEXT_CODER).withHeader(header); + } + + public static Bound withFooter(String footer) { + return new Bound<>(DEFAULT_TEXT_CODER).withFooter(footer); + } + + // TODO: appendingNewlines, etc. /** * A PTransform that writes a bounded PCollection to a text file (or @@ -465,6 +473,12 @@ public static class Bound extends PTransform, PDone> { /** The suffix of each file written, combined with prefix and shardTemplate. */ private final String filenameSuffix; + /** An optional header to add to each file. */ + private final String header; + + /** An optional footer to add to each file. */ + private final String footer; + /** The Coder to use to decode each line. */ private final Coder coder; @@ -478,12 +492,15 @@ public static class Bound extends PTransform, PDone> { private final boolean validate; Bound(Coder coder) { - this(null, null, "", coder, 0, DEFAULT_SHARD_TEMPLATE, true); + this(null, null, "", null, null, coder, 0, DEFAULT_SHARD_TEMPLATE, true); } - private Bound(String name, String filenamePrefix, String filenameSuffix, Coder coder, - int numShards, String shardTemplate, boolean validate) { + private Bound(String name, String filenamePrefix, String filenameSuffix, String header, + String footer, Coder coder, int numShards, String shardTemplate, + boolean validate) { super(name); + this.header = header; + this.footer = footer; this.coder = coder; this.filenamePrefix = filenamePrefix; this.filenameSuffix = filenameSuffix; @@ -502,7 +519,7 @@ private Bound(String name, String filenamePrefix, String filenameSuffix, Coder to(String filenamePrefix) { validateOutputComponent(filenamePrefix); - return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards, + return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards, shardTemplate, validate); } @@ -516,7 +533,7 @@ public Bound to(String filenamePrefix) { */ public Bound withSuffix(String nameExtension) { validateOutputComponent(nameExtension); - return new Bound<>(name, filenamePrefix, nameExtension, coder, numShards, + return new Bound<>(name, filenamePrefix, nameExtension, header, footer, coder, numShards, shardTemplate, validate); } @@ -536,7 +553,7 @@ public Bound withSuffix(String nameExtension) { */ public Bound withNumShards(int numShards) { checkArgument(numShards >= 0); - return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards, + return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards, shardTemplate, validate); } @@ -549,7 +566,7 @@ public Bound withNumShards(int numShards) { * @see ShardNameTemplate */ public Bound withShardNameTemplate(String shardTemplate) { - return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards, + return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards, shardTemplate, validate); } @@ -567,7 +584,8 @@ public Bound withShardNameTemplate(String shardTemplate) { *

              Does not modify this object. */ public Bound withoutSharding() { - return new Bound<>(name, filenamePrefix, filenameSuffix, coder, 1, "", validate); + return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, 1, "", + validate); } /** @@ -579,7 +597,7 @@ public Bound withoutSharding() { * @param the type of the elements of the input {@link PCollection} */ public Bound withCoder(Coder coder) { - return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards, + return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards, shardTemplate, validate); } @@ -594,7 +612,17 @@ public Bound withCoder(Coder coder) { *

              Does not modify this object. */ public Bound withoutValidation() { - return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards, + return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards, + shardTemplate, false); + } + + public Bound withHeader(String header) { + return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards, + shardTemplate, false); + } + + public Bound withFooter(String footer) { + return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards, shardTemplate, false); } @@ -607,7 +635,8 @@ public PDone apply(PCollection input) { org.apache.beam.sdk.io.Write.Bound write = org.apache.beam.sdk.io.Write.to( - new TextSink<>(filenamePrefix, filenameSuffix, shardTemplate, coder)); + new TextSink<>(filenamePrefix, filenameSuffix, header, footer, shardTemplate, + coder)); if (getNumShards() > 0) { write = write.withNumShards(getNumShards()); } @@ -629,7 +658,11 @@ public void populateDisplayData(DisplayData.Builder builder) { .addIfNotDefault(DisplayData.item("validation", validate) .withLabel("Validation Enabled"), true) .addIfNotDefault(DisplayData.item("numShards", numShards) - .withLabel("Maximum Output Shards"), 0); + .withLabel("Maximum Output Shards"), 0) + .addIfNotNull(DisplayData.item("fileHeader", header) + .withLabel("Output file header")) + .addIfNotNull(DisplayData.item("fileFooter", footer) + .withLabel("Output file footer")); } /** @@ -664,6 +697,14 @@ public Coder getCoder() { return coder; } + public String getHeader() { + return header; + } + + public String getFooter() { + return footer; + } + public boolean needsValidation() { return validate; } @@ -946,17 +987,22 @@ private boolean tryToEnsureNumberOfBytesInBuffer(int minCapacity) throws IOExcep @VisibleForTesting static class TextSink extends FileBasedSink { private final Coder coder; + private final String header; + private final String footer; @VisibleForTesting TextSink( - String baseOutputFilename, String extension, String fileNameTemplate, Coder coder) { + String baseOutputFilename, String extension, String header, String footer, + String fileNameTemplate, Coder coder) { super(baseOutputFilename, extension, fileNameTemplate); this.coder = coder; + this.header = header; + this.footer = footer; } @Override public FileBasedSink.FileBasedWriteOperation createWriteOperation(PipelineOptions options) { - return new TextWriteOperation<>(this, coder); + return new TextWriteOperation<>(this, coder, header, footer); } /** @@ -965,15 +1011,19 @@ public FileBasedSink.FileBasedWriteOperation createWriteOperation(PipelineOpt */ private static class TextWriteOperation extends FileBasedWriteOperation { private final Coder coder; + private final String header; + private final String footer; - private TextWriteOperation(TextSink sink, Coder coder) { + private TextWriteOperation(TextSink sink, Coder coder, String header, String footer) { super(sink); this.coder = coder; + this.header = header; + this.footer = footer; } @Override public FileBasedWriter createWriter(PipelineOptions options) throws Exception { - return new TextWriter<>(this, coder); + return new TextWriter<>(this, coder, header, footer); } } @@ -984,19 +1034,49 @@ public FileBasedWriter createWriter(PipelineOptions options) throws Exception private static class TextWriter extends FileBasedWriter { private static final byte[] NEWLINE = "\n".getBytes(StandardCharsets.UTF_8); private final Coder coder; + private final String header; + private final String footer; private OutputStream out; public TextWriter(FileBasedWriteOperation writeOperation, Coder coder) { + this(writeOperation, coder, null, null); + } + + public TextWriter(FileBasedWriteOperation writeOperation, Coder coder, String header) { + this(writeOperation, coder, header, null); + } + + public TextWriter(FileBasedWriteOperation writeOperation, Coder coder, String header, + String footer) { super(writeOperation); + this.header = header; + this.footer = footer; this.mimeType = MimeTypes.TEXT; this.coder = coder; } + private void writeLine(String line) throws IOException { + if (line != null) { + out.write(line.getBytes(StandardCharsets.UTF_8)); + out.write(NEWLINE); + } + } + @Override protected void prepareWrite(WritableByteChannel channel) throws Exception { out = Channels.newOutputStream(channel); } + @Override + protected void writeHeader() throws Exception { + writeLine(header); + } + + @Override + protected void writeFooter() throws Exception { + writeLine(footer); + } + @Override public void write(T value) throws Exception { coder.encode(value, out, Context.OUTER); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java index 8f9476606302..2ab268355697 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java @@ -34,6 +34,8 @@ import static org.junit.Assert.assertTrue; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; + import java.io.BufferedReader; import java.io.File; import java.io.FileOutputStream; @@ -48,6 +50,7 @@ import java.nio.file.StandardOpenOption; import java.util.ArrayList; import java.util.Arrays; +import java.util.LinkedList; import java.util.List; import java.util.Set; import java.util.zip.GZIPOutputStream; @@ -101,6 +104,9 @@ @SuppressWarnings("unchecked") public class TextIOTest { + private static final String MY_HEADER = "myHeader"; + private static final String MY_FOOTER = "myFooter"; + @Rule public TemporaryFolder tmpFolder = new TemporaryFolder(); @Rule @@ -216,10 +222,19 @@ public void testPrimitiveReadDisplayData() { } void runTestWrite(T[] elems, Coder coder) throws Exception { - runTestWrite(elems, coder, 1); + runTestWrite(elems, null, null, coder, 1); } void runTestWrite(T[] elems, Coder coder, int numShards) throws Exception { + runTestWrite(elems, null, null, coder, numShards); + } + + void runTestWrite(T[] elems, Coder coder, String header, String footer) throws Exception { + runTestWrite(elems, header, footer, coder, 1); + } + + void runTestWrite(T[] elems, String header, String footer, Coder coder, int numShards) + throws Exception { String outputName = "file.txt"; String baseFilename = tmpFolder.newFile(outputName).getPath(); @@ -235,6 +250,8 @@ void runTestWrite(T[] elems, Coder coder, int numShards) throws Exception } else { write = TextIO.Write.to(baseFilename).withCoder(coder); } + write = write.withHeader(header).withFooter(footer); + if (numShards == 1) { write = write.withoutSharding(); } else if (numShards > 0) { @@ -244,11 +261,14 @@ void runTestWrite(T[] elems, Coder coder, int numShards) throws Exception p.run(); - assertOutputFiles(elems, coder, numShards, tmpFolder, outputName, write.getShardNameTemplate()); + assertOutputFiles(elems, header, footer, coder, numShards, tmpFolder, outputName, + write.getShardNameTemplate()); } public static void assertOutputFiles( T[] elems, + String header, + String footer, Coder coder, int numShards, TemporaryFolder rootLocation, @@ -284,15 +304,23 @@ public static void assertOutputFiles( } } - String[] expected = new String[elems.length]; + LinkedList expected = Lists.newLinkedList(); + for (int i = 0; i < elems.length; i++) { T elem = elems[i]; byte[] encodedElem = CoderUtils.encodeToByteArray(coder, elem); String line = new String(encodedElem); - expected[i] = line; + expected.add(line); + } + + if (header != null) { + expected.addFirst(header); + } + if (footer != null) { + expected.addLast(footer); } - assertThat(actual, containsInAnyOrder(expected)); + assertThat(actual, containsInAnyOrder(expected.toArray())); } @Test @@ -331,6 +359,24 @@ public void testShardedWrite() throws Exception { runTestWrite(LINES_ARRAY, StringUtf8Coder.of(), 5); } + @Test + @Category(NeedsRunner.class) + public void testWriteWithHeader() throws Exception { + runTestWrite(LINES_ARRAY, StringUtf8Coder.of(), MY_HEADER, null); + } + + @Test + @Category(NeedsRunner.class) + public void testWriteWithFooter() throws Exception { + runTestWrite(LINES_ARRAY, StringUtf8Coder.of(), null, MY_FOOTER); + } + + @Test + @Category(NeedsRunner.class) + public void testWriteWithHeaderAndFooter() throws Exception { + runTestWrite(LINES_ARRAY, StringUtf8Coder.of(), MY_HEADER, MY_FOOTER); + } + @Test public void testWriteDisplayData() { TextIO.Write.Bound write = TextIO.Write @@ -338,12 +384,16 @@ public void testWriteDisplayData() { .withSuffix("bar") .withShardNameTemplate("-SS-of-NN-") .withNumShards(100) + .withFooter("myFooter") + .withHeader("myHeader") .withoutValidation(); DisplayData displayData = DisplayData.from(write); assertThat(displayData, hasDisplayItem("filePrefix", "foo")); assertThat(displayData, hasDisplayItem("fileSuffix", "bar")); + assertThat(displayData, hasDisplayItem("fileHeader", "myHeader")); + assertThat(displayData, hasDisplayItem("fileFooter", "myFooter")); assertThat(displayData, hasDisplayItem("shardNameTemplate", "-SS-of-NN-")); assertThat(displayData, hasDisplayItem("numShards", 100)); assertThat(displayData, hasDisplayItem("validation", false)); From af7437d1c8270c5b6f175b3d5ae90160bf6477b4 Mon Sep 17 00:00:00 2001 From: Stas Levin Date: Tue, 6 Sep 2016 23:22:11 +0300 Subject: [PATCH 106/112] Revised according to comments following a code review. --- .../java/org/apache/beam/sdk/io/TextIO.java | 72 ++++++++++-------- .../org/apache/beam/sdk/io/TextIOTest.java | 73 ++++++++++++++----- 2 files changed, 98 insertions(+), 47 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java index c754a0ba0c4f..eefa867d2ae5 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java @@ -22,6 +22,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.protobuf.ByteString; + import java.io.IOException; import java.io.OutputStream; import java.nio.ByteBuffer; @@ -35,12 +36,14 @@ import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.Coder.Context; +import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.io.Read.Bounded; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.util.MimeTypes; import org.apache.beam.sdk.values.PBegin; @@ -474,10 +477,10 @@ public static class Bound extends PTransform, PDone> { private final String filenameSuffix; /** An optional header to add to each file. */ - private final String header; + private final T header; /** An optional footer to add to each file. */ - private final String footer; + private final T footer; /** The Coder to use to decode each line. */ private final Coder coder; @@ -495,8 +498,8 @@ public static class Bound extends PTransform, PDone> { this(null, null, "", null, null, coder, 0, DEFAULT_SHARD_TEMPLATE, true); } - private Bound(String name, String filenamePrefix, String filenameSuffix, String header, - String footer, Coder coder, int numShards, String shardTemplate, + private Bound(String name, String filenamePrefix, String filenameSuffix, T header, + T footer, Coder coder, int numShards, String shardTemplate, boolean validate) { super(name); this.header = header; @@ -509,6 +512,14 @@ private Bound(String name, String filenamePrefix, String filenameSuffix, String this.validate = validate; } + private String asString(T obj, Coder coder) { + try { + return obj == null ? "" : new String(CoderUtils.encodeToByteArray(coder, obj)); + } catch (CoderException e) { + throw new RuntimeException(e); + } + } + /** * Returns a transform for writing to text files that's like this one but * that writes to the file(s) with the given filename prefix. @@ -594,9 +605,8 @@ public Bound withoutSharding() { * the elements of the input {@link PCollection PCollection} into an * output text line. Does not modify this object. * - * @param the type of the elements of the input {@link PCollection} */ - public Bound withCoder(Coder coder) { + public Bound withCoder(Coder coder) { return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards, shardTemplate, validate); } @@ -616,12 +626,12 @@ public Bound withoutValidation() { shardTemplate, false); } - public Bound withHeader(String header) { + public Bound withHeader(T header) { return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards, shardTemplate, false); } - public Bound withFooter(String footer) { + public Bound withFooter(T footer) { return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards, shardTemplate, false); } @@ -659,10 +669,10 @@ public void populateDisplayData(DisplayData.Builder builder) { .withLabel("Validation Enabled"), true) .addIfNotDefault(DisplayData.item("numShards", numShards) .withLabel("Maximum Output Shards"), 0) - .addIfNotNull(DisplayData.item("fileHeader", header) - .withLabel("Output file header")) - .addIfNotNull(DisplayData.item("fileFooter", footer) - .withLabel("Output file footer")); + .addIfNotNull(DisplayData.item("fileHeader", asString(header, coder)) + .withLabel("File Header")) + .addIfNotNull(DisplayData.item("fileFooter", asString(footer, coder)) + .withLabel("File Footer")); } /** @@ -697,11 +707,11 @@ public Coder getCoder() { return coder; } - public String getHeader() { + public T getHeader() { return header; } - public String getFooter() { + public T getFooter() { return footer; } @@ -987,17 +997,21 @@ private boolean tryToEnsureNumberOfBytesInBuffer(int minCapacity) throws IOExcep @VisibleForTesting static class TextSink extends FileBasedSink { private final Coder coder; - private final String header; - private final String footer; + private final byte[] header; + private final byte[] footer; @VisibleForTesting TextSink( - String baseOutputFilename, String extension, String header, String footer, + String baseOutputFilename, String extension, T header, T footer, String fileNameTemplate, Coder coder) { super(baseOutputFilename, extension, fileNameTemplate); this.coder = coder; - this.header = header; - this.footer = footer; + try { + this.header = header == null ? null : CoderUtils.encodeToByteArray(coder, header); + this.footer = footer == null ? null : CoderUtils.encodeToByteArray(coder, footer); + } catch (CoderException e) { + throw new RuntimeException(e); + } } @Override @@ -1011,10 +1025,10 @@ public FileBasedSink.FileBasedWriteOperation createWriteOperation(PipelineOpt */ private static class TextWriteOperation extends FileBasedWriteOperation { private final Coder coder; - private final String header; - private final String footer; + private final byte[] header; + private final byte[] footer; - private TextWriteOperation(TextSink sink, Coder coder, String header, String footer) { + private TextWriteOperation(TextSink sink, Coder coder, byte[] header, byte[] footer) { super(sink); this.coder = coder; this.header = header; @@ -1034,20 +1048,20 @@ public FileBasedWriter createWriter(PipelineOptions options) throws Exception private static class TextWriter extends FileBasedWriter { private static final byte[] NEWLINE = "\n".getBytes(StandardCharsets.UTF_8); private final Coder coder; - private final String header; - private final String footer; + private final byte[] header; + private final byte[] footer; private OutputStream out; public TextWriter(FileBasedWriteOperation writeOperation, Coder coder) { this(writeOperation, coder, null, null); } - public TextWriter(FileBasedWriteOperation writeOperation, Coder coder, String header) { + public TextWriter(FileBasedWriteOperation writeOperation, Coder coder, byte[] header) { this(writeOperation, coder, header, null); } - public TextWriter(FileBasedWriteOperation writeOperation, Coder coder, String header, - String footer) { + public TextWriter(FileBasedWriteOperation writeOperation, Coder coder, byte[] header, + byte[] footer) { super(writeOperation); this.header = header; this.footer = footer; @@ -1055,9 +1069,9 @@ public TextWriter(FileBasedWriteOperation writeOperation, Coder coder, Str this.coder = coder; } - private void writeLine(String line) throws IOException { + private void writeLine(byte[] line) throws IOException { if (line != null) { - out.write(line.getBytes(StandardCharsets.UTF_8)); + out.write(line); out.write(NEWLINE); } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java index 2ab268355697..7028761bb5e5 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java @@ -33,7 +33,11 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; +import com.google.common.base.Function; +import com.google.common.base.Predicate; +import com.google.common.collect.FluentIterable; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import java.io.BufferedReader; @@ -229,11 +233,11 @@ void runTestWrite(T[] elems, Coder coder, int numShards) throws Exception runTestWrite(elems, null, null, coder, numShards); } - void runTestWrite(T[] elems, Coder coder, String header, String footer) throws Exception { + void runTestWrite(T[] elems, Coder coder, T header, T footer) throws Exception { runTestWrite(elems, header, footer, coder, 1); } - void runTestWrite(T[] elems, String header, String footer, Coder coder, int numShards) + void runTestWrite(T[] elems, T header, T footer, Coder coder, int numShards) throws Exception { String outputName = "file.txt"; String baseFilename = tmpFolder.newFile(outputName).getPath(); @@ -248,7 +252,7 @@ void runTestWrite(T[] elems, String header, String footer, Coder coder, i // T==String write = (TextIO.Write.Bound) writeStrings; } else { - write = TextIO.Write.to(baseFilename).withCoder(coder); + write = TextIO.Write.withCoder(coder).to(baseFilename); } write = write.withHeader(header).withFooter(footer); @@ -267,9 +271,9 @@ void runTestWrite(T[] elems, String header, String footer, Coder coder, i public static void assertOutputFiles( T[] elems, - String header, - String footer, - Coder coder, + final T header, + final T footer, + final Coder coder, int numShards, TemporaryFolder rootLocation, String outputName, @@ -291,36 +295,69 @@ public static void assertOutputFiles( } } - List actual = new ArrayList<>(); + List> actual = new ArrayList<>(); + for (File tmpFile : expectedFiles) { try (BufferedReader reader = new BufferedReader(new FileReader(tmpFile))) { + List currentFile = Lists.newArrayList(); for (;;) { String line = reader.readLine(); if (line == null) { break; } - actual.add(line); + currentFile.add(line); } + actual.add(currentFile); } } - LinkedList expected = Lists.newLinkedList(); + LinkedList expectedElements = Lists.newLinkedList(); for (int i = 0; i < elems.length; i++) { T elem = elems[i]; byte[] encodedElem = CoderUtils.encodeToByteArray(coder, elem); String line = new String(encodedElem); - expected.add(line); + expectedElements.add(line); } - if (header != null) { - expected.addFirst(header); - } - if (footer != null) { - expected.addLast(footer); - } - - assertThat(actual, containsInAnyOrder(expected.toArray())); + final String headerString = + header == null ? null : new String(CoderUtils.encodeToByteArray(coder, header)); + + final String footerString = + footer == null ? null : new String(CoderUtils.encodeToByteArray(coder, footer)); + + ArrayList actualElements = + Lists.newArrayList( + Iterables.concat( + FluentIterable + .from(actual) + .transform(new Function, List>() { + @Nullable + @Override + public List apply(List lines) { + ArrayList newLines = Lists.newArrayList(lines); + if (headerString != null) { + newLines.remove(0); + } + if (footerString != null) { + int last = newLines.size() - 1; + newLines.remove(last); + } + return newLines; + } + }) + .toList())); + + assertThat(actualElements, containsInAnyOrder(expectedElements.toArray())); + + assertTrue(Iterables.all(actual, new Predicate>() { + @Override + public boolean apply(@Nullable List fileLines) { + int last = fileLines.size() - 1; + return (headerString == null || fileLines.get(0).equals(headerString)) + && (footerString == null || fileLines.get(last).equals(footerString)); + } + })); } @Test From 0e8ff41e34c9bc332148487c7c52ed99051b5ff7 Mon Sep 17 00:00:00 2001 From: Stas Levin Date: Wed, 7 Sep 2016 09:57:17 +0300 Subject: [PATCH 107/112] Reverted header and footer to be of type String. --- .../java/org/apache/beam/sdk/io/TextIO.java | 68 ++++++++----------- .../org/apache/beam/sdk/io/TextIOTest.java | 68 ++++++++++--------- 2 files changed, 63 insertions(+), 73 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java index eefa867d2ae5..0895123aba37 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java @@ -22,7 +22,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.protobuf.ByteString; - import java.io.IOException; import java.io.OutputStream; import java.nio.ByteBuffer; @@ -36,14 +35,12 @@ import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.Coder.Context; -import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.io.Read.Bounded; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.util.MimeTypes; import org.apache.beam.sdk.values.PBegin; @@ -477,10 +474,10 @@ public static class Bound extends PTransform, PDone> { private final String filenameSuffix; /** An optional header to add to each file. */ - private final T header; + private final String header; /** An optional footer to add to each file. */ - private final T footer; + private final String footer; /** The Coder to use to decode each line. */ private final Coder coder; @@ -498,8 +495,8 @@ public static class Bound extends PTransform, PDone> { this(null, null, "", null, null, coder, 0, DEFAULT_SHARD_TEMPLATE, true); } - private Bound(String name, String filenamePrefix, String filenameSuffix, T header, - T footer, Coder coder, int numShards, String shardTemplate, + private Bound(String name, String filenamePrefix, String filenameSuffix, String header, + String footer, Coder coder, int numShards, String shardTemplate, boolean validate) { super(name); this.header = header; @@ -512,14 +509,6 @@ private Bound(String name, String filenamePrefix, String filenameSuffix, T heade this.validate = validate; } - private String asString(T obj, Coder coder) { - try { - return obj == null ? "" : new String(CoderUtils.encodeToByteArray(coder, obj)); - } catch (CoderException e) { - throw new RuntimeException(e); - } - } - /** * Returns a transform for writing to text files that's like this one but * that writes to the file(s) with the given filename prefix. @@ -605,8 +594,9 @@ public Bound withoutSharding() { * the elements of the input {@link PCollection PCollection} into an * output text line. Does not modify this object. * + * @param the type of the elements of the input {@link PCollection} */ - public Bound withCoder(Coder coder) { + public Bound withCoder(Coder coder) { return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards, shardTemplate, validate); } @@ -626,12 +616,12 @@ public Bound withoutValidation() { shardTemplate, false); } - public Bound withHeader(T header) { + public Bound withHeader(String header) { return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards, shardTemplate, false); } - public Bound withFooter(T footer) { + public Bound withFooter(String footer) { return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards, shardTemplate, false); } @@ -669,9 +659,9 @@ public void populateDisplayData(DisplayData.Builder builder) { .withLabel("Validation Enabled"), true) .addIfNotDefault(DisplayData.item("numShards", numShards) .withLabel("Maximum Output Shards"), 0) - .addIfNotNull(DisplayData.item("fileHeader", asString(header, coder)) + .addIfNotNull(DisplayData.item("fileHeader", header) .withLabel("File Header")) - .addIfNotNull(DisplayData.item("fileFooter", asString(footer, coder)) + .addIfNotNull(DisplayData.item("fileFooter", footer) .withLabel("File Footer")); } @@ -707,11 +697,11 @@ public Coder getCoder() { return coder; } - public T getHeader() { + public String getHeader() { return header; } - public T getFooter() { + public String getFooter() { return footer; } @@ -997,21 +987,17 @@ private boolean tryToEnsureNumberOfBytesInBuffer(int minCapacity) throws IOExcep @VisibleForTesting static class TextSink extends FileBasedSink { private final Coder coder; - private final byte[] header; - private final byte[] footer; + private final String header; + private final String footer; @VisibleForTesting TextSink( - String baseOutputFilename, String extension, T header, T footer, + String baseOutputFilename, String extension, String header, String footer, String fileNameTemplate, Coder coder) { super(baseOutputFilename, extension, fileNameTemplate); this.coder = coder; - try { - this.header = header == null ? null : CoderUtils.encodeToByteArray(coder, header); - this.footer = footer == null ? null : CoderUtils.encodeToByteArray(coder, footer); - } catch (CoderException e) { - throw new RuntimeException(e); - } + this.header = header; + this.footer = footer; } @Override @@ -1025,10 +1011,10 @@ public FileBasedSink.FileBasedWriteOperation createWriteOperation(PipelineOpt */ private static class TextWriteOperation extends FileBasedWriteOperation { private final Coder coder; - private final byte[] header; - private final byte[] footer; + private final String header; + private final String footer; - private TextWriteOperation(TextSink sink, Coder coder, byte[] header, byte[] footer) { + private TextWriteOperation(TextSink sink, Coder coder, String header, String footer) { super(sink); this.coder = coder; this.header = header; @@ -1048,20 +1034,20 @@ public FileBasedWriter createWriter(PipelineOptions options) throws Exception private static class TextWriter extends FileBasedWriter { private static final byte[] NEWLINE = "\n".getBytes(StandardCharsets.UTF_8); private final Coder coder; - private final byte[] header; - private final byte[] footer; + private final String header; + private final String footer; private OutputStream out; public TextWriter(FileBasedWriteOperation writeOperation, Coder coder) { this(writeOperation, coder, null, null); } - public TextWriter(FileBasedWriteOperation writeOperation, Coder coder, byte[] header) { + public TextWriter(FileBasedWriteOperation writeOperation, Coder coder, String header) { this(writeOperation, coder, header, null); } - public TextWriter(FileBasedWriteOperation writeOperation, Coder coder, byte[] header, - byte[] footer) { + public TextWriter(FileBasedWriteOperation writeOperation, Coder coder, String header, + String footer) { super(writeOperation); this.header = header; this.footer = footer; @@ -1069,9 +1055,9 @@ public TextWriter(FileBasedWriteOperation writeOperation, Coder coder, byt this.coder = coder; } - private void writeLine(byte[] line) throws IOException { + private void writeLine(String line) throws IOException { if (line != null) { - out.write(line); + out.write(line.getBytes(StandardCharsets.UTF_8)); out.write(NEWLINE); } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java index 7028761bb5e5..c60b735b0129 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java @@ -233,11 +233,11 @@ void runTestWrite(T[] elems, Coder coder, int numShards) throws Exception runTestWrite(elems, null, null, coder, numShards); } - void runTestWrite(T[] elems, Coder coder, T header, T footer) throws Exception { + void runTestWrite(T[] elems, Coder coder, String header, String footer) throws Exception { runTestWrite(elems, header, footer, coder, 1); } - void runTestWrite(T[] elems, T header, T footer, Coder coder, int numShards) + void runTestWrite(T[] elems, String header, String footer, Coder coder, int numShards) throws Exception { String outputName = "file.txt"; String baseFilename = tmpFolder.newFile(outputName).getPath(); @@ -252,7 +252,7 @@ void runTestWrite(T[] elems, T header, T footer, Coder coder, int numShar // T==String write = (TextIO.Write.Bound) writeStrings; } else { - write = TextIO.Write.withCoder(coder).to(baseFilename); + write = TextIO.Write.to(baseFilename).withCoder(coder); } write = write.withHeader(header).withFooter(footer); @@ -271,9 +271,9 @@ void runTestWrite(T[] elems, T header, T footer, Coder coder, int numShar public static void assertOutputFiles( T[] elems, - final T header, - final T footer, - final Coder coder, + final String header, + final String footer, + Coder coder, int numShards, TemporaryFolder rootLocation, String outputName, @@ -320,44 +320,48 @@ public static void assertOutputFiles( expectedElements.add(line); } - final String headerString = - header == null ? null : new String(CoderUtils.encodeToByteArray(coder, header)); - - final String footerString = - footer == null ? null : new String(CoderUtils.encodeToByteArray(coder, footer)); - ArrayList actualElements = Lists.newArrayList( Iterables.concat( FluentIterable .from(actual) - .transform(new Function, List>() { - @Nullable - @Override - public List apply(List lines) { - ArrayList newLines = Lists.newArrayList(lines); - if (headerString != null) { - newLines.remove(0); - } - if (footerString != null) { - int last = newLines.size() - 1; - newLines.remove(last); - } - return newLines; - } - }) + .transform(removeHeaderAndFooter(header, footer)) .toList())); assertThat(actualElements, containsInAnyOrder(expectedElements.toArray())); - assertTrue(Iterables.all(actual, new Predicate>() { + assertTrue(Iterables.all(actual, haveProperHeaderAndFooter(header, footer))); + } + + private static Function, List> removeHeaderAndFooter(final String header, + final String footer) { + return new Function, List>() { + @Nullable + @Override + public List apply(List lines) { + ArrayList newLines = Lists.newArrayList(lines); + if (header != null) { + newLines.remove(0); + } + if (footer != null) { + int last = newLines.size() - 1; + newLines.remove(last); + } + return newLines; + } + }; + } + + private static Predicate> haveProperHeaderAndFooter(final String header, + final String footer) { + return new Predicate>() { @Override - public boolean apply(@Nullable List fileLines) { + public boolean apply(List fileLines) { int last = fileLines.size() - 1; - return (headerString == null || fileLines.get(0).equals(headerString)) - && (footerString == null || fileLines.get(last).equals(footerString)); + return (header == null || fileLines.get(0).equals(header)) + && (footer == null || fileLines.get(last).equals(footer)); } - })); + }; } @Test From 94c58b61411e78da8bb844e4167a51fb2894692e Mon Sep 17 00:00:00 2001 From: Stas Levin Date: Wed, 7 Sep 2016 18:33:48 +0300 Subject: [PATCH 108/112] Added javadoc to TextIO#withHeader and TextIO#withFooter. --- .../src/main/java/org/apache/beam/sdk/io/TextIO.java | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java index 0895123aba37..e75aaa9f2f5e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java @@ -448,10 +448,22 @@ public static Bound withoutValidation() { return new Bound<>(DEFAULT_TEXT_CODER).withoutValidation(); } + /** + * Returns a transform for writing to text files that adds a header string to the files + * it writes. + * + * @param header the string to be added as file header + */ public static Bound withHeader(String header) { return new Bound<>(DEFAULT_TEXT_CODER).withHeader(header); } + /** + * Returns a transform for writing to text files that adds a footer string to the files + * it writes. + * + * @param footer the string to be added as file footer + */ public static Bound withFooter(String footer) { return new Bound<>(DEFAULT_TEXT_CODER).withFooter(footer); } From b1176de93b0e5dcff17901f8b5049e677213d62e Mon Sep 17 00:00:00 2001 From: Stas Levin Date: Wed, 7 Sep 2016 18:38:28 +0300 Subject: [PATCH 109/112] Added even more javadoc to TextIO#withHeader and TextIO#withFooter. --- .../core/src/main/java/org/apache/beam/sdk/io/TextIO.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java index e75aaa9f2f5e..f09ab6c5654a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java @@ -452,6 +452,8 @@ public static Bound withoutValidation() { * Returns a transform for writing to text files that adds a header string to the files * it writes. * + *

              A null value will clear any previously configured header.

              + * * @param header the string to be added as file header */ public static Bound withHeader(String header) { @@ -462,6 +464,8 @@ public static Bound withHeader(String header) { * Returns a transform for writing to text files that adds a footer string to the files * it writes. * + *

              A null value will clear any previously configured footer.

              + * * @param footer the string to be added as file footer */ public static Bound withFooter(String footer) { From f21875e2fe85572e83c933ba409626d509209119 Mon Sep 17 00:00:00 2001 From: Stas Levin Date: Thu, 8 Sep 2016 00:02:52 +0300 Subject: [PATCH 110/112] Added even more javadoc to TextIO#withHeader and TextIO#withFooter (2). --- .../main/java/org/apache/beam/sdk/io/TextIO.java | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java index f09ab6c5654a..e4fcc327bd0d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java @@ -632,11 +632,27 @@ public Bound withoutValidation() { shardTemplate, false); } + /** + * Returns a transform for writing to text files that adds a header string to the files + * it writes. + * + *

              A null value will clear any previously configured header.

              + * + * @param header the string to be added as file header + */ public Bound withHeader(String header) { return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards, shardTemplate, false); } + /** + * Returns a transform for writing to text files that adds a footer string to the files + * it writes. + * + *

              A null value will clear any previously configured footer.

              + * + * @param footer the string to be added as file footer + */ public Bound withFooter(String footer) { return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards, shardTemplate, false); From ce1294dd5dfad80b7a154bb1ef52836b7770eb32 Mon Sep 17 00:00:00 2001 From: Luke Cwik Date: Wed, 7 Sep 2016 15:38:16 -0700 Subject: [PATCH 111/112] !fixup Minor javadoc clean-up --- .../src/main/java/org/apache/beam/sdk/io/TextIO.java | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java index e4fcc327bd0d..c0761b19ed88 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java @@ -452,7 +452,7 @@ public static Bound withoutValidation() { * Returns a transform for writing to text files that adds a header string to the files * it writes. * - *

              A null value will clear any previously configured header.

              + *

              A {@code null} value will clear any previously configured header. * * @param header the string to be added as file header */ @@ -464,7 +464,7 @@ public static Bound withHeader(String header) { * Returns a transform for writing to text files that adds a footer string to the files * it writes. * - *

              A null value will clear any previously configured footer.

              + *

              A {@code null} value will clear any previously configured footer. * * @param footer the string to be added as file footer */ @@ -636,7 +636,9 @@ public Bound withoutValidation() { * Returns a transform for writing to text files that adds a header string to the files * it writes. * - *

              A null value will clear any previously configured header.

              + *

              A {@code null} value will clear any previously configured header. + * + *

              Does not modify this object. * * @param header the string to be added as file header */ @@ -649,7 +651,9 @@ public Bound withHeader(String header) { * Returns a transform for writing to text files that adds a footer string to the files * it writes. * - *

              A null value will clear any previously configured footer.

              + *

              A {@code null} value will clear any previously configured footer. + * + *

              Does not modify this object. * * @param footer the string to be added as file footer */ From e3eccff4453a2c595f35487274154c8ec0054f45 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Mon, 12 Sep 2016 11:45:15 +0800 Subject: [PATCH 112/112] fix import order --- runners/gearpump/pom.xml | 7 ++--- .../gearpump/GearpumpPipelineOptions.java | 8 +++--- .../gearpump/GearpumpPipelineResult.java | 4 ++- .../gearpump/GearpumpPipelineRunner.java | 28 +++++++++---------- .../GearpumpPipelineRunnerRegistrar.java | 9 +++--- .../gearpump/GearpumpPipelineTranslator.java | 7 ++--- .../gearpump/examples/StreamingWordCount.java | 15 +++------- .../examples/UnboundedTextSource.java | 15 +++++----- .../translators/GroupByKeyTranslator.java | 14 ++++++---- .../ParDoBoundMultiTranslator.java | 16 +++++------ .../translators/TransformTranslator.java | 3 +- .../translators/TranslationContext.java | 6 ++-- .../translators/functions/DoFnFunction.java | 14 +++++----- .../translators/io/BoundedSourceWrapper.java | 4 +-- .../translators/io/GearpumpSource.java | 12 ++++---- .../io/UnboundedSourceWrapper.java | 4 +-- .../gearpump/translators/io/ValuesSource.java | 12 ++++---- .../translators/utils/GearpumpDoFnRunner.java | 28 +++++++++---------- .../utils/NoOpSideInputReader.java | 8 +++--- .../translators/utils/NoOpStepContext.java | 6 ++-- .../java/org/apache/beam/sdk/Pipeline.java | 2 +- .../beam/sdk/runners/PipelineRunner.java | 1 + .../beam/sdk/transforms/DoFnAdapters.java | 5 ++++ 23 files changed, 114 insertions(+), 114 deletions(-) diff --git a/runners/gearpump/pom.xml b/runners/gearpump/pom.xml index cc99a7a38489..296de6b52ff6 100644 --- a/runners/gearpump/pom.xml +++ b/runners/gearpump/pom.xml @@ -122,7 +122,6 @@ org.apache.gearpump gearpump-daemon_2.11 ${gearpump.version} - provided org.apache.gearpump @@ -185,10 +184,6 @@ com.fasterxml.jackson.core jackson-annotations - - com.google.http-client - google-http-client - com.google.guava guava @@ -225,6 +220,7 @@ auto-service 1.0-rc2 + @@ -287,6 +283,7 @@ org.apache.maven.plugins maven-checkstyle-plugin + diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineOptions.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineOptions.java index 5b6ee960745d..e02cbbc01a8d 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineOptions.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineOptions.java @@ -18,17 +18,17 @@ package org.apache.beam.runners.gearpump; +import com.fasterxml.jackson.annotation.JsonIgnore; + +import java.util.Map; + import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.PipelineOptions; -import com.fasterxml.jackson.annotation.JsonIgnore; - import org.apache.gearpump.cluster.client.ClientContext; import org.apache.gearpump.cluster.embedded.EmbeddedCluster; -import java.util.Map; - /** * Options that configure the Gearpump pipeline. */ diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java index 6184bc39a347..2011a4b68544 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java @@ -17,14 +17,16 @@ */ package org.apache.beam.runners.gearpump; +import java.io.IOException; + import org.apache.beam.sdk.AggregatorRetrievalException; import org.apache.beam.sdk.AggregatorValues; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.transforms.Aggregator; + import org.joda.time.Duration; -import java.io.IOException; /** * Result of executing a {@link Pipeline} with Gearpump. diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunner.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunner.java index 4182ee4785df..ad7bb3e06a71 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunner.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunner.java @@ -17,6 +17,13 @@ */ package org.apache.beam.runners.gearpump; +import com.typesafe.config.Config; +import com.typesafe.config.ConfigValueFactory; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.beam.runners.core.AssignWindows; import org.apache.beam.runners.gearpump.translators.TranslationContext; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.options.PipelineOptions; @@ -30,25 +37,18 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.AssignWindows; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; -import com.typesafe.config.Config; -import com.typesafe.config.ConfigValueFactory; - import org.apache.gearpump.cluster.ClusterConfig; import org.apache.gearpump.cluster.UserConfig; import org.apache.gearpump.cluster.client.ClientContext; import org.apache.gearpump.cluster.embedded.EmbeddedCluster; import org.apache.gearpump.streaming.dsl.javaapi.JavaStreamApp; -import java.util.HashMap; -import java.util.Map; - /** * A {@link PipelineRunner} that executes the operations in the * pipeline by first translating them to Gearpump Stream DSL @@ -79,16 +79,16 @@ public OutputT apply( PTransform transform, InputT input) { if (Window.Bound.class.equals(transform.getClass())) { return (OutputT) super.apply( - new AssignWindowsAndSetStrategy((Window.Bound) transform), input); + new AssignWindowsAndSetStrategy((Window.Bound) transform), input); } else if (Flatten.FlattenPCollectionList.class.equals(transform.getClass()) - && ((PCollectionList) input).size() == 0) { - return (OutputT) Pipeline.applyTransform(input, Create.of()); + && ((PCollectionList) input).size() == 0) { + return (OutputT) Pipeline.applyTransform(input.getPipeline().begin(), Create.of()); } else if (Create.Values.class.equals(transform.getClass())) { return (OutputT) PCollection - .createPrimitiveOutputInternal( - input.getPipeline(), - WindowingStrategy.globalDefault(), - PCollection.IsBounded.BOUNDED); + .createPrimitiveOutputInternal( + input.getPipeline(), + WindowingStrategy.globalDefault(), + PCollection.IsBounded.BOUNDED); } else { return super.apply(transform, input); } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunnerRegistrar.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunnerRegistrar.java index 2b9e89e6a4c4..ca173d10883e 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunnerRegistrar.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunnerRegistrar.java @@ -18,14 +18,14 @@ package org.apache.beam.runners.gearpump; +import com.google.auto.service.AutoService; +import com.google.common.collect.ImmutableList; + import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsRegistrar; import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.runners.PipelineRunnerRegistrar; -import com.google.auto.service.AutoService; -import com.google.common.collect.ImmutableList; - /** * Contains the {@link PipelineRunnerRegistrar} and {@link PipelineOptionsRegistrar} for the * {@link GearpumpPipelineRunner}. @@ -44,8 +44,7 @@ public static class Runner implements PipelineRunnerRegistrar { @Override public Iterable>> getPipelineRunners() { - return ImmutableList.>>of( - TestGearpumpRunner.class); + return ImmutableList.>>of(TestGearpumpRunner.class); } } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java index 59f0df7e8d3e..5045ae46e20b 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java @@ -18,6 +18,8 @@ package org.apache.beam.runners.gearpump; +import java.util.HashMap; +import java.util.Map; import org.apache.beam.runners.gearpump.translators.CreateValuesTranslator; import org.apache.beam.runners.gearpump.translators.FlattenPCollectionTranslator; import org.apache.beam.runners.gearpump.translators.GroupByKeyTranslator; @@ -41,9 +43,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.HashMap; -import java.util.Map; - /** * {@link GearpumpPipelineTranslator} knows how to translate {@link Pipeline} objects * into Gearpump {@link Graph}. @@ -109,7 +108,7 @@ public void visitPrimitiveTransform(TransformTreeNode node) { @Override public void visitValue(PValue value, TransformTreeNode producer) { - LOG.debug("visiting value {}", value); + LOG.info("visiting value {}", value); } /** diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/StreamingWordCount.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/StreamingWordCount.java index 5f35c6b155a6..ba50de79dda6 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/StreamingWordCount.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/StreamingWordCount.java @@ -23,11 +23,9 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; @@ -45,15 +43,9 @@ public class StreamingWordCount { static class ExtractWordsFn extends OldDoFn { - private final Aggregator emptyLines = - createAggregator("emptyLines", new Sum.SumLongFn()); @Override public void processElement(ProcessContext c) { - if (c.element().trim().isEmpty()) { - emptyLines.addValue(1L); - } - // Split the line into words. String[] words = c.element().split("[^a-zA-Z']+"); @@ -81,11 +73,12 @@ public void processElement(ProcessContext c) { public static void main(String[] args) { - GearpumpPipelineOptions options = PipelineOptionsFactory.fromArgs(args).withValidation() - .as(GearpumpPipelineOptions.class); - options.setApplicationName("StreamingWordCount"); + GearpumpPipelineOptions options = PipelineOptionsFactory + .fromArgs(args).as(GearpumpPipelineOptions.class); options.setRunner(GearpumpPipelineRunner.class); + options.setApplicationName("StreamingWordCount"); options.setParallelism(1); + Pipeline p = Pipeline.create(options); PCollection> wordCounts = diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/UnboundedTextSource.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/UnboundedTextSource.java index caf066c9710a..b01443273287 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/UnboundedTextSource.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/UnboundedTextSource.java @@ -18,13 +18,6 @@ package org.apache.beam.runners.gearpump.examples; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.io.UnboundedSource; -import org.apache.beam.sdk.options.PipelineOptions; - -import org.joda.time.Instant; - import java.io.IOException; import java.io.Serializable; import java.util.Collections; @@ -33,6 +26,14 @@ import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.io.UnboundedSource; +import org.apache.beam.sdk.options.PipelineOptions; + +import org.joda.time.Instant; + + /** * unbounded source that reads from text. */ diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java index f36b908f842d..43e3336b913a 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java @@ -18,23 +18,25 @@ package org.apache.beam.runners.gearpump.translators; +import com.google.common.collect.Iterables; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; + import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; -import com.google.common.collect.Iterables; - import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; import org.apache.gearpump.streaming.javaapi.dsl.functions.FlatMapFunction; import org.apache.gearpump.streaming.javaapi.dsl.functions.GroupByFunction; import org.apache.gearpump.streaming.javaapi.dsl.functions.MapFunction; import org.apache.gearpump.streaming.javaapi.dsl.functions.ReduceFunction; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; + /** * {@link GroupByKey} is translated to Gearpump groupBy function. diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java index d5ed0d2b06c1..2b49684c991d 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java @@ -18,6 +18,14 @@ package org.apache.beam.runners.gearpump.translators; +import com.google.common.collect.Lists; + +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +import org.apache.beam.runners.core.DoFnRunner; +import org.apache.beam.runners.core.DoFnRunners; import org.apache.beam.runners.gearpump.GearpumpPipelineOptions; import org.apache.beam.runners.gearpump.translators.utils.GearpumpDoFnRunner; import org.apache.beam.runners.gearpump.translators.utils.NoOpSideInputReader; @@ -25,8 +33,6 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.util.DoFnRunner; -import org.apache.beam.sdk.util.DoFnRunners; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; @@ -35,17 +41,11 @@ import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; -import com.google.common.collect.Lists; - import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; import org.apache.gearpump.streaming.javaapi.dsl.functions.FilterFunction; import org.apache.gearpump.streaming.javaapi.dsl.functions.FlatMapFunction; import org.apache.gearpump.streaming.javaapi.dsl.functions.MapFunction; -import java.util.Iterator; -import java.util.List; -import java.util.Map; - /** * {@link ParDo.BoundMulti} is translated to Gearpump flatMap function * with {@link DoFn} wrapped in {@link DoFnMultiFunction}. The outputs are diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TransformTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TransformTranslator.java index 1ed6d5daa28e..c8587d391429 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TransformTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TransformTranslator.java @@ -18,11 +18,10 @@ package org.apache.beam.runners.gearpump.translators; +import java.io.Serializable; import org.apache.beam.sdk.transforms.PTransform; -import java.io.Serializable; - /** * translates {@link PTransform} to Gearpump functions. */ diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java index b9b2c7aeeb91..d3bc75d39e7b 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java @@ -20,6 +20,9 @@ import static com.google.common.base.Preconditions.checkArgument; +import java.util.HashMap; +import java.util.Map; + import org.apache.beam.runners.gearpump.GearpumpPipelineOptions; import org.apache.beam.sdk.runners.TransformTreeNode; import org.apache.beam.sdk.transforms.AppliedPTransform; @@ -33,9 +36,6 @@ import org.apache.gearpump.streaming.dsl.javaapi.JavaStreamApp; import org.apache.gearpump.streaming.source.DataSource; -import java.util.HashMap; -import java.util.Map; - /** * Maintains context data for {@link TransformTranslator}s. */ diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java index b1ebd2a0d179..8d163569f00d 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java @@ -18,26 +18,26 @@ package org.apache.beam.runners.gearpump.translators.functions; +import com.google.common.collect.Lists; + +import java.util.Iterator; +import java.util.List; + +import org.apache.beam.runners.core.DoFnRunner; +import org.apache.beam.runners.core.DoFnRunners; import org.apache.beam.runners.gearpump.GearpumpPipelineOptions; import org.apache.beam.runners.gearpump.translators.utils.GearpumpDoFnRunner; import org.apache.beam.runners.gearpump.translators.utils.NoOpStepContext; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.OldDoFn; -import org.apache.beam.sdk.util.DoFnRunner; -import org.apache.beam.sdk.util.DoFnRunners; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; -import com.google.api.client.util.Lists; - import org.apache.gearpump.streaming.javaapi.dsl.functions.FlatMapFunction; -import java.util.Iterator; -import java.util.List; - /** * Gearpump {@link FlatMapFunction} wrapper over Beam {@link DoFn}. */ diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/BoundedSourceWrapper.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/BoundedSourceWrapper.java index f25d113e5c9d..f8891017688a 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/BoundedSourceWrapper.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/BoundedSourceWrapper.java @@ -18,12 +18,12 @@ package org.apache.beam.runners.gearpump.translators.io; +import java.io.IOException; + import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.Source; import org.apache.beam.sdk.options.PipelineOptions; -import java.io.IOException; - /** * wrapper over BoundedSource for Gearpump DataSource API. */ diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java index 892ccc3b39e9..8f2beb20195f 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java @@ -18,23 +18,23 @@ package org.apache.beam.runners.gearpump.translators.io; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; + +import java.io.IOException; + import org.apache.beam.sdk.io.Source; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowedValue; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectMapper; - import org.apache.gearpump.Message; import org.apache.gearpump.streaming.source.DataSource; import org.apache.gearpump.streaming.task.TaskContext; import org.joda.time.Instant; -import java.io.IOException; - /** * common methods for {@link BoundedSourceWrapper} and {@link UnboundedSourceWrapper}. */ @@ -61,6 +61,7 @@ public void open(TaskContext context, long startTime) { PipelineOptions options = new ObjectMapper() .readValue(serializedOptions, PipelineOptions.class); this.reader = createReader(options); + this.available = reader.start(); } catch (Exception e) { throw new RuntimeException(e); } finally { @@ -97,4 +98,5 @@ public void close() { throw new RuntimeException(e); } } + } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/UnboundedSourceWrapper.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/UnboundedSourceWrapper.java index b39f29f4219a..dfdecb2a21b4 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/UnboundedSourceWrapper.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/UnboundedSourceWrapper.java @@ -18,12 +18,12 @@ package org.apache.beam.runners.gearpump.translators.io; +import java.io.IOException; + import org.apache.beam.sdk.io.Source; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.options.PipelineOptions; -import java.io.IOException; - /** * wrapper over UnboundedSource for Gearpump DataSource API. */ diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java index 24055f7e4d0a..9359e35ac05c 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java @@ -18,12 +18,6 @@ package org.apache.beam.runners.gearpump.translators.io; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.io.UnboundedSource; -import org.apache.beam.sdk.options.PipelineOptions; - -import org.joda.time.Instant; - import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -36,6 +30,12 @@ import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.io.UnboundedSource; +import org.apache.beam.sdk.options.PipelineOptions; + +import org.joda.time.Instant; + /** * unbounded source that reads from a Java {@link Iterable}. */ diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/GearpumpDoFnRunner.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/GearpumpDoFnRunner.java index be0d0256290d..e2055759b604 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/GearpumpDoFnRunner.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/GearpumpDoFnRunner.java @@ -18,6 +18,20 @@ package org.apache.beam.runners.gearpump.translators.utils; +import com.google.common.base.Preconditions; +import com.google.common.collect.Iterables; +import com.google.common.collect.Sets; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.Set; + +import org.apache.beam.runners.core.DoFnRunner; +import org.apache.beam.runners.core.DoFnRunners; +import org.apache.beam.runners.core.SimpleDoFnRunner; import org.apache.beam.runners.gearpump.GearpumpPipelineOptions; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.IterableCoder; @@ -31,11 +45,8 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.DoFnRunner; -import org.apache.beam.sdk.util.DoFnRunners; import org.apache.beam.sdk.util.ExecutionContext; import org.apache.beam.sdk.util.SideInputReader; -import org.apache.beam.sdk.util.SimpleDoFnRunner; import org.apache.beam.sdk.util.SystemDoFnInternal; import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.UserCodeException; @@ -46,19 +57,8 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; -import com.google.common.base.Preconditions; -import com.google.common.collect.Iterables; -import com.google.common.collect.Sets; - import org.joda.time.Instant; -import java.io.IOException; -import java.io.Serializable; -import java.util.Collection; -import java.util.Iterator; -import java.util.List; -import java.util.Set; - /** * a serializable {@link SimpleDoFnRunner}. */ diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpSideInputReader.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpSideInputReader.java index 600ebfb225d0..d1a91988d962 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpSideInputReader.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpSideInputReader.java @@ -18,14 +18,14 @@ package org.apache.beam.runners.gearpump.translators.utils; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.SideInputReader; -import org.apache.beam.sdk.values.PCollectionView; - import java.io.Serializable; import javax.annotation.Nullable; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.SideInputReader; +import org.apache.beam.sdk.values.PCollectionView; + /** * no-op side input reader. */ diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpStepContext.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpStepContext.java index ce0935a38081..45f146b7d965 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpStepContext.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpStepContext.java @@ -18,6 +18,9 @@ package org.apache.beam.runners.gearpump.translators.utils; +import java.io.IOException; +import java.io.Serializable; + import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.ExecutionContext; @@ -26,9 +29,6 @@ import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.values.TupleTag; -import java.io.IOException; -import java.io.Serializable; - /** * serializable {@link ExecutionContext.StepContext} that basically does nothing. */ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java index 53f46f6f06ab..e95304dc81e2 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java @@ -134,7 +134,7 @@ public PipelineExecutionException(Throwable cause) { */ public static Pipeline create(PipelineOptions options) { Pipeline pipeline = new Pipeline(PipelineRunner.fromOptions(options), options); - LOG.debug("Creating {}", pipeline); + LOG.info("Creating {}", pipeline); return pipeline; } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunner.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunner.java index ede1507853a1..1ec4103fe6d2 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunner.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunner.java @@ -57,6 +57,7 @@ public static PipelineRunner fromOptions(PipelineOptio .fromFactoryMethod("fromOptions") .withArg(PipelineOptions.class, options) .build(); + System.out.println("runner: " + result.getClass().getName()); return result; } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java index 4803d77f57f1..642971f8c451 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.transforms; import java.io.IOException; + import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; @@ -31,6 +32,8 @@ import org.apache.beam.sdk.values.TypeDescriptor; import org.joda.time.Duration; import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Utility class containing adapters for running a {@link DoFn} as an {@link OldDoFn}. @@ -72,6 +75,8 @@ public static OldDoFn toOldDoFn(DoFn extends OldDoFn { private final DoFn fn; private transient DoFnInvoker invoker; + private static final Logger LOG = + LoggerFactory.getLogger(SimpleDoFnAdapter.class); SimpleDoFnAdapter(DoFn fn) { super(fn.aggregators);