Class TextIO.TypedWrite<UserT,DestinationT>
- java.lang.Object
-
- org.apache.beam.sdk.transforms.PTransform<PCollection<UserT>,WriteFilesResult<DestinationT>>
-
- org.apache.beam.sdk.io.TextIO.TypedWrite<UserT,DestinationT>
-
- All Implemented Interfaces:
java.io.Serializable
,HasDisplayData
- Enclosing class:
- TextIO
public abstract static class TextIO.TypedWrite<UserT,DestinationT> extends PTransform<PCollection<UserT>,WriteFilesResult<DestinationT>>
Implementation ofTextIO.write()
.- See Also:
- Serialized Form
-
-
Field Summary
-
Fields inherited from class org.apache.beam.sdk.transforms.PTransform
name, resourceHints
-
-
Constructor Summary
Constructors Constructor Description TypedWrite()
-
Method Summary
All Methods Instance Methods Concrete Methods Deprecated Methods Modifier and Type Method Description WriteFilesResult<DestinationT>
expand(PCollection<UserT> input)
Override this method to specify how thisPTransform
should be expanded on the givenInputT
.void
populateDisplayData(DisplayData.Builder builder)
Register display data for the given transform or component.TextIO.TypedWrite<UserT,DestinationT>
skipIfEmpty()
Don't write any output files if the PCollection is empty.TextIO.TypedWrite<UserT,DestinationT>
to(java.lang.String filenamePrefix)
Writes to text files with the given prefix.<NewDestinationT>
TextIO.TypedWrite<UserT,NewDestinationT>to(FileBasedSink.DynamicDestinations<UserT,NewDestinationT,java.lang.String> dynamicDestinations)
Deprecated.TextIO.TypedWrite<UserT,DestinationT>
to(FileBasedSink.FilenamePolicy filenamePolicy)
Writes to files named according to the givenFileBasedSink.FilenamePolicy
.TextIO.TypedWrite<UserT,DestinationT>
to(ResourceId filenamePrefix)
Liketo(String)
.TextIO.TypedWrite<UserT,DestinationT>
to(ValueProvider<java.lang.String> outputPrefix)
Liketo(String)
.TextIO.TypedWrite<UserT,DefaultFilenamePolicy.Params>
to(SerializableFunction<UserT,DefaultFilenamePolicy.Params> destinationFunction, DefaultFilenamePolicy.Params emptyDestination)
Deprecated.TextIO.TypedWrite<UserT,DestinationT>
toResource(ValueProvider<ResourceId> filenamePrefix)
Liketo(ResourceId)
.TextIO.TypedWrite<UserT,DestinationT>
withCompression(Compression compression)
Returns a transform for writing to text files like this one but that compresses output using the givenCompression
.TextIO.TypedWrite<UserT,DestinationT>
withDelimiter(char[] delimiter)
Specifies the delimiter after each string written.TextIO.TypedWrite<UserT,DestinationT>
withFooter(@Nullable java.lang.String footer)
Adds a footer string to each file.TextIO.TypedWrite<UserT,DestinationT>
withFormatFunction(@Nullable SerializableFunction<UserT,java.lang.String> formatFunction)
Deprecated.TextIO.TypedWrite<UserT,DestinationT>
withHeader(@Nullable java.lang.String header)
Adds a header string to each file.TextIO.TypedWrite<UserT,DestinationT>
withNoSpilling()
TextIO.TypedWrite<UserT,DestinationT>
withNumShards(int numShards)
Configures the number of output shards produced overall (when using unwindowed writes) or per-window (when using windowed writes).TextIO.TypedWrite<UserT,DestinationT>
withNumShards(@Nullable ValueProvider<java.lang.Integer> numShards)
LikewithNumShards(int)
.TextIO.TypedWrite<UserT,DestinationT>
withoutSharding()
Forces a single file as output and empty shard name template.TextIO.TypedWrite<UserT,DestinationT>
withShardNameTemplate(java.lang.String shardTemplate)
Uses the givenShardNameTemplate
for naming output files.TextIO.TypedWrite<UserT,DestinationT>
withSuffix(java.lang.String filenameSuffix)
Configures the filename suffix for written files.TextIO.TypedWrite<UserT,DestinationT>
withTempDirectory(ResourceId tempDirectory)
Set the base directory used to generate temporary files.TextIO.TypedWrite<UserT,DestinationT>
withTempDirectory(ValueProvider<ResourceId> tempDirectory)
Set the base directory used to generate temporary files.TextIO.TypedWrite<UserT,DestinationT>
withWindowedWrites()
Preserves windowing of input elements and writes them to files based on the element's window.TextIO.TypedWrite<UserT,DestinationT>
withWritableByteChannelFactory(FileBasedSink.WritableByteChannelFactory writableByteChannelFactory)
Returns a transform for writing to text files like this one but that has the givenFileBasedSink.WritableByteChannelFactory
to be used by theFileBasedSink
during output.-
Methods inherited from class org.apache.beam.sdk.transforms.PTransform
compose, compose, getAdditionalInputs, getDefaultOutputCoder, getDefaultOutputCoder, getDefaultOutputCoder, getKindString, getName, getResourceHints, setResourceHints, toString, validate, validate
-
-
-
-
Method Detail
-
to
public TextIO.TypedWrite<UserT,DestinationT> to(java.lang.String filenamePrefix)
Writes to text files with the given prefix. The givenprefix
can reference anyFileSystem
on the classpath. This prefix is used by theDefaultFilenamePolicy
to generate filenames.By default, a
DefaultFilenamePolicy
will be used built using the specified prefix to define the base output directory and file prefix, a shard identifier (seewithNumShards(int)
), and a common suffix (if supplied usingwithSuffix(String)
).This default policy can be overridden using
to(FilenamePolicy)
, in which casewithShardNameTemplate(String)
andwithSuffix(String)
should not be set. Custom filename policies do not automatically see this prefix - you should explicitly pass the prefix into yourFileBasedSink.FilenamePolicy
object if you need this.If
withTempDirectory(org.apache.beam.sdk.options.ValueProvider<org.apache.beam.sdk.io.fs.ResourceId>)
has not been called, this filename prefix will be used to infer a directory for temporary files.
-
to
@Experimental(FILESYSTEM) public TextIO.TypedWrite<UserT,DestinationT> to(ResourceId filenamePrefix)
Liketo(String)
.
-
to
public TextIO.TypedWrite<UserT,DestinationT> to(ValueProvider<java.lang.String> outputPrefix)
Liketo(String)
.
-
to
public TextIO.TypedWrite<UserT,DestinationT> to(FileBasedSink.FilenamePolicy filenamePolicy)
Writes to files named according to the givenFileBasedSink.FilenamePolicy
. A directory for temporary files must be specified usingwithTempDirectory(org.apache.beam.sdk.options.ValueProvider<org.apache.beam.sdk.io.fs.ResourceId>)
.
-
to
@Deprecated public <NewDestinationT> TextIO.TypedWrite<UserT,NewDestinationT> to(FileBasedSink.DynamicDestinations<UserT,NewDestinationT,java.lang.String> dynamicDestinations)
Deprecated.Use aFileBasedSink.DynamicDestinations
object to vendFileBasedSink.FilenamePolicy
objects. These objects can examine the input record when creating aFileBasedSink.FilenamePolicy
. A directory for temporary files must be specified usingwithTempDirectory(org.apache.beam.sdk.options.ValueProvider<org.apache.beam.sdk.io.fs.ResourceId>)
.
-
to
@Deprecated public TextIO.TypedWrite<UserT,DefaultFilenamePolicy.Params> to(SerializableFunction<UserT,DefaultFilenamePolicy.Params> destinationFunction, DefaultFilenamePolicy.Params emptyDestination)
Deprecated.Write to dynamic destinations using the default filename policy. The destinationFunction maps the input record to aDefaultFilenamePolicy.Params
object that specifies where the records should be written (base filename, file suffix, and shard template). The emptyDestination parameter specified where empty files should be written for when the writtenPCollection
is empty.
-
toResource
@Experimental(FILESYSTEM) public TextIO.TypedWrite<UserT,DestinationT> toResource(ValueProvider<ResourceId> filenamePrefix)
Liketo(ResourceId)
.
-
withFormatFunction
@Deprecated public TextIO.TypedWrite<UserT,DestinationT> withFormatFunction(@Nullable SerializableFunction<UserT,java.lang.String> formatFunction)
Deprecated.Specifies a format function to convertTextIO.TypedWrite
to the output type. Ifto(DynamicDestinations)
is used,FileBasedSink.DynamicDestinations.formatRecord(Object)
must be used instead.
-
withTempDirectory
@Experimental(FILESYSTEM) public TextIO.TypedWrite<UserT,DestinationT> withTempDirectory(ValueProvider<ResourceId> tempDirectory)
Set the base directory used to generate temporary files.
-
withTempDirectory
@Experimental(FILESYSTEM) public TextIO.TypedWrite<UserT,DestinationT> withTempDirectory(ResourceId tempDirectory)
Set the base directory used to generate temporary files.
-
withShardNameTemplate
public TextIO.TypedWrite<UserT,DestinationT> withShardNameTemplate(java.lang.String shardTemplate)
Uses the givenShardNameTemplate
for naming output files. This option may only be used when using one of the default filename-prefix to() overrides - i.e. not when using eitherto(FilenamePolicy)
orto(DynamicDestinations)
.See
DefaultFilenamePolicy
for how the prefix, shard name template, and suffix are used.
-
withSuffix
public TextIO.TypedWrite<UserT,DestinationT> withSuffix(java.lang.String filenameSuffix)
Configures the filename suffix for written files. This option may only be used when using one of the default filename-prefix to() overrides - i.e. not when using eitherto(FilenamePolicy)
orto(DynamicDestinations)
.See
DefaultFilenamePolicy
for how the prefix, shard name template, and suffix are used.
-
withNumShards
public TextIO.TypedWrite<UserT,DestinationT> withNumShards(int numShards)
Configures the number of output shards produced overall (when using unwindowed writes) or per-window (when using windowed writes).For unwindowed writes, constraining the number of shards is likely to reduce the performance of a pipeline. Setting this value is not recommended unless you require a specific number of output files.
- Parameters:
numShards
- the number of shards to use, or 0 to let the system decide.
-
withNumShards
public TextIO.TypedWrite<UserT,DestinationT> withNumShards(@Nullable ValueProvider<java.lang.Integer> numShards)
LikewithNumShards(int)
. Specifyingnull
means runner-determined sharding.
-
withoutSharding
public TextIO.TypedWrite<UserT,DestinationT> withoutSharding()
Forces a single file as output and empty shard name template.For unwindowed writes, constraining the number of shards is likely to reduce the performance of a pipeline. Setting this value is not recommended unless you require a specific number of output files.
This is equivalent to
.withNumShards(1).withShardNameTemplate("")
-
withDelimiter
public TextIO.TypedWrite<UserT,DestinationT> withDelimiter(char[] delimiter)
Specifies the delimiter after each string written.Defaults to '\n'.
-
withHeader
public TextIO.TypedWrite<UserT,DestinationT> withHeader(@Nullable java.lang.String header)
Adds a header string to each file. A newline after the header is added automatically.A
null
value will clear any previously configured header.
-
withFooter
public TextIO.TypedWrite<UserT,DestinationT> withFooter(@Nullable java.lang.String footer)
Adds a footer string to each file. A newline after the footer is added automatically.A
null
value will clear any previously configured footer.
-
withWritableByteChannelFactory
public TextIO.TypedWrite<UserT,DestinationT> withWritableByteChannelFactory(FileBasedSink.WritableByteChannelFactory writableByteChannelFactory)
Returns a transform for writing to text files like this one but that has the givenFileBasedSink.WritableByteChannelFactory
to be used by theFileBasedSink
during output. The default is value isCompression.UNCOMPRESSED
.A
null
value will reset the value to the default value mentioned above.
-
withCompression
public TextIO.TypedWrite<UserT,DestinationT> withCompression(Compression compression)
Returns a transform for writing to text files like this one but that compresses output using the givenCompression
. The default value isCompression.UNCOMPRESSED
.
-
withWindowedWrites
public TextIO.TypedWrite<UserT,DestinationT> withWindowedWrites()
Preserves windowing of input elements and writes them to files based on the element's window.If using
to(FileBasedSink.FilenamePolicy)
. Filenames will be generated usingFileBasedSink.FilenamePolicy.windowedFilename(int, int, org.apache.beam.sdk.transforms.windowing.BoundedWindow, org.apache.beam.sdk.transforms.windowing.PaneInfo, org.apache.beam.sdk.io.FileBasedSink.OutputFileHints)
. See alsoWriteFiles.withWindowedWrites()
.
-
withNoSpilling
public TextIO.TypedWrite<UserT,DestinationT> withNoSpilling()
-
skipIfEmpty
public TextIO.TypedWrite<UserT,DestinationT> skipIfEmpty()
Don't write any output files if the PCollection is empty.
-
expand
public WriteFilesResult<DestinationT> expand(PCollection<UserT> input)
Description copied from class:PTransform
Override this method to specify how thisPTransform
should be expanded on the givenInputT
.NOTE: This method should not be called directly. Instead apply the
PTransform
should be applied to theInputT
using theapply
method.Composite transforms, which are defined in terms of other transforms, should return the output of one of the composed transforms. Non-composite transforms, which do not apply any transforms internally, should return a new unbound output and register evaluators (via backend-specific registration methods).
- Specified by:
expand
in classPTransform<PCollection<UserT>,WriteFilesResult<DestinationT>>
-
populateDisplayData
public void populateDisplayData(DisplayData.Builder builder)
Description copied from class:PTransform
Register display data for the given transform or component.populateDisplayData(DisplayData.Builder)
is invoked by Pipeline runners to collect display data viaDisplayData.from(HasDisplayData)
. Implementations may callsuper.populateDisplayData(builder)
in order to register display data in the current namespace, but should otherwise usesubcomponent.populateDisplayData(builder)
to use the namespace of the subcomponent.By default, does not register any display data. Implementors may override this method to provide their own display data.
- Specified by:
populateDisplayData
in interfaceHasDisplayData
- Overrides:
populateDisplayData
in classPTransform<PCollection<UserT>,WriteFilesResult<DestinationT>>
- Parameters:
builder
- The builder to populate with display data.- See Also:
HasDisplayData
-
-