@Test @Category(NeedsRunner.class) public void testReadAll() throws IOException { Path tempFolderPath = tempFolder.getRoot().toPath(); writeToFile(TINY, tempFolder, "readAllTiny1.zip", ZIP); writeToFile(TINY, tempFolder, "readAllTiny2.txt", UNCOMPRESSED); writeToFile(LARGE, tempFolder, "readAllLarge1.zip", ZIP); writeToFile(LARGE, tempFolder, "readAllLarge2.txt", UNCOMPRESSED); PCollection<String> lines = p.apply( Create.of( tempFolderPath.resolve("readAllTiny*").toString(), tempFolderPath.resolve("readAllLarge*").toString())) .apply(TextIO.readAll().withCompression(AUTO)); PAssert.that(lines).containsInAnyOrder(Iterables.concat(TINY, TINY, LARGE, LARGE)); p.run(); }
@Override public PCollection<String> expand(PBegin input) { checkNotNull(getFilepattern(), "need to set the filepattern of a TextIO.Read transform"); if (getMatchConfiguration().getWatchInterval() == null && !getHintMatchesManyFiles()) { return input.apply("Read", org.apache.beam.sdk.io.Read.from(getSource())); } // All other cases go through ReadAll. return input .apply("Create filepattern", Create.ofProvider(getFilepattern(), StringUtf8Coder.of())) .apply( "Via ReadAll", readAll() .withCompression(getCompression()) .withMatchConfiguration(getMatchConfiguration()) .withDelimiter(getDelimiter())); }
/** * Helper method that runs a variety of ways to read a single file using TextIO and checks that * they all match the given expected output. * * <p>The transforms being verified are: * * <ul> * <li>TextIO.read().from(filename).withCompression(compressionType) * <li>TextIO.read().from(filename).withCompression(compressionType) .withHintMatchesManyFiles() * <li>TextIO.readAll().withCompression(compressionType) * </ul> */ private static void assertReadingCompressedFileMatchesExpected( File file, Compression compression, List<String> expected, Pipeline p) { TextIO.Read read = TextIO.read().from(file.getPath()).withCompression(compression); PAssert.that(p.apply("Read_" + file + "_" + compression.toString(), read)) .containsInAnyOrder(expected); PAssert.that( p.apply( "Read_" + file + "_" + compression.toString() + "_many", read.withHintMatchesManyFiles())) .containsInAnyOrder(expected); TextIO.ReadAll readAll = TextIO.readAll().withCompression(compression); PAssert.that( p.apply("Create_" + file, Create.of(file.getPath())) .apply("Read_" + compression.toString(), readAll)) .containsInAnyOrder(expected); }
@Test @Category(NeedsRunner.class) public void testWriteViaSink() throws Exception { List<String> data = ImmutableList.of("a", "b", "c", "d", "e", "f"); PAssert.that( p.apply(Create.of(data)) .apply( FileIO.<String>write() .to(tempFolder.getRoot().toString()) .withSuffix(".txt") .via(TextIO.sink()) .withIgnoreWindowing()) .getPerDestinationOutputFilenames() .apply(Values.create()) .apply(TextIO.readAll())) .containsInAnyOrder(data); p.run(); }
@Test @Category(NeedsRunner.class) public void testWindowedWritesWithOnceTrigger() throws Throwable { // Tests for https://issues.apache.org/jira/browse/BEAM-3169 PCollection<String> data = p.apply(Create.of("0", "1", "2")) .apply( Window.<String>into(FixedWindows.of(Duration.standardSeconds(1))) // According to this trigger, all data should be written. // However, the continuation of this trigger is elementCountAtLeast(1), // so with a buggy implementation that used a GBK before renaming files, // only 1 file would be renamed. .triggering(AfterPane.elementCountAtLeast(3)) .withAllowedLateness(Duration.standardMinutes(1)) .discardingFiredPanes()); PCollection<String> filenames = data.apply( TextIO.write() .to(new File(tempFolder.getRoot(), "windowed-writes").getAbsolutePath()) .withNumShards(2) .withWindowedWrites() .<Void>withOutputFilenames()) .getPerDestinationOutputFilenames() .apply(Values.create()); PAssert.that(filenames.apply(TextIO.readAll())).containsInAnyOrder("0", "1", "2"); p.run(); }