tomstepp commented on code in PR #37463:
URL: https://github.com/apache/beam/pull/37463#discussion_r2755585476
##########
sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileIOTest.java:
##########
@@ -547,4 +565,77 @@ public void testFileIoDynamicNaming() throws IOException {
"Output file shard 0 exists after pipeline completes",
new File(outputFileName + "-0").exists());
}
+
+ @Test
+ @Category({NeedsRunner.class, UsesUnboundedPCollections.class})
+ public void testWriteUnboundedWithCustomBatchParameters() throws IOException
{
+ File root = tmpFolder.getRoot();
+ List<String> inputs = Arrays.asList("one", "two", "three", "four", "five",
"six");
+
+ PTransform<PCollection<String>, PCollection<String>> transform =
+ Window.<String>into(FixedWindows.of(Duration.standardSeconds(10)))
+ .triggering(AfterWatermark.pastEndOfWindow())
+ .withAllowedLateness(Duration.ZERO)
+ .discardingFiredPanes();
+
+ FileIO.Write<Void, String> write =
+ FileIO.<String>write()
+ .via(TextIO.sink())
+ .to(root.getAbsolutePath())
+ .withPrefix("output")
+ .withSuffix(".txt")
+ .withAutoSharding()
+ .withBatchSize(CUSTOM_FILE_TRIGGERING_RECORD_COUNT)
+ .withBatchSizeBytes(CUSTOM_FILE_TRIGGERING_BYTE_COUNT)
+
.withBatchMaxBufferingDuration(CUSTOM_FILE_TRIGGERING_RECORD_BUFFERING_DURATION);
+
+ // Prepare timestamps for the elements.
+ List<Long> timestamps = new ArrayList<>();
+ for (long i = 0; i < inputs.size(); i++) {
+ timestamps.add(i + 1);
+ }
+
+ p.apply(Create.timestamped(inputs,
timestamps).withCoder(StringUtf8Coder.of()))
+ .setIsBoundedInternal(IsBounded.UNBOUNDED)
+ .apply(transform)
+ .apply(write);
+ p.run().waitUntilFinish();
+
+ // Verify that the custom batch parameters are set.
+ assertEquals(CUSTOM_FILE_TRIGGERING_RECORD_COUNT,
write.getBatchSize().intValue());
+ assertEquals(CUSTOM_FILE_TRIGGERING_BYTE_COUNT,
write.getBatchSizeBytes().intValue());
+ assertEquals(
+ CUSTOM_FILE_TRIGGERING_RECORD_BUFFERING_DURATION,
write.getBatchMaxBufferingDuration());
+
+ checkFileContents(root, "output", inputs);
Review Comment:
Could we check that the limits are propagated and used in processing? For
example we could set a small BatchSize value (such as 3), and verify two
batches/files are created instead of one.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]