@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); }
/** * Continuously watches for new files matching the filepattern, polling it at the given * interval, until the given termination condition is reached. The returned {@link PCollection} * is unbounded. * * <p>This works only in runners supporting {@link Kind#SPLITTABLE_DO_FN}. */ @Experimental(Kind.SPLITTABLE_DO_FN) public Read<T> watchForNewFiles( Duration pollInterval, TerminationCondition<String, ?> terminationCondition) { return withMatchConfiguration( getMatchConfiguration().continuously(pollInterval, terminationCondition)); }
/** Configures whether or not a filepattern matching no files is allowed. */ public Read<T> withEmptyMatchTreatment(EmptyMatchTreatment treatment) { return withMatchConfiguration(getMatchConfiguration().withEmptyMatchTreatment(treatment)); }
@Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder .addIfNotNull( DisplayData.item("filePattern", getFilepattern()).withLabel("Input File Pattern")) .include("matchConfiguration", getMatchConfiguration()); }