Interface PipelineOptions
- All Superinterfaces:
HasDisplayData
- All Known Subinterfaces:
ApplicationNameOptions,AwsOptions,AzureOptions,BeamSqlDataCatalogExample.DCExamplePipelineOptions,BeamSqlPipelineOptions,BigQueryOptions,BigtableChangeStreamTestOptions,BlobstoreOptions,CloudResourceManagerOptions,CosmosOptions,DataCatalogPipelineOptions,DataflowPipelineDebugOptions,DataflowPipelineOptions,DataflowPipelineWorkerPoolOptions,DataflowProfilingOptions,DataflowStreamingPipelineOptions,DataflowWorkerHarnessOptions,DataflowWorkerLoggingOptions,DirectOptions,DirectTestOptions,ElasticsearchIOITCommon.ElasticsearchPipelineOptions,ExecutorOptions,ExpansionServiceOptions,ExperimentalOptions,FileStagingOptions,FirestoreOptions,FlinkPipelineOptions,GcpOptions,GcsOptions,GoogleAdsOptions,GoogleApiDebugOptions,HadoopFileSystemOptions,JetPipelineOptions,KafkaIO.Read.FakeFlinkPipelineOptions,KinesisIOOptions,ManualDockerEnvironmentOptions,MemoryMonitorOptions,MetricsOptions,PortablePipelineOptions,PrismPipelineOptions,PubsubOptions,PythonExternalTransformOptions,RemoteEnvironmentOptions,ResourceHintsOptions,S3Options,SdkHarnessOptions,SnowflakePipelineOptions,SpannerIO.SpannerChangeStreamOptions,SparkCommonPipelineOptions,SparkContextOptions,SparkPipelineOptions,SparkPortableStreamingPipelineOptions,SparkStructuredStreamingPipelineOptions,StreamingOptions,TestBigQueryOptions,TestDataflowPipelineOptions,TestPipeline.TestValueProviderOptions,TestPipelineOptions,TestPortablePipelineOptions,TestPrismPipelineOptions,TestPubsubOptions,TestSparkPipelineOptions,TestUniversalRunner.Options,Twister2PipelineOptions,VersionDependentFlinkPipelineOptions,WordCount.WordCountOptions
PipelineOptions to create
custom configuration options specific to your Pipeline, for both local execution and
execution via a PipelineRunner.
PipelineOptions and their subinterfaces represent a collection of properties which can
be manipulated in a type safe manner. PipelineOptions is backed by a dynamic Proxy which allows for type safe manipulation of properties in an extensible fashion through
plain old Java interfaces.
PipelineOptions can be created with PipelineOptionsFactory.create() and PipelineOptionsFactory.as(Class). They can be created from command-line arguments with PipelineOptionsFactory.fromArgs(String[]). They can be converted to another type by invoking
as(Class) and can be accessed from within a DoFn by invoking
getPipelineOptions() on the input Context object.
Please don't implement PipelineOptions, it implies that it is backwards-incompatible
to add new options. User-implemented PipelineOptions is not accepted by Pipeline.
For example:
// The most common way to construct PipelineOptions is via command-line argument parsing:
public static void main(String[] args) {
// Will parse the arguments passed into the application and construct a PipelineOptions
// Note that --help will print registered options, and --help=PipelineOptionsClassName
// will print out usage for the specific class.
PipelineOptions options =
PipelineOptionsFactory.fromArgs(args).create();
Pipeline p = Pipeline.create(options);
...
p.run();
}
// To create options for the DirectRunner:
DirectOptions directRunnerOptions =
PipelineOptionsFactory.as(DirectOptions.class);
// To cast from one type to another using the as(Class) method:
ApplicationNameOptions applicationNameOptions =
directPipelineOptions.as(ApplicationNameOptions.class);
// Options for the same property are shared between types
// The statement below will print out the name of the enclosing class by default
System.out.println(applicationNameOptions.getApplicationName());
// Prints out registered options.
PipelineOptionsFactory.printHelp(System.out);
// Prints out options which are available to be set on ApplicationNameOptions
PipelineOptionsFactory.printHelp(System.out, ApplicationNameOptions.class);
Defining Your Own PipelineOptions
Defining your own PipelineOptions is the way for you to make configuration options
available for both local execution and execution via a PipelineRunner. By having
PipelineOptionsFactory as your command-line interpreter, you will provide a standardized way for
users to interact with your application via the command-line.
To define your own PipelineOptions, you create a public interface which extends PipelineOptions and define getter/setter pairs. These getter/setter pairs define a collection of
JavaBean
properties.
For example:
// Creates a user defined property called "myProperty"
public interface MyOptions extends PipelineOptions {
String getMyProperty();
void setMyProperty(String value);
}
Note: Please see the section on Registration below when using custom property types.
Restrictions
Since PipelineOptions can be "cast" to multiple types dynamically using as(Class), a property must conform to the following set of restrictions:
- Any property with the same name must have the same return type for all derived interfaces
of
PipelineOptions. - Every bean property of any interface derived from
PipelineOptionsmust have a getter and setter method. - Every method must conform to being a getter or setter for a JavaBean.
- The derived interface of
PipelineOptionsmust be composable with every interface part registered with the PipelineOptionsFactory. - Only getters may be annotated with
@JsonIgnore. - If any getter is annotated with
@JsonIgnore, then all getters for this property must be annotated with@JsonIgnore. - If any getter is annotated with
JsonDeserializeandJsonSerialize, then all getters for this property must also be.
Annotations For PipelineOptions
@Description can be used to annotate an interface or a getter with useful
information which is output when --help is invoked via PipelineOptionsFactory.fromArgs(String[]).
@Default represents a set of annotations that can be used to annotate getter
properties on PipelineOptions with information representing the default value to be
returned if no value is specified. Any default implementation (using the default keyword)
is ignored.
@Hidden hides an option from being listed when --help is invoked via
PipelineOptionsFactory.fromArgs(String[]).
@Validation represents a set of annotations that can be used to annotate
getter properties on PipelineOptions with information representing the validation
criteria to be used when validating with the PipelineOptionsValidator. Validation will be
performed if during construction of the PipelineOptions, PipelineOptionsFactory.withValidation() is invoked.
@JsonIgnore is used to prevent a property from being serialized and
available during execution of DoFn. See the Serialization section below for more details.
@JsonSerialize and @JsonDeserialize is used to
control how a property is (de)serialized when the PipelineOptions are (de)serialized to JSON. See
the Serialization section below for more details.
Registration Of PipelineOptions
Registration of PipelineOptions by an application guarantees that the PipelineOptions is composable during execution of their Pipeline and meets the
restrictions listed above or will fail during registration. Registration also lists the
registered PipelineOptions when --help is invoked via PipelineOptionsFactory.fromArgs(String[]).
Registration can be performed by invoking PipelineOptionsFactory.register(java.lang.Class<? extends org.apache.beam.sdk.options.PipelineOptions>) within a
users application or via automatic registration by creating a ServiceLoader entry and a
concrete implementation of the PipelineOptionsRegistrar interface.
It is optional but recommended to use one of the many build time tools such as AutoService to generate the necessary META-INF files automatically.
A list of registered options can be fetched from PipelineOptionsFactory.getRegisteredOptions().
Serialization Of PipelineOptions
PipelineOptions is intentionally not marked Serializable, in order
to discourage pipeline authors from capturing PipelineOptions at pipeline construction
time, because a pipeline may be saved as a template and run with a different set of options than
the ones it was constructed with. See Pipeline.run(PipelineOptions).
However, PipelineRunners require support for options to be serialized. Each property
within PipelineOptions must be able to be serialized using Jackson's ObjectMapper
or the getter method for the property annotated with @JsonIgnore.
Jackson supports serialization of many types and supports a useful set of annotations to aid in serialization
of custom types. We point you to the public Jackson documentation when attempting to add
serialization support for your custom types. Note that PipelineOptions relies on
Jackson's ability to automatically configure the ObjectMapper with additional modules via
ObjectMapper.findModules().
To further customize serialization, getter methods may be annotated with @JsonSerialize and @JsonDeserialize. @JsonDeserialize is also used when parsing command line arguments.
Note: A property must be annotated with BOTH@JsonDeserialize
and @JsonSerialize or neither. It is an error to have a property annotated
with only @JsonDeserialize or @JsonSerialize.
Note: It is an error to have the same property available in multiple interfaces with only some
of them being annotated with @JsonIgnore. It is also an error to mark a setter
for a property with @JsonIgnore.
-
Nested Class Summary
Nested ClassesModifier and TypeInterfaceDescriptionstatic classDefaultValueFactorywhich supplies an ID that is guaranteed to be unique within the given process.static enumEnumeration of the possible states for a given check.static classADefaultValueFactorythat obtains the class of theDirectRunnerif it exists on the classpath, and throws an exception otherwise.static classReturns a normalized job name constructed fromApplicationNameOptions.getAppName(), the local system user name (if available), the current time, and a random integer.static classReturns a user agent string constructed fromReleaseInfo.getName()andReleaseInfo.getVersion(), in the format[name]/[version]. -
Method Summary
Modifier and TypeMethodDescription<T extends PipelineOptions>
TTransforms this object into an object of type<T>saving each property that has been manipulated.getGbek()A string defining whether GroupByKey transforms should be replaced by GroupByEncryptedKeylongProvides a process wide unique ID for thisPipelineOptionsobject, assigned at graph construction time.Class<? extends PipelineRunner<?>> The pipeline runner that will be used to execute the pipeline.Whether to check for stable unique names on each transform.A pipeline level default location for storing temporary files.A user agent string as per RFC2616, describing the pipeline to external services.Returns a map of properties which correspond toValueProvider.RuntimeValueProvider, keyed by the property name.intrevision()A monotonically increasing revision number of thisPipelineOptionsobject that can be used to detect changes.voidvoidsetJobName(String jobName) voidsetOptionsId(long id) voidsetRunner(Class<? extends PipelineRunner<?>> kls) voidvoidsetTempLocation(String value) voidsetUserAgent(String userAgent) Methods inherited from interface org.apache.beam.sdk.transforms.display.HasDisplayData
populateDisplayData
-
Method Details
-
as
Transforms this object into an object of type<T>saving each property that has been manipulated.<T>must extendPipelineOptions.If
<T>is not registered with thePipelineOptionsFactory, then we attempt to verify that<T>is composable with every interface that this instance of thePipelineOptionshas seen.- Parameters:
kls- The class of the type to transform to.- Returns:
- An object of type kls.
-
getRunner
The pipeline runner that will be used to execute the pipeline. For registered runners, the class name can be specified, otherwise the fully qualified name needs to be specified. -
setRunner
-
getStableUniqueNames
Whether to check for stable unique names on each transform. This is necessary to support updating of pipelines. -
setStableUniqueNames
-
getTempLocation
String getTempLocation()A pipeline level default location for storing temporary files.This can be a path of any file system.
getTempLocation()can be used as a default location in otherPipelineOptions.If it is unset,
PipelineRunnercan override it. -
setTempLocation
-
getJobName
-
setJobName
-
outputRuntimeOptions
Returns a map of properties which correspond toValueProvider.RuntimeValueProvider, keyed by the property name. The value is a map containing type and default information. -
revision
int revision()A monotonically increasing revision number of thisPipelineOptionsobject that can be used to detect changes. -
getOptionsId
Provides a process wide unique ID for thisPipelineOptionsobject, assigned at graph construction time. -
setOptionsId
void setOptionsId(long id) -
getUserAgent
A user agent string as per RFC2616, describing the pipeline to external services.https://www.ietf.org/rfc/rfc2616.txt
It should follow the BNF Form:
Where a token is a series of characters without a separator.user agent = 1*(product | comment) product = token ["/" product-version] product-version = tokenThe string defaults to
[name]/[version]based on the properties of the Apache Beam release. -
setUserAgent
-
getGbek
A string defining whether GroupByKey transforms should be replaced by GroupByEncryptedKeyBeam will infer the secret type and value based on the secret itself. This guarantees that any data at rest during the performing a GBK, so this can be used to guarantee that data is not unencrypted. Runners with this behavior include the Dataflow, Flink, and Spark runners. The secret should be a url safe base64 encoded 32 byte value. The option should be structured like:
for example:--gbek=type:; :
All variables should use snake case to allow consistency across languages. For an example of generating a properly formatted secret, see https://github.com/apache/beam/blob/c8df4da229da49d533491857e1bb4ab5dbf4fd37/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyIT.java#L82--gbek=type:GcpSecret;version_name:my_secret/versions/latest" -
setGbek
-