@Test public void testInitialSplitAutoModeGz() throws Exception { PipelineOptions options = TestPipeline.testingPipelineOptions(); long desiredBundleSize = 1000; File largeGz = writeToFile(LARGE, tempFolder, "large.gz", GZIP); // Sanity check: file is at least 2 bundles long. assertThat(largeGz.length(), greaterThan(2 * desiredBundleSize)); FileBasedSource<String> source = TextIO.read().from(largeGz.getPath()).getSource(); List<? extends FileBasedSource<String>> splits = source.split(desiredBundleSize, options); // Exactly 1 split, even in AUTO mode, since it is a gzip file. assertThat(splits, hasSize(equalTo(1))); SourceTestUtils.assertSourcesEqualReferenceSource(source, splits, options); }
@Test public void testInitialSplitAutoModeTxt() throws Exception { PipelineOptions options = TestPipeline.testingPipelineOptions(); long desiredBundleSize = 1000; File largeTxt = writeToFile(LARGE, tempFolder, "large.txt", UNCOMPRESSED); // Sanity check: file is at least 2 bundles long. assertThat(largeTxt.length(), greaterThan(2 * desiredBundleSize)); FileBasedSource<String> source = TextIO.read().from(largeTxt.getPath()).getSource(); List<? extends FileBasedSource<String>> splits = source.split(desiredBundleSize, options); // At least 2 splits and they are equal to reading the whole file. assertThat(splits, hasSize(greaterThan(1))); SourceTestUtils.assertSourcesEqualReferenceSource(source, splits, options); }
@Test public void testInitialSplitGzipModeTxt() throws Exception { PipelineOptions options = TestPipeline.testingPipelineOptions(); long desiredBundleSize = 1000; File largeTxt = writeToFile(LARGE, tempFolder, "large.txt", UNCOMPRESSED); // Sanity check: file is at least 2 bundles long. assertThat(largeTxt.length(), greaterThan(2 * desiredBundleSize)); FileBasedSource<String> source = TextIO.read().from(largeTxt.getPath()).withCompression(GZIP).getSource(); List<? extends FileBasedSource<String>> splits = source.split(desiredBundleSize, options); // Exactly 1 split, even though splittable text file, since using GZIP mode. assertThat(splits, hasSize(equalTo(1))); SourceTestUtils.assertSourcesEqualReferenceSource(source, splits, options); }
@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())); }