1818import static com .google .common .base .Preconditions .checkState ;
1919
2020import com .google .cloud .dataflow .sdk .options .PipelineOptions ;
21+ import com .google .cloud .dataflow .sdk .options .ValueProvider ;
22+ import com .google .cloud .dataflow .sdk .options .ValueProvider .StaticValueProvider ;
2123import com .google .cloud .dataflow .sdk .transforms .display .DisplayData ;
2224import com .google .cloud .dataflow .sdk .util .IOChannelFactory ;
2325import com .google .cloud .dataflow .sdk .util .IOChannelUtils ;
@@ -73,7 +75,7 @@ public abstract class FileBasedSource<T> extends OffsetBasedSource<T> {
7375 // Package-private for testing.
7476 static final int THREAD_POOL_SIZE = 128 ;
7577
76- private final String fileOrPatternSpec ;
78+ private final ValueProvider < String > fileOrPatternSpec ;
7779 private final Mode mode ;
7880
7981 /**
@@ -95,6 +97,16 @@ public enum Mode {
9597 * @param minBundleSize minimum bundle size in bytes.
9698 */
9799 public FileBasedSource (String fileOrPatternSpec , long minBundleSize ) {
100+ this (StaticValueProvider .of (fileOrPatternSpec ), minBundleSize );
101+ }
102+
103+
104+ /**
105+ * Create a {@code FileBaseSource} based on a file or a file pattern specification.
106+ * Same as the {@code String} constructor, but accepting a {@link ValueProvider}
107+ * to allow for runtime configuration of the source.
108+ */
109+ public FileBasedSource (ValueProvider <String > fileOrPatternSpec , long minBundleSize ) {
98110 super (0 , Long .MAX_VALUE , minBundleSize );
99111 mode = Mode .FILEPATTERN ;
100112 this .fileOrPatternSpec = fileOrPatternSpec ;
@@ -120,10 +132,14 @@ public FileBasedSource(String fileName, long minBundleSize,
120132 long startOffset , long endOffset ) {
121133 super (startOffset , endOffset , minBundleSize );
122134 mode = Mode .SINGLE_FILE_OR_SUBRANGE ;
123- this .fileOrPatternSpec = fileName ;
135+ this .fileOrPatternSpec = StaticValueProvider . of ( fileName ) ;
124136 }
125137
126138 public final String getFileOrPatternSpec () {
139+ return fileOrPatternSpec .get ();
140+ }
141+
142+ public final ValueProvider <String > getFileOrPatternSpecProvider () {
127143 return fileOrPatternSpec ;
128144 }
129145
@@ -142,7 +158,9 @@ public final FileBasedSource<T> createSourceForSubrange(long start, long end) {
142158 + " of the subrange cannot be larger than the end offset value " + getEndOffset ()
143159 + " of the parent source" );
144160
145- FileBasedSource <T > source = createForSubrangeOfFile (fileOrPatternSpec , start , end );
161+ checkState (fileOrPatternSpec .isAccessible (),
162+ "Subrange creation should only happen at execution time." );
163+ FileBasedSource <T > source = createForSubrangeOfFile (fileOrPatternSpec .get (), start , end );
146164 if (start > 0 || end != Long .MAX_VALUE ) {
147165 checkArgument (source .getMode () == Mode .SINGLE_FILE_OR_SUBRANGE ,
148166 "Source created for the range [" + start + "," + end + ")"
@@ -180,19 +198,21 @@ public final long getEstimatedSizeBytes(PipelineOptions options) throws Exceptio
180198 // we perform the size estimation of files and file patterns using the interface provided by
181199 // IOChannelFactory.
182200
183- IOChannelFactory factory = IOChannelUtils .getFactory (fileOrPatternSpec );
184201 if (mode == Mode .FILEPATTERN ) {
202+ checkState (fileOrPatternSpec .isAccessible (),
203+ "Size estimation should be done at execution time." );
204+ IOChannelFactory factory = IOChannelUtils .getFactory (fileOrPatternSpec .get ());
185205 // TODO Implement a more efficient parallel/batch size estimation mechanism for file patterns.
186206 long startTime = System .currentTimeMillis ();
187207 long totalSize = 0 ;
188- Collection <String > inputs = factory .match (fileOrPatternSpec );
208+ Collection <String > inputs = factory .match (fileOrPatternSpec . get () );
189209 if (inputs .size () <= MAX_NUMBER_OF_FILES_FOR_AN_EXACT_STAT ) {
190210 totalSize = getExactTotalSizeOfFiles (inputs , factory );
191- LOG .debug ("Size estimation of all files of pattern " + fileOrPatternSpec + " took "
211+ LOG .debug ("Size estimation of all files of pattern " + fileOrPatternSpec . get () + " took "
192212 + (System .currentTimeMillis () - startTime ) + " ms" );
193213 } else {
194214 totalSize = getEstimatedSizeOfFilesBySampling (inputs , factory );
195- LOG .debug ("Size estimation of pattern " + fileOrPatternSpec + " by sampling took "
215+ LOG .debug ("Size estimation of pattern " + fileOrPatternSpec . get () + " by sampling took "
196216 + (System .currentTimeMillis () - startTime ) + " ms" );
197217 }
198218 return totalSize ;
@@ -261,7 +281,7 @@ private static long getEstimatedSizeOfFilesBySampling(
261281 @ Override
262282 public void populateDisplayData (DisplayData .Builder builder ) {
263283 super .populateDisplayData (builder );
264- builder .add (DisplayData .item ("filePattern" , getFileOrPatternSpec ())
284+ builder .add (DisplayData .item ("filePattern" , getFileOrPatternSpecProvider ())
265285 .withLabel ("File Pattern" ));
266286 }
267287
@@ -294,7 +314,9 @@ public final List<? extends FileBasedSource<T>> splitIntoBundles(
294314 ListeningExecutorService service =
295315 MoreExecutors .listeningDecorator (Executors .newFixedThreadPool (THREAD_POOL_SIZE ));
296316 try {
297- for (final String file : FileBasedSource .expandFilePattern (fileOrPatternSpec )) {
317+ checkState (fileOrPatternSpec .isAccessible (),
318+ "Bundle splitting should only happen at execution time." );
319+ for (final String file : FileBasedSource .expandFilePattern (fileOrPatternSpec .get ())) {
298320 futures .add (createFutureForFileSplit (file , desiredBundleSizeBytes , options , service ));
299321 }
300322 List <? extends FileBasedSource <T >> splitResults =
@@ -334,8 +356,10 @@ protected boolean isSplittable() throws Exception {
334356 // We split a file-based source into subranges only if the file is efficiently seekable.
335357 // If a file is not efficiently seekable it would be highly inefficient to create and read a
336358 // source based on a subrange of that file.
337- IOChannelFactory factory = IOChannelUtils .getFactory (fileOrPatternSpec );
338- return factory .isReadSeekEfficient (fileOrPatternSpec );
359+ checkState (fileOrPatternSpec .isAccessible (),
360+ "isSplittable should only be called at runtime." );
361+ IOChannelFactory factory = IOChannelUtils .getFactory (fileOrPatternSpec .get ());
362+ return factory .isReadSeekEfficient (fileOrPatternSpec .get ());
339363 }
340364
341365 @ Override
@@ -345,7 +369,7 @@ public final BoundedReader<T> createReader(PipelineOptions options) throws IOExc
345369
346370 if (mode == Mode .FILEPATTERN ) {
347371 long startTime = System .currentTimeMillis ();
348- Collection <String > files = FileBasedSource .expandFilePattern (fileOrPatternSpec );
372+ Collection <String > files = FileBasedSource .expandFilePattern (fileOrPatternSpec . get () );
349373 List <FileBasedReader <T >> fileReaders = new ArrayList <>();
350374 for (String fileName : files ) {
351375 long endOffset ;
@@ -373,9 +397,9 @@ public final BoundedReader<T> createReader(PipelineOptions options) throws IOExc
373397 public String toString () {
374398 switch (mode ) {
375399 case FILEPATTERN :
376- return fileOrPatternSpec ;
400+ return fileOrPatternSpec . toString () ;
377401 case SINGLE_FILE_OR_SUBRANGE :
378- return fileOrPatternSpec + " range " + super .toString ();
402+ return fileOrPatternSpec . toString () + " range " + super .toString ();
379403 default :
380404 throw new IllegalStateException ("Unexpected mode: " + mode );
381405 }
@@ -407,8 +431,8 @@ public final long getMaxEndOffset(PipelineOptions options) throws Exception {
407431 throw new IllegalArgumentException ("Cannot determine the exact end offset of a file pattern" );
408432 }
409433 if (getEndOffset () == Long .MAX_VALUE ) {
410- IOChannelFactory factory = IOChannelUtils .getFactory (fileOrPatternSpec );
411- return factory .getSizeBytes (fileOrPatternSpec );
434+ IOChannelFactory factory = IOChannelUtils .getFactory (fileOrPatternSpec . get () );
435+ return factory .getSizeBytes (fileOrPatternSpec . get () );
412436 } else {
413437 return getEndOffset ();
414438 }
@@ -480,8 +504,9 @@ public synchronized FileBasedSource<T> getCurrentSource() {
480504 @ Override
481505 protected final boolean startImpl () throws IOException {
482506 FileBasedSource <T > source = getCurrentSource ();
483- IOChannelFactory factory = IOChannelUtils .getFactory (source .getFileOrPatternSpec ());
484- this .channel = factory .open (source .getFileOrPatternSpec ());
507+ IOChannelFactory factory = IOChannelUtils .getFactory (
508+ source .getFileOrPatternSpecProvider ().get ());
509+ this .channel = factory .open (source .getFileOrPatternSpecProvider ().get ());
485510
486511 if (channel instanceof SeekableByteChannel ) {
487512 SeekableByteChannel seekChannel = (SeekableByteChannel ) channel ;
0 commit comments