@Override public PCollection<MatchResult.Metadata> expand(PBegin input) { return input .apply("Create filepattern", Create.ofProvider(getFilepattern(), StringUtf8Coder.of())) .apply("Via MatchAll", matchAll().withConfiguration(getConfiguration())); } }
@Test @Category(NeedsRunner.class) public void testCreateOfProvider() throws Exception { PAssert.that( p.apply( "Static", Create.ofProvider(StaticValueProvider.of("foo"), StringUtf8Coder.of()))) .containsInAnyOrder("foo"); PAssert.that( p.apply( "Static nested", Create.ofProvider( NestedValueProvider.of(StaticValueProvider.of("foo"), input -> input + "bar"), StringUtf8Coder.of()))) .containsInAnyOrder("foobar"); PAssert.that( p.apply( "Runtime", Create.ofProvider(p.newProvider("runtimeFoo"), StringUtf8Coder.of()))) .containsInAnyOrder("runtimeFoo"); p.run(); }
inputQuery = input .apply(Create.ofProvider(getLiteralGqlQuery(), StringUtf8Coder.of())) .apply(ParDo.of(new GqlQueryTranslateFn(v1Options)));
@Test @Category(NeedsRunner.class) public void testNewProvider() { ValueProvider<String> foo = pipeline.newProvider("foo"); ValueProvider<String> foobar = ValueProvider.NestedValueProvider.of(foo, input -> input + "bar"); assertFalse(foo.isAccessible()); assertFalse(foobar.isAccessible()); PAssert.that(pipeline.apply("create foo", Create.ofProvider(foo, StringUtf8Coder.of()))) .containsInAnyOrder("foo"); PAssert.that(pipeline.apply("create foobar", Create.ofProvider(foobar, StringUtf8Coder.of()))) .containsInAnyOrder("foobar"); pipeline.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())); }
@Override public PCollection<T> expand(PBegin input) { checkNotNull(getFilepattern(), "filepattern"); Coder<T> coder = inferCoder(getCoder(), getParseFn(), input.getPipeline().getCoderRegistry()); if (getMatchConfiguration().getWatchInterval() == null && !getHintMatchesManyFiles()) { return input.apply( org.apache.beam.sdk.io.Read.from( AvroSource.from(getFilepattern()).withParseFn(getParseFn(), coder))); } // All other cases go through ParseAllGenericRecords. return input .apply("Create filepattern", Create.ofProvider(getFilepattern(), StringUtf8Coder.of())) .apply( "Via ParseAll", parseAllGenericRecords(getParseFn()) .withCoder(coder) .withMatchConfiguration(getMatchConfiguration())); }
@Override public PCollection<T> expand(PBegin input) { checkNotNull(getFilepattern(), "filepattern"); checkNotNull(getSchema(), "schema"); if (getMatchConfiguration().getWatchInterval() == null && !getHintMatchesManyFiles()) { return input.apply( "Read", org.apache.beam.sdk.io.Read.from( createSource( getFilepattern(), getMatchConfiguration().getEmptyMatchTreatment(), getRecordClass(), getSchema()))); } // All other cases go through ReadAll. ReadAll<T> readAll = (getRecordClass() == GenericRecord.class) ? (ReadAll<T>) readAllGenericRecords(getSchema()) : readAll(getRecordClass()); readAll = readAll.withMatchConfiguration(getMatchConfiguration()); return input .apply("Create filepattern", Create.ofProvider(getFilepattern(), StringUtf8Coder.of())) .apply("Via ReadAll", readAll); }