diff --git a/website/www/site/content/en/documentation/programming-guide.md b/website/www/site/content/en/documentation/programming-guide.md index d8b3ec06342e..3eee8fef6d16 100644 --- a/website/www/site/content/en/documentation/programming-guide.md +++ b/website/www/site/content/en/documentation/programming-guide.md @@ -5491,3 +5491,283 @@ use case. {{< highlight py >}} {{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" BundleFinalize >}} {{< /highlight >}} + +## 13. Multi-language pipelines {#mulit-language-pipelines} + +Beam allows you to combine transforms written in any supported SDK language (currently, Java and Python) and use them in one multi-language pipeline. This capability makes it easy to provide new functionality simultaneously in different Apache Beam SDKs through a single cross-language transform. For example, the [Apache Kafka connector](https://github.com/apache/beam/blob/master/sdks/python/apache_beam/io/kafka.py) and [SQL transform](https://github.com/apache/beam/blob/master/sdks/python/apache_beam/transforms/sql.py) from the Java SDK can be used in Python streaming pipelines. + +Pipelines that use transforms from more than one SDK-language are known as *multi-language pipelines*. + +### 13.1. Creating cross-language transforms {#create-x-lang-transforms} + +To make transforms written in one language available to pipelines written in another language, an *expansion service* for transforms written in the same language is used to create and inject the appropriate language-specific pipeline fragments into your pipeline. + +In the following example, a Python pipeline written the Apache Beam SDK for Python starts up a local Java expansion service on your computer to create and inject the appropriate Java pipeline fragments for executing the Java Kafka cross-language transform into your Python pipeline. The SDK then downloads and stages the necessary Java dependencies needed to execute these transforms. + +![Diagram of multi-language pipeline execution flow.](/images/multi-language-pipelines-diagram.svg) + +At runtime, the Beam runner will execute both Python and Java transforms to execute your pipeline. + +In this section, we will use [KafkaIO.Read](https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/kafka/KafkaIO.Read.html) to illustrate how to create a cross-language transform for Java and a test example for Python. + +#### 13.1.1. Creating cross-language Java transforms + +To make your Apache Beam Java SDK transform portable across SDK languages, you must implement two interfaces: [ExternalTransformBuilder](https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ExternalTransformBuilder.java) and [ExternalTransformRegistrar](https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/expansion/ExternalTransformRegistrar.java). The `ExternalTransformBuilder` interface constructs the cross-language transform using configuration values passed in from the pipeline and the `ExternalTransformRegistrar` interface registers the cross-language transform for use with the expansion service. + +**Implementing the interfaces** + +1. Define a Builder class for your transform that implements the `ExternalTransformBuilder` interface and overrides the `buildExternal` method that will be used to build your transform object. Initial configuration values for your transform should be defined in the `buildExternal` method. In most cases, it is convenient to make the Java transform builder class implement `ExternalTransformBuilder`. + + > **Note:** `ExternalTransformBuilder` requires you to define a configuration object (a simple POJO) to capture a set of parameters sent by external SDKs to initiate the Java transform. Usually these parameters directly map to constructor parameters of the Java transform. + + {{< highlight >}} +@AutoValue.Builder +abstract static class Builder + implements ExternalTransformBuilder>> { + abstract Builder setConsumerConfig(Map config); + + abstract Builder setTopics(List topics); + + /** Remaining property declarations omitted for clarity. */ + + abstract Read build(); + + @Override + public PTransform>> buildExternal( + External.Configuration config) { + setTopics(ImmutableList.copyOf(config.topics)); + + /** Remaining property defaults omitted for clarity. */ + } +} + {{< /highlight >}} + + Note that `buildExternal` method may choose to perform additional operations before setting properties received from external SDKs in the transform. For example, `buildExternal` method may validates properties available in the configuration object before setting them in the transform. + +2. Register the transform as an external cross-language transform by defining a class that implements `ExternalTransformRegistrar`. You must annotate your class with the `AutoService` annotation to ensure that your transform is registered and instantiated properly by the expansion service. +3. In your registrar class, define a Uniform Resource Name (URN) for your transform. The URN must be a unique string that identifies your transform with the expansion service. +4. From within your registrar class, define a configuration class for the parameters used during the initialization of your transform by the external SDK. + + The following example from the KafkaIO transform shows how to implement steps two through four: + + {{< highlight >}} +@AutoService(ExternalTransformRegistrar.class) +public static class External implements ExternalTransformRegistrar { + + public static final String URN = "beam:external:java:kafka:read:v1"; + + @Override + public Map>> knownBuilders() { + return ImmutableMap.of( + URN, + (Class>) + (Class) AutoValue_KafkaIO_Read.Builder.class); + } + + /** Parameters class to expose the Read transform to an external SDK. */ + public static class Configuration { + private Map consumerConfig; + private List topics; + + public void setConsumerConfig(Map consumerConfig) { + this.consumerConfig = consumerConfig; + } + + public void setTopics(List topics) { + this.topics = topics; + } + + /** Remaining properties omitted for clarity. */ + } +} + {{< /highlight >}} + +After you have implemented the `ExternalTransformBuilder` and `ExternalTransformRegistrar` interfaces, your transform can be registered and created successfully by the default Java expansion service. + +**Starting the expansion service** + +An expansion service can be used with multiple transforms in the same pipeline. Java has a default expansion service included and available in the Apache Beam Java SDK for you to use with your Java transforms. You can write your own expansion service, but that is generally not needed, so it is not covered in this section. + +Perform the following to start up a Java expansion service directly: + +{{< highlight >}} +# Build a JAR with both your transform and the expansion service + +# Start the expansion service at the specified port. +$ jar -jar /path/to/expansion_service.jar +{{< /highlight >}} + +The expansion service is now ready to serve transforms on the specified port. + +When creating SDK-specific wrappers for your transform, SDKs may provide utilities that are readily available for easily starting up an expansion service. For example, the Python SDK provides the utilities [JavaJarExpansionService and BeamJarExpansionService](https://github.com/apache/beam/blob/master/sdks/python/apache_beam/transforms/external.py) utility for starting up a Java expansion service using a JAR file. + +**Including dependencies** + +If your transform requires external libraries, you can include them by adding them to the classpath of the expansion service. After they are included in the classpath, they will be staged when your transform is expanded by the expansion service. + +**Writing SDK-specific wrappers** + +Your cross-language Java transform can be called through the lower-level [ExternalTransform](https://github.com/apache/beam/blob/master/sdks/python/apache_beam/transforms/external.py) class in a multi-language pipeline (as described in the next section); however, if possible, you should create a SDK-specific wrapper written in the programming language of the pipeline (such as Python) to access the transform instead. This higher-level abstraction will make it easier for pipeline authors to use your transform. + +To create an SDK wrapper for use in a Python pipeline, do the following: + +1. Create a Python module for your cross-language transform(s). +2. In the module, build the payload that should be used to initiate the cross-language transform expansion request using one of the available [PayloadBuilder](https://github.com/apache/beam/blob/master/sdks/python/apache_beam/transforms/external.py) classes. + + The parameter names and types of the payload should map to parameter names and types of the configuration POJO provided to the Java `ExternalTransformBuilder`. Parameter types are mapped across SDKs using a [Beam schema](https://github.com/apache/beam/blob/master/model/pipeline/src/main/proto/schema.proto). Parameter names are mapped by simply converting Python underscore-separated variable names to camel-case (Java standard). + + In the following example, [kafka.py](https://github.com/apache/beam/blob/master/sdks/python/apache_beam/io/kafka.py) uses `NamedTupleBasedPayloadBuilder` to build the payload. The parameters map to the Java [KafkaIO.External.Configuration](https://github.com/apache/beam/blob/master/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java) config object defined previously in the **Implementing the interfaces** section. + + {{< highlight >}} +class ReadFromKafkaSchema(typing.NamedTuple): + consumer_config: typing.Mapping[str, str] + topics: typing.List[str] + # Other properties omitted for clarity. + +payload = NamedTupleBasedPayloadBuilder(ReadFromKafkaSchema(...)) + {{< /highlight >}} + +3. Start an expansion service unless one is specified by the pipeline creator. The Apache Beam Python SDK provides utilities `JavaJarExpansionService` and `BeamJarExpansionService` for easily starting up an expansion service using a JAR file.. `JavaJarExpansionService` can be used to startup an expansion service using path (a local path or a URL) to a given JAR file. `BeamJarExpansionService` can be used for easily starting an expansion service based on a JAR released with Beam. + + For transforms released with Beam do the following: + + 1. Add a Gradle target to Beam that can be used to build a shaded expansion service JAR for the target Java transform. This target should produce a Beam JAR that contains all dependencies needed for expanding the Java transform and the JAR should be released with Beam. Note that you might be able to use one of the existing Gradle target that offer an aggregated version of an expansion service jar (for example, for all GCP IO). + 2. In your Python module, instantiate `BeamJarExpansionService` with the Gradle target. + + {{< highlight >}} + expansion_service = BeamJarExpansionService('sdks:java:io:expansion-service:shadowJar') + {{< /highlight >}} +4. Add a Python wrapper transform class that extends [ExternalTransform](https://github.com/apache/beam/blob/master/sdks/python/apache_beam/transforms/external.py). Pass the payload and expansion service defined above as parameters to the constructor of the `ExternalTransform` parent class. + +#### 13.1.2. Creating cross-language Python transforms + +To make your Python transform usable with different SDK languages, you must create a Python module that registers an existing Python transform as a cross-language transform for use with the Python expansion service and calls into that existing transform to perform its intended operation. + +**Defining the Python module** + +1. Define a Uniform Resource Name (URN) for your transform. The URN must be a unique string that identifies your transform with the expansion service. + + {{< highlight >}} +TEST_COMPK_URN = "beam:transforms:xlang:test:compk" + {{< /highlight >}} +2. For an existing Python transform, create a new class to register the URN with the Python expansion service. + + {{< highlight >}} +@ptransform.PTransform.register_urn(TEST_COMPK_URN, None) +class CombinePerKeyTransform(ptransform.PTransform): + {{< /highlight >}} +3. From within the class, define an expand method that takes an input PCollection, runs the Python transform, and then returns the output PCollection. + + {{< highlight >}} +def expand(self, pcoll): + return pcoll \ + | beam.CombinePerKey(sum).with_output_types( + typing.Tuple[unicode, int]) + {{< /highlight >}} +4. As with other Python transforms, define a `to_runner_api_parameter` method that returns the URN. + + {{< highlight >}} +def to_runner_api_parameter(self, unused_context): + return TEST_COMPK_URN, None + {{< /highlight >}} +5. Define a static `from_runner_api_parameter` method that returns an instantiation of the cross-language Python transform. + + {{< highlight >}} +@staticmethod +def from_runner_api_parameter( + unused_ptransform, unused_parameter, unused_context): + return CombinePerKeyTransform() + {{< /highlight >}} + +**Starting the expansion service** + +An expansion service can be used with multiple transforms in the same pipeline. Python has a default expansion service included and available in the Apache Beam Python SDK for you to use with your Python transforms. You are free to write your own expansion service, but that is generally not needed, so it is not covered in this section. + +Perform the following steps to start up the default Python expansion service directly: + +1. Create a virtual environment and [install the Apache Beam SDK](https://beam.apache.org/get-started/quickstart-py/). +2. Start the Python SDK’s expansion service with a specified port. + + {{< highlight >}} +$ export PORT_FOR_EXPANSION_SERVICE=12345 +3. Import any modules that contain transforms to be made available using the expansion service. +$ python -m apache_beam.runners.portability.expansion_service -p $PORT_FOR_EXPANSION_SERVICE + {{< /highlight >}} + +4. This expansion service is not ready to serve up transforms on the address `localhost:$PORT_FOR_EXPANSION_SERVICE`. + +**Including dependencies** + +Currently Python external transforms are limited to dependencies available in core Beam SDK Harness. + +### 13.2. Using cross-language transforms {#use-x-lang-transforms} + +Depending on the SDK language of the pipeline, you can use a high-level SDK-wrapper class, or a low-level transform class to access a cross-language transform. + +#### 13.2.1. Using cross-language transforms in a Java pipeline + +Currently, to access cross-language transforms from the Java SDK, you have to use the lower-level [External](https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java) class. + +**Using the External class** + +1. Make sure you have any runtime environment dependencies (like JRE) installed on your local machine (either directly on the local machine or available through a container). See the expansion service section for more details. + + > **Note:** When including Python transforms from within a Java pipeline, all Python dependencies have to be included in the SDK harness container. +2. Start up the expansion service for the SDK that is in the language of the transform you're trying to consume, if not available. + + Make sure the transform you are trying to use is available and can be used by the expansion service. +3. Include [External.of(...)](https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java) when instantiating your pipeline. Reference the URN, payload, and expansion service. For examples, see the [cross-language transform test suite](https://github.com/apache/beam/blob/master/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java). +4. After the job has been submitted to the Beam runner, shutdown the expansion service by terminating the expansion service process. + +#### 13.2.2 Using cross-language transforms in a Python pipeline + +If a Python-specific wrapper for a cross-language transform is available, use that; otherwise, you have to use the lower-level [ExternalTransform](https://github.com/apache/beam/blob/master/sdks/python/apache_beam/transforms/external.py) class to access the transform. + +**Using an SDK wrapper** + +To use a cross-language transform through an SDK wrapper, import the module for the SDK wrapper and call it from your pipeline as shown in the example: + + {{< highlight >}} +from apache_beam.io.kafka import ReadFromKafka + +kafka_records = ( + pipeline + | 'ReadFromKafka' >> ReadFromKafka( + consumer_config={ + 'bootstrap.servers': self.bootstrap_servers, + 'auto.offset.reset': 'earliest' + }, + topics=[self.topic], + max_num_records=max_num_records, + expansion_service=
)) + {{< /highlight >}} + +**Using the ExternalTransform class** + +When an SDK-specific wrapper isn't available, you will have to access the cross-language transform through the `ExternalTransform` class. + +1. Make sure you have any runtime environment dependencies (like JRE) installed on your local machine. See the expansion service section for more details. +2. Start up the expansion service for the SDK that is in the language of the transform you're trying to consume, if not available. + + Make sure the transform you're trying to use is available and can be used by the expansion service. For Java, make sure the builder and registrar for the transform are available in the classpath of the expansion service. +3. Include `ExternalTransform` when instantiating your pipeline. Reference the URN, Payload, and expansion service. You can use one of the available [PayloadBuilder](https://github.com/apache/beam/blob/master/sdks/python/apache_beam/transforms/external.py) classes to build the payload for `ExternalTransform`. + + {{< highlight >}} +with pipeline as p: + res = ( + p + | beam.Create(['a', 'b']).with_output_types(unicode) + | beam.ExternalTransform( + TEST_PREFIX_URN, + ImplicitSchemaPayloadBuilder({'data': u'0'}), +
)) + assert_that(res, equal_to(['0a', '0b'])) + {{< /highlight >}} + +4. After the job has been submitted to the Beam runner, shutdown the expansion service by terminating the expansion service process. + +### 13.3. Runner Support {#x-lang-transform-runner-support} + +Currently, portable runners such as Flink, Spark, and the Direct runner can be used with multi-language pipelines. + +Google Cloud Dataflow supports multi-language pipelines through the Dataflow Runner v2 backend architecture. diff --git a/website/www/site/layouts/partials/section-menu/en/documentation.html b/website/www/site/layouts/partials/section-menu/en/documentation.html index a7ba2d45d751..bba2ff7f660e 100644 --- a/website/www/site/layouts/partials/section-menu/en/documentation.html +++ b/website/www/site/layouts/partials/section-menu/en/documentation.html @@ -172,6 +172,16 @@
  • Bundle finalization
  • + +
  • + Multi-language Pipelines + + +
  • diff --git a/website/www/site/static/images/multi-language-pipelines-diagram.svg b/website/www/site/static/images/multi-language-pipelines-diagram.svg new file mode 100644 index 000000000000..974e4e4b3145 --- /dev/null +++ b/website/www/site/static/images/multi-language-pipelines-diagram.svg @@ -0,0 +1,20 @@ + + + \ No newline at end of file