/** * Specifies to ignore windowing information in the input, and instead rewindow it to global * window with the default trigger. * * @deprecated Avoid usage of this method: its effects are complex and it will be removed in * future versions of Beam. Right now it exists for compatibility with {@link WriteFiles}. */ @Deprecated public Write<DestinationT, UserT> withIgnoreWindowing() { return toBuilder().setIgnoreWindowing(true).build(); }
/** * Like {@link #withNumShards(int)}. Specifying {@code null} means runner-determined sharding. */ public Write<DestinationT, UserT> withNumShards(@Nullable ValueProvider<Integer> numShards) { return toBuilder().setNumShards(numShards).build(); }
resolvedSpec.setIgnoreWindowing(getIgnoreWindowing()); Write<DestinationT, UserT> resolved = resolvedSpec.build(); WriteFiles<UserT, DestinationT, ?> writeFiles = WriteFiles.to(new ViaFileBasedSink<>(resolved))
/** * Specifies a {@link Coder} for the destination type, if it can not be inferred from {@link * #by}. */ public Write<DestinationT, UserT> withDestinationCoder(Coder<DestinationT> destinationCoder) { checkArgument(destinationCoder != null, "destinationCoder can not be null"); return toBuilder().setDestinationCoder(destinationCoder).build(); }
/** * If {@link #withIgnoreWindowing()} is specified, specifies a destination to be used in case * the collection is empty, to generate the (only, empty) output file. */ public Write<DestinationT, UserT> withEmptyGlobalWindowDestination( DestinationT emptyWindowDestination) { return toBuilder().setEmptyWindowDestination(emptyWindowDestination).build(); }
/** Like {@link #withSuffix(String)} but with a {@link ValueProvider}. */ public Write<DestinationT, UserT> withSuffix(ValueProvider<String> suffix) { checkArgument(suffix != null, "suffix can not be null"); return toBuilder().setFilenameSuffix(suffix).build(); }
/** * Like {@link #via(Contextful, Contextful)}, but the output type of the sink is the same as the * type of the input collection. The sink function must create a new {@link Sink} instance every * time it is called. */ public Write<DestinationT, UserT> via(Contextful<Fn<DestinationT, Sink<UserT>>> sinkFn) { checkArgument(sinkFn != null, "sinkFn can not be null"); return toBuilder() .setSinkFn((Contextful) sinkFn) .setOutputFn(fn(SerializableFunctions.<UserT>identity())) .build(); }
/** * Writes elements to files using a {@link Sink} and grouping the elements using "dynamic * destinations". See class-level documentation. */ public static <DestT, InputT> Write<DestT, InputT> writeDynamic() { return new AutoValue_FileIO_Write.Builder<DestT, InputT>() .setDynamic(true) .setCompression(Compression.UNCOMPRESSED) .setIgnoreWindowing(false) .build(); }
/** * Specifies a custom strategy for generating filenames. All generated filenames will be * resolved relative to the directory specified in {@link #to}, if any. * * <p>Incompatible with {@link #withSuffix}. * * <p>This can only be used in combination with {@link #write()} but not {@link * #writeDynamic()}. */ public Write<DestinationT, UserT> withNaming(FileNaming naming) { checkArgument(naming != null, "naming can not be null"); return toBuilder().setConstantFileNaming(naming).build(); }
/** * Like {@link #withNaming(SerializableFunction)} but allows accessing context, such as side * inputs, from the function. */ public Write<DestinationT, UserT> withNaming( Contextful<Fn<DestinationT, FileNaming>> namingFn) { checkArgument(namingFn != null, "namingFn can not be null"); return toBuilder().setFileNamingFn(namingFn).build(); }
/** Like {@link #withPrefix(String)} but with a {@link ValueProvider}. */ public Write<DestinationT, UserT> withPrefix(ValueProvider<String> prefix) { checkArgument(prefix != null, "prefix can not be null"); return toBuilder().setFilenamePrefix(prefix).build(); }
/** * Specifies how to create a {@link Sink} for a particular destination and how to map the * element type to the sink's output type. The sink function must create a new {@link Sink} * instance every time it is called. */ public <OutputT> Write<DestinationT, UserT> via( Contextful<Fn<UserT, OutputT>> outputFn, Contextful<Fn<DestinationT, Sink<OutputT>>> sinkFn) { checkArgument(sinkFn != null, "sinkFn can not be null"); checkArgument(outputFn != null, "outputFn can not be null"); return toBuilder().setSinkFn((Contextful) sinkFn).setOutputFn(outputFn).build(); }
/** Writes elements to files using a {@link Sink}. See class-level documentation. */ public static <InputT> Write<Void, InputT> write() { return new AutoValue_FileIO_Write.Builder<Void, InputT>() .setDynamic(false) .setCompression(Compression.UNCOMPRESSED) .setIgnoreWindowing(false) .build(); }
/** * Specifies a {@link PTransform} to use for computing the desired number of shards in each * window. */ public Write<DestinationT, UserT> withSharding( PTransform<PCollection<UserT>, PCollectionView<Integer>> sharding) { checkArgument(sharding != null, "sharding can not be null"); return toBuilder().setSharding(sharding).build(); }
/** Like {@link #withTempDirectory(String)}. */ public Write<DestinationT, UserT> withTempDirectory(ValueProvider<String> tempDirectory) { checkArgument(tempDirectory != null, "tempDirectory can not be null"); return toBuilder().setTempDirectory(tempDirectory).build(); }
/** Like {@link #by}, but with access to context such as side inputs. */ public Write<DestinationT, UserT> by(Contextful<Fn<UserT, DestinationT>> destinationFn) { checkArgument(destinationFn != null, "destinationFn can not be null"); return toBuilder().setDestinationFn(destinationFn).build(); }
/** Like {@link #to(String)} but with a {@link ValueProvider}. */ public Write<DestinationT, UserT> to(ValueProvider<String> directory) { checkArgument(directory != null, "directory can not be null"); return toBuilder().setOutputDirectory(directory).build(); }
/** * Specifies to compress all generated shard files using the given {@link Compression} and, by * default, append the respective extension to the filename. */ public Write<DestinationT, UserT> withCompression(Compression compression) { checkArgument(compression != null, "compression can not be null"); checkArgument( compression != Compression.AUTO, "AUTO compression is not supported for writing"); return toBuilder().setCompression(compression).build(); }